diff --git a/flink-core/src/main/java/org/apache/flink/types/DoubleValue.java b/flink-core/src/main/java/org/apache/flink/types/DoubleValue.java index 15f3a854cb495..45f0346b1c3e7 100644 --- a/flink-core/src/main/java/org/apache/flink/types/DoubleValue.java +++ b/flink-core/src/main/java/org/apache/flink/types/DoubleValue.java @@ -32,8 +32,7 @@ public class DoubleValue implements Comparable, ResettableValue, - CopyableValue, - Key { + CopyableValue { private static final long serialVersionUID = 1L; private double value; diff --git a/flink-core/src/main/java/org/apache/flink/types/FloatValue.java b/flink-core/src/main/java/org/apache/flink/types/FloatValue.java index 287fbaf08dfee..3bc83cd3e0423 100644 --- a/flink-core/src/main/java/org/apache/flink/types/FloatValue.java +++ b/flink-core/src/main/java/org/apache/flink/types/FloatValue.java @@ -30,10 +30,7 @@ */ @Public public class FloatValue - implements Comparable, - ResettableValue, - CopyableValue, - Key { + implements Comparable, ResettableValue, CopyableValue { private static final long serialVersionUID = 1L; private float value; diff --git a/flink-core/src/main/java/org/apache/flink/types/NormalizableKey.java b/flink-core/src/main/java/org/apache/flink/types/NormalizableKey.java index 72c0a3e483f1a..0433518af0fd9 100644 --- a/flink-core/src/main/java/org/apache/flink/types/NormalizableKey.java +++ b/flink-core/src/main/java/org/apache/flink/types/NormalizableKey.java @@ -34,7 +34,7 @@ * normalized key length. */ @Public -public interface NormalizableKey extends Comparable, Key { +public interface NormalizableKey extends Comparable { /** * Gets the maximal length of normalized keys that the data type would produce to determine the diff --git a/pom.xml b/pom.xml index d29130b179354..907b938e62da5 100644 --- a/pom.xml +++ b/pom.xml @@ -2393,6 +2393,10 @@ under the License. org.apache.flink.api.common.functions.RuntimeContext#getExecutionConfig() + + org.apache.flink.types.DoubleValue + org.apache.flink.types.FloatValue + org.apache.flink.types.NormalizableKey org.apache.flink.core.fs.Path