-
Notifications
You must be signed in to change notification settings - Fork 4.1k
/
SpoutOutputCollectorImpl.java
181 lines (162 loc) · 7.35 KB
/
SpoutOutputCollectorImpl.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
/**
* 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.storm.executor.spout;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
import org.apache.storm.daemon.Acker;
import org.apache.storm.daemon.Task;
import org.apache.storm.executor.TupleInfo;
import org.apache.storm.spout.ISpout;
import org.apache.storm.spout.ISpoutOutputCollector;
import org.apache.storm.tuple.AddressedTuple;
import org.apache.storm.tuple.MessageId;
import org.apache.storm.tuple.TupleImpl;
import org.apache.storm.tuple.Values;
import org.apache.storm.utils.MutableLong;
import org.apache.storm.utils.RotatingMap;
import org.apache.storm.utils.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Methods are not thread safe. Each thread expected to have a separate instance, or else synchronize externally
*/
public class SpoutOutputCollectorImpl implements ISpoutOutputCollector {
private static final Logger LOG = LoggerFactory.getLogger(SpoutOutputCollectorImpl.class);
private final SpoutExecutor executor;
private final Task taskData;
private final int taskId;
private final MutableLong emittedCount;
private final boolean hasAckers;
private final Random random;
private final Boolean isEventLoggers;
private final Boolean isDebug;
private final RotatingMap<Long, TupleInfo> pending;
private final long spoutExecutorThdId;
private TupleInfo globalTupleInfo = new TupleInfo();
// thread safety: assumes Collector.emit*() calls are externally synchronized (if needed).
@SuppressWarnings("unused")
public SpoutOutputCollectorImpl(ISpout spout, SpoutExecutor executor, Task taskData,
MutableLong emittedCount, boolean hasAckers, Random random,
Boolean isEventLoggers, Boolean isDebug, RotatingMap<Long, TupleInfo> pending) {
this.executor = executor;
this.taskData = taskData;
this.taskId = taskData.getTaskId();
this.emittedCount = emittedCount;
this.hasAckers = hasAckers;
this.random = random;
this.isEventLoggers = isEventLoggers;
this.isDebug = isDebug;
this.pending = pending;
this.spoutExecutorThdId = executor.getThreadId();
}
@Override
public List<Integer> emit(String streamId, List<Object> tuple, Object messageId) {
try {
return sendSpoutMsg(streamId, tuple, messageId, null);
} catch (InterruptedException e) {
LOG.warn("Spout thread interrupted during emit().");
throw new RuntimeException(e);
}
}
@Override
public void emitDirect(int taskId, String streamId, List<Object> tuple, Object messageId) {
try {
sendSpoutMsg(streamId, tuple, messageId, taskId);
} catch (InterruptedException e) {
LOG.warn("Spout thread interrupted during emitDirect().");
throw new RuntimeException(e);
}
}
@Override
public void flush() {
try {
executor.getExecutorTransfer().flush();
} catch (InterruptedException e) {
LOG.warn("Spout thread interrupted during flush().");
throw new RuntimeException(e);
}
}
@Override
public long getPendingCount() {
return pending.size();
}
@Override
public void reportError(Throwable error) {
executor.getErrorReportingMetrics().incrReportedErrorCount();
executor.getReportError().report(error);
}
private List<Integer> sendSpoutMsg(String stream, List<Object> values, Object messageId, Integer outTaskId) throws
InterruptedException {
emittedCount.increment();
List<Integer> outTasks;
if (outTaskId != null) {
outTasks = taskData.getOutgoingTasks(outTaskId, stream, values);
} else {
outTasks = taskData.getOutgoingTasks(stream, values);
}
final boolean needAck = (messageId != null) && hasAckers;
final List<Long> ackSeq = needAck ? new ArrayList<>() : null;
final long rootId = needAck ? MessageId.generateId(random) : 0;
for (int i = 0; i < outTasks.size(); i++) { // perf critical path. don't use iterators.
Integer t = outTasks.get(i);
MessageId msgId;
if (needAck) {
long as = MessageId.generateId(random);
msgId = MessageId.makeRootId(rootId, as);
ackSeq.add(as);
} else {
msgId = MessageId.makeUnanchored();
}
final TupleImpl tuple =
new TupleImpl(executor.getWorkerTopologyContext(), values, executor.getComponentId(), this.taskId, stream, msgId);
AddressedTuple adrTuple = new AddressedTuple(t, tuple);
executor.getExecutorTransfer().tryTransfer(adrTuple, executor.getPendingEmits());
}
if (isEventLoggers) {
taskData.sendToEventLogger(executor, values, executor.getComponentId(), messageId, random, executor.getPendingEmits());
}
if (needAck) {
boolean sample = executor.samplerCheck();
TupleInfo info = new TupleInfo();
info.setTaskId(this.taskId);
info.setStream(stream);
info.setMessageId(messageId);
if (isDebug) {
info.setValues(values);
}
if (sample) {
info.setTimestamp(System.currentTimeMillis());
}
pending.put(rootId, info);
List<Object> ackInitTuple = new Values(rootId, Utils.bitXorVals(ackSeq), this.taskId);
taskData.sendUnanchored(Acker.ACKER_INIT_STREAM_ID, ackInitTuple, executor.getExecutorTransfer(), executor.getPendingEmits());
} else if (messageId != null) {
// Reusing TupleInfo object as we directly call executor.ackSpoutMsg() & are not sending msgs. perf critical
if (isDebug) {
if (spoutExecutorThdId != Thread.currentThread().getId()) {
throw new RuntimeException("Detected background thread emitting tuples for the spout. " +
"Spout Output Collector should only emit from the main spout executor thread.");
}
}
globalTupleInfo.clear();
globalTupleInfo.setStream(stream);
globalTupleInfo.setValues(values);
globalTupleInfo.setMessageId(messageId);
globalTupleInfo.setTimestamp(0);
globalTupleInfo.setId("0:");
Long timeDelta = 0L;
executor.ackSpoutMsg(executor, taskData, timeDelta, globalTupleInfo);
}
return outTasks;
}
}