From 003ce18efc0249fae874e56c3df6acf19f5f2429 Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 9 May 2016 14:54:32 +0200 Subject: [PATCH] [FLINK-1502] [core] Add basic metric system --- .../flink-statebackend-rocksdb/pom.xml | 7 + flink-contrib/flink-storm/pom.xml | 8 + .../flink/storm/wrappers/BoltWrapperTest.java | 7 + flink-core/pom.xml | 6 + .../api/common/functions/RuntimeContext.java | 8 + .../util/AbstractRuntimeUDFContext.java | 12 +- .../functions/util/RuntimeUDFContext.java | 6 +- .../common/operators/CollectionExecutor.java | 48 +++- .../org/apache/flink/metrics/Counter.java | 69 ++++++ .../java/org/apache/flink/metrics/Gauge.java | 33 +++ .../java/org/apache/flink/metrics/Metric.java | 27 +++ .../org/apache/flink/metrics/MetricGroup.java | 99 ++++++++ .../apache/flink/metrics/MetricRegistry.java | 170 ++++++++++++++ .../metrics/groups/AbstractMetricGroup.java | 145 ++++++++++++ .../metrics/groups/ComponentMetricGroup.java | 113 +++++++++ .../metrics/groups/GenericMetricGroup.java | 57 +++++ .../flink/metrics/groups/IOMetricGroup.java | 70 ++++++ .../flink/metrics/groups/JobMetricGroup.java | 71 ++++++ .../metrics/groups/OperatorMetricGroup.java | 46 ++++ .../apache/flink/metrics/groups/Scope.java | 120 ++++++++++ .../groups/TaskManagerMetricGroup.java | 70 ++++++ .../flink/metrics/groups/TaskMetricGroup.java | 87 +++++++ .../metrics/reporter/AbstractReporter.java | 48 ++++ .../flink/metrics/reporter/JMXReporter.java | 167 ++++++++++++++ .../metrics/reporter/MetricReporter.java | 73 ++++++ .../flink/metrics/reporter/Scheduled.java | 32 +++ .../functions/util/RuntimeUDFContextTest.java | 11 +- .../api/common/io/RichInputFormatTest.java | 3 +- .../api/common/io/RichOutputFormatTest.java | 3 +- .../operators/GenericDataSinkBaseTest.java | 5 +- .../operators/GenericDataSourceBaseTest.java | 5 +- .../base/FlatMapOperatorCollectionTest.java | 3 +- .../base/InnerJoinOperatorBaseTest.java | 5 +- .../operators/base/MapOperatorTest.java | 5 +- .../base/PartitionMapOperatorTest.java | 5 +- .../apache/flink/metrics/MetricGroupTest.java | 93 ++++++++ .../flink/metrics/MetricRegistryTest.java | 217 ++++++++++++++++++ .../flink/metrics/groups/JobGroupTest.java | 71 ++++++ .../metrics/groups/OperatorGroupTest.java | 86 +++++++ .../flink/metrics/groups/TaskGroupTest.java | 80 +++++++ .../metrics/groups/TaskManagerGroupTest.java | 70 ++++++ .../metrics/reporter/JMXReporterTest.java | 43 ++++ .../metrics/util/DummyJobMetricGroup.java | 50 ++++ .../flink/metrics/util/DummyMetricGroup.java | 57 +++++ .../metrics/util/DummyMetricRegistry.java | 34 +++ .../util/DummyOperatorMetricGroup.java | 43 ++++ .../flink/metrics/util/DummyReporter.java | 47 ++++ .../util/DummyTaskManagerMetricGroup.java | 48 ++++ .../metrics/util/DummyTaskMetricGroup.java | 48 ++++ .../flink/metrics/util/TestReporter.java | 40 ++++ flink-dist/src/main/flink-bin/bin/config.sh | 6 + .../src/main/flink-bin/bin/flink-daemon.sh | 18 +- .../base/CoGroupOperatorCollectionTest.java | 3 +- .../base/GroupReduceOperatorTest.java | 5 +- .../base/InnerJoinOperatorBaseTest.java | 5 +- .../operators/base/ReduceOperatorTest.java | 5 +- .../flink-metrics-dropwizard/pom.xml | 72 ++++++ .../ScheduledDropwizardReporter.java | 99 ++++++++ .../dropwizard/metrics/CounterWrapper.java | 33 +++ .../dropwizard/metrics/GaugeWrapper.java | 33 +++ flink-metrics/flink-metrics-ganglia/pom.xml | 90 ++++++++ .../metrics/graphite/GangliaReporter.java | 73 ++++++ flink-metrics/flink-metrics-graphite/pom.xml | 84 +++++++ .../metrics/graphite/GraphiteReporter.java | 70 ++++++ flink-metrics/flink-metrics-statsd/pom.xml | 43 ++++ .../flink/metrics/statsd/StatsDReporter.java | 132 +++++++++++ flink-metrics/pom.xml | 42 ++++ flink-runtime/pom.xml | 4 - .../deployment/TaskDeploymentDescriptor.java | 7 + .../flink/runtime/execution/Environment.java | 8 + .../executiongraph/ExecutionVertex.java | 1 + .../api/reader/AbstractRecordReader.java | 8 + .../io/network/api/reader/BufferReader.java | 5 + .../io/network/api/reader/ReaderBase.java | 8 + .../AdaptiveSpanningRecordDeserializer.java | 20 ++ .../api/serialization/RecordDeserializer.java | 8 + .../api/serialization/RecordSerializer.java | 8 + .../SpanningRecordSerializer.java | 19 ++ ...ingAdaptiveSpanningRecordDeserializer.java | 21 ++ .../io/network/api/writer/RecordWriter.java | 11 + .../iterative/task/AbstractIterativeTask.java | 9 +- .../flink/runtime/operators/BatchTask.java | 19 +- .../flink/runtime/operators/DataSinkTask.java | 5 +- .../runtime/operators/DataSourceTask.java | 5 +- .../runtime/operators/GroupReduceDriver.java | 2 +- .../flink/runtime/operators/TaskContext.java | 3 + .../operators/chaining/ChainedDriver.java | 8 +- .../util/DistributedRuntimeUDFContext.java | 5 +- .../taskmanager/RuntimeEnvironment.java | 11 +- .../flink/runtime/taskmanager/Task.java | 12 +- .../runtime/taskmanager/TaskManager.scala | 169 +++++++++++++- .../TaskDeploymentDescriptorTest.java | 3 +- .../api/reader/AbstractReaderTest.java | 5 + .../operators/drivers/TestTaskContext.java | 7 + .../testutils/BinaryOperatorTestBase.java | 7 + .../operators/testutils/DriverTestBase.java | 7 + .../operators/testutils/DummyEnvironment.java | 7 + .../operators/testutils/MockEnvironment.java | 7 + .../testutils/UnaryOperatorTestBase.java | 6 + .../taskmanager/TaskAsyncCallTest.java | 6 +- .../runtime/taskmanager/TaskManagerTest.java | 27 +-- .../runtime/taskmanager/TaskStopTest.java | 3 +- .../flink/runtime/taskmanager/TaskTest.java | 6 +- .../streaming/api/graph/StreamConfig.java | 9 + .../api/graph/StreamingJobGraphGenerator.java | 11 +- .../api/operators/AbstractStreamOperator.java | 9 + .../api/operators/StreamOperator.java | 3 + .../operators/StreamingRuntimeContext.java | 3 +- .../runtime/io/StreamInputProcessor.java | 12 + .../runtime/io/StreamTwoInputProcessor.java | 12 + .../runtime/tasks/OneInputStreamTask.java | 1 + .../runtime/tasks/OperatorChain.java | 1 + .../runtime/tasks/TwoInputStreamTask.java | 1 + ...ignedProcessingTimeWindowOperatorTest.java | 2 + ...ignedProcessingTimeWindowOperatorTest.java | 2 + .../runtime/tasks/StreamMockEnvironment.java | 7 + .../runtime/tasks/StreamTaskTest.java | 6 +- pom.xml | 2 + 118 files changed, 3936 insertions(+), 96 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/Counter.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/Gauge.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/Metric.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/groups/AbstractMetricGroup.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/groups/ComponentMetricGroup.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/groups/GenericMetricGroup.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/groups/IOMetricGroup.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/groups/JobMetricGroup.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/groups/OperatorMetricGroup.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/groups/Scope.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/groups/TaskManagerMetricGroup.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/reporter/JMXReporter.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporter.java create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/reporter/Scheduled.java create mode 100644 flink-core/src/test/java/org/apache/flink/metrics/MetricGroupTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/metrics/MetricRegistryTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/metrics/groups/JobGroupTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/metrics/groups/OperatorGroupTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/metrics/groups/TaskGroupTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/metrics/groups/TaskManagerGroupTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/metrics/reporter/JMXReporterTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/metrics/util/DummyJobMetricGroup.java create mode 100644 flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricGroup.java create mode 100644 flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricRegistry.java create mode 100644 flink-core/src/test/java/org/apache/flink/metrics/util/DummyOperatorMetricGroup.java create mode 100644 flink-core/src/test/java/org/apache/flink/metrics/util/DummyReporter.java create mode 100644 flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskManagerMetricGroup.java create mode 100644 flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskMetricGroup.java create mode 100644 flink-core/src/test/java/org/apache/flink/metrics/util/TestReporter.java create mode 100644 flink-metrics/flink-metrics-dropwizard/pom.xml create mode 100644 flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java create mode 100644 flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/CounterWrapper.java create mode 100644 flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/GaugeWrapper.java create mode 100644 flink-metrics/flink-metrics-ganglia/pom.xml create mode 100644 flink-metrics/flink-metrics-ganglia/src/main/java/org/apache/flink/metrics/graphite/GangliaReporter.java create mode 100644 flink-metrics/flink-metrics-graphite/pom.xml create mode 100644 flink-metrics/flink-metrics-graphite/src/main/java/org/apache/flink/metrics/graphite/GraphiteReporter.java create mode 100644 flink-metrics/flink-metrics-statsd/pom.xml create mode 100644 flink-metrics/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java create mode 100644 flink-metrics/pom.xml diff --git a/flink-contrib/flink-statebackend-rocksdb/pom.xml b/flink-contrib/flink-statebackend-rocksdb/pom.xml index b96637152e384..cccdc207d97a7 100644 --- a/flink-contrib/flink-statebackend-rocksdb/pom.xml +++ b/flink-contrib/flink-statebackend-rocksdb/pom.xml @@ -52,6 +52,13 @@ under the License. rocksdbjni 4.1.0 + + org.apache.flink + flink-core + ${project.version} + test-jar + test + org.apache.flink flink-runtime_2.10 diff --git a/flink-contrib/flink-storm/pom.xml b/flink-contrib/flink-storm/pom.xml index 24b0b4bcead1a..a080a030e9c04 100644 --- a/flink-contrib/flink-storm/pom.xml +++ b/flink-contrib/flink-storm/pom.xml @@ -48,6 +48,14 @@ under the License. ${project.version} + + org.apache.flink + flink-core + ${project.version} + test-jar + test + + org.apache.storm storm-core diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java index e65dc45d2d8cf..cb9ac1cefdcea 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java @@ -32,7 +32,10 @@ import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.UnmodifiableConfiguration; +import org.apache.flink.metrics.util.DummyMetricGroup; +import org.apache.flink.metrics.util.DummyTaskMetricGroup; import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.taskmanager.RuntimeEnvironment; import org.apache.flink.storm.util.AbstractTest; import org.apache.flink.storm.util.SplitStreamType; import org.apache.flink.storm.util.StormConfig; @@ -141,6 +144,7 @@ private void testWrapper(final int numberOfAttributes) throws Exception { final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); when(taskContext.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class)); when(taskContext.getTaskName()).thenReturn("name"); + when(taskContext.getMetricGroup()).thenReturn(new DummyMetricGroup()); final IRichBolt bolt = mock(IRichBolt.class); @@ -225,6 +229,7 @@ public void testOpen() throws Exception { final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); when(taskContext.getExecutionConfig()).thenReturn(taskConfig); when(taskContext.getTaskName()).thenReturn("name"); + when(taskContext.getMetricGroup()).thenReturn(new DummyMetricGroup()); final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); declarer.declare(new Fields("dummy")); @@ -289,6 +294,7 @@ public void testOpenSink() throws Exception { final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); when(taskContext.getExecutionConfig()).thenReturn(taskConfig); when(taskContext.getTaskName()).thenReturn("name"); + when(taskContext.getMetricGroup()).thenReturn(new DummyMetricGroup()); final IRichBolt bolt = mock(IRichBolt.class); BoltWrapper wrapper = new BoltWrapper(bolt); @@ -361,6 +367,7 @@ public Map getComponentConfiguration() { Environment env = mock(Environment.class); when(env.getTaskInfo()).thenReturn(new TaskInfo("Mock Task", 0, 1, 0)); when(env.getUserClassLoader()).thenReturn(BoltWrapperTest.class.getClassLoader()); + when(env.getMetricGroup()).thenReturn(new DummyTaskMetricGroup()); StreamTask mockTask = mock(StreamTask.class); when(mockTask.getCheckpointLock()).thenReturn(new Object()); diff --git a/flink-core/pom.xml b/flink-core/pom.xml index eb55bdd136c35..0e0b54bba1a71 100644 --- a/flink-core/pom.xml +++ b/flink-core/pom.xml @@ -47,6 +47,12 @@ under the License. + + io.dropwizard.metrics + metrics-core + ${metrics.version} + + org.apache.avro diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java index 3225c003550d4..ed2f613987fd4 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java @@ -39,6 +39,7 @@ import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.metrics.MetricGroup; /** * A RuntimeContext contains information about the context in which functions are executed. Each parallel instance @@ -58,6 +59,13 @@ public interface RuntimeContext { */ String getTaskName(); + /** + * Returns the metric group for this parallel subtask. + * + * @return The metric group for this parallel subtask. + */ + MetricGroup getMetricGroup(); + /** * Gets the parallelism with which the parallel task runs. * diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java index 74b78df624ee9..6645964433edc 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java @@ -43,6 +43,7 @@ import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.MetricGroup; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -61,17 +62,21 @@ public abstract class AbstractRuntimeUDFContext implements RuntimeContext { private final Map> accumulators; private final DistributedCache distributedCache; + + private final MetricGroup metrics; public AbstractRuntimeUDFContext(TaskInfo taskInfo, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig, Map> accumulators, - Map> cpTasks) { + Map> cpTasks, + MetricGroup metrics) { this.taskInfo = checkNotNull(taskInfo); this.userCodeClassLoader = userCodeClassLoader; this.executionConfig = executionConfig; this.distributedCache = new DistributedCache(checkNotNull(cpTasks)); this.accumulators = checkNotNull(accumulators); + this.metrics = metrics; } @Override @@ -93,6 +98,11 @@ public int getNumberOfParallelSubtasks() { public int getIndexOfThisSubtask() { return taskInfo.getIndexOfThisSubtask(); } + + @Override + public MetricGroup getMetricGroup() { + return metrics; + } @Override public int getAttemptNumber() { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java index 2337afaa5f5be..6571d0d44686e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java @@ -30,6 +30,7 @@ import org.apache.flink.api.common.functions.BroadcastVariableInitializer; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.MetricGroup; /** * A standalone implementation of the {@link RuntimeContext}, created by runtime UDF operators. @@ -42,8 +43,9 @@ public class RuntimeUDFContext extends AbstractRuntimeUDFContext { private final HashMap> uninitializedBroadcastVars = new HashMap>(); public RuntimeUDFContext(TaskInfo taskInfo, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig, - Map> cpTasks, Map> accumulators) { - super(taskInfo, userCodeClassLoader, executionConfig, accumulators, cpTasks); + Map> cpTasks, Map> accumulators, + MetricGroup metrics) { + super(taskInfo, userCodeClassLoader, executionConfig, accumulators, cpTasks, metrics); } @Override diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java index 44042c432db2d..7e5269e84fe87 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java @@ -36,6 +36,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; @@ -58,9 +59,15 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.local.LocalFileSystem; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.MetricRegistry; +import org.apache.flink.metrics.groups.JobMetricGroup; +import org.apache.flink.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.types.Value; +import org.apache.flink.util.AbstractID; import org.apache.flink.util.Visitor; /** @@ -86,6 +93,8 @@ public class CollectionExecutor { private final ExecutionConfig executionConfig; private int iterationSuperstep; + + private JobMetricGroup jobMetricGroup; // -------------------------------------------------------------------------------------------- @@ -106,6 +115,14 @@ public CollectionExecutor(ExecutionConfig executionConfig) { public JobExecutionResult execute(Plan program) throws Exception { long startTime = System.currentTimeMillis(); + + JobID jobID = program.getJobId(); + if (jobID == null) { + jobID = new JobID(); + } + this.jobMetricGroup = + new TaskManagerMetricGroup(new MetricRegistry(new Configuration()), "localhost", new AbstractID().toString()) + .addJob(jobID, program.getJobName()); initCache(program.getCachedFiles()); Collection> sinks = program.getDataSinks(); for (Operator sink : sinks) { @@ -184,9 +201,12 @@ private void executeDataSink(GenericDataSinkBase sink, int superStep) th // build the runtime context and compute broadcast variables, if necessary TaskInfo taskInfo = new TaskInfo(typedSink.getName(), 0, 1, 0); RuntimeUDFContext ctx; + + MetricGroup metrics = this.jobMetricGroup.addTask(new AbstractID(), new AbstractID(), 0, typedSink.getName()); + if (RichOutputFormat.class.isAssignableFrom(typedSink.getUserCodeWrapper().getUserCodeClass())) { - ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators) : - new IterationRuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators); + ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics) : + new IterationRuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics); } else { ctx = null; } @@ -200,10 +220,13 @@ private List executeDataSource(GenericDataSourceBase source, in GenericDataSourceBase typedSource = (GenericDataSourceBase) source; // build the runtime context and compute broadcast variables, if necessary TaskInfo taskInfo = new TaskInfo(typedSource.getName(), 0, 1, 0); + RuntimeUDFContext ctx; + + MetricGroup metrics = this.jobMetricGroup.addTask(new AbstractID(), new AbstractID(), 0, source.getName()); if (RichInputFormat.class.isAssignableFrom(typedSource.getUserCodeWrapper().getUserCodeClass())) { - ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators) : - new IterationRuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators); + ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics) : + new IterationRuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics); } else { ctx = null; } @@ -225,9 +248,11 @@ private List executeUnaryOperator(SingleInputOperator op // build the runtime context and compute broadcast variables, if necessary TaskInfo taskInfo = new TaskInfo(typedOp.getName(), 0, 1, 0); RuntimeUDFContext ctx; + + MetricGroup metrics = this.jobMetricGroup.addTask(new AbstractID(), new AbstractID(), 0, typedOp.getName()); if (RichFunction.class.isAssignableFrom(typedOp.getUserCodeWrapper().getUserCodeClass())) { - ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators) : - new IterationRuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators); + ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics) : + new IterationRuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics); for (Map.Entry> bcInputs : operator.getBroadcastInputs().entrySet()) { List bcData = execute(bcInputs.getValue()); @@ -265,9 +290,11 @@ private List executeBinaryOperator(DualInputOperator> bcInputs : operator.getBroadcastInputs().entrySet()) { List bcData = execute(bcInputs.getValue()); @@ -523,8 +550,9 @@ else if (op instanceof GenericDataSourceBase) { private class IterationRuntimeUDFContext extends RuntimeUDFContext implements IterationRuntimeContext { public IterationRuntimeUDFContext(TaskInfo taskInfo, ClassLoader classloader, ExecutionConfig executionConfig, - Map> cpTasks, Map> accumulators) { - super(taskInfo, classloader, executionConfig, cpTasks, accumulators); + Map> cpTasks, Map> accumulators, + MetricGroup metrics) { + super(taskInfo, classloader, executionConfig, cpTasks, accumulators, metrics); } @Override diff --git a/flink-core/src/main/java/org/apache/flink/metrics/Counter.java b/flink-core/src/main/java/org/apache/flink/metrics/Counter.java new file mode 100644 index 0000000000000..b18da4fe2672e --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/Counter.java @@ -0,0 +1,69 @@ +/* + * 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.metrics; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * A Counter is a {@link org.apache.flink.metrics.Metric} that measures a count. + */ +@PublicEvolving +public final class Counter implements Metric { + private long count = 0; + + /** + * Increment the current count by 1. + */ + public void inc() { + count++; + } + + /** + * Increment the current count by the given value. + * + * @param n value to increment the current count by + */ + public void inc(long n) { + count += n; + } + + /** + * Decrement the current count by 1. + */ + public void dec() { + count--; + } + + /** + * Decrement the current count by the given value. + * + * @param n value to decrement the current count by + */ + public void dec(long n) { + count -= n; + } + + /** + * Returns the current count. + * + * @return current count + */ + public long getCount() { + return count; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/Gauge.java b/flink-core/src/main/java/org/apache/flink/metrics/Gauge.java new file mode 100644 index 0000000000000..455587ad743eb --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/Gauge.java @@ -0,0 +1,33 @@ +/* + * 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.metrics; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * A Gauge is a {@link org.apache.flink.metrics.Metric} that calculates a specific value at a point in time. + */ +@PublicEvolving +public abstract class Gauge implements Metric { + /** + * Calculates and returns the measured value. + * + * @return calculated value + */ + public abstract T getValue(); +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/Metric.java b/flink-core/src/main/java/org/apache/flink/metrics/Metric.java new file mode 100644 index 0000000000000..11cfcc6df3831 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/Metric.java @@ -0,0 +1,27 @@ +/* + * 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.metrics; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * Common interface for all metrics. + */ +@PublicEvolving +public interface Metric { +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java new file mode 100644 index 0000000000000..403612958e7d5 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java @@ -0,0 +1,99 @@ +/* + * 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.metrics; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * A MetricGroup is a named container for {@link org.apache.flink.metrics.Metric}s and + * {@link org.apache.flink.metrics.MetricGroup}s. + *

+ * Instances of this class can be used to register new metrics with Flink and to create a nested hierarchy based on the + * group names. + *

+ * A MetricGroup is uniquely identified by it's place in the hierarchy and name. + */ +@PublicEvolving +public interface MetricGroup { + + /** + * Recursively unregisters all {@link org.apache.flink.metrics.Metric}s contained in this + * {@link org.apache.flink.metrics.MetricGroup} + */ + void close(); + + // ----------------------------------------------------------------------------------------------------------------- + // Metrics + // ----------------------------------------------------------------------------------------------------------------- + + /** + * Creates and registers a new {@link org.apache.flink.metrics.Counter} with Flink. + * + * @param name name of the counter + * @return the registered counter + */ + Counter counter(int name); + + /** + * Creates and registers a new {@link org.apache.flink.metrics.Counter} with Flink. + * + * @param name name of the counter + * @return the registered counter + */ + Counter counter(String name); + + /** + * Registers a new {@link org.apache.flink.metrics.Gauge} with Flink. + * + * @param name name of the gauge + * @param gauge gauge to register + * @param return type of the gauge + * @return the registered gauge + */ + Gauge gauge(int name, Gauge gauge); + + /** + * Registers a new {@link org.apache.flink.metrics.Gauge} with Flink. + * + * @param name name of the gauge + * @param gauge gauge to register + * @param return type of the gauge + * @return the registered gauge + */ + Gauge gauge(String name, Gauge gauge); + + // ----------------------------------------------------------------------------------------------------------------- + // Groups + // ----------------------------------------------------------------------------------------------------------------- + + /** + * Creates a new {@link org.apache.flink.metrics.MetricGroup} and adds it to this groups sub-groups. + * + * @param name name of the group + * @return the created group + */ + MetricGroup addGroup(int name); + + /** + * Creates a new {@link org.apache.flink.metrics.MetricGroup} and adds it to this groups sub-groups. + * + * @param name name of the group + * @return the created group + */ + MetricGroup addGroup(String name); +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java b/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java new file mode 100644 index 0000000000000..0e8b0d5bfecd6 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java @@ -0,0 +1,170 @@ +/* + * 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.metrics; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.groups.AbstractMetricGroup; +import org.apache.flink.metrics.groups.Scope; +import org.apache.flink.metrics.reporter.JMXReporter; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.metrics.reporter.Scheduled; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.TimerTask; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.metrics.groups.JobMetricGroup.DEFAULT_SCOPE_JOB; +import static org.apache.flink.metrics.groups.OperatorMetricGroup.DEFAULT_SCOPE_OPERATOR; +import static org.apache.flink.metrics.groups.TaskManagerMetricGroup.DEFAULT_SCOPE_TM; +import static org.apache.flink.metrics.groups.TaskMetricGroup.DEFAULT_SCOPE_TASK; + +/** + * A MetricRegistry keeps track of all registered {@link org.apache.flink.metrics.Metric}s. It serves as the + * connection between {@link org.apache.flink.metrics.MetricGroup}s and {@link org.apache.flink.metrics.reporter.MetricReporter}s. + */ +@Internal +public class MetricRegistry { + private static final Logger LOG = LoggerFactory.getLogger(MetricRegistry.class); + + private final MetricReporter reporter; + private java.util.Timer timer; + + public static final String KEY_METRICS_REPORTER_CLASS = "metrics.reporter.class"; + public static final String KEY_METRICS_REPORTER_ARGUMENTS = "metrics.reporter.arguments"; + public static final String KEY_METRICS_REPORTER_INTERVAL = "metrics.reporter.interval"; + + public static final String KEY_METRICS_SCOPE_NAMING_TM = "metrics.scope.tm"; + public static final String KEY_METRICS_SCOPE_NAMING_JOB = "metrics.scope.job"; + public static final String KEY_METRICS_SCOPE_NAMING_TASK = "metrics.scope.task"; + public static final String KEY_METRICS_SCOPE_NAMING_OPERATOR = "metrics.scope.operator"; + + private final Scope.ScopeFormat scopeConfig; + + /** + * Creates a new {@link MetricRegistry} and starts the configured reporter. + */ + public MetricRegistry(Configuration config) { + try { + String className = config.getString(KEY_METRICS_REPORTER_CLASS, null); + if (className == null) { + LOG.info("No reporter class name defined in flink-conf.yaml, defaulting to " + JMXReporter.class.getName() + "."); + className = JMXReporter.class.getName(); + } + + this.scopeConfig = createScopeConfig(config); + + Configuration reporterConfig = createReporterConfig(config); + Class reporterClass = Class.forName(className); + reporter = (MetricReporter) reporterClass.newInstance(); + reporter.open(reporterConfig); + + if (reporter instanceof Scheduled) { + String[] interval = config.getString(KEY_METRICS_REPORTER_INTERVAL, "10 SECONDS").split(" "); + long millis = TimeUnit.valueOf(interval[1]).toMillis(Long.parseLong(interval[0])); + timer = new java.util.Timer(true); + timer.schedule(new TimerTask() { + @Override + public void run() { + ((Scheduled) reporter).report(); + } + }, millis, millis); + } + } catch (InstantiationException | ClassNotFoundException e) { + throw new RuntimeException("Error while instantiating reporter.", e); + } catch (IllegalAccessException e) { + throw new RuntimeException("Implementation error.", e); + } + } + + private static Configuration createReporterConfig(Configuration config) { + String[] interval = config.getString(KEY_METRICS_REPORTER_INTERVAL, "10 SECONDS").split(" "); + + String[] arguments = config.getString(KEY_METRICS_REPORTER_ARGUMENTS, "").split(" "); + + Configuration reporterConfig = new Configuration(); + reporterConfig.setString("period", interval[0]); + reporterConfig.setString("timeunit", interval[1]); + + if (arguments.length > 1) { + for (int x = 0; x < arguments.length; x += 2) { + reporterConfig.setString(arguments[x].replace("--", ""), arguments[x + 1]); + } + } + return reporterConfig; + } + + private static Scope.ScopeFormat createScopeConfig(Configuration config) { + String tmFormat = config.getString(KEY_METRICS_SCOPE_NAMING_TM, DEFAULT_SCOPE_TM); + String jobFormat = config.getString(KEY_METRICS_SCOPE_NAMING_JOB, DEFAULT_SCOPE_JOB); + String taskFormat = config.getString(KEY_METRICS_SCOPE_NAMING_TASK, DEFAULT_SCOPE_TASK); + String operatorFormat = config.getString(KEY_METRICS_SCOPE_NAMING_OPERATOR, DEFAULT_SCOPE_OPERATOR); + + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setTaskManagerFormat(tmFormat); + format.setJobFormat(jobFormat); + format.setTaskFormat(taskFormat); + format.setOperatorFormat(operatorFormat); + return format; + } + + public Scope.ScopeFormat getScopeConfig() { + return this.scopeConfig; + } + + /** + * Shuts down this registry and the associated {@link org.apache.flink.metrics.reporter.MetricReporter}. + */ + public void shutdown() { + if (timer != null) { + timer.cancel(); + } + if (reporter != null) { + reporter.close(); + } + } + + /** + * Registers a new {@link org.apache.flink.metrics.Metric} with this registry. + * + * @param metric metric to register + * @param name name of the metric + * @param parent group that contains the metric + */ + public void register(Metric metric, String name, AbstractMetricGroup parent) { + String metricName = reporter.generateName(name, parent.generateScope()); + + + this.reporter.notifyOfAddedMetric(metric, metricName); + } + + /** + * Un-registers the given {@link org.apache.flink.metrics.Metric} with this registry. + * + * @param metric metric to un-register + * @param name name of the metric + * @param parent group that contains the metric + */ + public void unregister(Metric metric, String name, AbstractMetricGroup parent) { + String metricName = reporter.generateName(name, parent.generateScope()); + + this.reporter.notifyOfRemovedMetric(metric, metricName); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/AbstractMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/AbstractMetricGroup.java new file mode 100644 index 0000000000000..373ac0945cb8a --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/AbstractMetricGroup.java @@ -0,0 +1,145 @@ +/* + * 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.metrics.groups; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.MetricRegistry; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Abstract {@link org.apache.flink.metrics.MetricGroup} that contains key functionality for adding metrics and groups. + */ +@Internal +public abstract class AbstractMetricGroup implements MetricGroup { + private static final Logger LOG = LoggerFactory.getLogger(MetricGroup.class); + protected final MetricRegistry registry; + + // all metrics that are directly contained in this group + protected final Map metrics = new HashMap<>(); + // all generic groups that are directly contained in this group + protected final Map groups = new HashMap<>(); + + public AbstractMetricGroup(MetricRegistry registry) { + this.registry = registry; + } + + @Override + public void close() { + for (MetricGroup group : groups.values()) { + group.close(); + } + this.groups.clear(); + for (Map.Entry metric : metrics.entrySet()) { + registry.unregister(metric.getValue(), metric.getKey(), this); + } + this.metrics.clear(); + } + + // ----------------------------------------------------------------------------------------------------------------- + // Scope + // ----------------------------------------------------------------------------------------------------------------- + + /** + * Generates the full scope based on the default/configured format that applies to all metrics within this group. + * + * @return generated scope + */ + public abstract List generateScope(); + + /** + * Generates the full scope based on the given format that applies to all metrics within this group. + * + * @param format format string + * @return generated scope + */ + public abstract List generateScope(Scope.ScopeFormat format); + + // ----------------------------------------------------------------------------------------------------------------- + // Metrics + // ----------------------------------------------------------------------------------------------------------------- + + @Override + public Counter counter(int name) { + return counter("" + name); + } + + @Override + public Counter counter(String name) { + Counter counter = new Counter(); + addMetric(name, counter); + return counter; + } + + @Override + public Gauge gauge(int name, Gauge gauge) { + return gauge("" + name, gauge); + } + + @Override + public Gauge gauge(String name, Gauge gauge) { + addMetric(name, gauge); + return gauge; + } + + protected MetricGroup addMetric(String name, Metric metric) { + if (!name.matches("[a-zA-Z0-9]*")) { + throw new IllegalArgumentException("Metric names may not contain special characters."); + } + if (metrics.containsKey(name)) { + LOG.warn("Detected metric name collision. This group already contains a group for the given group name. " + + this.generateScope().toString() + "." + name); + } + if (groups.containsKey(name)) { + LOG.warn("Detected metric name collision. This group already contains a group for the given metric name." + + this.generateScope().toString() + ")." + name); + } + metrics.put(name, metric); + registry.register(metric, name, this); + return this; + } + + // ----------------------------------------------------------------------------------------------------------------- + // Groups + // ----------------------------------------------------------------------------------------------------------------- + + @Override + public MetricGroup addGroup(int name) { + return addGroup("" + name); + } + + @Override + public MetricGroup addGroup(String name) { + if (metrics.containsKey(name)) { + LOG.warn("Detected metric name collision. This group already contains a metric for the given group name." + + this.generateScope().toString() + "." + name); + } + if (!groups.containsKey(name)) { + groups.put(name, new GenericMetricGroup(registry, this, name)); + } + return groups.get(name); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/ComponentMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/ComponentMetricGroup.java new file mode 100644 index 0000000000000..81851e2e767f4 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/ComponentMetricGroup.java @@ -0,0 +1,113 @@ +/* + * 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.metrics.groups; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.MetricRegistry; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.metrics.groups.Scope.SCOPE_WILDCARD; + +/** + * Abstract {@link org.apache.flink.metrics.MetricGroup} that contains key functionality for modifying the scope. + */ +@Internal +public abstract class ComponentMetricGroup extends AbstractMetricGroup { + private final ComponentMetricGroup parent; + private final String format; + + + // Map: scope variable -> specific value + protected final Map formats; + + /** + * Creates a new ComponentMetricGroup. + * + * @param registry registry to register new metrics with + * @param parentGroup parent group, may be null + * @param scopeFormat default format string + */ + public ComponentMetricGroup(MetricRegistry registry, ComponentMetricGroup parentGroup, String scopeFormat) { + super(registry); + this.formats = new HashMap<>(); + this.parent = parentGroup; + this.format = scopeFormat; + } + + @Override + public List generateScope() { + return this.generateScope(this.format); + } + + @Override + public List generateScope(Scope.ScopeFormat format) { + return generateScope(getScopeFormat(format)); + } + + protected abstract String getScopeFormat(Scope.ScopeFormat format); + + private List generateScope(String format) { + String[] components = Scope.split(format); + + List scope = new ArrayList<>(); + if (components[0].equals(SCOPE_WILDCARD)) { + if (this.parent != null) { + scope = this.parent.generateScope(); + } + this.replaceFormats(components); + addToList(scope, components, 1); + } else { + if (this.parent != null) { + this.parent.replaceFormats(components); + } + this.replaceFormats(components); + addToList(scope, components, 0); + } + return scope; + } + + private void replaceFormats(String[] components) { + if (this.parent != null) { + this.parent.replaceFormats(components); + } + for (int x = 0; x < components.length; x++) { + if (components[x].startsWith("<")) { + if (this.formats.containsKey(components[x])) { + components[x] = this.formats.get(components[x]); + } + } + } + } + + /** + * Adds all elements from the given array, starting from the given index, to the given list. + * + * @param list destination + * @param array source + * @param startIndex array index to start from + */ + private static void addToList(List list, String[] array, int startIndex) { + for (int x = startIndex; x < array.length; x++) { + list.add(array[x]); + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/GenericMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/GenericMetricGroup.java new file mode 100644 index 0000000000000..5886312aca647 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/GenericMetricGroup.java @@ -0,0 +1,57 @@ +/* + * 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.metrics.groups; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.MetricRegistry; + +import java.util.List; + +/** + * A simple named {@link org.apache.flink.metrics.MetricGroup} with no special properties. + */ +@Internal +public class GenericMetricGroup extends AbstractMetricGroup { + private final AbstractMetricGroup parent; + + private final String name; + + protected GenericMetricGroup(MetricRegistry registry, AbstractMetricGroup parent, int name) { + this(registry, parent, "" + name); + } + + protected GenericMetricGroup(MetricRegistry registry, AbstractMetricGroup parent, String name) { + super(registry); + this.parent = parent; + this.name = name; + } + + @Override + public List generateScope() { + List scope = parent.generateScope(); + scope.add(name); + return scope; + } + + @Override + public List generateScope(Scope.ScopeFormat format) { + List scope = parent.generateScope(format); + scope.add(name); + return scope; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/IOMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/IOMetricGroup.java new file mode 100644 index 0000000000000..68d91c448712b --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/IOMetricGroup.java @@ -0,0 +1,70 @@ +/* + * 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.metrics.groups; + +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.MetricRegistry; + +import java.util.List; + +/** + * Special {@link org.apache.flink.metrics.MetricGroup} that contains shareable pre-defined IO-related metrics. + */ +public class IOMetricGroup extends AbstractMetricGroup { + private final TaskMetricGroup parent; + + private transient final Counter numBytesIn; + private transient final Counter numBytesOut; + private transient final Counter numRecordsIn; + private transient final Counter numRecordsOut; + + public IOMetricGroup(MetricRegistry registry, TaskMetricGroup parent) { + super(registry); + this.parent = parent; + this.numBytesIn = parent.counter("numBytesIn"); + this.numBytesOut = parent.counter("numBytesOut"); + this.numRecordsIn = parent.counter("numRecordsIn"); + this.numRecordsOut = parent.counter("numRecordsOut"); + } + + @Override + public List generateScope() { + return parent.generateScope(); + } + + @Override + public List generateScope(Scope.ScopeFormat format) { + return parent.generateScope(format); + } + + public Counter getBytesInCounter() { + return this.numBytesIn; + } + + public Counter getBytesOutCounter() { + return this.numBytesOut; + } + + public Counter getRecordsInCounter() { + return this.numRecordsIn; + } + + public Counter getRecordsOutCounter() { + return this.numRecordsOut; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/JobMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/JobMetricGroup.java new file mode 100644 index 0000000000000..35a01f8d30e0c --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/JobMetricGroup.java @@ -0,0 +1,71 @@ +/* + * 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.metrics.groups; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.JobID; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.MetricRegistry; +import org.apache.flink.util.AbstractID; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.metrics.groups.TaskManagerMetricGroup.DEFAULT_SCOPE_TM; + +/** + * Special {@link org.apache.flink.metrics.MetricGroup} representing a Job. + * tasks = new HashMap<>(); + + public JobMetricGroup(MetricRegistry registry, TaskManagerMetricGroup taskManager, JobID id, String name) { + super(registry, taskManager, registry.getScopeConfig().getJobFormat()); + this.formats.put(SCOPE_JOB_ID, id.toString()); + this.formats.put(SCOPE_JOB_NAME, name); + } + + public TaskMetricGroup addTask(AbstractID id, AbstractID attemptID, int subtaskIndex, String name) { + TaskMetricGroup task = new TaskMetricGroup(this.registry, this, id, attemptID, subtaskIndex, name); + tasks.put(id, task); + return task; + } + + @Override + public void close() { + super.close(); + for (MetricGroup group : tasks.values()) { + group.close(); + } + tasks.clear(); + } + + @Override + protected String getScopeFormat(Scope.ScopeFormat format) { + return format.getJobFormat(); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/OperatorMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/OperatorMetricGroup.java new file mode 100644 index 0000000000000..6475eecc31a8d --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/OperatorMetricGroup.java @@ -0,0 +1,46 @@ +/* + * 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.metrics.groups; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.MetricRegistry; + +import static org.apache.flink.metrics.groups.JobMetricGroup.DEFAULT_SCOPE_JOB; + +/** + * Special {@link org.apache.flink.metrics.MetricGroup} representing an Operator. + */ +@Internal +public class OperatorMetricGroup extends ComponentMetricGroup { + public static final String SCOPE_OPERATOR_DESCRIPTOR = "operator"; + public static final String SCOPE_OPERATOR_NAME = Scope.format("operator_name"); + public static final String SCOPE_OPERATOR_SUBTASK_INDEX = Scope.format("subtask_index"); + public static final String DEFAULT_SCOPE_OPERATOR_COMPONENT = Scope.concat(SCOPE_OPERATOR_NAME, SCOPE_OPERATOR_SUBTASK_INDEX); + public static final String DEFAULT_SCOPE_OPERATOR = Scope.concat(DEFAULT_SCOPE_JOB, DEFAULT_SCOPE_OPERATOR_COMPONENT); + + protected OperatorMetricGroup(MetricRegistry registry, TaskMetricGroup task, String name, int subTaskIndex) { + super(registry, task, registry.getScopeConfig().getOperatorFormat()); + this.formats.put(SCOPE_OPERATOR_NAME, name); + this.formats.put(SCOPE_OPERATOR_SUBTASK_INDEX, "" + subTaskIndex); + } + + @Override + protected String getScopeFormat(Scope.ScopeFormat format) { + return format.getOperatorFormat(); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/Scope.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/Scope.java new file mode 100644 index 0000000000000..47bae37ec4641 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/Scope.java @@ -0,0 +1,120 @@ +/* + * 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.metrics.groups; + +import org.apache.flink.annotation.Internal; + +/** + * This class provides utility-functions for handling scopes. + */ +@Internal +public class Scope { + public static final String SCOPE_WILDCARD = "*"; + + private static final String SCOPE_PREFIX = "<"; + private static final String SCOPE_SUFFIX = ">"; + private static final String SCOPE_SPLIT = "."; + + private Scope() { + } + + /** + * Modifies the given string to resemble a scope variable. + * + * @param scope string to format + * @return formatted string + */ + public static String format(String scope) { + return SCOPE_PREFIX + scope + SCOPE_SUFFIX; + } + + /** + * Joins the given components into a single scope. + * + * @param components components to join + * @return joined scoped + */ + public static String concat(String... components) { + StringBuilder sb = new StringBuilder(); + sb.append(components[0]); + for (int x = 1; x < components.length; x++) { + sb.append(SCOPE_SPLIT); + sb.append(components[x]); + } + return sb.toString(); + } + + /** + * Splits the given scope into it's individual components. + * + * @param scope scope to split + * @return array of components + */ + public static String[] split(String scope) { + return scope.split("\\" + SCOPE_SPLIT); + } + + /** + * Simple container for component scope format strings. + */ + public static class ScopeFormat { + private String operatorFormat = OperatorMetricGroup.DEFAULT_SCOPE_OPERATOR; + private String taskFormat = TaskMetricGroup.DEFAULT_SCOPE_TASK; + private String jobFormat = JobMetricGroup.DEFAULT_SCOPE_JOB; + private String taskManagerFormat = TaskManagerMetricGroup.DEFAULT_SCOPE_TM; + + public ScopeFormat() { + } + + public ScopeFormat setOperatorFormat(String format) { + this.operatorFormat = format; + return this; + } + + public ScopeFormat setTaskFormat(String format) { + this.taskFormat = format; + return this; + } + + public ScopeFormat setJobFormat(String format) { + this.jobFormat = format; + return this; + } + + public ScopeFormat setTaskManagerFormat(String format) { + this.taskManagerFormat = format; + return this; + } + + public String getOperatorFormat() { + return this.operatorFormat; + } + + public String getTaskFormat() { + return this.taskFormat; + } + + public String getJobFormat() { + return this.jobFormat; + } + + public String getTaskManagerFormat() { + return this.taskManagerFormat; + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskManagerMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskManagerMetricGroup.java new file mode 100644 index 0000000000000..e199ca7ea76c1 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskManagerMetricGroup.java @@ -0,0 +1,70 @@ +/* + * 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.metrics.groups; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.JobID; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.MetricRegistry; + +import java.util.HashMap; +import java.util.Map; + +/** + * Special {@link org.apache.flink.metrics.MetricGroup} representing a TaskManager. + * jobs = new HashMap<>(); + + public TaskManagerMetricGroup(MetricRegistry registry, String host, String id) { + super(registry, null, registry.getScopeConfig().getTaskManagerFormat()); + this.formats.put(SCOPE_TM_HOST, host); + this.formats.put(SCOPE_TM_ID, id); + } + + public JobMetricGroup addJob(JobID id, String name) { + JobMetricGroup task = new JobMetricGroup(this.registry, this, id, name); + jobs.put(id, task); + return task; + } + + @Override + public void close() { + super.close(); + for (MetricGroup group : jobs.values()) { + group.close(); + } + jobs.clear(); + } + + @Override + protected String getScopeFormat(Scope.ScopeFormat format) { + return format.getTaskManagerFormat(); + } +} + diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java new file mode 100644 index 0000000000000..4f8e010a528f7 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java @@ -0,0 +1,87 @@ +/* + * 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.metrics.groups; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.MetricRegistry; +import org.apache.flink.util.AbstractID; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.metrics.groups.JobMetricGroup.DEFAULT_SCOPE_JOB; + +/** + * Special {@link org.apache.flink.metrics.MetricGroup} representing a Task. + * operators = new HashMap<>(); + private IOMetricGroup ioMetrics; + + protected TaskMetricGroup(MetricRegistry registry, JobMetricGroup job, AbstractID id, AbstractID attemptID, int subtaskIndex, String name) { + super(registry, job, registry.getScopeConfig().getTaskFormat()); + this.formats.put(SCOPE_TASK_ID, id.toString()); + this.formats.put(SCOPE_TASK_ATTEMPT, attemptID.toString()); + this.formats.put(SCOPE_TASK_NAME, name); + this.formats.put(SCOPE_TASK_SUBTASK_INDEX, "" + subtaskIndex); + this.subtaskIndex = subtaskIndex; + this.ioMetrics = new IOMetricGroup(registry, this); + } + + @Override + public void close() { + super.close(); + for (MetricGroup group : operators.values()) { + group.close(); + } + operators.clear(); + } + + public OperatorMetricGroup addOperator(String name) { + OperatorMetricGroup operator = new OperatorMetricGroup(this.registry, this, name, this.subtaskIndex); + operators.put(name, operator); + return operator; + } + + /** + * Returns the IOMetricGroup for this task. + * + * @return IOMetricGroup for this task. + */ + public IOMetricGroup getIOMetricGroup() { + return this.ioMetrics; + } + + @Override + protected String getScopeFormat(Scope.ScopeFormat format) { + return format.getTaskFormat(); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java b/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java new file mode 100644 index 0000000000000..43f09b2293f16 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java @@ -0,0 +1,48 @@ +/* + * 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.metrics.reporter; + +import org.apache.flink.hadoop.shaded.org.jboss.netty.util.internal.ConcurrentHashMap; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; + +import java.util.Map; + +public abstract class AbstractReporter implements MetricReporter { + protected Map gauges = new ConcurrentHashMap<>(); + protected Map counters = new ConcurrentHashMap<>(); + + @Override + public void notifyOfAddedMetric(Metric metric, String name) { + if (metric instanceof Counter) { + counters.put(name, (Counter) metric); + } else if (metric instanceof Gauge) { + gauges.put(name, (Gauge) metric); + } + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String name) { + if (metric instanceof Counter) { + counters.remove(name); + } else if (metric instanceof Gauge) { + gauges.remove(name); + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/reporter/JMXReporter.java b/flink-core/src/main/java/org/apache/flink/metrics/reporter/JMXReporter.java new file mode 100644 index 0000000000000..0b2efe4b5de4e --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/reporter/JMXReporter.java @@ -0,0 +1,167 @@ +/* + * 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.metrics.reporter; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.management.InstanceAlreadyExistsException; +import javax.management.InstanceNotFoundException; +import javax.management.MBeanRegistrationException; +import javax.management.MBeanServer; +import javax.management.MalformedObjectNameException; +import javax.management.NotCompliantMBeanException; +import javax.management.ObjectName; +import java.lang.management.ManagementFactory; +import java.util.List; + +/** + * {@link org.apache.flink.metrics.reporter.MetricReporter} that exports {@link org.apache.flink.metrics.Metric}s via JMX. + * + * Largely based on the JmxReporter class of the dropwizard metrics library + * https://github.com/dropwizard/metrics/blob/master/metrics-core/src/main/java/io/dropwizard/metrics/JmxReporter.java + */ +@Internal +public class JMXReporter implements MetricReporter { + private static final Logger LOG = LoggerFactory.getLogger(JMXReporter.class); + + private MBeanServer mBeanServer; + + private static final String PREFIX = "org.apache.flink.metrics:"; + private static final String KEY_PREFIX = "key"; + + public JMXReporter() { + this.mBeanServer = ManagementFactory.getPlatformMBeanServer(); + } + + @Override + public void notifyOfAddedMetric(Metric metric, String name) { + AbstractBean jmxMetric; + ObjectName jmxName; + try { + jmxName = new ObjectName(name); + } catch (MalformedObjectNameException e) { + throw new IllegalArgumentException("Metric name did not conform to JMX ObjectName rules: " + name, e); + } + + if (metric instanceof Gauge) { + jmxMetric = new JmxGauge((Gauge) metric); + } else if (metric instanceof Counter) { + jmxMetric = new JmxCounter((Counter) metric); + } else { + throw new IllegalArgumentException("Unknown metric type: " + metric.getClass()); + } + + try { + mBeanServer.registerMBean(jmxMetric, jmxName); + } catch (NotCompliantMBeanException e) { //implementation error on our side + LOG.error("Metric did not comply with JMX MBean naming rules.", e); + } catch (InstanceAlreadyExistsException e) { + LOG.error("A metric with the name " + jmxName + " was already registered.", e); + } catch (MBeanRegistrationException e) { + LOG.error("Failed to register metric.", e); + } + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String name) { + try { + mBeanServer.unregisterMBean(new ObjectName(name)); + } catch (MBeanRegistrationException e) { + LOG.error("Un-registering metric failed.", e); + } catch (MalformedObjectNameException e) { + LOG.error("Un-registering metric failed due to invalid name.", e); + } catch (InstanceNotFoundException e) { + //alright then + } + } + + @Override + public void open(Configuration config) { + } + + @Override + public void close() { + } + + @Override + public String generateName(String name, List origin) { + StringBuilder fullName = new StringBuilder(); + + fullName.append(PREFIX); + for (int x = 0; x < origin.size(); x++) { + fullName.append(KEY_PREFIX); + fullName.append(x); + fullName.append("="); + String value = origin.get(x); + value = value.replaceAll("\"", ""); + value = value.replaceAll(" ", "_"); + value = value.replaceAll("[,=;:?'*]", "-"); + fullName.append(value); + fullName.append(","); + } + fullName.append("name=" + name); + + return fullName.toString(); + } + + public interface MetricMBean { + } + + private abstract static class AbstractBean implements MetricMBean { + } + + public interface JmxCounterMBean extends MetricMBean { + long getCount(); + } + + private static class JmxCounter extends AbstractBean implements JmxCounterMBean { + private Counter counter; + + public JmxCounter(Counter counter) { + this.counter = counter; + } + + @Override + public long getCount() { + return counter.getCount(); + } + } + + public interface JmxGaugeMBean extends MetricMBean { + Object getValue(); + } + + private static class JmxGauge extends AbstractBean implements JmxGaugeMBean { + private final Gauge gauge; + + public JmxGauge(Gauge gauge) { + this.gauge = gauge; + } + + @Override + public Object getValue() { + return gauge.getValue(); + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporter.java b/flink-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporter.java new file mode 100644 index 0000000000000..2bca606e215f1 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporter.java @@ -0,0 +1,73 @@ +/* + * 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.metrics.reporter; + +import com.codahale.metrics.Reporter; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Metric; + +import java.util.List; + +/** + * Reporters are used to export {@link org.apache.flink.metrics.Metric}s to an external backend. + *

+ * Reporters are instantiated generically and must have a no-argument constructor. + */ +@PublicEvolving +public interface MetricReporter extends Reporter { + /** + * Configures this reporter. Since reporters are instantiated generically and hence parameter-less, + * this method is the place where the reporters set their basic fields based on configuration values. + *

+ * This method is always called first on a newly instantiated reporter. + * + * @param config The configuration with all parameters. + */ + void open(Configuration config); + + /** + * Closes this reporter. Should be used to close channels, streams and release resources. + */ + void close(); + + /** + * Called when a new {@link org.apache.flink.metrics.Metric} was added. + * + * @param metric metric that was added + * @param name name of the metric + */ + void notifyOfAddedMetric(Metric metric, String name); + + /** + * Called when a {@link org.apache.flink.metrics.Metric} was removed. + * + * @param metric metric that was removed + * @param name name of the metric + */ + void notifyOfRemovedMetric(Metric metric, String name); + + /** + * Generates the reported name of a metric based on it's hierarchy/scope and associated name. + * + * @param name name of the metric + * @param scope hierarchy/scope of the metric + * @return reported name + */ + String generateName(String name, List scope); +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/reporter/Scheduled.java b/flink-core/src/main/java/org/apache/flink/metrics/reporter/Scheduled.java new file mode 100644 index 0000000000000..3638f7a5308a7 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/reporter/Scheduled.java @@ -0,0 +1,32 @@ +/* + * 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.metrics.reporter; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * Marker interface for reporters that actively send out data periodically. + */ +@PublicEvolving +public interface Scheduled { + /** + * Report the current measurements. + * This method is called in regular intervals + */ + void report(); +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java b/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java index 858bc49403b15..83c88cc3608ae 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java @@ -31,6 +31,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.BroadcastVariableInitializer; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.junit.Test; @@ -41,7 +42,7 @@ public class RuntimeUDFContextTest { @Test public void testBroadcastVariableNotFound() { try { - RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(),new HashMap>()); + RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(),new HashMap>(), new DummyMetricGroup()); try { ctx.getBroadcastVariable("some name"); @@ -71,7 +72,7 @@ public void testBroadcastVariableNotFound() { @Test public void testBroadcastVariableSimple() { try { - RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>()); + RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>(), new DummyMetricGroup()); ctx.setBroadcastVariable("name1", Arrays.asList(1, 2, 3, 4)); ctx.setBroadcastVariable("name2", Arrays.asList(1.0, 2.0, 3.0, 4.0)); @@ -105,7 +106,7 @@ public void testBroadcastVariableSimple() { @Test public void testBroadcastVariableWithInitializer() { try { - RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>()); + RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>(), new DummyMetricGroup()); ctx.setBroadcastVariable("name", Arrays.asList(1, 2, 3, 4)); @@ -130,7 +131,7 @@ public void testBroadcastVariableWithInitializer() { @Test public void testResetBroadcastVariableWithInitializer() { try { - RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>()); + RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>(), new DummyMetricGroup()); ctx.setBroadcastVariable("name", Arrays.asList(1, 2, 3, 4)); @@ -153,7 +154,7 @@ public void testResetBroadcastVariableWithInitializer() { @Test public void testBroadcastVariableWithInitializerAndMismatch() { try { - RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>()); + RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>(), new DummyMetricGroup()); ctx.setBroadcastVariable("name", Arrays.asList(1, 2, 3, 4)); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/RichInputFormatTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/RichInputFormatTest.java index 7ea007174c1c3..554820e981f4d 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/io/RichInputFormatTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/io/RichInputFormatTest.java @@ -27,6 +27,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.util.RuntimeUDFContext; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.types.Value; import org.junit.Assert; import org.junit.Test; @@ -40,7 +41,7 @@ public class RichInputFormatTest { public void testCheckRuntimeContextAccess() { final SerializedInputFormat inputFormat = new SerializedInputFormat(); final TaskInfo taskInfo = new TaskInfo("test name", 1, 3, 0); - inputFormat.setRuntimeContext(new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>())); + inputFormat.setRuntimeContext(new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>(), new DummyMetricGroup())); Assert.assertEquals(inputFormat.getRuntimeContext().getIndexOfThisSubtask(), 1); Assert.assertEquals(inputFormat.getRuntimeContext().getNumberOfParallelSubtasks(),3); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/RichOutputFormatTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/RichOutputFormatTest.java index 273f4f54e7389..09db3a99efff7 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/io/RichOutputFormatTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/io/RichOutputFormatTest.java @@ -28,6 +28,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.util.RuntimeUDFContext; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.types.Value; import org.junit.Assert; import org.junit.Test; @@ -41,7 +42,7 @@ public class RichOutputFormatTest { public void testCheckRuntimeContextAccess() { final SerializedOutputFormat inputFormat = new SerializedOutputFormat(); final TaskInfo taskInfo = new TaskInfo("test name", 1, 3, 0); - inputFormat.setRuntimeContext(new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>())); + inputFormat.setRuntimeContext(new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>(), new DummyMetricGroup())); Assert.assertEquals(inputFormat.getRuntimeContext().getIndexOfThisSubtask(), 1); Assert.assertEquals(inputFormat.getRuntimeContext().getNumberOfParallelSubtasks(),3); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSinkBaseTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSinkBaseTest.java index 5ca4c4cfe2d85..7c905c186d878 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSinkBaseTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSinkBaseTest.java @@ -28,6 +28,7 @@ import org.apache.flink.api.common.operators.util.TestRichOutputFormat; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.types.Nothing; import org.junit.Test; @@ -94,13 +95,13 @@ public void testDataSourceWithRuntimeContext() { final TaskInfo taskInfo = new TaskInfo("test_sink", 0, 1, 0); executionConfig.disableObjectReuse(); in.reset(); - sink.executeOnCollections(asList(TestIOData.NAMES), new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap), executionConfig); + sink.executeOnCollections(asList(TestIOData.NAMES), new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap, new DummyMetricGroup()), executionConfig); assertEquals(out.output, asList(TestIOData.RICH_NAMES)); executionConfig.enableObjectReuse(); out.clear(); in.reset(); - sink.executeOnCollections(asList(TestIOData.NAMES), new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap), executionConfig); + sink.executeOnCollections(asList(TestIOData.NAMES), new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap, new DummyMetricGroup()), executionConfig); assertEquals(out.output, asList(TestIOData.RICH_NAMES)); } catch(Exception e){ e.printStackTrace(); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSourceBaseTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSourceBaseTest.java index 083039af10f71..c360c62fdbb5a 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSourceBaseTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSourceBaseTest.java @@ -27,6 +27,7 @@ import org.apache.flink.api.common.operators.util.TestRichInputFormat; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.junit.Test; import java.util.HashMap; @@ -83,7 +84,7 @@ public void testDataSourceWithRuntimeContext() { executionConfig.disableObjectReuse(); assertEquals(false, in.hasBeenClosed()); assertEquals(false, in.hasBeenOpened()); - List resultMutableSafe = source.executeOnCollections(new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap), executionConfig); + List resultMutableSafe = source.executeOnCollections(new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap, new DummyMetricGroup()), executionConfig); assertEquals(true, in.hasBeenClosed()); assertEquals(true, in.hasBeenOpened()); @@ -91,7 +92,7 @@ public void testDataSourceWithRuntimeContext() { executionConfig.enableObjectReuse(); assertEquals(false, in.hasBeenClosed()); assertEquals(false, in.hasBeenOpened()); - List resultRegular = source.executeOnCollections(new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap), executionConfig); + List resultRegular = source.executeOnCollections(new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap, new DummyMetricGroup()), executionConfig); assertEquals(true, in.hasBeenClosed()); assertEquals(true, in.hasBeenOpened()); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/FlatMapOperatorCollectionTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/FlatMapOperatorCollectionTest.java index cda324526fe0c..9447efd865ec6 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/FlatMapOperatorCollectionTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/FlatMapOperatorCollectionTest.java @@ -29,6 +29,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.util.Collector; import org.junit.Assert; import org.junit.Test; @@ -78,7 +79,7 @@ private void testExecuteOnCollection(FlatMapFunction udf, List result = getTestFlatMapOperator(udf) - .executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + .executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); Assert.assertEquals(input.size(), result.size()); Assert.assertEquals(input, result); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java index d119fe22e6a7b..a610a4dc5ffaf 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java @@ -30,6 +30,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.util.Collector; import org.junit.Test; @@ -125,9 +126,9 @@ public void join(String first, String second, Collector out) throws Exc ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.disableObjectReuse(); - List resultSafe = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap), executionConfig); + List resultSafe = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap, new DummyMetricGroup()), executionConfig); executionConfig.enableObjectReuse(); - List resultRegular = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap), executionConfig); + List resultRegular = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap, new DummyMetricGroup()), executionConfig); assertEquals(expected, resultSafe); assertEquals(expected, resultRegular); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/MapOperatorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/MapOperatorTest.java index 6059ab1e438d5..7ecdefa83b1e0 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/MapOperatorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/MapOperatorTest.java @@ -39,6 +39,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.junit.Test; @SuppressWarnings("serial") @@ -112,9 +113,9 @@ public void close() throws Exception { final TaskInfo taskInfo = new TaskInfo(taskName, 0, 1, 0); ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.disableObjectReuse(); - List resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap), executionConfig); + List resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap, new DummyMetricGroup()), executionConfig); executionConfig.enableObjectReuse(); - List resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap), executionConfig); + List resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap, new DummyMetricGroup()), executionConfig); assertEquals(asList(1, 2, 3, 4, 5, 6), resultMutableSafe); assertEquals(asList(1, 2, 3, 4, 5, 6), resultRegular); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/PartitionMapOperatorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/PartitionMapOperatorTest.java index 71486a51bf1f6..50127185af76c 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/PartitionMapOperatorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/PartitionMapOperatorTest.java @@ -31,6 +31,7 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.util.Collector; import org.apache.flink.api.common.functions.MapPartitionFunction; import org.apache.flink.api.common.functions.RichMapPartitionFunction; @@ -85,9 +86,9 @@ public void close() throws Exception { ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.disableObjectReuse(); - List resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + List resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); executionConfig.enableObjectReuse(); - List resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + List resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); assertEquals(asList(1, 2, 3, 4, 5, 6), resultMutableSafe); assertEquals(asList(1, 2, 3, 4, 5, 6), resultRegular); diff --git a/flink-core/src/test/java/org/apache/flink/metrics/MetricGroupTest.java b/flink-core/src/test/java/org/apache/flink/metrics/MetricGroupTest.java new file mode 100644 index 0000000000000..f7502e58dbb9b --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/MetricGroupTest.java @@ -0,0 +1,93 @@ +/* + * 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.metrics; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.groups.TaskManagerMetricGroup; +import org.apache.flink.metrics.util.TestReporter; +import org.junit.Assert; +import org.junit.Test; + +public class MetricGroupTest { + /** + * Verifies that group methods instantiate the correct metric with the given name. + */ + @Test + public void testMetricInstantiation() { + Configuration config = new Configuration(); + + config.setString(MetricRegistry.KEY_METRICS_REPORTER_CLASS, TestReporter1.class.getName()); + + MetricGroup root = new TaskManagerMetricGroup(new MetricRegistry(config), "host", "id"); + + root.counter("counter"); + Assert.assertTrue(TestReporter1.lastPassedMetric instanceof Counter); + Assert.assertEquals("counter", TestReporter1.lastPassedName); + + root.gauge("gauge", new Gauge() { + @Override + public Object getValue() { + return null; + } + }); + Assert.assertTrue(TestReporter1.lastPassedMetric instanceof Gauge); + Assert.assertEquals("gauge", TestReporter1.lastPassedName); + } + + protected static class TestReporter1 extends TestReporter { + public static Metric lastPassedMetric; + public static String lastPassedName; + + @Override + public void notifyOfAddedMetric(Metric metric, String name) { + lastPassedMetric = metric; + lastPassedName = name; + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String name) { + } + } + + /** + * Verifies that metric names containing special characters are rejected. + */ + @Test(expected = IllegalArgumentException.class) + public void testInvalidMetricName() { + Configuration config = new Configuration(); + + MetricGroup root = new TaskManagerMetricGroup(new MetricRegistry(config), "host", "id"); + root.counter("=)(/!"); + } + + /** + * Verifies that when attempting to create a group with the name of an existing one the existing one will be returned instead. + */ + @Test + public void testDuplicateGroupName() { + Configuration config = new Configuration(); + + MetricGroup root = new TaskManagerMetricGroup(new MetricRegistry(config), "host", "id"); + + MetricGroup group1 = root.addGroup("group"); + MetricGroup group2 = root.addGroup("group"); + MetricGroup group3 = root.addGroup("group"); + Assert.assertTrue(group1 == group2 && group2 == group3); + + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/MetricRegistryTest.java b/flink-core/src/test/java/org/apache/flink/metrics/MetricRegistryTest.java new file mode 100644 index 0000000000000..32cc11ce940a8 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/MetricRegistryTest.java @@ -0,0 +1,217 @@ +/* + * 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.metrics; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.groups.JobMetricGroup; +import org.apache.flink.metrics.groups.OperatorMetricGroup; +import org.apache.flink.metrics.groups.Scope; +import org.apache.flink.metrics.groups.TaskManagerMetricGroup; +import org.apache.flink.metrics.groups.TaskMetricGroup; +import org.apache.flink.metrics.reporter.Scheduled; +import org.apache.flink.metrics.util.TestReporter; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class MetricRegistryTest { + /** + * Verifies that the reporter class argument is correctly used to instantiate and open the reporter. + */ + @Test + public void testReporterInstantiation() { + Configuration config = new Configuration(); + + config.setString(MetricRegistry.KEY_METRICS_REPORTER_CLASS, TestReporter1.class.getName()); + + new MetricRegistry(config); + + Assert.assertTrue(TestReporter1.wasOpened); + } + + protected static class TestReporter1 extends TestReporter { + public static boolean wasOpened = false; + + @Override + public void open(Configuration config) { + wasOpened = true; + } + } + + /** + * Verifies that configured arguments are properly forwarded to the reporter. + */ + @Test + public void testReporterArgumentForwarding() { + Configuration config = new Configuration(); + + config.setString(MetricRegistry.KEY_METRICS_REPORTER_CLASS, TestReporter2.class.getName()); + config.setString(MetricRegistry.KEY_METRICS_REPORTER_ARGUMENTS, "--arg1 hello --arg2 world"); + + new MetricRegistry(config); + } + + protected static class TestReporter2 extends TestReporter { + @Override + public void open(Configuration config) { + Assert.assertEquals("hello", config.getString("arg1", null)); + Assert.assertEquals("world", config.getString("arg2", null)); + } + } + + /** + * Verifies that reporters implementing the Scheduled interface are regularly called to report the metrics. + * + * @throws InterruptedException + */ + @Test + public void testReporterScheduling() throws InterruptedException { + Configuration config = new Configuration(); + + config.setString(MetricRegistry.KEY_METRICS_REPORTER_CLASS, TestReporter3.class.getName()); + config.setString(MetricRegistry.KEY_METRICS_REPORTER_INTERVAL, "50 MILLISECONDS"); + + new MetricRegistry(config); + + long start = System.currentTimeMillis(); + for (int x = 0; x < 10; x++) { + Thread.sleep(100); + int reportCount = TestReporter3.reportCount; + long curT = System.currentTimeMillis(); + /** + * Within a given time-frame T only T/500 reports may be triggered due to the interval between reports. + * This value however does not not take the first triggered report into account (=> +1). + * Furthermore we have to account for the mis-alignment between reports being triggered and our time + * measurement (=> +1); for T=200 a total of 4-6 reports may have been + * triggered depending on whether the end of the interval for the first reports ends before + * or after T=50. + */ + long maxAllowedReports = (curT - start) / 50 + 2; + Assert.assertTrue("Too many report were triggered.", maxAllowedReports >= reportCount); + } + Assert.assertTrue("No report was triggered.", TestReporter3.reportCount > 0); + } + + protected static class TestReporter3 extends TestReporter implements Scheduled { + public static int reportCount = 0; + + @Override + public void report() { + reportCount++; + } + } + + /** + * Verifies that groups are correctly created, nesting works, and names are properly forwarded to generate names. + */ + @Test + public void testMetricGroupGeneration() { + Configuration config = new Configuration(); + + config.setString(MetricRegistry.KEY_METRICS_REPORTER_CLASS, TestReporter4.class.getName()); + + MetricRegistry registry = new MetricRegistry(config); + + MetricGroup root = new TaskManagerMetricGroup(registry, "host", "id"); + root.counter("rootCounter"); + root.addGroup("top").counter("topCounter"); + } + + protected static class TestReporter4 extends TestReporter { + @Override + public String generateName(String name, List scope) { + if (name.compareTo("rootCounter") == 0) { + Assert.assertEquals("host", scope.get(0)); + return "success"; + } else if (name.compareTo("topCounter") == 0) { + Assert.assertEquals("host", scope.get(0)); + Assert.assertEquals("taskmanager", scope.get(1)); + return "success"; + } else { + Assert.fail(); + return null; + } + } + } + + /** + * Verifies that reporters implementing the Listener interface are notified when Metrics are added or removed. + */ + @Test + public void testListener() { + Configuration config = new Configuration(); + + config.setString(MetricRegistry.KEY_METRICS_REPORTER_CLASS, TestReporter6.class.getName()); + + MetricRegistry registry = new MetricRegistry(config); + + MetricGroup root = new TaskManagerMetricGroup(registry, "host", "id"); + root.counter("rootCounter"); + root.close(); + + Assert.assertTrue(TestReporter6.addCalled); + Assert.assertTrue(TestReporter6.removeCalled); + } + + protected static class TestReporter6 extends TestReporter { + public static boolean addCalled = false; + public static boolean removeCalled = false; + + @Override + public void notifyOfAddedMetric(Metric metric, String name) { + addCalled = true; + Assert.assertTrue(metric instanceof Counter); + Assert.assertEquals("rootCounter", name); + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String name) { + removeCalled = true; + Assert.assertTrue(metric instanceof Counter); + Assert.assertEquals("rootCounter", name); + } + } + + /** + * Verifies that the scope configuration is properly extracted. + */ + @Test + public void testScopeConfig() { + Configuration config = new Configuration(); + + config.setString(MetricRegistry.KEY_METRICS_SCOPE_NAMING_TM, "A"); + config.setString(MetricRegistry.KEY_METRICS_SCOPE_NAMING_JOB, "B"); + config.setString(MetricRegistry.KEY_METRICS_SCOPE_NAMING_TASK, "C"); + config.setString(MetricRegistry.KEY_METRICS_SCOPE_NAMING_OPERATOR, "D"); + + Scope.ScopeFormat scopeConfig = new MetricRegistry(config).getScopeConfig(); + + Assert.assertEquals("A", scopeConfig.getTaskManagerFormat()); + Assert.assertEquals("B", scopeConfig.getJobFormat()); + Assert.assertEquals("C", scopeConfig.getTaskFormat()); + Assert.assertEquals("D", scopeConfig.getOperatorFormat()); + + Scope.ScopeFormat emptyScopeConfig = new MetricRegistry(new Configuration()).getScopeConfig(); + + Assert.assertEquals(TaskManagerMetricGroup.DEFAULT_SCOPE_TM, emptyScopeConfig.getTaskManagerFormat()); + Assert.assertEquals(JobMetricGroup.DEFAULT_SCOPE_JOB, emptyScopeConfig.getJobFormat()); + Assert.assertEquals(TaskMetricGroup.DEFAULT_SCOPE_TASK, emptyScopeConfig.getTaskFormat()); + Assert.assertEquals(OperatorMetricGroup.DEFAULT_SCOPE_OPERATOR, emptyScopeConfig.getOperatorFormat()); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/groups/JobGroupTest.java b/flink-core/src/test/java/org/apache/flink/metrics/groups/JobGroupTest.java new file mode 100644 index 0000000000000..89483b3dbcccf --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/groups/JobGroupTest.java @@ -0,0 +1,71 @@ +/* + * 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.metrics.groups; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricRegistry; +import org.junit.Test; + +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class JobGroupTest { + @Test + public void testGenerateScopeDefault() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + JobMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job"); + + List scope = operator.generateScope(); + assertEquals(4, scope.size()); + assertEquals("job", scope.get(3)); + } + + @Test + public void testGenerateScopeWildcard() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + JobMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job"); + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setJobFormat(Scope.concat(Scope.SCOPE_WILDCARD, "superjob", JobMetricGroup.SCOPE_JOB_NAME)); + + List scope = operator.generateScope(format); + assertEquals(5, scope.size()); + assertEquals("superjob", scope.get(3)); + assertEquals("job", scope.get(4)); + } + + @Test + public void testGenerateScopeCustom() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + JobMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job"); + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setJobFormat(Scope.concat(TaskManagerMetricGroup.SCOPE_TM_HOST, "superjob", JobMetricGroup.SCOPE_JOB_NAME)); + + List scope = operator.generateScope(format); + assertEquals(3, scope.size()); + assertEquals("host", scope.get(0)); + assertEquals("superjob", scope.get(1)); + assertEquals("job", scope.get(2)); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/groups/OperatorGroupTest.java b/flink-core/src/test/java/org/apache/flink/metrics/groups/OperatorGroupTest.java new file mode 100644 index 0000000000000..4f33d2a54edf0 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/groups/OperatorGroupTest.java @@ -0,0 +1,86 @@ +/* + * 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.metrics.groups; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricRegistry; +import org.apache.flink.util.AbstractID; +import org.junit.Test; + +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class OperatorGroupTest { + @Test + public void testGenerateScopeDefault() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + OperatorMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job") + .addTask(new AbstractID(), new AbstractID(), 0, "task").addOperator("operator"); + + List scope = operator.generateScope(); + assertEquals(6, scope.size()); + assertEquals("host", scope.get(0)); + assertEquals("taskmanager", scope.get(1)); + assertEquals("id", scope.get(2)); + assertEquals("job", scope.get(3)); + assertEquals("operator", scope.get(4)); + assertEquals("0", scope.get(5)); + } + + @Test + public void testGenerateScopeWildcard() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + OperatorMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job") + .addTask(new AbstractID(), new AbstractID(), 0, "task").addOperator("operator"); + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setOperatorFormat(Scope.concat(Scope.SCOPE_WILDCARD, "op", OperatorMetricGroup.SCOPE_OPERATOR_NAME)); + + List scope = operator.generateScope(format); + assertEquals(7, scope.size()); + assertEquals("host", scope.get(0)); + assertEquals("taskmanager", scope.get(1)); + assertEquals("id", scope.get(2)); + assertEquals("job", scope.get(3)); + assertEquals("task", scope.get(4)); + assertEquals("op", scope.get(5)); + assertEquals("operator", scope.get(6)); + } + + @Test + public void testGenerateScopeCustom() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + OperatorMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job") + .addTask(new AbstractID(), new AbstractID(), 0, "task").addOperator("operator"); + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setOperatorFormat(Scope.concat("jobs", JobMetricGroup.SCOPE_JOB_NAME, "op", OperatorMetricGroup.SCOPE_OPERATOR_NAME)); + + List scope = operator.generateScope(format); + assertEquals(4, scope.size()); + assertEquals("jobs", scope.get(0)); + assertEquals("job", scope.get(1)); + assertEquals("op", scope.get(2)); + assertEquals("operator", scope.get(3)); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskGroupTest.java b/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskGroupTest.java new file mode 100644 index 0000000000000..c49fdcdc95f1f --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskGroupTest.java @@ -0,0 +1,80 @@ +/* + * 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.metrics.groups; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricRegistry; +import org.apache.flink.util.AbstractID; +import org.junit.Test; + +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class TaskGroupTest { + @Test + public void testGenerateScopeDefault() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + TaskMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job") + .addTask(new AbstractID(), new AbstractID(), 0, "task"); + + List scope = operator.generateScope(); + assertEquals(5, scope.size()); + assertEquals("task", scope.get(4)); + } + + @Test + public void testGenerateScopeWilcard() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + TaskMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job") + .addTask(new AbstractID(), new AbstractID(), 0, "task"); + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setTaskFormat(Scope.concat(Scope.SCOPE_WILDCARD, "supertask", TaskMetricGroup.SCOPE_TASK_NAME)); + + List scope = operator.generateScope(format); + assertEquals(6, scope.size()); + assertEquals("host", scope.get(0)); + assertEquals("taskmanager", scope.get(1)); + assertEquals("id", scope.get(2)); + assertEquals("job", scope.get(3)); + assertEquals("supertask", scope.get(4)); + assertEquals("task", scope.get(5)); + } + + @Test + public void testGenerateScopeCustom() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + TaskMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job") + .addTask(new AbstractID(), new AbstractID(), 0, "task"); + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setTaskFormat(Scope.concat(TaskManagerMetricGroup.SCOPE_TM_HOST, JobMetricGroup.SCOPE_JOB_NAME, "supertask", TaskMetricGroup.SCOPE_TASK_NAME)); + + List scope = operator.generateScope(format); + assertEquals(4, scope.size()); + assertEquals("host", scope.get(0)); + assertEquals("job", scope.get(1)); + assertEquals("supertask", scope.get(2)); + assertEquals("task", scope.get(3)); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskManagerGroupTest.java b/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskManagerGroupTest.java new file mode 100644 index 0000000000000..7b3286d05aacb --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskManagerGroupTest.java @@ -0,0 +1,70 @@ +/* + * 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.metrics.groups; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricRegistry; +import org.junit.Test; + +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class TaskManagerGroupTest { + @Test + public void testGenerateScopeDefault() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + TaskManagerMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id"); + + List scope = operator.generateScope(); + assertEquals(3, scope.size()); + assertEquals("host", scope.get(0)); + assertEquals("taskmanager", scope.get(1)); + assertEquals("id", scope.get(2)); + } + + @Test + public void testGenerateScopeWildcard() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + TaskManagerMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id"); + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setTaskManagerFormat(Scope.concat(Scope.SCOPE_WILDCARD, "superhost", TaskManagerMetricGroup.SCOPE_TM_HOST)); + + List scope = operator.generateScope(format); + assertEquals(2, scope.size()); + assertEquals("superhost", scope.get(0)); + assertEquals("host", scope.get(1)); + } + + @Test + public void testGenerateScopeCustom() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + TaskManagerMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id"); + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setTaskManagerFormat(Scope.concat("h", TaskManagerMetricGroup.SCOPE_TM_HOST, "t", TaskManagerMetricGroup.SCOPE_TM_ID)); + + List scope = operator.generateScope(format); + assertEquals(4, scope.size()); + assertEquals("h", scope.get(0)); + assertEquals("host", scope.get(1)); + assertEquals("t", scope.get(2)); + assertEquals("id", scope.get(3)); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/reporter/JMXReporterTest.java b/flink-core/src/test/java/org/apache/flink/metrics/reporter/JMXReporterTest.java new file mode 100644 index 0000000000000..0d683c25c9c99 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/reporter/JMXReporterTest.java @@ -0,0 +1,43 @@ +/* + * 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.metrics.reporter; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +public class JMXReporterTest { + /** + * Verifies that the JMXReporter properly generates the JMX name. + */ + @Test + public void testGenerateName() { + String name = "metric"; + + List scope = new ArrayList<>(); + scope.add("value0"); + scope.add("value1"); + scope.add("\"value2 (test),=;:?'"); + + String jmxName = new JMXReporter().generateName(name, scope); + + Assert.assertEquals("org.apache.flink.metrics:key0=value0,key1=value1,key2=value2_(test)------,name=metric", jmxName); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyJobMetricGroup.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyJobMetricGroup.java new file mode 100644 index 0000000000000..d607072e8fea2 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyJobMetricGroup.java @@ -0,0 +1,50 @@ +/* + * 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.metrics.util; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.JobMetricGroup; +import org.apache.flink.util.AbstractID; + +public class DummyJobMetricGroup extends JobMetricGroup { + public DummyJobMetricGroup() { + super(new DummyMetricRegistry(), new DummyTaskManagerMetricGroup(), new JobID(), "job"); + } + + @Override + public DummyTaskMetricGroup addTask(AbstractID id, AbstractID attemptID, int subtaskIndex, String name) { + return new DummyTaskMetricGroup(); + } + + @Override + protected MetricGroup addMetric(String name, Metric metric) { + return this; + } + + @Override + public MetricGroup addGroup(int name) { + return addGroup("" + name); + } + + @Override + public MetricGroup addGroup(String name) { + return new DummyMetricGroup(); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricGroup.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricGroup.java new file mode 100644 index 0000000000000..26df8749348f9 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricGroup.java @@ -0,0 +1,57 @@ +/* + * 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.metrics.util; + +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.AbstractMetricGroup; +import org.apache.flink.metrics.groups.Scope; + +import java.util.ArrayList; +import java.util.List; + +public class DummyMetricGroup extends AbstractMetricGroup { + public DummyMetricGroup() { + super(new DummyMetricRegistry()); + } + + @Override + public List generateScope() { + return new ArrayList<>(); + } + + @Override + public List generateScope(Scope.ScopeFormat format) { + return new ArrayList<>(); + } + + @Override + protected MetricGroup addMetric(String name, Metric metric) { + return this; + } + + @Override + public MetricGroup addGroup(int name) { + return addGroup("" + name); + } + + @Override + public MetricGroup addGroup(String name) { + return new DummyMetricGroup(); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricRegistry.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricRegistry.java new file mode 100644 index 0000000000000..f0d6d3f4cd780 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricRegistry.java @@ -0,0 +1,34 @@ +/* + * 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.metrics.util; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricRegistry; + +public class DummyMetricRegistry extends MetricRegistry { + private static final Configuration config; + + static { + config = new Configuration(); + config.setString(KEY_METRICS_REPORTER_CLASS, DummyReporter.class.getCanonicalName()); + } + + public DummyMetricRegistry() { + super(new Configuration()); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyOperatorMetricGroup.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyOperatorMetricGroup.java new file mode 100644 index 0000000000000..eb45f6ae31cd4 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyOperatorMetricGroup.java @@ -0,0 +1,43 @@ +/* + * 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.metrics.util; + +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.OperatorMetricGroup; + +public class DummyOperatorMetricGroup extends OperatorMetricGroup { + public DummyOperatorMetricGroup() { + super(new DummyMetricRegistry(), new DummyTaskMetricGroup(), "operator", 0); + } + + @Override + protected MetricGroup addMetric(String name, Metric metric) { + return this; + } + + @Override + public MetricGroup addGroup(int name) { + return addGroup("" + name); + } + + @Override + public MetricGroup addGroup(String name) { + return new DummyMetricGroup(); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyReporter.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyReporter.java new file mode 100644 index 0000000000000..23a77683d993e --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyReporter.java @@ -0,0 +1,47 @@ +/* + * 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.metrics.util; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.reporter.MetricReporter; + +import java.util.List; + +public class DummyReporter implements MetricReporter { + @Override + public void open(Configuration config) { + } + + @Override + public void close() { + } + + @Override + public void notifyOfAddedMetric(Metric metric, String name) { + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String name) { + } + + @Override + public String generateName(String name, List scope) { + return ""; + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskManagerMetricGroup.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskManagerMetricGroup.java new file mode 100644 index 0000000000000..1c7d33bc4db64 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskManagerMetricGroup.java @@ -0,0 +1,48 @@ +/* + * 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.metrics.util; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.TaskManagerMetricGroup; + +public class DummyTaskManagerMetricGroup extends TaskManagerMetricGroup { + public DummyTaskManagerMetricGroup() { + super(new DummyMetricRegistry(), "host", "id"); + } + + public DummyJobMetricGroup addJob(JobID id, String name) { + return new DummyJobMetricGroup(); + } + + @Override + protected MetricGroup addMetric(String name, Metric metric) { + return this; + } + + @Override + public MetricGroup addGroup(int name) { + return addGroup("" + name); + } + + @Override + public MetricGroup addGroup(String name) { + return new DummyMetricGroup(); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskMetricGroup.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskMetricGroup.java new file mode 100644 index 0000000000000..53683f4f8430b --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskMetricGroup.java @@ -0,0 +1,48 @@ +/* + * 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.metrics.util; + +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.TaskMetricGroup; +import org.apache.flink.util.AbstractID; + +public class DummyTaskMetricGroup extends TaskMetricGroup { + public DummyTaskMetricGroup() { + super(new DummyMetricRegistry(), new DummyJobMetricGroup(), new AbstractID(), new AbstractID(), 0, "task"); + } + + public DummyOperatorMetricGroup addOperator(String name) { + return new DummyOperatorMetricGroup(); + } + + @Override + protected MetricGroup addMetric(String name, Metric metric) { + return this; + } + + @Override + public MetricGroup addGroup(int name) { + return addGroup("" + name); + } + + @Override + public MetricGroup addGroup(String name) { + return new DummyMetricGroup(); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/TestReporter.java b/flink-core/src/test/java/org/apache/flink/metrics/util/TestReporter.java new file mode 100644 index 0000000000000..482d1e8aa22c7 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/TestReporter.java @@ -0,0 +1,40 @@ +/* + * 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.metrics.util; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.reporter.AbstractReporter; +import org.apache.flink.metrics.reporter.MetricReporter; + +import java.util.List; + +public class TestReporter extends AbstractReporter { + @Override + public void open(Configuration config) { + } + + @Override + public void close() { + } + + @Override + public String generateName(String name, List scope) { + return name; + } +} diff --git a/flink-dist/src/main/flink-bin/bin/config.sh b/flink-dist/src/main/flink-bin/bin/config.sh index ffbec07b71bab..b6bdbed805de7 100755 --- a/flink-dist/src/main/flink-bin/bin/config.sh +++ b/flink-dist/src/main/flink-bin/bin/config.sh @@ -103,6 +103,8 @@ KEY_ENV_SSH_OPTS="env.ssh.opts" KEY_RECOVERY_MODE="recovery.mode" KEY_ZK_HEAP_MB="zookeeper.heap.mb" +KEY_METRICS_JMX_PORT="metrics.jmx.port" + ######################################################################################################################## # PATHS AND CONFIG ######################################################################################################################## @@ -240,6 +242,10 @@ if [ -z "${RECOVERY_MODE}" ]; then RECOVERY_MODE=$(readFromConfig ${KEY_RECOVERY_MODE} "standalone" "${YAML_CONF}") fi +if [ -z "${JMX_PORT}" ]; then + JMX_PORT=$(readFromConfig ${KEY_METRICS_JMX_PORT} 9010 "${YAML_CONF}") +fi + # Arguments for the JVM. Used for job and task manager JVMs. # DO NOT USE FOR MEMORY SETTINGS! Use conf/flink-conf.yaml with keys # KEY_JOBM_MEM_SIZE and KEY_TASKM_MEM_SIZE for that! diff --git a/flink-dist/src/main/flink-bin/bin/flink-daemon.sh b/flink-dist/src/main/flink-bin/bin/flink-daemon.sh index 1ef743945443a..cc7163f254361 100644 --- a/flink-dist/src/main/flink-bin/bin/flink-daemon.sh +++ b/flink-dist/src/main/flink-bin/bin/flink-daemon.sh @@ -23,14 +23,24 @@ USAGE="Usage: flink-daemon.sh (start|stop|stop-all) (jobmanager|taskmanager|zook STARTSTOP=$1 DAEMON=$2 ARGS=("${@:3}") # get remaining arguments as array +JMX_ARGS="" + +bin=`dirname "$0"` +bin=`cd "$bin"; pwd` + +. "$bin"/config.sh case $DAEMON in (jobmanager) CLASS_TO_RUN=org.apache.flink.runtime.jobmanager.JobManager + if [ "${ARGS[3]}" == "local" ]; then + JMX_ARGS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=${JMX_PORT} -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false" + fi ;; (taskmanager) CLASS_TO_RUN=org.apache.flink.runtime.taskmanager.TaskManager + JMX_ARGS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=${JMX_PORT} -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false" ;; (zookeeper) @@ -43,11 +53,6 @@ case $DAEMON in ;; esac -bin=`dirname "$0"` -bin=`cd "$bin"; pwd` - -. "$bin"/config.sh - if [ "$FLINK_IDENT_STRING" = "" ]; then FLINK_IDENT_STRING="$USER" fi @@ -96,12 +101,13 @@ case $STARTSTOP in count="${#active[@]}" if [ ${count} -gt 0 ]; then + JMX_ARGS="" echo "[INFO] $count instance(s) of $DAEMON are already running on $HOSTNAME." fi fi echo "Starting $DAEMON daemon on host $HOSTNAME." - $JAVA_RUN $JVM_ARGS ${FLINK_ENV_JAVA_OPTS} "${log_setting[@]}" -classpath "`manglePathList "$FLINK_TM_CLASSPATH:$INTERNAL_HADOOP_CLASSPATHS"`" ${CLASS_TO_RUN} "${ARGS[@]}" > "$out" 2>&1 < /dev/null & + $JAVA_RUN $JVM_ARGS ${FLINK_ENV_JAVA_OPTS} ${JMX_ARGS} "${log_setting[@]}" -classpath "`manglePathList "$FLINK_TM_CLASSPATH:$INTERNAL_HADOOP_CLASSPATHS"`" ${CLASS_TO_RUN} "${ARGS[@]}" > "$out" 2>&1 < /dev/null & mypid=$! diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java index 8ce83f3415665..31b3ba2af8cd2 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java @@ -31,6 +31,7 @@ import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.util.Collector; import org.junit.Assert; import org.junit.Test; @@ -76,7 +77,7 @@ public void testExecuteOnCollection() { final HashMap> accumulators = new HashMap>(); final HashMap> cpTasks = new HashMap<>(); final TaskInfo taskInfo = new TaskInfo("Test UDF", 0, 4, 0); - final RuntimeContext ctx = new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulators); + final RuntimeContext ctx = new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulators, new DummyMetricGroup()); { SumCoGroup udf1 = new SumCoGroup(); diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java index a5632812a8b7f..1b627c403ce15 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java @@ -30,6 +30,7 @@ import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.util.Collector; import org.junit.Test; @@ -168,9 +169,9 @@ public void close() throws Exception { ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.disableObjectReuse(); - List> resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + List> resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); executionConfig.enableObjectReuse(); - List> resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + List> resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); Set> resultSetMutableSafe = new HashSet>(resultMutableSafe); diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java index a6b3debf83be4..1d5668bd9cb8c 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java @@ -30,6 +30,7 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.util.Collector; import org.junit.Test; @@ -107,9 +108,9 @@ public void join(Tuple3 first, Tuple2 final TaskInfo taskInfo = new TaskInfo("op", 0, 1, 0); ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.disableObjectReuse(); - List> resultSafe = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + List> resultSafe = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); executionConfig.enableObjectReuse(); - List> resultRegular = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + List> resultRegular = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); assertEquals(expected, new HashSet<>(resultSafe)); assertEquals(expected, new HashSet<>(resultRegular)); diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java index c04916d0477ab..4317c03ffaf8f 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java @@ -30,6 +30,7 @@ import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.junit.Test; import java.util.ArrayList; @@ -145,9 +146,9 @@ public void close() throws Exception { ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.disableObjectReuse(); - List> resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + List> resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); executionConfig.enableObjectReuse(); - List> resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + List> resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); Set> resultSetMutableSafe = new HashSet>(resultMutableSafe); Set> resultSetRegular = new HashSet>(resultRegular); diff --git a/flink-metrics/flink-metrics-dropwizard/pom.xml b/flink-metrics/flink-metrics-dropwizard/pom.xml new file mode 100644 index 0000000000000..84d97228c08f6 --- /dev/null +++ b/flink-metrics/flink-metrics-dropwizard/pom.xml @@ -0,0 +1,72 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-metrics + 1.1-SNAPSHOT + .. + + + flink-metrics-dropwizard + flink-metrics-dropwizard + + + + org.apache.flink + flink-core + ${project.version} + provided + + + + io.dropwizard.metrics + metrics-core + ${metrics.version} + + + + + + + maven-assembly-plugin + 2.4 + + + jar-with-dependencies + + + + + make-assembly + package + + single + + + + + + + diff --git a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java new file mode 100644 index 0000000000000..a7309be39aa9a --- /dev/null +++ b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java @@ -0,0 +1,99 @@ +/* + * 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.dropwizard; + +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.ScheduledReporter; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.dropwizard.metrics.CounterWrapper; +import org.apache.flink.dropwizard.metrics.GaugeWrapper; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.metrics.reporter.Scheduled; + +import java.util.List; + +/** + * Base class for {@link org.apache.flink.metrics.reporter.MetricReporter} that wraps a + * Dropwizard {@link com.codahale.metrics.Reporter}. + */ +@PublicEvolving +public abstract class ScheduledDropwizardReporter implements MetricReporter, Scheduled { + protected MetricRegistry registry; + protected ScheduledReporter reporter; + + public static final String ARG_HOST = "host"; + public static final String ARG_PORT = "port"; + public static final String ARG_PREFIX = "prefix"; + public static final String ARG_CONVERSION_RATE = "rateConversion"; + public static final String ARG_CONVERSION_DURATION = "durationConversion"; + + protected ScheduledDropwizardReporter() { + this.registry = new MetricRegistry(); + } + + @Override + public synchronized void notifyOfAddedMetric(Metric metric, String name) { + if (metric instanceof Counter) { + registry.register(name, new CounterWrapper((Counter) metric)); + } else if (metric instanceof Gauge) { + registry.register(name, new GaugeWrapper((Gauge) metric)); + } + } + + @Override + public synchronized void notifyOfRemovedMetric(Metric metric, String name) { + registry.remove(name); + } + + public abstract ScheduledReporter getReporter(Configuration config); + + @Override + public void open(Configuration config) { + this.reporter = getReporter(config); + } + + @Override + public void close() { + this.reporter.stop(); + } + + @Override + public String generateName(String name, List scope) { + StringBuilder sb = new StringBuilder(); + for (String s : scope) { + sb.append(s); + sb.append('.'); + } + sb.append(name); + return sb.toString(); + } + + @Override + public synchronized void report() { + this.reporter.report( + this.registry.getGauges(), + this.registry.getCounters(), + this.registry.getHistograms(), + this.registry.getMeters(), + this.registry.getTimers()); + } +} diff --git a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/CounterWrapper.java b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/CounterWrapper.java new file mode 100644 index 0000000000000..f6630b9854125 --- /dev/null +++ b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/CounterWrapper.java @@ -0,0 +1,33 @@ +/* + * 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.dropwizard.metrics; + +import org.apache.flink.metrics.Counter; + +public class CounterWrapper extends com.codahale.metrics.Counter { + private final Counter counter; + + public CounterWrapper(Counter counter) { + this.counter = counter; + } + + @Override + public long getCount() { + return this.counter.getCount(); + } +} diff --git a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/GaugeWrapper.java b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/GaugeWrapper.java new file mode 100644 index 0000000000000..d47090d8b36d5 --- /dev/null +++ b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/GaugeWrapper.java @@ -0,0 +1,33 @@ +/* + * 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.dropwizard.metrics; + +import org.apache.flink.metrics.Gauge; + +public class GaugeWrapper implements com.codahale.metrics.Gauge { + private final Gauge gauge; + + public GaugeWrapper(Gauge gauge) { + this.gauge = gauge; + } + + @Override + public Object getValue() { + return this.gauge.getValue(); + } +} diff --git a/flink-metrics/flink-metrics-ganglia/pom.xml b/flink-metrics/flink-metrics-ganglia/pom.xml new file mode 100644 index 0000000000000..c4f51da9a2cec --- /dev/null +++ b/flink-metrics/flink-metrics-ganglia/pom.xml @@ -0,0 +1,90 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-metrics + 1.1-SNAPSHOT + .. + + + flink-metrics-ganglia + flink-metrics-ganglia + + + + org.apache.flink + flink-core + ${project.version} + provided + + + + org.apache.flink + flink-metrics-dropwizard + ${project.version} + + + + info.ganglia.gmetric4j + gmetric4j + 1.0.7 + + + + io.dropwizard.metrics + metrics-core + ${metrics.version} + + + + io.dropwizard.metrics + metrics-ganglia + ${metrics.version} + + + + + + + maven-assembly-plugin + 2.4 + + + jar-with-dependencies + + + + + make-assembly + package + + single + + + + + + + diff --git a/flink-metrics/flink-metrics-ganglia/src/main/java/org/apache/flink/metrics/graphite/GangliaReporter.java b/flink-metrics/flink-metrics-ganglia/src/main/java/org/apache/flink/metrics/graphite/GangliaReporter.java new file mode 100644 index 0000000000000..a1dafc9c71a7c --- /dev/null +++ b/flink-metrics/flink-metrics-ganglia/src/main/java/org/apache/flink/metrics/graphite/GangliaReporter.java @@ -0,0 +1,73 @@ +/* + * 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.metrics.graphite; + +import com.codahale.metrics.ScheduledReporter; +import info.ganglia.gmetric4j.gmetric.GMetric; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.dropwizard.ScheduledDropwizardReporter; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +public class GangliaReporter extends ScheduledDropwizardReporter { + public static final String ARG_DMAX = "dmax"; + public static final String ARG_TMAX = "tmax"; + public static final String ARG_TTL = "ttl"; + public static final String ARG_MODE_ADDRESSING = "addressingMode"; + + @Override + public ScheduledReporter getReporter(Configuration config) { + + try { + String host = config.getString(ARG_HOST, null); + int port = config.getInteger(ARG_PORT, -1); + if (host == null || host.length() == 0 || port < 1) { + throw new IllegalArgumentException("Invalid host/port configuration. Host: " + host + " Port: " + port); + } + String addressingMode = config.getString(ARG_MODE_ADDRESSING, "MULTICAST"); + int ttl = config.getInteger(ARG_TTL, -1); + GMetric gMetric = new GMetric(host, port, GMetric.UDPAddressingMode.valueOf(addressingMode), ttl); + + String prefix = config.getString(ARG_PREFIX, null); + String conversionRate = config.getString(ARG_CONVERSION_RATE, null); + String conversionDuration = config.getString(ARG_CONVERSION_DURATION, null); + int dMax = config.getInteger(ARG_DMAX, 0); + int tMax = config.getInteger(ARG_TMAX, 60); + + com.codahale.metrics.ganglia.GangliaReporter.Builder builder = + com.codahale.metrics.ganglia.GangliaReporter.forRegistry(registry); + + if (prefix != null) { + builder.prefixedWith(prefix); + } + if (conversionRate != null) { + builder.convertRatesTo(TimeUnit.valueOf(conversionRate)); + } + if (conversionDuration != null) { + builder.convertDurationsTo(TimeUnit.valueOf(conversionDuration)); + } + builder.withDMax(dMax); + builder.withTMax(tMax); + + return builder.build(gMetric); + } catch (IOException e) { + throw new RuntimeException("Error while instantiating GangliaReporter.", e); + } + } +} diff --git a/flink-metrics/flink-metrics-graphite/pom.xml b/flink-metrics/flink-metrics-graphite/pom.xml new file mode 100644 index 0000000000000..45fb01857df4c --- /dev/null +++ b/flink-metrics/flink-metrics-graphite/pom.xml @@ -0,0 +1,84 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-metrics + 1.1-SNAPSHOT + .. + + + flink-metrics-graphite + flink-metrics-graphite + + + + org.apache.flink + flink-core + ${project.version} + provided + + + + org.apache.flink + flink-metrics-dropwizard + ${project.version} + + + + io.dropwizard.metrics + metrics-core + ${metrics.version} + + + + io.dropwizard.metrics + metrics-graphite + ${metrics.version} + + + + + + + maven-assembly-plugin + 2.4 + + + jar-with-dependencies + + + + + make-assembly + package + + single + + + + + + + diff --git a/flink-metrics/flink-metrics-graphite/src/main/java/org/apache/flink/metrics/graphite/GraphiteReporter.java b/flink-metrics/flink-metrics-graphite/src/main/java/org/apache/flink/metrics/graphite/GraphiteReporter.java new file mode 100644 index 0000000000000..b28d7a4904643 --- /dev/null +++ b/flink-metrics/flink-metrics-graphite/src/main/java/org/apache/flink/metrics/graphite/GraphiteReporter.java @@ -0,0 +1,70 @@ +/* + * 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.metrics.graphite; + +import com.codahale.metrics.ScheduledReporter; +import com.codahale.metrics.graphite.Graphite; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.dropwizard.ScheduledDropwizardReporter; + +import java.util.List; +import java.util.concurrent.TimeUnit; + +public class GraphiteReporter extends ScheduledDropwizardReporter { + @Override + public ScheduledReporter getReporter(Configuration config) { + String host = config.getString(ARG_HOST, null); + int port = config.getInteger(ARG_PORT, -1); + + if (host == null || host.length() == 0 || port < 1) { + throw new IllegalArgumentException("Invalid host/port configuration. Host: " + host + " Port: " + port); + } + + String prefix = config.getString(ARG_PREFIX, null); + String conversionRate = config.getString(ARG_CONVERSION_RATE, null); + String conversionDuration = config.getString(ARG_CONVERSION_DURATION, null); + + com.codahale.metrics.graphite.GraphiteReporter.Builder builder = + com.codahale.metrics.graphite.GraphiteReporter.forRegistry(registry); + + if (prefix != null) { + builder.prefixedWith(prefix); + } + + if (conversionRate != null) { + builder.convertRatesTo(TimeUnit.valueOf(conversionRate)); + } + + if (conversionDuration != null) { + builder.convertDurationsTo(TimeUnit.valueOf(conversionDuration)); + } + + return builder.build(new Graphite(host, port)); + } + + @Override + public String generateName(String name, List scope) { + StringBuilder sb = new StringBuilder(); + for (String s : scope) { + sb.append(s.replace(".", "_").replace("\"", "")); + sb.append("."); + } + sb.append(name); + return sb.toString(); + } +} diff --git a/flink-metrics/flink-metrics-statsd/pom.xml b/flink-metrics/flink-metrics-statsd/pom.xml new file mode 100644 index 0000000000000..5d639087067d8 --- /dev/null +++ b/flink-metrics/flink-metrics-statsd/pom.xml @@ -0,0 +1,43 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-metrics + 1.1-SNAPSHOT + .. + + + flink-metrics-statsd + flink-metrics-statsd + + + + org.apache.flink + flink-core + ${project.version} + provided + + + diff --git a/flink-metrics/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java b/flink-metrics/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java new file mode 100644 index 0000000000000..288b8b86aa99b --- /dev/null +++ b/flink-metrics/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java @@ -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. + */ +package org.apache.flink.metrics.statsd; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.reporter.AbstractReporter; +import org.apache.flink.metrics.reporter.Scheduled; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.DatagramPacket; +import java.net.DatagramSocket; +import java.net.InetSocketAddress; +import java.net.SocketException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * Largely based on the StatsDReporter class by ReadyTalk + * https://github.com/ReadyTalk/metrics-statsd/blob/master/metrics3-statsd/src/main/java/com/readytalk/metrics/StatsDReporter.java + * + * Ported since it was not present in maven central. + */ +public class StatsDReporter extends AbstractReporter implements Scheduled { + private static final Logger LOG = LoggerFactory.getLogger(StatsDReporter.class); + + public static final String ARG_HOST = "host"; + public static final String ARG_PORT = "port"; + public static final String ARG_CONVERSION_RATE = "rateConversion"; + public static final String ARG_CONVERSION_DURATION = "durationConversion"; + + private DatagramSocket socket; + private InetSocketAddress address; + + private double durationFactor; + private double rateFactor; + + @Override + public void open(Configuration config) { + String host = config.getString(ARG_HOST, null); + int port = config.getInteger(ARG_PORT, -1); + + if (host == null || host.length() == 0 || port < 1) { + throw new IllegalArgumentException("Invalid host/port configuration. Host: " + host + " Port: " + port); + } + + String conversionRate = config.getString(ARG_CONVERSION_RATE, "SECONDS"); + String conversionDuration = config.getString(ARG_CONVERSION_DURATION, "MILLISECONDS"); + + this.address = new InetSocketAddress(host, port); + this.rateFactor = TimeUnit.valueOf(conversionRate).toSeconds(1); + this.durationFactor = 1.0 / TimeUnit.valueOf(conversionDuration).toNanos(1); + try { + this.socket = new DatagramSocket(0); + } catch (SocketException e) { + throw new RuntimeException("Failure while creating socket. ", e); + } + } + + @Override + public void close() { + if (socket != null && !socket.isClosed()) { + socket.close(); + } + } + + @Override + public String generateName(String name, List scope) { + StringBuilder sb = new StringBuilder(); + for (String s : scope) { + sb.append(s); + sb.append('.'); + } + sb.append(name); + return sb.toString(); + } + + public void send(final String name, final double value) { + send(name, "" + value); + } + + public void send(final String name, final String value) { + try { + String formatted = String.format("%s:%s|g", name, value); + byte[] data = formatted.getBytes(); + socket.send(new DatagramPacket(data, data.length, this.address)); + } catch (IOException e) { + LOG.error("unable to send packet to statsd at '{}:{}'", address.getHostName(), address.getPort()); + } + } + + @Override + public void report() { + for (Map.Entry entry : gauges.entrySet()) { + reportGauge(entry.getKey(), entry.getValue()); + } + + for (Map.Entry entry : counters.entrySet()) { + reportCounter(entry.getKey(), entry.getValue()); + } + } + + private void reportCounter(final String name, final Counter counter) { + send(name, counter.getCount()); + } + + private void reportGauge(final String name, final Gauge gauge) { + final String value = gauge.getValue().toString(); + if (value != null) { + send((name), value); + } + } +} diff --git a/flink-metrics/pom.xml b/flink-metrics/pom.xml new file mode 100644 index 0000000000000..542f49c26bc32 --- /dev/null +++ b/flink-metrics/pom.xml @@ -0,0 +1,42 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-parent + 1.1-SNAPSHOT + .. + + + flink-metrics + flink-metrics + pom + + + flink-metrics-dropwizard + flink-metrics-ganglia + flink-metrics-graphite + flink-metrics-statsd + + diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index 13c4fa51d0c86..8007428c0073b 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -34,10 +34,6 @@ under the License. jar - - 3.1.0 - - org.apache.flink diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java index 2b1c224812877..d3ebc952b98a2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java @@ -45,6 +45,7 @@ public final class TaskDeploymentDescriptor implements Serializable { /** The ID of the job the tasks belongs to. */ private final JobID jobID; + private final String jobName; /** The task's job vertex ID. */ private final JobVertexID vertexID; @@ -99,6 +100,7 @@ public final class TaskDeploymentDescriptor implements Serializable { */ public TaskDeploymentDescriptor( JobID jobID, + String jobName, JobVertexID vertexID, ExecutionAttemptID executionId, SerializedValue serializedExecutionConfig, @@ -123,6 +125,7 @@ public TaskDeploymentDescriptor( checkArgument(attemptNumber >= 0); this.jobID = checkNotNull(jobID); + this.jobName = checkNotNull(jobName); this.vertexID = checkNotNull(vertexID); this.executionId = checkNotNull(executionId); this.serializedExecutionConfig = checkNotNull(serializedExecutionConfig); @@ -144,6 +147,7 @@ public TaskDeploymentDescriptor( public TaskDeploymentDescriptor( JobID jobID, + String jobName, JobVertexID vertexID, ExecutionAttemptID executionId, SerializedValue serializedExecutionConfig, @@ -162,6 +166,7 @@ public TaskDeploymentDescriptor( this( jobID, + jobName, vertexID, executionId, serializedExecutionConfig, @@ -195,6 +200,8 @@ public SerializedValue getSerializedExecutionConfig() { public JobID getJobID() { return jobID; } + + public String getJobName() { return jobName; } /** * Returns the task's execution vertex ID. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java index a10c463a74440..121936c82a1bd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -88,6 +89,13 @@ public interface Environment { */ TaskManagerRuntimeInfo getTaskManagerInfo(); + /** + * Returns the task specific metric group. + * + * @return The MetricGroup of this task. + */ + TaskMetricGroup getMetricGroup(); + /** * Returns the job-wide configuration object that was attached to the JobGraph. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index cbc47a4aba89c..a85f32ac2eec6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -673,6 +673,7 @@ TaskDeploymentDescriptor createDeploymentDescriptor( return new TaskDeploymentDescriptor( getJobId(), + getExecutionGraph().getJobName(), getJobvertexId(), executionId, serializedConfig, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java index 7aa57a8ec339b..a784f5444ca7d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.io.network.api.reader; import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer; import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult; @@ -122,4 +123,11 @@ public void setReporter(AccumulatorRegistry.Reporter reporter) { deserializer.setReporter(reporter); } } + + @Override + public void setMetricGroup(IOMetricGroup metrics) { + for (RecordDeserializer deserializer : recordDeserializers) { + deserializer.instantiateMetrics(metrics); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/BufferReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/BufferReader.java index debb352589bcf..e5f5930698785 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/BufferReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/BufferReader.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.api.reader; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; @@ -53,4 +54,8 @@ public Buffer getNextBuffer() throws IOException, InterruptedException { public void setReporter(AccumulatorRegistry.Reporter reporter) { } + + @Override + public void setMetricGroup(IOMetricGroup metrics) { + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/ReaderBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/ReaderBase.java index a1d705f47a151..192a9abed5df5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/ReaderBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/ReaderBase.java @@ -20,6 +20,7 @@ import java.io.IOException; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.util.event.EventListener; @@ -57,4 +58,11 @@ public interface ReaderBase { */ void setReporter(AccumulatorRegistry.Reporter reporter); + /** + * Setter for the metric group. + * + * @param metrics metric group to set + */ + void setMetricGroup(IOMetricGroup metrics); + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java index cdd8731c3a74c..1c1747601b8fc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java @@ -21,6 +21,8 @@ import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.util.DataInputDeserializer; @@ -47,6 +49,9 @@ public class AdaptiveSpanningRecordDeserializer im private Buffer currentBuffer; private AccumulatorRegistry.Reporter reporter; + + private transient Counter numRecordsIn; + private transient Counter numBytesIn; public AdaptiveSpanningRecordDeserializer() { this.nonSpanningWrapper = new NonSpanningWrapper(); @@ -96,6 +101,9 @@ public DeserializationResult getNextRecord(T target) throws IOException { if (reporter != null) { reporter.reportNumBytesIn(len); } + if (numBytesIn != null) { + numBytesIn.inc(len); + } if (len <= nonSpanningRemaining - 4) { // we can get a full record from here @@ -104,6 +112,9 @@ public DeserializationResult getNextRecord(T target) throws IOException { if (reporter != null) { reporter.reportNumRecordsIn(1); } + if (numRecordsIn != null) { + numRecordsIn.inc(); + } return (this.nonSpanningWrapper.remaining() == 0) ? DeserializationResult.LAST_RECORD_FROM_BUFFER : @@ -131,6 +142,9 @@ public DeserializationResult getNextRecord(T target) throws IOException { if (reporter != null) { reporter.reportNumRecordsIn(1); } + if (numRecordsIn != null) { + numRecordsIn.inc(); + } // move the remainder to the non-spanning wrapper // this does not copy it, only sets the memory segment @@ -165,6 +179,12 @@ public void setReporter(AccumulatorRegistry.Reporter reporter) { this.spanningWrapper.setReporter(reporter); } + @Override + public void instantiateMetrics(IOMetricGroup metrics) { + numBytesIn = metrics.getBytesInCounter(); + numRecordsIn = metrics.getRecordsInCounter(); + } + // ----------------------------------------------------------------------------------------------------------------- private static final class NonSpanningWrapper implements DataInputView { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordDeserializer.java index e4c7890bd69e6..2f0c1ac65486e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordDeserializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordDeserializer.java @@ -23,6 +23,7 @@ import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.io.network.buffer.Buffer; @@ -70,4 +71,11 @@ public boolean isBufferConsumed() { * Setter for the reporter, e.g. for the number of records emitted and the number of bytes read. */ void setReporter(AccumulatorRegistry.Reporter reporter); + + /** + * Instantiates all metrics. + * + * @param metrics metric group + */ + void instantiateMetrics(IOMetricGroup metrics); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordSerializer.java index e9f339a716d97..9e7dfc1f3cf7e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordSerializer.java @@ -22,6 +22,7 @@ import java.io.IOException; import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.io.network.buffer.Buffer; @@ -69,4 +70,11 @@ public boolean isFullBuffer() { * Setter for the reporter, e.g. for the number of records emitted and the number of bytes read. */ void setReporter(AccumulatorRegistry.Reporter reporter); + + /** + * Insantiates all metrics. + * + * @param metrics metric group + */ + void instantiateMetrics(IOMetricGroup metrics); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializer.java index f163e050c9fb6..64956507f7714 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializer.java @@ -24,6 +24,8 @@ import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.util.DataOutputSerializer; @@ -53,6 +55,9 @@ public class SpanningRecordSerializer implements R private AccumulatorRegistry.Reporter reporter; + private transient Counter numRecordsOut; + private transient Counter numBytesOut; + public SpanningRecordSerializer() { this.serializationBuffer = new DataOutputSerializer(128); @@ -85,6 +90,14 @@ public SerializationResult addRecord(T record) throws IOException { reporter.reportNumBytesOut(len); reporter.reportNumRecordsOut(1); } + + if (numBytesOut != null) { + numBytesOut.inc(len); + } + + if (numRecordsOut != null) { + numRecordsOut.inc(); + } this.dataBuffer = this.serializationBuffer.wrapAsByteBuffer(); @@ -187,4 +200,10 @@ public boolean hasData() { public void setReporter(AccumulatorRegistry.Reporter reporter) { this.reporter = reporter; } + + @Override + public void instantiateMetrics(IOMetricGroup metrics) { + numBytesOut = metrics.getBytesOutCounter(); + numRecordsOut = metrics.getRecordsOutCounter(); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java index 6a4692e7076b2..49f7584497c85 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java @@ -24,6 +24,8 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.util.DataInputDeserializer; @@ -63,6 +65,9 @@ public class SpillingAdaptiveSpanningRecordDeserializer 0) { @@ -165,6 +176,9 @@ else if (remaining == 0) { if (reporter != null) { reporter.reportNumRecordsIn(1); } + if (numRecordsIn != null) { + numRecordsIn.inc(); + } // move the remainder to the non-spanning wrapper // this does not copy it, only sets the memory segment @@ -196,6 +210,13 @@ public void setReporter(AccumulatorRegistry.Reporter reporter) { this.spanningWrapper.setReporter(reporter); } + @Override + public void instantiateMetrics(IOMetricGroup metrics) { + numBytesIn = metrics.getBytesInCounter(); + numRecordsIn = metrics.getRecordsInCounter(); + } + + // ----------------------------------------------------------------------------------------------------------------- private static final class NonSpanningWrapper implements DataInputView { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java index c534aa2188d28..f93cdfcd5343f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.io.network.api.writer; import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer; @@ -209,6 +210,16 @@ public void setReporter(AccumulatorRegistry.Reporter reporter) { } } + /** + * Sets the metric group for this RecordWriter. + * @param metrics + */ + public void setMetricGroup(IOMetricGroup metrics) { + for(RecordSerializer serializer : serializers) { + serializer.instantiateMetrics(metrics); + } + } + /** * Writes the buffer to the {@link ResultPartitionWriter}. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativeTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativeTask.java index 434f7d4e4ed96..047fd7ea0b84f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativeTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativeTask.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.operators.BatchTask; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -169,10 +170,10 @@ protected void closeLocalStrategiesAndCaches() { } @Override - public DistributedRuntimeUDFContext createRuntimeContext() { + public DistributedRuntimeUDFContext createRuntimeContext(MetricGroup metrics) { Environment env = getEnvironment(); return new IterativeRuntimeUdfContext(env.getTaskInfo(), getUserCodeClassLoader(), - getExecutionConfig(), env.getDistributedCacheEntries(), this.accumulatorMap); + getExecutionConfig(), env.getDistributedCacheEntries(), this.accumulatorMap, metrics); } // -------------------------------------------------------------------------------------------- @@ -362,8 +363,8 @@ private TypeSerializer getOutputSerializer() { private class IterativeRuntimeUdfContext extends DistributedRuntimeUDFContext implements IterationRuntimeContext { public IterativeRuntimeUdfContext(TaskInfo taskInfo, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig, - Map> cpTasks, Map> accumulatorMap) { - super(taskInfo, userCodeClassLoader, executionConfig, cpTasks, accumulatorMap); + Map> cpTasks, Map> accumulatorMap, MetricGroup metrics) { + super(taskInfo, userCodeClassLoader, executionConfig, cpTasks, accumulatorMap, metrics); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java index 51e00b38597b5..546193c4892d6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java @@ -30,6 +30,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableMaterialization; import org.apache.flink.runtime.execution.CancelTaskException; @@ -212,6 +213,7 @@ public class BatchTask extends AbstractInvokable impleme * The accumulator map used in the RuntimeContext. */ protected Map> accumulatorMap; + private MetricGroup metrics; // -------------------------------------------------------------------------------------------- // Task Interface @@ -237,6 +239,10 @@ public void invoke() throws Exception { final Class> driverClass = this.config.getDriver(); this.driver = InstantiationUtil.instantiate(driverClass, Driver.class); + String headName = getEnvironment().getTaskInfo().getTaskName().split("->")[0].trim(); + this.metrics = getEnvironment().getMetricGroup() + .addOperator(headName.startsWith("CHAIN") ? headName.substring(6) : headName); + // initialize the readers. // this does not yet trigger any stream consuming or processing. initInputReaders(); @@ -256,7 +262,7 @@ public void invoke() throws Exception { LOG.debug(formatLogString("Start task code.")); } - this.runtimeUdfContext = createRuntimeContext(); + this.runtimeUdfContext = createRuntimeContext(metrics); // whatever happens in this scope, make sure that the local strategies are cleaned up! // note that the initialization of the local strategies is in the try-finally block as well, @@ -666,6 +672,7 @@ protected void initInputReaders() throws Exception { } inputReaders[i].setReporter(reporter); + inputReaders[i].setMetricGroup(getEnvironment().getMetricGroup().getIOMetricGroup()); currentReaderOffset += groupSize; } @@ -1005,11 +1012,11 @@ protected void initOutputs() throws Exception { this.getExecutionConfig(), reporter, this.accumulatorMap); } - public DistributedRuntimeUDFContext createRuntimeContext() { + public DistributedRuntimeUDFContext createRuntimeContext(MetricGroup metrics) { Environment env = getEnvironment(); return new DistributedRuntimeUDFContext(env.getTaskInfo(), getUserCodeClassLoader(), - getExecutionConfig(), env.getDistributedCacheEntries(), this.accumulatorMap); + getExecutionConfig(), env.getDistributedCacheEntries(), this.accumulatorMap, metrics); } // -------------------------------------------------------------------------------------------- @@ -1056,6 +1063,11 @@ public String formatLogString(String message) { return constructLogString(message, getEnvironment().getTaskInfo().getTaskName(), this); } + @Override + public MetricGroup getMetricGroup() { + return metrics; + } + @Override public MutableObjectIterator getInput(int index) { if (index < 0 || index > this.driver.getNumberOfInputs()) { @@ -1226,6 +1238,7 @@ public static Collector getOutputCollector(AbstractInvokable task, TaskCo // setup live accumulator counters recordWriter.setReporter(reporter); + recordWriter.setMetricGroup(task.getEnvironment().getMetricGroup().getIOMetricGroup()); writers.add(recordWriter); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java index 21e87843bbd74..39bf23f126167 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java @@ -344,6 +344,8 @@ private void initInputReaders() throws Exception { final AccumulatorRegistry.Reporter reporter = accumulatorRegistry.getReadWriteReporter(); inputReader.setReporter(reporter); + + inputReader.setMetricGroup(getEnvironment().getMetricGroup().getIOMetricGroup()); this.inputTypeSerializerFactory = this.config.getInputSerializer(0, getUserCodeClassLoader()); @SuppressWarnings({ "rawtypes" }) @@ -375,6 +377,7 @@ public DistributedRuntimeUDFContext createRuntimeContext() { Environment env = getEnvironment(); return new DistributedRuntimeUDFContext(env.getTaskInfo(), getUserCodeClassLoader(), - getExecutionConfig(), env.getDistributedCacheEntries(), env.getAccumulatorRegistry().getUserMap()); + getExecutionConfig(), env.getDistributedCacheEntries(), env.getAccumulatorRegistry().getUserMap(), + getEnvironment().getMetricGroup().addOperator(getEnvironment().getTaskInfo().getTaskName())); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java index 0c525ea80978e..819b84f99dd5f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java @@ -358,7 +358,10 @@ public void remove() { public DistributedRuntimeUDFContext createRuntimeContext() { Environment env = getEnvironment(); + String sourceName = getEnvironment().getTaskInfo().getTaskName().split("->")[0].trim(); + sourceName = sourceName.startsWith("CHAIN") ? sourceName.substring(6) : sourceName; return new DistributedRuntimeUDFContext(env.getTaskInfo(), getUserCodeClassLoader(), - getExecutionConfig(), env.getDistributedCacheEntries(), env.getAccumulatorRegistry().getUserMap()); + getExecutionConfig(), env.getDistributedCacheEntries(), env.getAccumulatorRegistry().getUserMap(), + getEnvironment().getMetricGroup().addOperator(sourceName)); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java index 220527d6f8d58..d6825acb102be 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java @@ -134,4 +134,4 @@ public void cleanup() {} public void cancel() { this.running = false; } -} \ No newline at end of file +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TaskContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TaskContext.java index fd5d2385d2338..df225288ac407 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TaskContext.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TaskContext.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; @@ -67,4 +68,6 @@ public interface TaskContext { AbstractInvokable getOwningNepheleTask(); String formatLogString(String message); + + MetricGroup getMetricGroup(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java index 2fb52cde980e2..407716f92f453 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.operators.BatchTask; @@ -50,6 +51,8 @@ public abstract class ChainedDriver implements Collector { protected ExecutionConfig executionConfig; protected boolean objectReuseEnabled = false; + + protected MetricGroup metrics; public void setup(TaskConfig config, String taskName, Collector outputCollector, @@ -60,14 +63,15 @@ public void setup(TaskConfig config, String taskName, Collector outputCollec this.taskName = taskName; this.outputCollector = outputCollector; this.userCodeClassLoader = userCodeClassLoader; + this.metrics = parent.getEnvironment().getMetricGroup().addOperator(taskName); Environment env = parent.getEnvironment(); if (parent instanceof BatchTask) { - this.udfContext = ((BatchTask) parent).createRuntimeContext(); + this.udfContext = ((BatchTask) parent).createRuntimeContext(metrics); } else { this.udfContext = new DistributedRuntimeUDFContext(env.getTaskInfo(), userCodeClassLoader, - parent.getExecutionConfig(), env.getDistributedCacheEntries(), accumulatorMap + parent.getExecutionConfig(), env.getDistributedCacheEntries(), accumulatorMap, metrics ); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/DistributedRuntimeUDFContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/DistributedRuntimeUDFContext.java index b5ac4d788ad5f..293d34f44a283 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/DistributedRuntimeUDFContext.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/DistributedRuntimeUDFContext.java @@ -30,6 +30,7 @@ import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.AbstractRuntimeUDFContext; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.broadcast.BroadcastVariableMaterialization; import org.apache.flink.runtime.broadcast.InitializationTypeConflictException; @@ -43,8 +44,8 @@ public class DistributedRuntimeUDFContext extends AbstractRuntimeUDFContext { private final HashMap> broadcastVars = new HashMap>(); public DistributedRuntimeUDFContext(TaskInfo taskInfo, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig, - Map> cpTasks, Map> accumulators) { - super(taskInfo, userCodeClassLoader, executionConfig, accumulators, cpTasks); + Map> cpTasks, Map> accumulators, MetricGroup metrics) { + super(taskInfo, userCodeClassLoader, executionConfig, accumulators, cpTasks, metrics); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java index 51e7e34b3064f..1f93a0d9bfc80 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.execution.Environment; @@ -75,6 +76,7 @@ public class RuntimeEnvironment implements Environment { private final AccumulatorRegistry accumulatorRegistry; private final TaskManagerRuntimeInfo taskManagerInfo; + private final TaskMetricGroup metrics; // ------------------------------------------------------------------------ @@ -96,7 +98,8 @@ public RuntimeEnvironment( ResultPartitionWriter[] writers, InputGate[] inputGates, ActorGateway jobManager, - TaskManagerRuntimeInfo taskManagerInfo) { + TaskManagerRuntimeInfo taskManagerInfo, + TaskMetricGroup metrics) { this.jobId = checkNotNull(jobId); this.jobVertexId = checkNotNull(jobVertexId); @@ -116,6 +119,7 @@ public RuntimeEnvironment( this.inputGates = checkNotNull(inputGates); this.jobManager = checkNotNull(jobManager); this.taskManagerInfo = checkNotNull(taskManagerInfo); + this.metrics = metrics; } // ------------------------------------------------------------------------ @@ -160,6 +164,11 @@ public TaskManagerRuntimeInfo getTaskManagerInfo() { return taskManagerInfo; } + @Override + public TaskMetricGroup getMetricGroup() { + return metrics; + } + @Override public ClassLoader getUserClassLoader() { return userCodeClassLoader; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 251673fcf6288..bc3ba0d9bc906 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -25,6 +25,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; @@ -201,6 +202,8 @@ public class Task implements Runnable { /** atomic flag that makes sure the invokable is canceled exactly once upon error */ private final AtomicBoolean invokableHasBeenCanceled; + private final TaskMetricGroup metrics; + /** The invokable of this task, if initialized */ private volatile AbstractInvokable invokable; @@ -239,7 +242,8 @@ public Task(TaskDeploymentDescriptor tdd, FiniteDuration actorAskTimeout, LibraryCacheManager libraryCache, FileCache fileCache, - TaskManagerRuntimeInfo taskManagerConfig) + TaskManagerRuntimeInfo taskManagerConfig, + TaskMetricGroup metricGroup) { this.taskInfo = checkNotNull(tdd.getTaskInfo()); this.jobId = checkNotNull(tdd.getJobID()); @@ -274,6 +278,7 @@ public Task(TaskDeploymentDescriptor tdd, this.taskManagerConfig = checkNotNull(taskManagerConfig); this.executionListenerActors = new CopyOnWriteArrayList(); + this.metrics = metricGroup; // create the reader and writer structures @@ -518,7 +523,7 @@ else if (current == ExecutionState.CANCELING) { userCodeClassLoader, memoryManager, ioManager, broadcastVariableManager, accumulatorRegistry, splitProvider, distributedCacheEntries, - writers, inputGates, jobManager, taskManagerConfig); + writers, inputGates, jobManager, taskManagerConfig, metrics); // let the task code create its readers and writers invokable.setEnvironment(env); @@ -683,6 +688,9 @@ else if (STATE_UPDATER.compareAndSet(this, current, ExecutionState.FAILED)) { // remove all of the tasks library resources libraryCache.unregisterTask(jobId, executionId); + + //Uncomment before Merging!!! + //metrics.close(); // remove all files in the distributed cache removeCachedFiles(distributedCacheEntries, fileCache); diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 1c7815c17984b..beb012c9bed1f 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -18,13 +18,14 @@ package org.apache.flink.runtime.taskmanager -import java.io.{FileInputStream, File, IOException} +import java.io.{File, FileInputStream, IOException} import java.lang.management.{ManagementFactory, OperatingSystemMXBean} import java.lang.reflect.Method import java.net.{InetAddress, InetSocketAddress} import java.util import java.util.UUID import java.util.concurrent.TimeUnit +import javax.management.ObjectName import _root_.akka.actor._ import _root_.akka.pattern.ask @@ -36,12 +37,14 @@ import com.fasterxml.jackson.databind.ObjectMapper import grizzled.slf4j.Logger import org.apache.flink.configuration._ import org.apache.flink.core.fs.FileSystem -import org.apache.flink.core.memory.{HybridMemorySegment, HeapMemorySegment, MemorySegmentFactory, MemoryType} +import org.apache.flink.core.memory.{HeapMemorySegment, HybridMemorySegment, MemorySegmentFactory, MemoryType} +import org.apache.flink.metrics.groups.TaskManagerMetricGroup +import org.apache.flink.metrics.{MetricGroup, Gauge => FlinkGauge, MetricRegistry => FlinkMetricRegistry} import org.apache.flink.runtime.accumulators.AccumulatorSnapshot import org.apache.flink.runtime.clusterframework.messages.StopCluster import org.apache.flink.runtime.clusterframework.types.ResourceID import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.blob.{BlobClient, BlobCache, BlobService} +import org.apache.flink.runtime.blob.{BlobCache, BlobClient, BlobService} import org.apache.flink.runtime.broadcast.BroadcastVariableManager import org.apache.flink.runtime.deployment.{InputChannelDeploymentDescriptor, TaskDeploymentDescriptor} import org.apache.flink.runtime.execution.ExecutionState @@ -58,7 +61,7 @@ import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalListener, Leader import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.messages.Messages._ import org.apache.flink.runtime.messages.RegistrationMessages._ -import org.apache.flink.runtime.messages.StackTraceSampleMessages.{SampleTaskStackTrace, ResponseStackTraceSampleFailure, ResponseStackTraceSampleSuccess, StackTraceSampleMessages, TriggerStackTraceSample} +import org.apache.flink.runtime.messages.StackTraceSampleMessages.{ResponseStackTraceSampleFailure, ResponseStackTraceSampleSuccess, SampleTaskStackTrace, StackTraceSampleMessages, TriggerStackTraceSample} import org.apache.flink.runtime.messages.TaskManagerMessages._ import org.apache.flink.runtime.messages.TaskMessages._ import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, NotifyCheckpointComplete, TriggerCheckpoint} @@ -152,6 +155,9 @@ class TaskManager( /** Registry of metrics periodically transmitted to the JobManager */ private val metricRegistry = TaskManager.createMetricsRegistry() + private var metricsRegistry : FlinkMetricRegistry = null + private var taskManagerMetricGroup : TaskManagerMetricGroup = null + /** Metric serialization */ private val metricRegistryMapper: ObjectMapper = new ObjectMapper() .registerModule( @@ -257,6 +263,13 @@ class TaskManager( } catch { case t: Exception => log.error("FileCache did not shutdown properly.", t) } + + try { + //enable this before merging + //metricsRegistry.shutdown() + } catch { + case t: Exception => log.error("MetricRegistry did not shutdown properly.", t) + } log.info(s"Task manager ${self.path} is completely shut down.") } @@ -924,6 +937,14 @@ class TaskManager( else { libraryCacheManager = Some(new FallbackLibraryCacheManager) } + + metricsRegistry = new FlinkMetricRegistry(this.config.configuration); + + taskManagerMetricGroup = + new TaskManagerMetricGroup(metricsRegistry, this.runtimeInfo.getHostname, id.toString) + + TaskManager.instantiateStatusMetrics(taskManagerMetricGroup) + // watch job manager to detect when it dies context.watch(jobManager) @@ -1056,6 +1077,17 @@ class TaskManager( val jobManagerGateway = new AkkaActorGateway(jobManagerActor, leaderSessionID.orNull) val selfGateway = new AkkaActorGateway(self, leaderSessionID.orNull) + var jobName = tdd.getJobName + if (tdd.getJobName.length == 0) { + jobName = tdd.getJobID.toString() + } else { + jobName = tdd.getJobName + } + + val taskMetricGroup = taskManagerMetricGroup + .addJob(tdd.getJobID, jobName) + .addTask(tdd.getVertexID, tdd.getExecutionId, tdd.getIndexInSubtaskGroup, tdd.getTaskName) + val task = new Task( tdd, memoryManager, @@ -1067,7 +1099,8 @@ class TaskManager( config.timeout, libCache, fileCache, - runtimeInfo) + runtimeInfo, + taskMetricGroup) log.info(s"Received task ${task.getTaskInfo.getTaskNameWithSubtasks()}") @@ -2234,4 +2267,130 @@ object TaskManager { }) metricRegistry } + + private def instantiateStatusMetrics(taskManagerMetricGroup: MetricGroup) : Unit = { + val jvm = taskManagerMetricGroup + .addGroup("Status") + .addGroup("JVM") + + instantiateClassLoaderMetrics(jvm.addGroup("ClassLoader")) + instantiateGarbageCollectorMetrics(jvm.addGroup("GarbageCollector")) + instantiateMemoryMetrics(jvm.addGroup("Memory")) + instantiateThreadMetrics(jvm.addGroup("Threads")) + instantiateCPUMetrics(jvm.addGroup("CPU")) + } + + private def instantiateClassLoaderMetrics(metrics: MetricGroup) { + val mxBean = ManagementFactory.getClassLoadingMXBean + + metrics + .gauge("ClassesLoaded", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getTotalLoadedClassCount + }) + metrics.gauge("ClassesUnloaded", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getUnloadedClassCount + }) + } + + private def instantiateGarbageCollectorMetrics(metrics: MetricGroup) { + val garbageCollectors = ManagementFactory.getGarbageCollectorMXBeans + + for (garbageCollector <- garbageCollectors) { + val gcGroup = metrics.addGroup("\"" + garbageCollector.getName + "\"") + gcGroup.gauge("Count", new FlinkGauge[Long] { + override def getValue: Long = garbageCollector.getCollectionCount + }) + gcGroup.gauge("Time", new FlinkGauge[Long] { + override def getValue: Long = garbageCollector.getCollectionTime + }) + } + } + + private def instantiateMemoryMetrics(metrics: MetricGroup) { + val mxBean = ManagementFactory.getMemoryMXBean + val heap = metrics.addGroup("Heap") + heap.gauge("Used", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getHeapMemoryUsage.getUsed + }) + heap.gauge("Committed", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getHeapMemoryUsage.getCommitted + }) + heap.gauge("Max", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getHeapMemoryUsage.getMax + }) + + val nonHeap = metrics.addGroup("NonHeap") + nonHeap.gauge("Used", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getNonHeapMemoryUsage.getUsed + }) + nonHeap.gauge("Committed", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getNonHeapMemoryUsage.getCommitted + }) + nonHeap.gauge("Max", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getNonHeapMemoryUsage.getMax + }) + + val con = ManagementFactory.getPlatformMBeanServer; + + val directObjectName = new ObjectName("java.nio:type=BufferPool,name=direct") + + val direct = metrics.addGroup("Direct") + direct.gauge("Count", new FlinkGauge[Long] { + override def getValue: Long = con + .getAttribute(directObjectName, "Count").asInstanceOf[Long] + }) + direct.gauge("MemoryUsed", new FlinkGauge[Long] { + override def getValue: Long = con + .getAttribute(directObjectName, "MemoryUsed").asInstanceOf[Long] + }) + direct.gauge("TotalCapacity", new FlinkGauge[Long] { + override def getValue: Long = con + .getAttribute(directObjectName, "TotalCapacity").asInstanceOf[Long] + }) + + val mappedObjectName = new ObjectName("java.nio:type=BufferPool,name=direct") + + val mapped = metrics.addGroup("Mapped") + mapped.gauge("Count", new FlinkGauge[Long] { + override def getValue: Long = con + .getAttribute(mappedObjectName, "Count").asInstanceOf[Long] + }) + mapped.gauge("MemoryUsed", new FlinkGauge[Long] { + override def getValue: Long = con + .getAttribute(mappedObjectName, "MemoryUsed").asInstanceOf[Long] + }) + mapped.gauge("TotalCapacity", new FlinkGauge[Long] { + override def getValue: Long = con + .getAttribute(mappedObjectName, "TotalCapacity").asInstanceOf[Long] + }) + } + + private def instantiateThreadMetrics(metrics: MetricGroup): Unit = { + val mxBean = ManagementFactory.getThreadMXBean + + metrics + .gauge("Count", new FlinkGauge[Int] { + override def getValue: Int = mxBean.getThreadCount + }) + } + + private def instantiateCPUMetrics(metrics: MetricGroup): Unit = { + try { + val mxBean = ManagementFactory.getOperatingSystemMXBean + .asInstanceOf[com.sun.management.OperatingSystemMXBean] + + metrics + .gauge("Load", new FlinkGauge[Double] { + override def getValue: Double = mxBean.getProcessCpuLoad + }) + metrics.gauge("Time", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getProcessCpuTime + }) + } + catch { + case t: Throwable => + LOG.warn("Cannot access com.sun.management.OperatingSystemMXBean.getProcessCpuLoad()" + + " - CPU load metrics will not be available.") + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java index 36744a91815f6..f0bf69428c47b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java @@ -46,6 +46,7 @@ public void testSerialization() { final JobID jobID = new JobID(); final JobVertexID vertexID = new JobVertexID(); final ExecutionAttemptID execId = new ExecutionAttemptID(); + final String jobName = "job name"; final String taskName = "task name"; final int indexInSubtaskGroup = 0; final int currentNumberOfSubtasks = 1; @@ -59,7 +60,7 @@ public void testSerialization() { final List requiredClasspaths = new ArrayList(0); final SerializedValue executionConfig = ExecutionConfigTest.getSerializedConfig(); - final TaskDeploymentDescriptor orig = new TaskDeploymentDescriptor(jobID, vertexID, execId, + final TaskDeploymentDescriptor orig = new TaskDeploymentDescriptor(jobID, jobName, vertexID, execId, executionConfig, taskName, indexInSubtaskGroup, currentNumberOfSubtasks, attemptNumber, jobConfiguration, taskConfiguration, invokableClass.getName(), producedResults, inputGates, requiredJars, requiredClasspaths, 47); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java index 6853722330c7a..9724a809c628e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java @@ -20,6 +20,7 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; @@ -189,5 +190,9 @@ protected MockReader(InputGate inputGate) { public void setReporter(AccumulatorRegistry.Reporter reporter) { } + + @Override + public void setMetricGroup(IOMetricGroup metrics) { + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java index 9be957aa8077d..0300a079b1f39 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java @@ -25,6 +25,8 @@ import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.MemoryType; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; @@ -221,4 +223,9 @@ public AbstractInvokable getOwningNepheleTask() { public String formatLogString(String message) { return message; } + + @Override + public MetricGroup getMetricGroup() { + return new DummyMetricGroup(); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java index 7043a637b0c42..2c3dcf144ede2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java @@ -26,6 +26,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; @@ -366,6 +368,11 @@ public String formatLogString(String message) { return "Driver Tester: " + message; } + @Override + public MetricGroup getMetricGroup() { + return new DummyMetricGroup(); + } + // -------------------------------------------------------------------------------------------- @After diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java index ab58cea536d98..638173359c8db 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java @@ -24,6 +24,8 @@ import java.util.List; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.runtime.operators.Driver; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.util.TestLogger; @@ -363,6 +365,11 @@ public String formatLogString(String message) { return "Driver Tester: " + message; } + @Override + public MetricGroup getMetricGroup() { + return new DummyMetricGroup(); + } + // -------------------------------------------------------------------------------------------- @After diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java index 233dace253e9b..9b54383e5ca1f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java @@ -27,6 +27,8 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.groups.TaskMetricGroup; +import org.apache.flink.metrics.util.DummyTaskMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.execution.Environment; @@ -81,6 +83,11 @@ public TaskManagerRuntimeInfo getTaskManagerInfo() { return null; } + @Override + public TaskMetricGroup getMetricGroup() { + return new DummyTaskMetricGroup(); + } + @Override public Configuration getJobConfiguration() { return null; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java index d29b20670617c..31fd08c2957f6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java @@ -24,6 +24,8 @@ import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.core.fs.Path; import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.metrics.groups.TaskMetricGroup; +import org.apache.flink.metrics.util.DummyTaskMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.execution.Environment; @@ -209,6 +211,11 @@ public TaskManagerRuntimeInfo getTaskManagerInfo() { return new TaskManagerRuntimeInfo("localhost", new UnmodifiableConfiguration(new Configuration())); } + @Override + public TaskMetricGroup getMetricGroup() { + return new DummyTaskMetricGroup(); + } + @Override public InputSplitProvider getInputSplitProvider() { return this.inputSplitProvider; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java index 886c881bf6734..77e18c60eac2a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java @@ -26,6 +26,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; @@ -358,6 +359,11 @@ public String formatLogString(String message) { return "Driver Tester: " + message; } + @Override + public MetricGroup getMetricGroup() { + return null; + } + // -------------------------------------------------------------------------------------------- @After diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java index 1b463bcd38702..ae05ae9e7a9cf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -148,7 +149,7 @@ private static Task createTask() { when(networkEnvironment.getDefaultIOMode()).thenReturn(IOManager.IOMode.SYNC); TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( - new JobID(), new JobVertexID(), new ExecutionAttemptID(), + new JobID(), "Job Name", new JobVertexID(), new ExecutionAttemptID(), ExecutionConfigTest.getSerializedConfig(), "Test Task", 0, 1, 0, new Configuration(), new Configuration(), @@ -170,7 +171,8 @@ private static Task createTask() { new FiniteDuration(60, TimeUnit.SECONDS), libCache, mock(FileCache.class), - new TaskManagerRuntimeInfo("localhost", new Configuration())); + new TaskManagerRuntimeInfo("localhost", new Configuration()), + mock(TaskMetricGroup.class)); } public static class CheckpointsInOrderInvokable extends AbstractInvokable implements StatefulTask> { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java index 3ee9a842aa39e..b3ad5894e9206 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java @@ -162,7 +162,7 @@ protected void run() { final ExecutionAttemptID eid = new ExecutionAttemptID(); final SerializedValue executionConfig = ExecutionConfigTest.getSerializedConfig(); - final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(jid, vid, eid, executionConfig, + final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(jid, "TestJob", vid, eid, executionConfig, "TestTask", 2, 7, 0, new Configuration(), new Configuration(), TestInvokableCorrect.class.getName(), Collections.emptyList(), @@ -263,14 +263,14 @@ public void testJobSubmissionAndCanceling() { final ExecutionAttemptID eid1 = new ExecutionAttemptID(); final ExecutionAttemptID eid2 = new ExecutionAttemptID(); - final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, vid1, eid1, + final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, "TestJob1", vid1, eid1, ExecutionConfigTest.getSerializedConfig(), "TestTask1", 1, 5, 0, new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(), Collections.emptyList(), Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); - final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, vid2, eid2, + final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, "TestJob2", vid2, eid2, ExecutionConfigTest.getSerializedConfig(), "TestTask2", 2, 7, 0, new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(), Collections.emptyList(), @@ -399,13 +399,13 @@ public void testJobSubmissionAndStop() { final SerializedValue executionConfig = ExecutionConfigTest.getSerializedConfig(); - final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, vid1, eid1, executionConfig, + final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, "TestJob", vid1, eid1, executionConfig, "TestTask1", 1, 5, 0, new Configuration(), new Configuration(), StoppableInvokable.class.getName(), Collections.emptyList(), Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); - final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, vid2, eid2, executionConfig, + final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, "TestJob", vid2, eid2, executionConfig, "TestTask2", 2, 7, 0, new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(), Collections.emptyList(), Collections.emptyList(), @@ -525,14 +525,14 @@ public void testGateChannelEdgeMismatch() { final ExecutionAttemptID eid1 = new ExecutionAttemptID(); final ExecutionAttemptID eid2 = new ExecutionAttemptID(); - final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, + final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, "TestJob", vid1, eid1, ExecutionConfigTest.getSerializedConfig(), "Sender", 0, 1, 0, new Configuration(), new Configuration(), Tasks.Sender.class.getName(), Collections.emptyList(), Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); - final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, + final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, "TestJob", vid2, eid2, ExecutionConfigTest.getSerializedConfig(), "Receiver", 2, 7, 0, new Configuration(), new Configuration(), Tasks.Receiver.class.getName(), Collections.emptyList(), @@ -626,13 +626,13 @@ public void testRunJobWithForwardChannel() { } ); - final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, + final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, "TestJob", vid1, eid1, ExecutionConfigTest.getSerializedConfig(), "Sender", 0, 1, 0, new Configuration(), new Configuration(), Tasks.Sender.class.getName(), irpdd, Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); - final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, + final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, "TestJob", vid2, eid2, ExecutionConfigTest.getSerializedConfig(), "Receiver", 2, 7, 0, new Configuration(), new Configuration(), Tasks.Receiver.class.getName(), Collections.emptyList(), @@ -767,13 +767,13 @@ public void testCancellingDependentAndStateUpdateFails() { } ); - final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, + final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, "TestJob", vid1, eid1, ExecutionConfigTest.getSerializedConfig(), "Sender", 0, 1, 0, new Configuration(), new Configuration(), Tasks.Sender.class.getName(), irpdd, Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); - final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, + final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, "TestJob", vid2, eid2, ExecutionConfigTest.getSerializedConfig(), "Receiver", 2, 7, 0, new Configuration(), new Configuration(), Tasks.BlockingReceiver.class.getName(), Collections.emptyList(), @@ -912,7 +912,7 @@ public void testRemotePartitionNotFound() throws Exception { new InputGateDeploymentDescriptor(resultId, 0, icdd); final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( - jid, vid, eid, + jid, "TestJob", vid, eid, ExecutionConfigTest.getSerializedConfig(), "Receiver", 0, 1, 0, new Configuration(), new Configuration(), Tasks.AgnosticReceiver.class.getName(), @@ -1007,7 +1007,7 @@ public void testLocalPartitionNotFound() throws Exception { new InputGateDeploymentDescriptor(resultId, 0, icdd); final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( - jid, vid, eid, ExecutionConfigTest.getSerializedConfig(), "Receiver", 0, 1, 0, + jid, "TestJob", vid, eid, ExecutionConfigTest.getSerializedConfig(), "Receiver", 0, 1, 0, new Configuration(), new Configuration(), Tasks.AgnosticReceiver.class.getName(), Collections.emptyList(), @@ -1081,6 +1081,7 @@ public void testTriggerStackTraceSampleMessage() throws Exception { // Single blocking task final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( new JobID(), + "Job", new JobVertexID(), new ExecutionAttemptID(), ExecutionConfigTest.getSerializedConfig(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java index 99e037d452b00..7660893ab2969 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; @@ -69,7 +70,7 @@ public void doMocking(AbstractInvokable taskMock) throws Exception { task = new Task(tddMock, mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), mock(BroadcastVariableManager.class), mock(ActorGateway.class), mock(ActorGateway.class), mock(FiniteDuration.class), mock(LibraryCacheManager.class), mock(FileCache.class), - mock(TaskManagerRuntimeInfo.class)); + mock(TaskManagerRuntimeInfo.class), mock(TaskMetricGroup.class)); Field f = task.getClass().getDeclaredField("invokable"); f.setAccessible(true); f.set(task, taskMock); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index 06f393f30c4a9..f237c877aaea9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -622,12 +623,13 @@ private Task createTask(Class invokable, new FiniteDuration(60, TimeUnit.SECONDS), libCache, mock(FileCache.class), - new TaskManagerRuntimeInfo("localhost", new Configuration())); + new TaskManagerRuntimeInfo("localhost", new Configuration()), + mock(TaskMetricGroup.class)); } private TaskDeploymentDescriptor createTaskDeploymentDescriptor(Class invokable) { return new TaskDeploymentDescriptor( - new JobID(), new JobVertexID(), new ExecutionAttemptID(), + new JobID(), "Test Job", new JobVertexID(), new ExecutionAttemptID(), ExecutionConfigTest.getSerializedConfig(), "Test Task", 0, 1, 0, new Configuration(), new Configuration(), diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java index c9b1d38de8be1..783b3e27af5ba 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java @@ -53,6 +53,7 @@ public class StreamConfig implements Serializable { private static final String CHAINED_OUTPUTS = "chainedOutputs"; private static final String CHAINED_TASK_CONFIG = "chainedTaskConfig_"; private static final String IS_CHAINED_VERTEX = "isChainedSubtask"; + private static final String CHAIN_INDEX = "chainIndex"; private static final String VERTEX_NAME = "vertexID"; private static final String ITERATION_ID = "iterationId"; private static final String OUTPUT_SELECTOR_WRAPPER = "outputSelectorWrapper"; @@ -387,6 +388,14 @@ public Map getTransitiveChainedTaskConfigs(ClassLoader cl throw new StreamTaskException("Could not instantiate configuration.", e); } } + + public void setChainIndex(int index) { + this.config.setInteger(CHAIN_INDEX, index); + } + + public int getChainIndex() { + return this.config.getInteger(CHAIN_INDEX, 0); + } // ------------------------------------------------------------------------ // State backend diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index 9adabae33ec04..71cc7f2010f9b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -167,14 +167,15 @@ private void setPhysicalEdges() { */ private void setChaining(Map hashes) { for (Integer sourceNodeId : streamGraph.getSourceIDs()) { - createChain(sourceNodeId, sourceNodeId, hashes); + createChain(sourceNodeId, sourceNodeId, hashes, 0); } } private List createChain( Integer startNodeId, Integer currentNodeId, - Map hashes) { + Map hashes, + int chainIndex) { if (!builtVertices.contains(startNodeId)) { @@ -192,12 +193,12 @@ private List createChain( } for (StreamEdge chainable : chainableOutputs) { - transitiveOutEdges.addAll(createChain(startNodeId, chainable.getTargetId(), hashes)); + transitiveOutEdges.addAll(createChain(startNodeId, chainable.getTargetId(), hashes, chainIndex + 1)); } for (StreamEdge nonChainable : nonChainableOutputs) { transitiveOutEdges.add(nonChainable); - createChain(nonChainable.getTargetId(), nonChainable.getTargetId(), hashes); + createChain(nonChainable.getTargetId(), nonChainable.getTargetId(), hashes, 0); } chainedNames.put(currentNodeId, createChainedName(currentNodeId, chainableOutputs)); @@ -211,6 +212,7 @@ private List createChain( if (currentNodeId.equals(startNodeId)) { config.setChainStart(); + config.setChainIndex(0); config.setOutEdgesInOrder(transitiveOutEdges); config.setOutEdges(streamGraph.getStreamNode(currentNodeId).getOutEdges()); @@ -227,6 +229,7 @@ private List createChain( if (chainedConfs == null) { chainedConfigs.put(startNodeId, new HashMap()); } + config.setChainIndex(chainIndex); chainedConfigs.get(startNodeId).put(currentNodeId, config); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index 9673f874e80f0..326a42f619f2e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -25,6 +25,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.VoidSerializer; import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.runtime.state.KvStateSnapshot; import org.apache.flink.runtime.state.AbstractStateBackend; @@ -91,6 +92,7 @@ public abstract class AbstractStreamOperator /** The state backend that stores the state and checkpoints for this task */ private AbstractStateBackend stateBackend = null; + protected MetricGroup metrics; // ------------------------------------------------------------------------ // Life Cycle @@ -101,6 +103,9 @@ public void setup(StreamTask containingTask, StreamConfig config, Output")[config.getChainIndex()].trim(); + + this.metrics = container.getEnvironment().getMetricGroup().addOperator(operatorName); this.runtimeContext = new StreamingRuntimeContext(this, container.getEnvironment(), container.getAccumulatorMap()); stateKeySelector1 = config.getStatePartitioner(0, getUserCodeClassloader()); @@ -116,6 +121,10 @@ public void setup(StreamTask containingTask, StreamConfig config, Output extends Serializable { ChainingStrategy getChainingStrategy(); void setChainingStrategy(ChainingStrategy strategy); + + MetricGroup getMetricGroup(); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java index 4500ee73a1161..5fef3c797dbd5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java @@ -64,7 +64,8 @@ public StreamingRuntimeContext(AbstractStreamOperator operator, env.getUserClassLoader(), operator.getExecutionConfig(), accumulators, - env.getDistributedCacheEntries()); + env.getDistributedCacheEntries(), + operator.getMetricGroup()); this.operator = operator; this.taskEnvironment = env; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java index 8c0b63f8eef88..1dde85b3c9f10 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.disk.iomanager.IOManager; @@ -202,6 +203,17 @@ public void setReporter(AccumulatorRegistry.Reporter reporter) { deserializer.setReporter(reporter); } } + + /** + * Sets the metric group for this StreamInputProcessor. + * + * @param metrics metric group + */ + public void setMetricGroup(IOMetricGroup metrics) { + for (RecordDeserializer deserializer : recordDeserializers) { + deserializer.instantiateMetrics(metrics); + } + } public void cleanup() throws IOException { // clear the buffers first. this part should not ever fail diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java index 4adea3bf8ab0a..07ada23502ccf 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.disk.iomanager.IOManager; @@ -278,6 +279,17 @@ public void setReporter(AccumulatorRegistry.Reporter reporter) { deserializer.setReporter(reporter); } } + + /** + * Sets the metric group for this StreamTwoInputProcessor. + * + * @param metrics metric group + */ + public void setMetricGroup(IOMetricGroup metrics) { + for (RecordDeserializer deserializer : recordDeserializers) { + deserializer.instantiateMetrics(metrics); + } + } public void cleanup() throws IOException { // clear the buffers first. this part should not ever fail diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java index a1ed8288c26e7..938d8c147bcc5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java @@ -52,6 +52,7 @@ public void init() throws Exception { AccumulatorRegistry registry = getEnvironment().getAccumulatorRegistry(); AccumulatorRegistry.Reporter reporter = registry.getReadWriteReporter(); inputProcessor.setReporter(reporter); + inputProcessor.setMetricGroup(getEnvironment().getMetricGroup().getIOMetricGroup()); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java index f3d3482ce692c..90abea40a5600 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java @@ -286,6 +286,7 @@ private static RecordWriterOutput createStreamOutput( StreamRecordWriter>> output = new StreamRecordWriter<>(bufferWriter, outputPartitioner, upStreamConfig.getBufferTimeout()); output.setReporter(reporter); + output.setMetricGroup(taskEnvironment.getMetricGroup().getIOMetricGroup()); return new RecordWriterOutput(output, outSerializer, withTimestamps); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java index ebc7789c04852..c3305eb6e2d07 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java @@ -77,6 +77,7 @@ public void init() throws Exception { AccumulatorRegistry registry = getEnvironment().getAccumulatorRegistry(); AccumulatorRegistry.Reporter reporter = registry.getReadWriteReporter(); this.inputProcessor.setReporter(reporter); + inputProcessor.setMetricGroup(getEnvironment().getMetricGroup().getIOMetricGroup()); } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java index dff118424631a..812507f509cff 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java @@ -29,6 +29,7 @@ import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.util.DummyTaskMetricGroup; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction; import org.apache.flink.streaming.api.functions.windowing.WindowFunction; @@ -794,6 +795,7 @@ public void apply(Integer key, final Environment env = mock(Environment.class); when(env.getTaskInfo()).thenReturn(new TaskInfo("Test task name", 0, 1, 0)); when(env.getUserClassLoader()).thenReturn(AggregatingAlignedProcessingTimeWindowOperatorTest.class.getClassLoader()); + when(env.getMetricGroup()).thenReturn(new DummyTaskMetricGroup()); when(task.getEnvironment()).thenReturn(env); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java index 342921510e5a2..c89ac5080e609 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java @@ -34,6 +34,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.util.DummyTaskMetricGroup; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.Output; @@ -966,6 +967,7 @@ public Tuple2 reduce(Tuple2 value1, Tuple2 state) { public TaskManagerRuntimeInfo getTaskManagerInfo() { return new TaskManagerRuntimeInfo("localhost", new UnmodifiableConfiguration(new Configuration())); } + + @Override + public TaskMetricGroup getMetricGroup() { + return new DummyTaskMetricGroup(); + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index ed8bf019f7e02..deda82fde386c 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -134,7 +135,7 @@ private Task createTask(Class invokable, StreamConf when(network.getDefaultIOMode()).thenReturn(IOManager.IOMode.SYNC); TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( - new JobID(), new JobVertexID(), new ExecutionAttemptID(), + new JobID(), "Job Name", new JobVertexID(), new ExecutionAttemptID(), ExecutionConfigTest.getSerializedConfig(), "Test Task", 0, 1, 0, new Configuration(), @@ -157,7 +158,8 @@ private Task createTask(Class invokable, StreamConf new FiniteDuration(60, TimeUnit.SECONDS), libCache, mock(FileCache.class), - new TaskManagerRuntimeInfo("localhost", new Configuration())); + new TaskManagerRuntimeInfo("localhost", new Configuration()), + mock(TaskMetricGroup.class)); } // ------------------------------------------------------------------------ diff --git a/pom.xml b/pom.xml index 1ba237f596e9b..c97815af63576 100644 --- a/pom.xml +++ b/pom.xml @@ -74,6 +74,7 @@ under the License. flink-quickstart flink-contrib flink-dist + flink-metrics @@ -103,6 +104,7 @@ under the License. 3.4.6 2.8.0 2.7.4 + 3.1.0