rkhachatryan commented on a change in pull request #15119:
URL: https://github.com/apache/flink/pull/15119#discussion_r594929467
##########
File path:
flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
##########
@@ -26,42 +26,9 @@
public class CheckpointingOptions {
// ------------------------------------------------------------------------
- // general checkpoint and state backend options
+ // general checkpoint options
// ------------------------------------------------------------------------
- /**
- * The checkpoint storage used to store operator state locally within the
cluster during
- * execution.
- *
- * <p>The implementation can be specified either via their shortcut name,
or via the class name
- * of a {@code StateBackendFactory}. If a StateBackendFactory class name
is specified, the
- * factory is instantiated (via its zero-argument constructor) and its
{@code
- * StateBackendFactory#createFromConfig(ReadableConfig, ClassLoader)}
method is called.
- *
- * <p>Recognized shortcut names are 'hashmap' and 'rocksdb'.
- */
- @Documentation.Section(value =
Documentation.Sections.COMMON_STATE_BACKENDS, position = 1)
- public static final ConfigOption<String> STATE_BACKEND =
Review comment:
Probably this field should only be deprecated and hidden from the
configuration, I guess it's used in some apps directly.
##########
File path: flink-runtime/pom.xml
##########
@@ -56,6 +56,12 @@ under the License.
<version>${project.version}</version>
</dependency>
+ <dependency>
+ <groupId>io.dropwizard.metrics</groupId>
+ <artifactId>metrics-core</artifactId>
+ <version>${dropwizard.version}</version>
+ </dependency>
+
Review comment:
Why do we need this dependency for `flink-runtime`?
I couldn't find any imports of dropwizard.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/state/metrics/LatencyTrackingStateConfig.java
##########
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state.metrics;
+
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.configuration.StateBackendOptions;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.util.Preconditions;
+
+import java.io.Serializable;
+
+/** Config to create latency tracking state metric. */
+public class LatencyTrackingStateConfig {
Review comment:
`@Internal`?
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/state/metrics/AbstractLatencyTrackingStateMetric.java
##########
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state.metrics;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.metrics.Histogram;
+import org.apache.flink.metrics.HistogramStatistics;
+import org.apache.flink.metrics.MetricGroup;
+
+import com.codahale.metrics.SlidingTimeWindowReservoir;
+import com.codahale.metrics.Snapshot;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/** Abstract class of latency tracking state metric which counts and histogram
the state metric. */
+class AbstractLatencyTrackingStateMetric implements AutoCloseable {
+ protected static final String STATE_CLEAR_LATENCY = "stateClearLatency";
+ private final MetricGroup metricGroup;
+ private final int sampleInterval;
+ private final Map<String, Histogram> histogramMetrics;
+ private final Map<String, Counter> countersPerMetric;
+ private final Supplier<com.codahale.metrics.Histogram> histogramSupplier;
Review comment:
Why not `org.apache.flink.metrics.Histogram`?
##########
File path:
flink-core/src/main/java/org/apache/flink/configuration/StateBackendOptions.java
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.description.Description;
+import org.apache.flink.configuration.description.TextElement;
+
+/** A collection of all configuration options that relate to state backend. */
+public class StateBackendOptions {
+
+ // ------------------------------------------------------------------------
+ // general state backend options
+ // ------------------------------------------------------------------------
+
+ /**
+ * The checkpoint storage used to store operator state locally within the
cluster during
+ * execution.
+ *
+ * <p>The implementation can be specified either via their shortcut name,
or via the class name
+ * of a {@code StateBackendFactory}. If a StateBackendFactory class name
is specified, the
+ * factory is instantiated (via its zero-argument constructor) and its
{@code
+ * StateBackendFactory#createFromConfig(ReadableConfig, ClassLoader)}
method is called.
+ *
+ * <p>Recognized shortcut names are 'hashmap' and 'rocksdb'.
+ */
+ @Documentation.Section(value =
Documentation.Sections.COMMON_STATE_BACKENDS, position = 1)
+ public static final ConfigOption<String> STATE_BACKEND =
+ ConfigOptions.key("state.backend")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text("The state backend to be used to
store state.")
+ .linebreak()
+ .text(
+ "The implementation can be
specified either via their shortcut "
+ + " name, or via the class
name of a %s. "
+ + "If a factory is
specified it is instantiated via its "
+ + "zero argument
constructor and its %s "
+ + "method is called.",
+
TextElement.code("StateBackendFactory"),
+ TextElement.code(
+
"StateBackendFactory#createFromConfig(ReadableConfig, ClassLoader)"))
+ .linebreak()
+ .text("Recognized shortcut names are
'hashmap' and 'rocksdb'.")
+ .build());
+
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption<Boolean> LATENCY_TRACK_ENABLED =
+ ConfigOptions.key("state.backend.latency-track-enabled")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription(
+ "Whether to track latency of state operations, e.g
value state put/get/clear.");
+
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption<Integer> LATENCY_TRACK_SAMPLE_INTERVAL =
+ ConfigOptions.key("state.backend.latency-track-sample-interval")
+ .intType()
+ .defaultValue(100)
Review comment:
I'd either use `durationType` (or similar) or specify unit in the key
name. I always end up looking up the unit in such cases :)
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/state/metrics/AbstractLatencyTrackingStateMetric.java
##########
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state.metrics;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.metrics.Histogram;
+import org.apache.flink.metrics.HistogramStatistics;
+import org.apache.flink.metrics.MetricGroup;
+
+import com.codahale.metrics.SlidingTimeWindowReservoir;
+import com.codahale.metrics.Snapshot;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/** Abstract class of latency tracking state metric which counts and histogram
the state metric. */
+class AbstractLatencyTrackingStateMetric implements AutoCloseable {
+ protected static final String STATE_CLEAR_LATENCY = "stateClearLatency";
+ private final MetricGroup metricGroup;
+ private final int sampleInterval;
+ private final Map<String, Histogram> histogramMetrics;
+ private final Map<String, Counter> countersPerMetric;
+ private final Supplier<com.codahale.metrics.Histogram> histogramSupplier;
+
+ AbstractLatencyTrackingStateMetric(
+ String stateName, MetricGroup metricGroup, int sampleInterval,
long slidingWindow) {
+ this.metricGroup = metricGroup.addGroup(stateName);
Review comment:
How does the metric name look like in the end?
Can we distinguish between different subtasks if they are running in the
same TM?
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
##########
@@ -289,8 +297,11 @@ public KeyGroupRange getKeyGroupRange() {
stateDescriptor.initializeSerializerUnlessSet(executionConfig);
}
kvState =
- TtlStateFactory.createStateAndWrapWithTtlIfEnabled(
- namespaceSerializer, stateDescriptor, this,
ttlTimeProvider);
+ LatencyTrackingStateFactory.trackLatencyIfEnabled(
+ TtlStateFactory.createStateAndWrapWithTtlIfEnabled(
+ namespaceSerializer, stateDescriptor,
this, ttlTimeProvider),
+ stateDescriptor,
+ latencyTrackingStateConfig);
Review comment:
This will only work for `AbstractKeyedStateBackend` descendants, right?
Why not to use delegation to enable tracking for any backend?
##########
File path:
flink-core/src/main/java/org/apache/flink/configuration/StateBackendOptions.java
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.description.Description;
+import org.apache.flink.configuration.description.TextElement;
+
+/** A collection of all configuration options that relate to state backend. */
+public class StateBackendOptions {
+
+ // ------------------------------------------------------------------------
+ // general state backend options
+ // ------------------------------------------------------------------------
+
+ /**
+ * The checkpoint storage used to store operator state locally within the
cluster during
+ * execution.
+ *
+ * <p>The implementation can be specified either via their shortcut name,
or via the class name
+ * of a {@code StateBackendFactory}. If a StateBackendFactory class name
is specified, the
+ * factory is instantiated (via its zero-argument constructor) and its
{@code
+ * StateBackendFactory#createFromConfig(ReadableConfig, ClassLoader)}
method is called.
+ *
+ * <p>Recognized shortcut names are 'hashmap' and 'rocksdb'.
+ */
+ @Documentation.Section(value =
Documentation.Sections.COMMON_STATE_BACKENDS, position = 1)
+ public static final ConfigOption<String> STATE_BACKEND =
+ ConfigOptions.key("state.backend")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text("The state backend to be used to
store state.")
+ .linebreak()
+ .text(
+ "The implementation can be
specified either via their shortcut "
+ + " name, or via the class
name of a %s. "
+ + "If a factory is
specified it is instantiated via its "
+ + "zero argument
constructor and its %s "
+ + "method is called.",
+
TextElement.code("StateBackendFactory"),
+ TextElement.code(
+
"StateBackendFactory#createFromConfig(ReadableConfig, ClassLoader)"))
+ .linebreak()
+ .text("Recognized shortcut names are
'hashmap' and 'rocksdb'.")
+ .build());
+
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption<Boolean> LATENCY_TRACK_ENABLED =
+ ConfigOptions.key("state.backend.latency-track-enabled")
Review comment:
Did you consider a dedicated section for latency tracking?
Both in documentation and in key space
(`state.backend.latency-track.enabled`).
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/state/metrics/LatencyTrackingMapState.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state.metrics;
+
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.internal.InternalMapState;
+
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * This class wraps map state with latency tracking logic.
+ *
+ * @param <K> The type of key the state is associated to
+ * @param <N> The type of the namespace
+ * @param <UK> Type of the user entry key of state
+ * @param <UV> Type of the user entry value of state
+ */
+public class LatencyTrackingMapState<K, N, UK, UV>
Review comment:
Package-private?
ditto: other states
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/state/metrics/LatencyTrackingMapState.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state.metrics;
+
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.internal.InternalMapState;
+
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * This class wraps map state with latency tracking logic.
+ *
+ * @param <K> The type of key the state is associated to
+ * @param <N> The type of the namespace
+ * @param <UK> Type of the user entry key of state
+ * @param <UV> Type of the user entry value of state
+ */
+public class LatencyTrackingMapState<K, N, UK, UV>
+ extends AbstractLatencyTrackState<
+ K,
+ N,
+ Map<UK, UV>,
+ InternalMapState<K, N, UK, UV>,
+ LatencyTrackingMapState.LatencyTrackingMapStateMetrics>
+ implements InternalMapState<K, N, UK, UV> {
+ LatencyTrackingMapState(
+ String stateName,
+ InternalMapState<K, N, UK, UV> original,
+ LatencyTrackingStateConfig latencyTrackingStateConfig) {
+ super(
+ original,
+ new LatencyTrackingMapStateMetrics(
+ stateName,
+ latencyTrackingStateConfig.getMetricGroup(),
+ latencyTrackingStateConfig.getSampleInterval(),
+ latencyTrackingStateConfig.getSlidingWindow()));
+ }
+
+ @Override
+ public UV get(UK key) throws Exception {
+ if (latencyTrackingStateMetric.checkGetCounter()) {
+ return trackLatencyWithException(
+ () -> original.get(key),
latencyTrackingStateMetric::updateGetLatency);
+ } else {
+ return original.get(key);
+ }
+ }
+
+ @Override
+ public void put(UK key, UV value) throws Exception {
+ if (latencyTrackingStateMetric.checkPutCounter()) {
+ trackLatencyWithException(
+ () -> original.put(key, value),
latencyTrackingStateMetric::updatePutLatency);
+ } else {
+ original.put(key, value);
Review comment:
I think this code could be de-duplicated by passing
`latencyTrackingStateMetric::checkPutCounter` to
`trackLatencyWithException()` and calling `updatePutLatency` there
conditionally.
WDYT?
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/state/metrics/AbstractLatencyTrackingStateMetric.java
##########
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state.metrics;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.metrics.Histogram;
+import org.apache.flink.metrics.HistogramStatistics;
+import org.apache.flink.metrics.MetricGroup;
+
+import com.codahale.metrics.SlidingTimeWindowReservoir;
+import com.codahale.metrics.Snapshot;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/** Abstract class of latency tracking state metric which counts and histogram
the state metric. */
+class AbstractLatencyTrackingStateMetric implements AutoCloseable {
Review comment:
I think it either should be `asbtract` or called `...Base`.
The name isn't very clear to me. How about `AbstractStateLatencyMetric` (or
`StateLatencyMetricBase`)?
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/state/metrics/LatencyTrackingStateConfig.java
##########
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state.metrics;
+
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.configuration.StateBackendOptions;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.util.Preconditions;
+
+import java.io.Serializable;
+
+/** Config to create latency tracking state metric. */
+public class LatencyTrackingStateConfig {
+
+ private final MetricGroup metricGroup;
+
+ private final boolean enabled;
+ private final int sampleInterval;
+ private final long slidingWindow;
+
+ LatencyTrackingStateConfig(
+ MetricGroup metricGroup, boolean enabled, int sampleInterval, long
slidingWindow) {
+ this.metricGroup = metricGroup;
+ this.enabled = enabled;
+ this.sampleInterval = sampleInterval;
+ this.slidingWindow = slidingWindow;
+ }
+
+ public boolean isEnabled() {
+ return enabled;
+ }
+
+ public MetricGroup getMetricGroup() {
+ return metricGroup;
+ }
+
+ public long getSlidingWindow() {
+ return slidingWindow;
+ }
+
+ public int getSampleInterval() {
+ return sampleInterval;
+ }
+
+ public static LatencyTrackingStateConfig disabled() {
+ return newBuilder().setEnabled(false).build();
+ }
+
+ public static Builder newBuilder() {
+ return new Builder();
+ }
+
+ public static class Builder implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private boolean enabled =
StateBackendOptions.LATENCY_TRACK_ENABLED.defaultValue();
+ private int sampleInterval =
+
StateBackendOptions.LATENCY_TRACK_SAMPLE_INTERVAL.defaultValue();
+ private long slidingWindow =
+
StateBackendOptions.LATENCY_TRACK_SLIDING_WINDOW.defaultValue();
+ private MetricGroup metricGroup;
+
+ public Builder setEnabled(boolean enabled) {
+ this.enabled = enabled;
+ return this;
+ }
+
+ public Builder setSampleInterval(int sampleInterval) {
+ this.sampleInterval = sampleInterval;
+ return this;
+ }
+
+ public Builder setSlidingWindow(long slidingWindow) {
+ this.slidingWindow = slidingWindow;
+ return this;
+ }
+
+ public Builder setMetricGroup(MetricGroup metricGroup) {
+ this.metricGroup = metricGroup;
+ return this;
+ }
+
+ public Builder configure(ReadableConfig config) {
+
this.setEnabled(config.get(StateBackendOptions.LATENCY_TRACK_ENABLED))
+ .setSampleInterval(
+
config.get(StateBackendOptions.LATENCY_TRACK_SAMPLE_INTERVAL))
+
.setSlidingWindow(config.get(StateBackendOptions.LATENCY_TRACK_SLIDING_WINDOW));
+ return this;
+ }
+
+ public LatencyTrackingStateConfig build() {
+ if (enabled) {
+ Preconditions.checkNotNull(
+ metricGroup, "Metric group cannot be null if latency
tracking is enabled.");
+ Preconditions.checkArgument(sampleInterval >= 1);
+ }
+ return new LatencyTrackingStateConfig(
+ metricGroup, enabled, sampleInterval, slidingWindow);
Review comment:
I'd move these check to constructor, as it potentially can be called
from other places.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/state/metrics/LatencyTrackingStateConfig.java
##########
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state.metrics;
+
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.configuration.StateBackendOptions;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.util.Preconditions;
+
+import java.io.Serializable;
+
+/** Config to create latency tracking state metric. */
+public class LatencyTrackingStateConfig {
+
+ private final MetricGroup metricGroup;
+
+ private final boolean enabled;
+ private final int sampleInterval;
+ private final long slidingWindow;
+
+ LatencyTrackingStateConfig(
+ MetricGroup metricGroup, boolean enabled, int sampleInterval, long
slidingWindow) {
+ this.metricGroup = metricGroup;
+ this.enabled = enabled;
+ this.sampleInterval = sampleInterval;
+ this.slidingWindow = slidingWindow;
+ }
+
+ public boolean isEnabled() {
+ return enabled;
+ }
+
+ public MetricGroup getMetricGroup() {
+ return metricGroup;
+ }
+
+ public long getSlidingWindow() {
+ return slidingWindow;
+ }
+
+ public int getSampleInterval() {
+ return sampleInterval;
+ }
+
+ public static LatencyTrackingStateConfig disabled() {
+ return newBuilder().setEnabled(false).build();
+ }
+
+ public static Builder newBuilder() {
+ return new Builder();
+ }
+
+ public static class Builder implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private boolean enabled =
StateBackendOptions.LATENCY_TRACK_ENABLED.defaultValue();
+ private int sampleInterval =
+
StateBackendOptions.LATENCY_TRACK_SAMPLE_INTERVAL.defaultValue();
+ private long slidingWindow =
+
StateBackendOptions.LATENCY_TRACK_SLIDING_WINDOW.defaultValue();
+ private MetricGroup metricGroup;
+
+ public Builder setEnabled(boolean enabled) {
+ this.enabled = enabled;
+ return this;
+ }
+
+ public Builder setSampleInterval(int sampleInterval) {
+ this.sampleInterval = sampleInterval;
+ return this;
+ }
+
+ public Builder setSlidingWindow(long slidingWindow) {
+ this.slidingWindow = slidingWindow;
+ return this;
+ }
+
+ public Builder setMetricGroup(MetricGroup metricGroup) {
+ this.metricGroup = metricGroup;
+ return this;
+ }
+
+ public Builder configure(ReadableConfig config) {
+
this.setEnabled(config.get(StateBackendOptions.LATENCY_TRACK_ENABLED))
+ .setSampleInterval(
+
config.get(StateBackendOptions.LATENCY_TRACK_SAMPLE_INTERVAL))
+
.setSlidingWindow(config.get(StateBackendOptions.LATENCY_TRACK_SLIDING_WINDOW));
+ return this;
+ }
+
+ public LatencyTrackingStateConfig build() {
+ if (enabled) {
+ Preconditions.checkNotNull(
+ metricGroup, "Metric group cannot be null if latency
tracking is enabled.");
+ Preconditions.checkArgument(sampleInterval >= 1);
Review comment:
Why `0` is not allowed? I.e., track **every** state access?
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/state/metrics/LatencyTrackingMapState.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state.metrics;
+
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.internal.InternalMapState;
+
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * This class wraps map state with latency tracking logic.
+ *
+ * @param <K> The type of key the state is associated to
+ * @param <N> The type of the namespace
+ * @param <UK> Type of the user entry key of state
+ * @param <UV> Type of the user entry value of state
+ */
+public class LatencyTrackingMapState<K, N, UK, UV>
+ extends AbstractLatencyTrackState<
+ K,
+ N,
+ Map<UK, UV>,
+ InternalMapState<K, N, UK, UV>,
+ LatencyTrackingMapState.LatencyTrackingMapStateMetrics>
+ implements InternalMapState<K, N, UK, UV> {
+ LatencyTrackingMapState(
+ String stateName,
+ InternalMapState<K, N, UK, UV> original,
+ LatencyTrackingStateConfig latencyTrackingStateConfig) {
+ super(
+ original,
+ new LatencyTrackingMapStateMetrics(
+ stateName,
+ latencyTrackingStateConfig.getMetricGroup(),
+ latencyTrackingStateConfig.getSampleInterval(),
+ latencyTrackingStateConfig.getSlidingWindow()));
+ }
+
+ @Override
+ public UV get(UK key) throws Exception {
+ if (latencyTrackingStateMetric.checkGetCounter()) {
+ return trackLatencyWithException(
+ () -> original.get(key),
latencyTrackingStateMetric::updateGetLatency);
+ } else {
+ return original.get(key);
+ }
+ }
+
+ @Override
+ public void put(UK key, UV value) throws Exception {
+ if (latencyTrackingStateMetric.checkPutCounter()) {
+ trackLatencyWithException(
+ () -> original.put(key, value),
latencyTrackingStateMetric::updatePutLatency);
+ } else {
+ original.put(key, value);
+ }
+ }
+
+ @Override
+ public void putAll(Map<UK, UV> map) throws Exception {
+ if (latencyTrackingStateMetric.checkPuAllCounter()) {
+ trackLatencyWithException(
+ () -> original.putAll(map),
latencyTrackingStateMetric::updatePutAllLatency);
+ } else {
+ original.putAll(map);
+ }
+ }
+
+ @Override
+ public void remove(UK key) throws Exception {
+ if (latencyTrackingStateMetric.checkRemoveCounter()) {
+ trackLatencyWithException(
+ () -> original.remove(key),
latencyTrackingStateMetric::updateRemoveLatency);
+ } else {
+ original.remove(key);
+ }
+ }
+
+ @Override
+ public boolean contains(UK key) throws Exception {
+ if (latencyTrackingStateMetric.checkContainsCounter()) {
+ return trackLatencyWithException(
+ () -> original.contains(key),
+ latencyTrackingStateMetric::updateContainsLatency);
+ } else {
+ return original.contains(key);
+ }
+ }
+
+ @Override
+ public Iterable<Map.Entry<UK, UV>> entries() throws Exception {
+ if (latencyTrackingStateMetric.checkEntriesCounter()) {
+ return trackLatencyWithException(
+ () -> original.entries(),
latencyTrackingStateMetric::updateEntriesLatency);
+ } else {
+ return original.entries();
+ }
+ }
+
+ @Override
+ public Iterable<UK> keys() throws Exception {
+ if (latencyTrackingStateMetric.checkKeysCounter()) {
+ return trackLatencyWithException(
+ () -> original.keys(),
latencyTrackingStateMetric::updateKeysLatency);
+ } else {
+ return original.keys();
+ }
+ }
+
+ @Override
+ public Iterable<UV> values() throws Exception {
+ if (latencyTrackingStateMetric.checkValuesCounter()) {
+ return trackLatencyWithException(
+ () -> original.values(),
latencyTrackingStateMetric::updateValuesLatency);
+ } else {
+ return original.values();
+ }
+ }
+
+ @Override
+ public Iterator<Map.Entry<UK, UV>> iterator() throws Exception {
+ if (latencyTrackingStateMetric.checkIteratorCounter()) {
+ return trackLatencyWithException(
+ () -> original.iterator(),
latencyTrackingStateMetric::updateIteratorLatency);
Review comment:
I think tracking the iterator creation doesn't make much sense - it's
often just creation of an object (at least rocksdb and heap).
Tracking the iteration itself would make more sense. This could be done by
wrapping `next()` call.
But I guess this can also be implemented later.
ditto: other iterator/iterables and other states
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]