22
22
import org .apache .flink .api .common .TaskInfo ;
23
23
import org .apache .flink .api .common .accumulators .Accumulator ;
24
24
import org .apache .flink .api .common .functions .ReduceFunction ;
25
+ import org .apache .flink .api .common .state .ListState ;
25
26
import org .apache .flink .api .common .state .ListStateDescriptor ;
26
27
import org .apache .flink .api .common .state .ReducingStateDescriptor ;
27
28
import org .apache .flink .api .common .state .StateDescriptor ;
28
29
import org .apache .flink .api .common .state .ValueStateDescriptor ;
29
30
import org .apache .flink .api .common .typeutils .TypeSerializer ;
31
+ import org .apache .flink .api .common .typeutils .base .StringSerializer ;
32
+ import org .apache .flink .api .common .typeutils .base .VoidSerializer ;
30
33
import org .apache .flink .api .java .typeutils .runtime .kryo .KryoSerializer ;
31
34
import org .apache .flink .core .fs .Path ;
32
35
import org .apache .flink .runtime .execution .Environment ;
33
36
37
+ import org .apache .flink .runtime .state .memory .MemListState ;
34
38
import org .junit .Test ;
35
39
36
40
import org .mockito .invocation .InvocationOnMock ;
@@ -54,7 +58,7 @@ public void testValueStateInstantiation() throws Exception {
54
58
final AtomicReference <Object > descriptorCapture = new AtomicReference <>();
55
59
56
60
StreamingRuntimeContext context = new StreamingRuntimeContext (
57
- createMockOp (descriptorCapture , config ),
61
+ createDescriptorCapturingMockOp (descriptorCapture , config ),
58
62
createMockEnvironment (),
59
63
Collections .<String , Accumulator <?, ?>>emptyMap ());
60
64
@@ -78,7 +82,7 @@ public void testReduceingStateInstantiation() throws Exception {
78
82
final AtomicReference <Object > descriptorCapture = new AtomicReference <>();
79
83
80
84
StreamingRuntimeContext context = new StreamingRuntimeContext (
81
- createMockOp (descriptorCapture , config ),
85
+ createDescriptorCapturingMockOp (descriptorCapture , config ),
82
86
createMockEnvironment (),
83
87
Collections .<String , Accumulator <?, ?>>emptyMap ());
84
88
@@ -107,7 +111,7 @@ public void testListStateInstantiation() throws Exception {
107
111
final AtomicReference <Object > descriptorCapture = new AtomicReference <>();
108
112
109
113
StreamingRuntimeContext context = new StreamingRuntimeContext (
110
- createMockOp (descriptorCapture , config ),
114
+ createDescriptorCapturingMockOp (descriptorCapture , config ),
111
115
createMockEnvironment (),
112
116
Collections .<String , Accumulator <?, ?>>emptyMap ());
113
117
@@ -121,13 +125,29 @@ public void testListStateInstantiation() throws Exception {
121
125
assertTrue (serializer instanceof KryoSerializer );
122
126
assertTrue (((KryoSerializer <?>) serializer ).getKryo ().getRegistration (Path .class ).getId () > 0 );
123
127
}
128
+
129
+ @ Test
130
+ public void testListStateReturnsEmptyListByDefault () throws Exception {
131
+
132
+ StreamingRuntimeContext context = new StreamingRuntimeContext (
133
+ createPlainMockOp (),
134
+ createMockEnvironment (),
135
+ Collections .<String , Accumulator <?, ?>>emptyMap ());
136
+
137
+ ListStateDescriptor <String > descr = new ListStateDescriptor <>("name" , String .class );
138
+ ListState <String > state = context .getListState (descr );
139
+
140
+ Iterable <String > value = state .get ();
141
+ assertNotNull (value );
142
+ assertFalse (value .iterator ().hasNext ());
143
+ }
124
144
125
145
// ------------------------------------------------------------------------
126
146
//
127
147
// ------------------------------------------------------------------------
128
148
129
149
@ SuppressWarnings ("unchecked" )
130
- private static AbstractStreamOperator <?> createMockOp (
150
+ private static AbstractStreamOperator <?> createDescriptorCapturingMockOp (
131
151
final AtomicReference <Object > ref , final ExecutionConfig config ) throws Exception {
132
152
133
153
AbstractStreamOperator <?> operatorMock = mock (AbstractStreamOperator .class );
@@ -145,6 +165,27 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
145
165
146
166
return operatorMock ;
147
167
}
168
+
169
+ @ SuppressWarnings ("unchecked" )
170
+ private static AbstractStreamOperator <?> createPlainMockOp () throws Exception {
171
+
172
+ AbstractStreamOperator <?> operatorMock = mock (AbstractStreamOperator .class );
173
+ when (operatorMock .getExecutionConfig ()).thenReturn (new ExecutionConfig ());
174
+
175
+ when (operatorMock .getPartitionedState (any (ListStateDescriptor .class ))).thenAnswer (
176
+ new Answer <ListState <String >>() {
177
+
178
+ @ Override
179
+ public ListState <String > answer (InvocationOnMock invocationOnMock ) throws Throwable {
180
+ ListStateDescriptor <String > descr =
181
+ (ListStateDescriptor <String >) invocationOnMock .getArguments ()[0 ];
182
+ return new MemListState <String , Void , String >(
183
+ StringSerializer .INSTANCE , VoidSerializer .INSTANCE , descr );
184
+ }
185
+ });
186
+
187
+ return operatorMock ;
188
+ }
148
189
149
190
private static Environment createMockEnvironment () {
150
191
Environment env = mock (Environment .class );
0 commit comments