forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-32091][checkpoint] Add file size metrics for file-merging (apa…
- Loading branch information
Showing
12 changed files
with
264 additions
and
13 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
54 changes: 54 additions & 0 deletions
54
...src/main/java/org/apache/flink/runtime/checkpoint/filemerging/FileMergingMetricGroup.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,54 @@ | ||
/* | ||
* 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.flink.runtime.checkpoint.filemerging; | ||
|
||
import org.apache.flink.annotation.VisibleForTesting; | ||
import org.apache.flink.metrics.MetricGroup; | ||
import org.apache.flink.runtime.metrics.groups.ProxyMetricGroup; | ||
|
||
import net.jcip.annotations.ThreadSafe; | ||
|
||
/** | ||
* Metrics related to the file merging snapshot manager. Thread-safety is required because it is | ||
* used by multiple task threads. | ||
*/ | ||
@ThreadSafe | ||
public class FileMergingMetricGroup extends ProxyMetricGroup<MetricGroup> { | ||
|
||
private static final String PREFIX = "fileMerging"; | ||
@VisibleForTesting public static final String LOGICAL_FILE_COUNT = PREFIX + ".logicalFileCount"; | ||
@VisibleForTesting public static final String LOGICAL_FILE_SIZE = PREFIX + ".logicalFileSize"; | ||
|
||
@VisibleForTesting | ||
public static final String PHYSICAL_FILE_COUNT = PREFIX + ".physicalFileCount"; | ||
|
||
@VisibleForTesting public static final String PHYSICAL_FILE_SIZE = PREFIX + ".physicalFileSize"; | ||
|
||
public FileMergingMetricGroup( | ||
MetricGroup parentMetricGroup, FileMergingSnapshotManager.SpaceStat spaceStat) { | ||
super(parentMetricGroup); | ||
registerMetrics(spaceStat); | ||
} | ||
|
||
public void registerMetrics(FileMergingSnapshotManager.SpaceStat spaceStat) { | ||
gauge(LOGICAL_FILE_COUNT, spaceStat.logicalFileCount::get); | ||
gauge(LOGICAL_FILE_SIZE, spaceStat.logicalFileSize::get); | ||
gauge(PHYSICAL_FILE_COUNT, spaceStat.physicalFileCount::get); | ||
gauge(PHYSICAL_FILE_SIZE, spaceStat.physicalFileSize::get); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
103 changes: 103 additions & 0 deletions
103
...src/test/java/org/apache/flink/runtime/checkpoint/filemerging/FileMergingMetricsTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,103 @@ | ||
/* | ||
* 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.flink.runtime.checkpoint.filemerging; | ||
|
||
import org.apache.flink.metrics.Gauge; | ||
import org.apache.flink.metrics.MetricGroup; | ||
import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; | ||
|
||
import org.junit.jupiter.api.Test; | ||
|
||
import java.util.ArrayList; | ||
import java.util.Arrays; | ||
import java.util.Collection; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
|
||
import static org.assertj.core.api.Assertions.assertThat; | ||
|
||
/** Test for {@link FileMergingMetricGroup}. */ | ||
class FileMergingMetricsTest { | ||
@Test | ||
void testMetricsRegistration() { | ||
final Collection<String> registeredGaugeNames = new ArrayList<>(); | ||
|
||
MetricGroup metricGroup = | ||
new UnregisteredMetricsGroup() { | ||
@Override | ||
public <T, G extends Gauge<T>> G gauge(String name, G gauge) { | ||
if (gauge != null) { | ||
registeredGaugeNames.add(name); | ||
} | ||
return gauge; | ||
} | ||
}; | ||
FileMergingSnapshotManager.SpaceStat spaceStat = new FileMergingSnapshotManager.SpaceStat(); | ||
FileMergingMetricGroup fileMergingMetricGroup = | ||
new FileMergingMetricGroup(metricGroup, spaceStat); | ||
|
||
assertThat(registeredGaugeNames) | ||
.containsAll( | ||
Arrays.asList( | ||
FileMergingMetricGroup.LOGICAL_FILE_COUNT, | ||
FileMergingMetricGroup.LOGICAL_FILE_SIZE, | ||
FileMergingMetricGroup.PHYSICAL_FILE_COUNT, | ||
FileMergingMetricGroup.PHYSICAL_FILE_SIZE)); | ||
assertThat(registeredGaugeNames.size()).isEqualTo(4); | ||
} | ||
|
||
@Test | ||
@SuppressWarnings("unchecked") | ||
void testMetricsAreUpdated() { | ||
final Map<String, Gauge<?>> registeredGauges = new HashMap<>(); | ||
|
||
MetricGroup metricGroup = | ||
new UnregisteredMetricsGroup() { | ||
@Override | ||
public <T, G extends Gauge<T>> G gauge(String name, G gauge) { | ||
registeredGauges.put(name, gauge); | ||
return gauge; | ||
} | ||
}; | ||
FileMergingSnapshotManager.SpaceStat spaceStat = new FileMergingSnapshotManager.SpaceStat(); | ||
FileMergingMetricGroup fileMergingMetricGroup = | ||
new FileMergingMetricGroup(metricGroup, spaceStat); | ||
Gauge<Long> logicalFileCountGauge = | ||
(Gauge<Long>) registeredGauges.get(FileMergingMetricGroup.LOGICAL_FILE_COUNT); | ||
Gauge<Long> logicalFileSizeGauge = | ||
(Gauge<Long>) registeredGauges.get(FileMergingMetricGroup.LOGICAL_FILE_SIZE); | ||
Gauge<Long> physicalFileCountGauge = | ||
(Gauge<Long>) registeredGauges.get(FileMergingMetricGroup.PHYSICAL_FILE_COUNT); | ||
Gauge<Long> physicalFileSizeGauge = | ||
(Gauge<Long>) registeredGauges.get(FileMergingMetricGroup.PHYSICAL_FILE_SIZE); | ||
|
||
assertThat(logicalFileCountGauge.getValue()).isEqualTo(0L); | ||
assertThat(logicalFileSizeGauge.getValue()).isEqualTo(0L); | ||
assertThat(physicalFileCountGauge.getValue()).isEqualTo(0L); | ||
assertThat(physicalFileSizeGauge.getValue()).isEqualTo(0L); | ||
|
||
// update space stat | ||
spaceStat.onLogicalFileCreate(100L); | ||
spaceStat.onPhysicalFileCreate(); | ||
spaceStat.onPhysicalFileUpdate(100L); | ||
assertThat(logicalFileCountGauge.getValue()).isEqualTo(1L); | ||
assertThat(logicalFileSizeGauge.getValue()).isEqualTo(100L); | ||
assertThat(physicalFileCountGauge.getValue()).isEqualTo(1L); | ||
assertThat(physicalFileSizeGauge.getValue()).isEqualTo(100L); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.