/
StormBoltWrapperTest.java
190 lines (154 loc) · 6.82 KB
/
StormBoltWrapperTest.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
/*
* 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.stormcompatibility.wrappers;
import backtype.storm.task.OutputCollector;
import backtype.storm.task.TopologyContext;
import backtype.storm.topology.IRichBolt;
import backtype.storm.tuple.Fields;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.powermock.api.mockito.PowerMockito;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
import java.util.Map;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.eq;
import static org.mockito.Matchers.isNull;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@RunWith(PowerMockRunner.class)
@PrepareForTest({StreamRecordSerializer.class, StormWrapperSetupHelper.class})
public class StormBoltWrapperTest {
@SuppressWarnings("unused")
@Test(expected = IllegalArgumentException.class)
public void testWrapperRawType() throws Exception {
final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer();
declarer.declare(new Fields("dummy1", "dummy2"));
PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
new StormBoltWrapper<Object, Object>(mock(IRichBolt.class), true);
}
@SuppressWarnings("unused")
@Test(expected = IllegalArgumentException.class)
public void testWrapperToManyAttributes1() throws Exception {
final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer();
final String[] schema = new String[26];
for (int i = 0; i < schema.length; ++i) {
schema[i] = "a" + i;
}
declarer.declare(new Fields(schema));
PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
new StormBoltWrapper<Object, Object>(mock(IRichBolt.class));
}
@SuppressWarnings("unused")
@Test(expected = IllegalArgumentException.class)
public void testWrapperToManyAttributes2() throws Exception {
final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer();
final String[] schema = new String[26];
for (int i = 0; i < schema.length; ++i) {
schema[i] = "a" + i;
}
declarer.declare(new Fields(schema));
PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
new StormBoltWrapper<Object, Object>(mock(IRichBolt.class), false);
}
@Test
public void testWrapper() throws Exception {
for (int i = 0; i < 26; ++i) {
this.testWrapper(i);
}
}
@SuppressWarnings({"rawtypes", "unchecked"})
private void testWrapper(final int numberOfAttributes) throws Exception {
assert ((0 <= numberOfAttributes) && (numberOfAttributes <= 25));
Tuple flinkTuple = null;
String rawTuple = null;
if (numberOfAttributes == 0) {
rawTuple = "test";
} else {
flinkTuple = Tuple.getTupleClass(numberOfAttributes).newInstance();
}
String[] schema = new String[numberOfAttributes];
if (numberOfAttributes == 0) {
schema = new String[1];
}
for (int i = 0; i < schema.length; ++i) {
schema[i] = "a" + i;
}
final StreamRecord record = mock(StreamRecord.class);
if (numberOfAttributes == 0) {
when(record.getValue()).thenReturn(rawTuple);
} else {
when(record.getValue()).thenReturn(flinkTuple);
}
final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class);
final IRichBolt bolt = mock(IRichBolt.class);
final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer();
declarer.declare(new Fields(schema));
PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
final StormBoltWrapper wrapper = new StormBoltWrapper(bolt);
wrapper.setup(mock(Output.class), taskContext);
wrapper.open(new Configuration());
wrapper.processElement(record);
if (numberOfAttributes == 0) {
verify(bolt).execute(eq(new StormTuple<String>(rawTuple)));
} else {
verify(bolt).execute(eq(new StormTuple<Tuple>(flinkTuple)));
}
}
@SuppressWarnings("unchecked")
@Test
public void testOpen() throws Exception {
final IRichBolt bolt = mock(IRichBolt.class);
final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer();
declarer.declare(new Fields("dummy"));
PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
final StormBoltWrapper<Object, Object> wrapper = new StormBoltWrapper<Object, Object>(bolt);
wrapper.setup(mock(Output.class), mock(StreamingRuntimeContext.class));
wrapper.open(mock(Configuration.class));
verify(bolt).prepare(any(Map.class), any(TopologyContext.class), any(OutputCollector.class));
}
@SuppressWarnings("unchecked")
@Test
public void testOpenSink() throws Exception {
final IRichBolt bolt = mock(IRichBolt.class);
final StormBoltWrapper<Object, Object> wrapper = new StormBoltWrapper<Object, Object>(bolt);
wrapper.setup(mock(Output.class), mock(StreamingRuntimeContext.class));
wrapper.open(mock(Configuration.class));
verify(bolt).prepare(any(Map.class), any(TopologyContext.class), isNull(OutputCollector.class));
}
@SuppressWarnings("unchecked")
@Test
public void testClose() throws Exception {
final IRichBolt bolt = mock(IRichBolt.class);
final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer();
declarer.declare(new Fields("dummy"));
PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
final StormBoltWrapper<Object, Object> wrapper = new StormBoltWrapper<Object, Object>(bolt);
final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class);
// when(taskContext.getOutputCollector()).thenReturn(mock(Collector.class));
wrapper.setup(mock(Output.class), taskContext);
wrapper.close();
verify(bolt).cleanup();
}
}