Skip to content

Commit

Permalink
[FLINK-34457] Rename options for latency tracking
Browse files Browse the repository at this point in the history
  • Loading branch information
Zakelly committed Feb 18, 2024
1 parent 050503c commit 0a40ca8
Show file tree
Hide file tree
Showing 10 changed files with 123 additions and 34 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,13 @@

package org.apache.flink.configuration;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.annotation.docs.Documentation;
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. */
@PublicEvolving
public class StateBackendOptions {

// ------------------------------------------------------------------------
Expand Down Expand Up @@ -63,15 +65,19 @@ public class StateBackendOptions {
.text("Recognized shortcut names are 'hashmap' and 'rocksdb'.")
.build());

@Documentation.Section(Documentation.Sections.STATE_BACKEND_LATENCY_TRACKING)
/** @deprecated Use {@link StateLatencyTrackOptions#LATENCY_TRACK_ENABLED} instead. */
@Deprecated
@Documentation.ExcludeFromDocumentation("Hidden for deprecated")
public static final ConfigOption<Boolean> LATENCY_TRACK_ENABLED =
ConfigOptions.key("state.backend.latency-track.keyed-state-enabled")
.booleanType()
.defaultValue(false)
.withDescription(
"Whether to track latency of keyed state operations, e.g value state put/get/clear.");

@Documentation.Section(Documentation.Sections.STATE_BACKEND_LATENCY_TRACKING)
/** @deprecated Use {@link StateLatencyTrackOptions#LATENCY_TRACK_SAMPLE_INTERVAL} instead. */
@Deprecated
@Documentation.ExcludeFromDocumentation("Hidden for deprecated")
public static final ConfigOption<Integer> LATENCY_TRACK_SAMPLE_INTERVAL =
ConfigOptions.key("state.backend.latency-track.sample-interval")
.intType()
Expand All @@ -82,15 +88,22 @@ public class StateBackendOptions {
+ "The default value is 100, which means we would track the latency every 100 access requests.",
LATENCY_TRACK_ENABLED.key()));

@Documentation.Section(Documentation.Sections.STATE_BACKEND_LATENCY_TRACKING)
/** @deprecated Use {@link StateLatencyTrackOptions#LATENCY_TRACK_HISTORY_SIZE} instead. */
@Deprecated
@Documentation.ExcludeFromDocumentation("Hidden for deprecated")
public static final ConfigOption<Integer> LATENCY_TRACK_HISTORY_SIZE =
ConfigOptions.key("state.backend.latency-track.history-size")
.intType()
.defaultValue(128)
.withDescription(
"Defines the number of measured latencies to maintain at each state access operation.");

@Documentation.Section(Documentation.Sections.STATE_BACKEND_LATENCY_TRACKING)
/**
* @deprecated Use {@link StateLatencyTrackOptions#LATENCY_TRACK_STATE_NAME_AS_VARIABLE}
* instead.
*/
@Deprecated
@Documentation.ExcludeFromDocumentation("Hidden for deprecated")
public static final ConfigOption<Boolean> LATENCY_TRACK_STATE_NAME_AS_VARIABLE =
ConfigOptions.key("state.backend.latency-track.state-name-as-variable")
.booleanType()
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
/*
* 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.PublicEvolving;
import org.apache.flink.annotation.docs.Documentation;

/**
* A collection of all configuration options that relate to the latency tracking for state access.
*/
@PublicEvolving
public class StateLatencyTrackOptions {

@Documentation.Section(Documentation.Sections.STATE_BACKEND_LATENCY_TRACKING)
public static final ConfigOption<Boolean> LATENCY_TRACK_ENABLED =
ConfigOptions.key("state.latency-track.keyed-state-enabled")
.booleanType()
.defaultValue(false)
.withDeprecatedKeys("state.backend.latency-track.keyed-state-enabled")
.withDescription(
"Whether to track latency of keyed state operations, e.g value state put/get/clear.");

@Documentation.Section(Documentation.Sections.STATE_BACKEND_LATENCY_TRACKING)
public static final ConfigOption<Integer> LATENCY_TRACK_SAMPLE_INTERVAL =
ConfigOptions.key("state.latency-track.sample-interval")
.intType()
.defaultValue(100)
.withDeprecatedKeys("state.backend.latency-track.sample-interval")
.withDescription(
String.format(
"The sample interval of latency track once '%s' is enabled. "
+ "The default value is 100, which means we would track the latency every 100 access requests.",
LATENCY_TRACK_ENABLED.key()));

@Documentation.Section(Documentation.Sections.STATE_BACKEND_LATENCY_TRACKING)
public static final ConfigOption<Integer> LATENCY_TRACK_HISTORY_SIZE =
ConfigOptions.key("state.latency-track.history-size")
.intType()
.defaultValue(128)
.withDeprecatedKeys("state.backend.latency-track.history-size")
.withDescription(
"Defines the number of measured latencies to maintain at each state access operation.");

@Documentation.Section(Documentation.Sections.STATE_BACKEND_LATENCY_TRACKING)
public static final ConfigOption<Boolean> LATENCY_TRACK_STATE_NAME_AS_VARIABLE =
ConfigOptions.key("state.latency-track.state-name-as-variable")
.booleanType()
.defaultValue(true)
.withDeprecatedKeys("state.backend.latency-track.state-name-as-variable")
.withDescription(
"Whether to expose state name as a variable if tracking latency.");
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@

import org.apache.flink.annotation.Internal;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.configuration.StateBackendOptions;
import org.apache.flink.configuration.StateLatencyTrackOptions;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.util.Preconditions;

Expand Down Expand Up @@ -86,12 +86,13 @@ public static Builder newBuilder() {
public static class Builder implements Serializable {
private static final long serialVersionUID = 1L;

private boolean enabled = StateBackendOptions.LATENCY_TRACK_ENABLED.defaultValue();
private boolean enabled = StateLatencyTrackOptions.LATENCY_TRACK_ENABLED.defaultValue();
private int sampleInterval =
StateBackendOptions.LATENCY_TRACK_SAMPLE_INTERVAL.defaultValue();
private int historySize = StateBackendOptions.LATENCY_TRACK_HISTORY_SIZE.defaultValue();
StateLatencyTrackOptions.LATENCY_TRACK_SAMPLE_INTERVAL.defaultValue();
private int historySize =
StateLatencyTrackOptions.LATENCY_TRACK_HISTORY_SIZE.defaultValue();
private boolean stateNameAsVariable =
StateBackendOptions.LATENCY_TRACK_STATE_NAME_AS_VARIABLE.defaultValue();
StateLatencyTrackOptions.LATENCY_TRACK_STATE_NAME_AS_VARIABLE.defaultValue();
private MetricGroup metricGroup;

public Builder setEnabled(boolean enabled) {
Expand Down Expand Up @@ -120,12 +121,13 @@ public Builder setMetricGroup(MetricGroup metricGroup) {
}

public Builder configure(ReadableConfig config) {
this.setEnabled(config.get(StateBackendOptions.LATENCY_TRACK_ENABLED))
this.setEnabled(config.get(StateLatencyTrackOptions.LATENCY_TRACK_ENABLED))
.setSampleInterval(
config.get(StateBackendOptions.LATENCY_TRACK_SAMPLE_INTERVAL))
.setHistorySize(config.get(StateBackendOptions.LATENCY_TRACK_HISTORY_SIZE))
config.get(StateLatencyTrackOptions.LATENCY_TRACK_SAMPLE_INTERVAL))
.setHistorySize(config.get(StateLatencyTrackOptions.LATENCY_TRACK_HISTORY_SIZE))
.setStateNameAsVariable(
config.get(StateBackendOptions.LATENCY_TRACK_STATE_NAME_AS_VARIABLE));
config.get(
StateLatencyTrackOptions.LATENCY_TRACK_STATE_NAME_AS_VARIABLE));
return this;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@
import org.apache.flink.api.java.typeutils.runtime.kryo.JavaSerializer;
import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.StateBackendOptions;
import org.apache.flink.configuration.StateLatencyTrackOptions;
import org.apache.flink.core.fs.CloseableRegistry;
import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
import org.apache.flink.core.testutils.CheckedThread;
Expand Down Expand Up @@ -277,7 +277,7 @@ protected MetricGroup getMetricGroup() {
void testEnableStateLatencyTracking() throws Exception {
ConfigurableStateBackend stateBackend = getStateBackend();
Configuration config = new Configuration();
config.set(StateBackendOptions.LATENCY_TRACK_ENABLED, true);
config.set(StateLatencyTrackOptions.LATENCY_TRACK_ENABLED, true);
StateBackend configuredBackend =
stateBackend.configure(config, Thread.currentThread().getContextClassLoader());
KeyGroupRange groupRange = new KeyGroupRange(0, 1);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@
package org.apache.flink.runtime.state.metrics;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.StateBackendOptions;
import org.apache.flink.configuration.StateLatencyTrackOptions;
import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;

import org.junit.jupiter.api.Test;
Expand All @@ -46,11 +46,17 @@ void testDefaultEnabledLatencyTrackingStateConfig() {
.build();
assertThat(latencyTrackingStateConfig.isEnabled()).isTrue();
assertThat(latencyTrackingStateConfig.getSampleInterval())
.isEqualTo((int) StateBackendOptions.LATENCY_TRACK_SAMPLE_INTERVAL.defaultValue());
.isEqualTo(
(int)
StateLatencyTrackOptions.LATENCY_TRACK_SAMPLE_INTERVAL
.defaultValue());
assertThat(latencyTrackingStateConfig.getHistorySize())
.isEqualTo((long) StateBackendOptions.LATENCY_TRACK_HISTORY_SIZE.defaultValue());
.isEqualTo(
(long) StateLatencyTrackOptions.LATENCY_TRACK_HISTORY_SIZE.defaultValue());
assertThat(latencyTrackingStateConfig.isStateNameAsVariable())
.isEqualTo(StateBackendOptions.LATENCY_TRACK_STATE_NAME_AS_VARIABLE.defaultValue());
.isEqualTo(
StateLatencyTrackOptions.LATENCY_TRACK_STATE_NAME_AS_VARIABLE
.defaultValue());
}

@Test
Expand All @@ -72,9 +78,9 @@ void testSetLatencyTrackingStateConfig() {
void testConfigureFromReadableConfig() {
LatencyTrackingStateConfig.Builder builder = LatencyTrackingStateConfig.newBuilder();
Configuration configuration = new Configuration();
configuration.set(StateBackendOptions.LATENCY_TRACK_ENABLED, true);
configuration.set(StateBackendOptions.LATENCY_TRACK_SAMPLE_INTERVAL, 10);
configuration.set(StateBackendOptions.LATENCY_TRACK_HISTORY_SIZE, 500);
configuration.set(StateLatencyTrackOptions.LATENCY_TRACK_ENABLED, true);
configuration.set(StateLatencyTrackOptions.LATENCY_TRACK_SAMPLE_INTERVAL, 10);
configuration.set(StateLatencyTrackOptions.LATENCY_TRACK_HISTORY_SIZE, 500);
LatencyTrackingStateConfig latencyTrackingStateConfig =
builder.configure(configuration)
.setMetricGroup(new UnregisteredMetricsGroup())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
import org.apache.flink.api.common.state.StateDescriptor;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.StateBackendOptions;
import org.apache.flink.configuration.StateLatencyTrackOptions;
import org.apache.flink.core.fs.CloseableRegistry;
import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
import org.apache.flink.runtime.execution.Environment;
Expand Down Expand Up @@ -58,11 +58,11 @@ protected AbstractKeyedStateBackend<K> createKeyedBackend(TypeSerializer<K> keyS
KeyGroupRange keyGroupRange = new KeyGroupRange(0, 127);
int numberOfKeyGroups = keyGroupRange.getNumberOfKeyGroups();
Configuration configuration = new Configuration();
configuration.set(StateBackendOptions.LATENCY_TRACK_ENABLED, true);
configuration.set(StateBackendOptions.LATENCY_TRACK_SAMPLE_INTERVAL, SAMPLE_INTERVAL);
configuration.set(StateLatencyTrackOptions.LATENCY_TRACK_ENABLED, true);
configuration.set(StateLatencyTrackOptions.LATENCY_TRACK_SAMPLE_INTERVAL, SAMPLE_INTERVAL);
// use a very large value to not let metrics data overridden.
int historySize = 1000_000;
configuration.set(StateBackendOptions.LATENCY_TRACK_HISTORY_SIZE, historySize);
configuration.set(StateLatencyTrackOptions.LATENCY_TRACK_HISTORY_SIZE, historySize);
HashMapStateBackend stateBackend =
new HashMapStateBackend()
.configure(configuration, Thread.currentThread().getContextClassLoader());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
import org.apache.flink.api.common.time.Time;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.StateBackendOptions;
import org.apache.flink.configuration.StateLatencyTrackOptions;
import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend;
import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackendTest;
import org.apache.flink.runtime.execution.Environment;
Expand Down Expand Up @@ -105,7 +105,7 @@ public void testMaterializedRestoreWithWrappedState() throws Exception {
CheckpointStreamFactory streamFactory = createStreamFactory();

Configuration configuration = new Configuration();
configuration.set(StateBackendOptions.LATENCY_TRACK_ENABLED, true);
configuration.set(StateLatencyTrackOptions.LATENCY_TRACK_ENABLED, true);
StateBackend stateBackend =
getStateBackend()
.configure(configuration, Thread.currentThread().getContextClassLoader());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
import org.apache.flink.api.common.time.Time;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.StateBackendOptions;
import org.apache.flink.configuration.StateLatencyTrackOptions;
import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.runtime.state.CheckpointStorage;
import org.apache.flink.runtime.state.CheckpointStreamFactory;
Expand Down Expand Up @@ -101,7 +101,7 @@ public void testMaterializedRestoreWithWrappedState() throws Exception {
CheckpointStreamFactory streamFactory = createStreamFactory();

Configuration configuration = new Configuration();
configuration.set(StateBackendOptions.LATENCY_TRACK_ENABLED, true);
configuration.set(StateLatencyTrackOptions.LATENCY_TRACK_ENABLED, true);
StateBackend stateBackend =
getStateBackend()
.configure(configuration, Thread.currentThread().getContextClassLoader());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
import org.apache.flink.api.common.time.Time;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.StateBackendOptions;
import org.apache.flink.configuration.StateLatencyTrackOptions;
import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.runtime.state.CheckpointStreamFactory;
import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend;
Expand Down Expand Up @@ -92,7 +92,7 @@ public void testMaterializedRestoreWithWrappedState() throws Exception {
CheckpointStreamFactory streamFactory = createStreamFactory();

Configuration configuration = new Configuration();
configuration.set(StateBackendOptions.LATENCY_TRACK_ENABLED, true);
configuration.set(StateLatencyTrackOptions.LATENCY_TRACK_ENABLED, true);
StateBackend stateBackend =
getStateBackend()
.configure(configuration, Thread.currentThread().getContextClassLoader());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
import org.apache.flink.api.common.time.Time;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.StateBackendOptions;
import org.apache.flink.configuration.StateLatencyTrackOptions;
import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.runtime.state.CheckpointStorage;
import org.apache.flink.runtime.state.CheckpointStreamFactory;
Expand Down Expand Up @@ -101,7 +101,7 @@ public void testMaterializedRestoreWithWrappedState() throws Exception {
CheckpointStreamFactory streamFactory = createStreamFactory();

Configuration configuration = new Configuration();
configuration.set(StateBackendOptions.LATENCY_TRACK_ENABLED, true);
configuration.set(StateLatencyTrackOptions.LATENCY_TRACK_ENABLED, true);
StateBackend stateBackend =
getStateBackend()
.configure(configuration, Thread.currentThread().getContextClassLoader());
Expand Down

0 comments on commit 0a40ca8

Please sign in to comment.