Hi Igniters,
In the current implementation, cache metrics are collected on each node and sent across the whole cluster with discovery message (TcpDiscoveryMetricsUpdateMessage) with configured frequency (MetricsUpdateFrequency, 2 seconds by default) even if no one requested them. If there are a lot of caches and a lot of nodes in the cluster, metrics update message (which contain each metric for each cache on each node) can reach a critical size. Also frequently collecting all cache metrics have a negative performance impact (some of them just get values from AtomicLong, but some of them need an iteration over all cache partitions). The only way now to disable cache metrics collecting and sending with discovery message is to disable statistics for each cache. But this also makes impossible to request some of cache metrics locally (for the current node only). Requesting a limited set of cache metrics on the current node doesn't have such performance impact as the frequent collecting of all cache metrics, but sometimes it's enough for diagnostic purposes. As a workaround I have filled and implemented ticket [1], which introduces new system property to disable cache metrics sending with TcpDiscoveryMetricsUpdateMessage (in case this property is set, the message will contain only node metrics). But system property is not good for a permanent solution. Perhaps it's better to move such property to public API (to IgniteConfiguration for example). Also maybe we should change cache metrics distributing strategy? For example, collect metrics by request via communication SPI or subscribe to a limited set of cache/metrics, etc. Thoughts? [1]: https://issues.apache.org/jira/browse/IGNITE-10172 |
Alex,
Did you measure the impact of metrics collection? What is the overhead you are trying to avoid? Just to make it clear, MetricUpdateMessage-s are used as heartbeats. So they are sent anyways, even if no metrics are distributed between nodes. Denis вт, 4 дек. 2018 г. в 12:46, Alex Plehanov <[hidden email]>: > Hi Igniters, > > In the current implementation, cache metrics are collected on each node and > sent across the whole cluster with discovery message > (TcpDiscoveryMetricsUpdateMessage) with configured frequency > (MetricsUpdateFrequency, 2 seconds by default) even if no one requested > them. > If there are a lot of caches and a lot of nodes in the cluster, metrics > update message (which contain each metric for each cache on each node) can > reach a critical size. > > Also frequently collecting all cache metrics have a negative performance > impact (some of them just get values from AtomicLong, but some of them need > an iteration over all cache partitions). > The only way now to disable cache metrics collecting and sending with > discovery message is to disable statistics for each cache. But this also > makes impossible to request some of cache metrics locally (for the current > node only). Requesting a limited set of cache metrics on the current node > doesn't have such performance impact as the frequent collecting of all > cache metrics, but sometimes it's enough for diagnostic purposes. > > As a workaround I have filled and implemented ticket [1], which introduces > new system property to disable cache metrics sending with > TcpDiscoveryMetricsUpdateMessage (in case this property is set, the message > will contain only node metrics). But system property is not good for a > permanent solution. Perhaps it's better to move such property to public API > (to IgniteConfiguration for example). > > Also maybe we should change cache metrics distributing strategy? For > example, collect metrics by request via communication SPI or subscribe to a > limited set of cache/metrics, etc. > > Thoughts? > > [1]: https://issues.apache.org/jira/browse/IGNITE-10172 > |
In reply to this post by Alexey Plekhanov
hi, Alex.
imo: 1. metrics through discovery require refactoring. 2. local cache metrics should be available (if configured) on each node. 3. there must be an opportunity to configure metrics in runtime. thanks. > > >Hi Igniters, > >In the current implementation, cache metrics are collected on each node and >sent across the whole cluster with discovery message >(TcpDiscoveryMetricsUpdateMessage) with configured frequency >(MetricsUpdateFrequency, 2 seconds by default) even if no one requested >them. >If there are a lot of caches and a lot of nodes in the cluster, metrics >update message (which contain each metric for each cache on each node) can >reach a critical size. > >Also frequently collecting all cache metrics have a negative performance >impact (some of them just get values from AtomicLong, but some of them need >an iteration over all cache partitions). >The only way now to disable cache metrics collecting and sending with >discovery message is to disable statistics for each cache. But this also >makes impossible to request some of cache metrics locally (for the current >node only). Requesting a limited set of cache metrics on the current node >doesn't have such performance impact as the frequent collecting of all >cache metrics, but sometimes it's enough for diagnostic purposes. > >As a workaround I have filled and implemented ticket [1], which introduces >new system property to disable cache metrics sending with >TcpDiscoveryMetricsUpdateMessage (in case this property is set, the message >will contain only node metrics). But system property is not good for a >permanent solution. Perhaps it's better to move such property to public API >(to IgniteConfiguration for example). > >Also maybe we should change cache metrics distributing strategy? For >example, collect metrics by request via communication SPI or subscribe to a >limited set of cache/metrics, etc. > >Thoughts? > >[1]: https://issues.apache.org/jira/browse/IGNITE-10172 -- Zhenya Stanilovsky |
In reply to this post by Alexey Plekhanov
Hi Alex,
Agree with you. Most of the time these distribution of metrics is not needed. In future we will have more and more information which potentially needs to be shared between nodes. E.g. IO statistics, SQL statistics for query optimizer, SQL execution history, etc. We need common mechanics for this, so I vote for your proposal: 1) Data is collected locally 2) If a node needs to collect data from the cluster, it sends explicit request over communication SPI 3) For performance reasons we may consider caching - return previously collected metrics without re-requesting them again if they are not too old (configurable) On Tue, Dec 4, 2018 at 12:46 PM Alex Plehanov <[hidden email]> wrote: > Hi Igniters, > > In the current implementation, cache metrics are collected on each node and > sent across the whole cluster with discovery message > (TcpDiscoveryMetricsUpdateMessage) with configured frequency > (MetricsUpdateFrequency, 2 seconds by default) even if no one requested > them. > If there are a lot of caches and a lot of nodes in the cluster, metrics > update message (which contain each metric for each cache on each node) can > reach a critical size. > > Also frequently collecting all cache metrics have a negative performance > impact (some of them just get values from AtomicLong, but some of them need > an iteration over all cache partitions). > The only way now to disable cache metrics collecting and sending with > discovery message is to disable statistics for each cache. But this also > makes impossible to request some of cache metrics locally (for the current > node only). Requesting a limited set of cache metrics on the current node > doesn't have such performance impact as the frequent collecting of all > cache metrics, but sometimes it's enough for diagnostic purposes. > > As a workaround I have filled and implemented ticket [1], which introduces > new system property to disable cache metrics sending with > TcpDiscoveryMetricsUpdateMessage (in case this property is set, the message > will contain only node metrics). But system property is not good for a > permanent solution. Perhaps it's better to move such property to public API > (to IgniteConfiguration for example). > > Also maybe we should change cache metrics distributing strategy? For > example, collect metrics by request via communication SPI or subscribe to a > limited set of cache/metrics, etc. > > Thoughts? > > [1]: https://issues.apache.org/jira/browse/IGNITE-10172 > |
Hi,
One of the problems with metrics is a huge size in case when a lot caches started on node (for example, I see 7000 caches). We have to think how to compact them. Not all metrics changed frequently, so, we may store locally and send over wire only a difference from previous collect. And think carefully about store format. For example, if current cache metrics will be passed as JSON object, then 70% of it will be strings with metrics names. On Tue, Dec 4, 2018 at 7:22 PM Vladimir Ozerov <[hidden email]> wrote: > Hi Alex, > > Agree with you. Most of the time these distribution of metrics is not > needed. In future we will have more and more information which potentially > needs to be shared between nodes. E.g. IO statistics, SQL statistics for > query optimizer, SQL execution history, etc. We need common mechanics for > this, so I vote for your proposal: > 1) Data is collected locally > 2) If a node needs to collect data from the cluster, it sends explicit > request over communication SPI > 3) For performance reasons we may consider caching - return previously > collected metrics without re-requesting them again if they are not too old > (configurable) > > On Tue, Dec 4, 2018 at 12:46 PM Alex Plehanov <[hidden email]> > wrote: > > > Hi Igniters, > > > > In the current implementation, cache metrics are collected on each node > and > > sent across the whole cluster with discovery message > > (TcpDiscoveryMetricsUpdateMessage) with configured frequency > > (MetricsUpdateFrequency, 2 seconds by default) even if no one requested > > them. > > If there are a lot of caches and a lot of nodes in the cluster, metrics > > update message (which contain each metric for each cache on each node) > can > > reach a critical size. > > > > Also frequently collecting all cache metrics have a negative performance > > impact (some of them just get values from AtomicLong, but some of them > need > > an iteration over all cache partitions). > > The only way now to disable cache metrics collecting and sending with > > discovery message is to disable statistics for each cache. But this also > > makes impossible to request some of cache metrics locally (for the > current > > node only). Requesting a limited set of cache metrics on the current node > > doesn't have such performance impact as the frequent collecting of all > > cache metrics, but sometimes it's enough for diagnostic purposes. > > > > As a workaround I have filled and implemented ticket [1], which > introduces > > new system property to disable cache metrics sending with > > TcpDiscoveryMetricsUpdateMessage (in case this property is set, the > message > > will contain only node metrics). But system property is not good for a > > permanent solution. Perhaps it's better to move such property to public > API > > (to IgniteConfiguration for example). > > > > Also maybe we should change cache metrics distributing strategy? For > > example, collect metrics by request via communication SPI or subscribe > to a > > limited set of cache/metrics, etc. > > > > Thoughts? > > > > [1]: https://issues.apache.org/jira/browse/IGNITE-10172 > > > -- Alexey Kuznetsov |
Denis, I measure the impact of metrics collecting on my laptop, it's about
5 seconds on each node for collecting metrics of 1000 caches (all caches in one cache group) with 32000 partitions. All this time tcp-disco-msg-worker is blocked. Guys, thanks for your proposals, I'd filled ticket [1]. [1]: https://issues.apache.org/jira/browse/IGNITE-10642 вт, 4 дек. 2018 г. в 18:07, Alexey Kuznetsov <[hidden email]>: > Hi, > > One of the problems with metrics is a huge size in case when a lot caches > started on node (for example, I see 7000 caches). > We have to think how to compact them. > Not all metrics changed frequently, so, we may store locally and send over > wire only a difference from previous collect. > > And think carefully about store format. For example, if current cache > metrics will be passed as JSON object, > then 70% of it will be strings with metrics names. > > > On Tue, Dec 4, 2018 at 7:22 PM Vladimir Ozerov <[hidden email]> > wrote: > > > Hi Alex, > > > > Agree with you. Most of the time these distribution of metrics is not > > needed. In future we will have more and more information which > potentially > > needs to be shared between nodes. E.g. IO statistics, SQL statistics for > > query optimizer, SQL execution history, etc. We need common mechanics for > > this, so I vote for your proposal: > > 1) Data is collected locally > > 2) If a node needs to collect data from the cluster, it sends explicit > > request over communication SPI > > 3) For performance reasons we may consider caching - return previously > > collected metrics without re-requesting them again if they are not too > old > > (configurable) > > > > On Tue, Dec 4, 2018 at 12:46 PM Alex Plehanov <[hidden email]> > > wrote: > > > > > Hi Igniters, > > > > > > In the current implementation, cache metrics are collected on each node > > and > > > sent across the whole cluster with discovery message > > > (TcpDiscoveryMetricsUpdateMessage) with configured frequency > > > (MetricsUpdateFrequency, 2 seconds by default) even if no one requested > > > them. > > > If there are a lot of caches and a lot of nodes in the cluster, metrics > > > update message (which contain each metric for each cache on each node) > > can > > > reach a critical size. > > > > > > Also frequently collecting all cache metrics have a negative > performance > > > impact (some of them just get values from AtomicLong, but some of them > > need > > > an iteration over all cache partitions). > > > The only way now to disable cache metrics collecting and sending with > > > discovery message is to disable statistics for each cache. But this > also > > > makes impossible to request some of cache metrics locally (for the > > current > > > node only). Requesting a limited set of cache metrics on the current > node > > > doesn't have such performance impact as the frequent collecting of all > > > cache metrics, but sometimes it's enough for diagnostic purposes. > > > > > > As a workaround I have filled and implemented ticket [1], which > > introduces > > > new system property to disable cache metrics sending with > > > TcpDiscoveryMetricsUpdateMessage (in case this property is set, the > > message > > > will contain only node metrics). But system property is not good for a > > > permanent solution. Perhaps it's better to move such property to public > > API > > > (to IgniteConfiguration for example). > > > > > > Also maybe we should change cache metrics distributing strategy? For > > > example, collect metrics by request via communication SPI or subscribe > > to a > > > limited set of cache/metrics, etc. > > > > > > Thoughts? > > > > > > [1]: https://issues.apache.org/jira/browse/IGNITE-10172 > > > > > > > > -- > Alexey Kuznetsov > |
Free forum by Nabble | Edit this page |