提交 edb79b0a 编写于 作者: Z zentol

[FLINK-6898] [metrics] Limit size of operator component in metric name

This closes #4109.
上级 08580765
......@@ -63,7 +63,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
@Internal
public abstract class AbstractMetricGroup<A extends AbstractMetricGroup<?>> implements MetricGroup {
private static final Logger LOG = LoggerFactory.getLogger(MetricGroup.class);
protected static final Logger LOG = LoggerFactory.getLogger(MetricGroup.class);
// ------------------------------------------------------------------------
......
......@@ -42,6 +42,8 @@ public class TaskMetricGroup extends ComponentMetricGroup<TaskManagerJobMetricGr
private final Map<String, OperatorMetricGroup> operators = new HashMap<>();
static final int METRICS_OPERATOR_NAME_MAX_LENGTH = 80;
private final TaskIOMetricGroup ioMetrics;
/** The execution Id uniquely identifying the executed task represented by this metrics group. */
......@@ -131,6 +133,10 @@ public class TaskMetricGroup extends ComponentMetricGroup<TaskManagerJobMetricGr
// ------------------------------------------------------------------------
public OperatorMetricGroup addOperator(String name) {
if (name != null && name.length() > METRICS_OPERATOR_NAME_MAX_LENGTH) {
LOG.warn("The operator name {} exceeded the {} characters length limit and was truncated.", name, METRICS_OPERATOR_NAME_MAX_LENGTH);
name = name.substring(0, METRICS_OPERATOR_NAME_MAX_LENGTH);
}
OperatorMetricGroup operator = new OperatorMetricGroup(this.registry, this, name);
synchronized (this) {
......
......@@ -25,10 +25,12 @@ import org.apache.flink.metrics.Metric;
import org.apache.flink.runtime.metrics.MetricRegistry;
import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
import org.apache.flink.runtime.metrics.dump.QueryScopeInfo;
import org.apache.flink.runtime.metrics.scope.ScopeFormat;
import org.apache.flink.runtime.metrics.util.DummyCharacterFilter;
import org.apache.flink.util.AbstractID;
import org.apache.flink.util.TestLogger;
import org.junit.Assert;
import org.junit.Test;
import static org.junit.Assert.assertArrayEquals;
......@@ -150,6 +152,23 @@ public class TaskMetricGroupTest extends TestLogger {
registry.shutdown();
}
@Test
public void testOperatorNameTruncation() {
Configuration cfg = new Configuration();
cfg.setString(MetricOptions.SCOPE_NAMING_OPERATOR, ScopeFormat.SCOPE_OPERATOR_NAME);
MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(cfg));
TaskManagerMetricGroup tm = new TaskManagerMetricGroup(registry, "host", "id");
TaskManagerJobMetricGroup job = new TaskManagerJobMetricGroup(registry, tm, new JobID(), "jobname");
TaskMetricGroup taskMetricGroup = new TaskMetricGroup(registry, job, new AbstractID(), new AbstractID(), "task", 0, 0);
String originalName = new String(new char[100]).replace("\0", "-");
OperatorMetricGroup operatorMetricGroup = taskMetricGroup.addOperator(originalName);
String storedName = operatorMetricGroup.getScopeComponents()[0];
Assert.assertEquals(TaskMetricGroup.METRICS_OPERATOR_NAME_MAX_LENGTH, storedName.length());
Assert.assertEquals(originalName.substring(0, TaskMetricGroup.METRICS_OPERATOR_NAME_MAX_LENGTH), storedName);
}
private static class CountingMetricRegistry extends MetricRegistry {
private int counter = 0;
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册