[CELEBORN-1237][METRICS] Refactor metrics name

### What changes were proposed in this pull request?
Refactor metrics name.

### Why are the changes needed?
Easier to understand the meaning of metrics

### Does this PR introduce _any_ user-facing change?
METRICS.md
migration.md
monitoring.md

### How was this patch tested?
Existing UTs.

Closes #2240 from leixm/metrics_name.

Authored-by: xianminglei <xianming.lei@shopee.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
This commit is contained in:
xianminglei 2024-01-18 18:15:43 +08:00 committed by zky.zhoukeyong
parent 749a0fa439
commit b90fb1fdb2
6 changed files with 20 additions and 14 deletions

View File

@ -92,7 +92,7 @@ Here is an example of Grafana dashboard importing.
| OfferSlotsTime | master | The time of offer slots. |
| OpenStreamTime | worker | OpenStream means read a shuffle file and send client about chunks size and stream index. |
| FetchChunkTime | worker | FetchChunk means read a chunk from a shuffle file and send to client. |
| ChunkStreamCount | worker | The stream count for reduce partition reading streams in current worker. |
| ActiveChunkStreamCount | worker | The active stream count for reduce partition reading streams in current worker. |
| OpenStreamFailCount | worker | The count of opening stream failed in current worker. |
| FetchChunkFailCount | worker | The count of fetching chunk failed in current worker. |
| PrimaryPushDataTime | worker | PrimaryPushData means handle PushData of primary partition location. |
@ -133,7 +133,7 @@ Here is an example of Grafana dashboard importing.
| BufferStreamReadBuffer | worker | This value means memory used by credit stream read buffer. |
| ReadBufferDispatcherRequestsLength | worker | This value means the queue size of read buffer allocation requests. |
| ReadBufferAllocatedCount | worker | This value means count of allocated read buffer. |
| CreditStreamCount | worker | This value means count of stream for map partition reading streams. |
| ActiveCreditStreamCount | worker | This value means active count of stream for map partition reading streams. |
| ActiveMapPartitionCount | worker | This value means count of active map partition reading streams. |
| DeviceOSFreeBytes | worker | This value means actual usable space of OS for device monitor. |
| DeviceOSTotalBytes | worker | This value means total usable space of OS for device monitor. |

View File

@ -3475,13 +3475,13 @@
"uid": "${DS_PROMETHEUS}"
},
"editorMode": "code",
"expr": "metrics_ChunkStreamCount_Value",
"expr": "metrics_ActiveChunkStreamCount_Value",
"legendFormat": "${baseLegend}",
"range": true,
"refId": "A"
}
],
"title": "metrics_ChunkStreamCount_Value",
"title": "metrics_ActiveChunkStreamCount_Value",
"type": "timeseries"
},
{
@ -7578,13 +7578,13 @@
"uid": "${DS_PROMETHEUS}"
},
"editorMode": "builder",
"expr": "metrics_CreditStreamCount_Value",
"expr": "metrics_ActiveCreditStreamCount_Value",
"legendFormat": "${baseLegend}",
"range": true,
"refId": "A"
}
],
"title": "CreditStreamCount",
"title": "ActiveCreditStreamCount",
"type": "timeseries"
},
{

View File

@ -21,6 +21,12 @@ license: |
# Migration Guide
## Upgrading from 0.4 to 0.5
- Since 0.5.0 Celeborn worker metrics `ChunkStreamCount` is renamed as `ActiveChunkStreamCount`.
- Since 0.5.0 Celeborn worker metrics `CreditStreamCount` is renamed as `ActiveCreditStreamCount`.
## Upgrading from 0.3 to 0.4
- Since 0.4.0, Celeborn won't be compatible with Celeborn client that versions below 0.3.0.

View File

@ -141,8 +141,8 @@ These metrics are exposed by Celeborn worker.
- The time for a worker to process openStream RPC and return StreamHandle.
- FetchChunkTime
- The time for a worker to fetch a chunk which is 8MB by default from a reduced partition.
- ChunkStreamCount
- Stream count for reduce partition reading streams.
- ActiveChunkStreamCount
- Active stream count for reduce partition reading streams.
- OpenStreamFailCount
- FetchChunkFailCount
- PrimaryPushDataTime
@ -199,8 +199,8 @@ These metrics are exposed by Celeborn worker.
- The queue size of read buffer allocation requests.
- ReadBufferAllocatedCount
- Allocated read buffer count.
- CreditStreamCount
- Stream count for map partition reading streams.
- ActiveCreditStreamCount
- Active stream count for map partition reading streams.
- ActiveMapPartitionCount
- DeviceOSFreeBytes
- DeviceOSTotalBytes

View File

@ -60,11 +60,11 @@ class FetchHandler(
var registered: AtomicBoolean = new AtomicBoolean(false)
def init(worker: Worker): Unit = {
workerSource.addGauge(WorkerSource.CHUNK_STREAM_COUNT) { () =>
workerSource.addGauge(WorkerSource.ACTIVE_CHUNK_STREAM_COUNT) { () =>
chunkStreamManager.getStreamsCount
}
workerSource.addGauge(WorkerSource.CREDIT_STREAM_COUNT) { () =>
workerSource.addGauge(WorkerSource.ACTIVE_CREDIT_STREAM_COUNT) { () =>
creditStreamManager.getStreamsCount
}

View File

@ -78,7 +78,7 @@ object WorkerSource {
// fetch data
val OPEN_STREAM_TIME = "OpenStreamTime"
val FETCH_CHUNK_TIME = "FetchChunkTime"
val CHUNK_STREAM_COUNT = "ChunkStreamCount"
val ACTIVE_CHUNK_STREAM_COUNT = "ActiveChunkStreamCount"
val OPEN_STREAM_FAIL_COUNT = "OpenStreamFailCount"
val FETCH_CHUNK_FAIL_COUNT = "FetchChunkFailCount"
@ -133,7 +133,7 @@ object WorkerSource {
val READ_BUFFER_ALLOCATED_COUNT = "ReadBufferAllocatedCount"
// credit
val CREDIT_STREAM_COUNT = "CreditStreamCount"
val ACTIVE_CREDIT_STREAM_COUNT = "ActiveCreditStreamCount"
val ACTIVE_MAP_PARTITION_COUNT = "ActiveMapPartitionCount"
// local device