Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
6f9a8840
F
flink
项目概览
doujutun3207
/
flink
与 Fork 源项目一致
从无法访问的项目Fork
通知
24
Star
0
Fork
0
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
0
列表
看板
标记
里程碑
合并请求
0
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
F
flink
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
0
Issue
0
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
提交
Issue看板
体验新版 GitCode,发现更多精彩内容 >>
提交
6f9a8840
编写于
12月 11, 2018
作者:
S
Stefan Richter
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[hotfix] Remove unused generic parameter from RocksDB states
上级
28e0b83d
变更
9
隐藏空白更改
内联
并排
Showing
9 changed file
with
10 addition
and
12 deletion
+10
-12
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBAppendingState.java
...ontrib/streaming/state/AbstractRocksDBAppendingState.java
+2
-3
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java
...e/flink/contrib/streaming/state/AbstractRocksDBState.java
+1
-2
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java
...link/contrib/streaming/state/RocksDBAggregatingState.java
+1
-1
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBFoldingState.java
...he/flink/contrib/streaming/state/RocksDBFoldingState.java
+1
-1
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
...ink/contrib/streaming/state/RocksDBKeyedStateBackend.java
+1
-1
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
...pache/flink/contrib/streaming/state/RocksDBListState.java
+1
-1
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java
...apache/flink/contrib/streaming/state/RocksDBMapState.java
+1
-1
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBReducingState.java
...e/flink/contrib/streaming/state/RocksDBReducingState.java
+1
-1
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBValueState.java
...ache/flink/contrib/streaming/state/RocksDBValueState.java
+1
-1
未找到文件。
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBAppendingState.java
浏览文件 @
6f9a8840
...
...
@@ -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
<
K
,
N
,
IN
,
SV
,
OUT
,
S
extends
State
>
extends
AbstractRocksDBState
<
K
,
N
,
SV
,
S
>
abstract
class
AbstractRocksDBAppendingState
<
K
,
N
,
IN
,
SV
,
OUT
>
extends
AbstractRocksDBState
<
K
,
N
,
SV
>
implements
InternalAppendingState
<
K
,
N
,
IN
,
SV
,
OUT
>
{
/**
...
...
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java
浏览文件 @
6f9a8840
...
...
@@ -44,9 +44,8 @@ import java.io.IOException;
* @param <K> The type of the key.
* @param <N> The type of the namespace.
* @param <V> The type of values kept internally in state.
* @param <S> The type of {@link State}.
*/
public
abstract
class
AbstractRocksDBState
<
K
,
N
,
V
,
S
extends
State
>
implements
InternalKvState
<
K
,
N
,
V
>,
State
{
public
abstract
class
AbstractRocksDBState
<
K
,
N
,
V
>
implements
InternalKvState
<
K
,
N
,
V
>,
State
{
/** Serializer for the namespace. */
final
TypeSerializer
<
N
>
namespaceSerializer
;
...
...
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java
浏览文件 @
6f9a8840
...
...
@@ -43,7 +43,7 @@ import java.util.Collection;
* @param <R> The type of the value returned from the state
*/
class
RocksDBAggregatingState
<
K
,
N
,
T
,
ACC
,
R
>
extends
AbstractRocksDBAppendingState
<
K
,
N
,
T
,
ACC
,
R
,
AggregatingState
<
T
,
R
>
>
extends
AbstractRocksDBAppendingState
<
K
,
N
,
T
,
ACC
,
R
>
implements
InternalAggregatingState
<
K
,
N
,
T
,
ACC
,
R
>
{
/** User-specified aggregation function. */
...
...
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBFoldingState.java
浏览文件 @
6f9a8840
...
...
@@ -42,7 +42,7 @@ import org.rocksdb.ColumnFamilyHandle;
*/
@Deprecated
class
RocksDBFoldingState
<
K
,
N
,
T
,
ACC
>
extends
AbstractRocksDBAppendingState
<
K
,
N
,
T
,
ACC
,
ACC
,
FoldingState
<
T
,
ACC
>
>
extends
AbstractRocksDBAppendingState
<
K
,
N
,
T
,
ACC
,
ACC
>
implements
InternalFoldingState
<
K
,
N
,
T
,
ACC
>
{
/** User-specified fold function. */
...
...
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
浏览文件 @
6f9a8840
...
...
@@ -1400,7 +1400,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
}
@SuppressWarnings
(
"unchecked"
)
AbstractRocksDBState
<?,
?,
SV
,
S
>
rocksDBState
=
(
AbstractRocksDBState
<?,
?,
SV
,
S
>)
state
;
AbstractRocksDBState
<?,
?,
SV
>
rocksDBState
=
(
AbstractRocksDBState
<?,
?,
SV
>)
state
;
Snapshot
rocksDBSnapshot
=
db
.
getSnapshot
();
try
(
...
...
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
浏览文件 @
6f9a8840
...
...
@@ -59,7 +59,7 @@ import static org.apache.flink.runtime.state.StateSnapshotTransformer.Collection
* @param <V> The type of the values in the list state.
*/
class
RocksDBListState
<
K
,
N
,
V
>
extends
AbstractRocksDBState
<
K
,
N
,
List
<
V
>
,
ListState
<
V
>
>
extends
AbstractRocksDBState
<
K
,
N
,
List
<
V
>>
implements
InternalListState
<
K
,
N
,
V
>
{
/** Serializer for the values. */
...
...
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java
浏览文件 @
6f9a8840
...
...
@@ -64,7 +64,7 @@ import java.util.Map;
* @param <UV> The type of the values in the map state.
*/
class
RocksDBMapState
<
K
,
N
,
UK
,
UV
>
extends
AbstractRocksDBState
<
K
,
N
,
Map
<
UK
,
UV
>
,
MapState
<
UK
,
UV
>
>
extends
AbstractRocksDBState
<
K
,
N
,
Map
<
UK
,
UV
>>
implements
InternalMapState
<
K
,
N
,
UK
,
UV
>
{
private
static
final
Logger
LOG
=
LoggerFactory
.
getLogger
(
RocksDBMapState
.
class
);
...
...
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBReducingState.java
浏览文件 @
6f9a8840
...
...
@@ -41,7 +41,7 @@ import java.util.Collection;
* @param <V> The type of value that the state state stores.
*/
class
RocksDBReducingState
<
K
,
N
,
V
>
extends
AbstractRocksDBAppendingState
<
K
,
N
,
V
,
V
,
V
,
ReducingState
<
V
>
>
extends
AbstractRocksDBAppendingState
<
K
,
N
,
V
,
V
,
V
>
implements
InternalReducingState
<
K
,
N
,
V
>
{
/** User-specified reduce function. */
...
...
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBValueState.java
浏览文件 @
6f9a8840
...
...
@@ -40,7 +40,7 @@ import java.io.IOException;
* @param <V> The type of value that the state state stores.
*/
class
RocksDBValueState
<
K
,
N
,
V
>
extends
AbstractRocksDBState
<
K
,
N
,
V
,
ValueState
<
V
>
>
extends
AbstractRocksDBState
<
K
,
N
,
V
>
implements
InternalValueState
<
K
,
N
,
V
>
{
/**
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录