From 6f9a8840fc3cf58e3b43ae4ce1da5a9023888442 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Tue, 11 Dec 2018 15:21:25 +0100 Subject: [PATCH] [hotfix] Remove unused generic parameter from RocksDB states --- .../streaming/state/AbstractRocksDBAppendingState.java | 5 ++--- .../flink/contrib/streaming/state/AbstractRocksDBState.java | 3 +-- .../contrib/streaming/state/RocksDBAggregatingState.java | 2 +- .../flink/contrib/streaming/state/RocksDBFoldingState.java | 2 +- .../contrib/streaming/state/RocksDBKeyedStateBackend.java | 2 +- .../flink/contrib/streaming/state/RocksDBListState.java | 2 +- .../flink/contrib/streaming/state/RocksDBMapState.java | 2 +- .../flink/contrib/streaming/state/RocksDBReducingState.java | 2 +- .../flink/contrib/streaming/state/RocksDBValueState.java | 2 +- 9 files changed, 10 insertions(+), 12 deletions(-) diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBAppendingState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBAppendingState.java index 8c0f4d7da7b..04e665a6e87 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBAppendingState.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBAppendingState.java @@ -18,7 +18,6 @@ package org.apache.flink.contrib.streaming.state; -import org.apache.flink.api.common.state.State; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.state.internal.InternalAppendingState; import org.apache.flink.util.FlinkRuntimeException; @@ -28,8 +27,8 @@ import org.rocksdb.RocksDBException; import java.io.IOException; -abstract class AbstractRocksDBAppendingState - extends AbstractRocksDBState +abstract class AbstractRocksDBAppendingState + extends AbstractRocksDBState implements InternalAppendingState { /** diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java index 2218bc0d3e0..b5ab996b1f6 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java @@ -44,9 +44,8 @@ import java.io.IOException; * @param The type of the key. * @param The type of the namespace. * @param The type of values kept internally in state. - * @param The type of {@link State}. */ -public abstract class AbstractRocksDBState implements InternalKvState, State { +public abstract class AbstractRocksDBState implements InternalKvState, State { /** Serializer for the namespace. */ final TypeSerializer namespaceSerializer; diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java index 2085fb86256..770c558c827 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java @@ -43,7 +43,7 @@ import java.util.Collection; * @param The type of the value returned from the state */ class RocksDBAggregatingState - extends AbstractRocksDBAppendingState> + extends AbstractRocksDBAppendingState implements InternalAggregatingState { /** User-specified aggregation function. */ diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBFoldingState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBFoldingState.java index 4d6635704b9..c5e830fcedf 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBFoldingState.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBFoldingState.java @@ -42,7 +42,7 @@ import org.rocksdb.ColumnFamilyHandle; */ @Deprecated class RocksDBFoldingState - extends AbstractRocksDBAppendingState> + extends AbstractRocksDBAppendingState implements InternalFoldingState { /** User-specified fold function. */ diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index 700c5468c8c..72445ca32dd 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -1400,7 +1400,7 @@ public class RocksDBKeyedStateBackend extends AbstractKeyedStateBackend { } @SuppressWarnings("unchecked") - AbstractRocksDBState rocksDBState = (AbstractRocksDBState) state; + AbstractRocksDBState rocksDBState = (AbstractRocksDBState) state; Snapshot rocksDBSnapshot = db.getSnapshot(); try ( diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java index 6904c853fe3..cedeed8753c 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java @@ -59,7 +59,7 @@ import static org.apache.flink.runtime.state.StateSnapshotTransformer.Collection * @param The type of the values in the list state. */ class RocksDBListState - extends AbstractRocksDBState, ListState> + extends AbstractRocksDBState> implements InternalListState { /** Serializer for the values. */ diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java index cb656b53b1b..7abbe3d2c45 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java @@ -64,7 +64,7 @@ import java.util.Map; * @param The type of the values in the map state. */ class RocksDBMapState - extends AbstractRocksDBState, MapState> + extends AbstractRocksDBState> implements InternalMapState { private static final Logger LOG = LoggerFactory.getLogger(RocksDBMapState.class); diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBReducingState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBReducingState.java index 138357b0d77..69736ae03f7 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBReducingState.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBReducingState.java @@ -41,7 +41,7 @@ import java.util.Collection; * @param The type of value that the state state stores. */ class RocksDBReducingState - extends AbstractRocksDBAppendingState> + extends AbstractRocksDBAppendingState implements InternalReducingState { /** User-specified reduce function. */ diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBValueState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBValueState.java index 0ca90d4a521..97b83dfa49c 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBValueState.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBValueState.java @@ -40,7 +40,7 @@ import java.io.IOException; * @param The type of value that the state state stores. */ class RocksDBValueState - extends AbstractRocksDBState> + extends AbstractRocksDBState implements InternalValueState { /** -- GitLab