This repository has been archived by the owner on Nov 18, 2024. It is now read-only.
-
Notifications
You must be signed in to change notification settings - Fork 2
/
Copy pathstorm.js
325 lines (280 loc) · 10.1 KB
/
storm.js
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
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
/*
* 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.
*/
/**
* Base classes in node-js for storm Bolt and Spout.
* Implements the storm multilang protocol for nodejs.
*/
var fs = require('fs');
function Storm() {
this.messagePart = "";
this.taskIdsCallbacks = [];
this.isFirstMessage = true;
this.separator = '\nend\n';
}
Storm.prototype.sendMsgToParent = function(msg) {
process.stdout.write(JSON.stringify(msg) + this.separator);
}
Storm.prototype.sync = function() {
this.sendMsgToParent({"command":"sync"});
}
Storm.prototype.sendPid = function(heartbeatdir) {
var pid = process.pid;
fs.closeSync(fs.openSync(heartbeatdir + "/" + pid, "w"));
this.sendMsgToParent({"pid": pid})
}
Storm.prototype.log = function(msg) {
this.sendMsgToParent({"command": "log", "msg": msg});
}
Storm.prototype.error = function(msg) {
this.sendMsgToParent({"command": "error", "msg": msg});
}
Storm.prototype.initSetupInfo = function(setupInfo) {
var self = this;
var callback = function(err) {
if (err) {
throw err;
}
self.sendPid(setupInfo['pidDir']);
}
this.initialize(setupInfo['conf'], setupInfo['context'], callback);
}
Storm.prototype.startReadingInput = function() {
var self = this;
process.stdin.on('readable', function() {
var chunk = process.stdin.read();
var messages = self.handleNewChunk(chunk);
messages.forEach(function(message) {
self.handleNewMessage(message);
})
});
}
/**
* receives a new string chunk and returns a list of new messages with the separator removed
* stores state in this.messagePart
* @param chunk
*/
Storm.prototype.handleNewChunk = function(chunk) {
//invariant: this.messagePart has no separator otherwise we would have parsed it already
var messages = [];
if (chunk && chunk.length !== 0) {
//"{}".split("\nend\n") ==> ['{}']
//"\nend\n".split("\nend\n") ==> ['' , '']
//"{}\nend\n".split("\nend\n") ==> ['{}', '']
//"\nend\n{}".split("\nend\n") ==> ['' , '{}']
// "{}\nend\n{}".split("\nend\n") ==> ['{}', '{}' ]
this.messagePart = this.messagePart + chunk;
var newMessageParts = this.messagePart.split(this.separator);
while (newMessageParts.length > 0) {
var potentialMessage = newMessageParts.shift();
var anotherMessageAhead = newMessageParts.length > 0;
if (!anotherMessageAhead) {
this.messagePart = potentialMessage;
}
else if (potentialMessage.length > 0) {
messages.push(potentialMessage);
}
}
}
return messages;
}
Storm.prototype.isTaskIds = function(msg) {
return (msg instanceof Array);
}
Storm.prototype.handleNewMessage = function(msg) {
var parsedMsg;
try {
parsedMsg = JSON.parse(msg);
}
catch (err) {
throw err;
}
if (this.isFirstMessage) {
this.initSetupInfo(parsedMsg);
this.isFirstMessage = false;
} else if (this.isTaskIds(parsedMsg)) {
this.handleNewTaskId(parsedMsg);
} else {
this.handleNewCommand(parsedMsg);
}
}
Storm.prototype.handleNewTaskId = function(taskIds) {
//When new list of task ids arrives, the callback that was passed with the corresponding emit should be called.
//Storm assures that the task ids will be sent in the same order as their corresponding emits so it we can simply
//take the first callback in the list and be sure it is the right one.
var callback = this.taskIdsCallbacks.shift();
if (callback) {
callback(taskIds);
} else {
throw new Error('Something went wrong, we off the split of task id callbacks');
}
}
/**
*
* @param messageDetails json with the emit details.
*
* For bolt, the json must contain the required fields:
* - tuple - the value to emit
* - anchorTupleId - the value of the anchor tuple (the input tuple that lead to this emit). Used to track the source
* tuple and return ack when all components successfully finished to process it.
* and may contain the optional fields:
* - stream (if empty - emit to default stream)
*
* For spout, the json must contain the required fields:
* - tuple - the value to emit
*
* and may contain the optional fields:
* - id - pass id for reliable emit (and receive ack/fail later).
* - stream - if empty - emit to default stream.
*
* @param onTaskIds function than will be called with list of task ids the message was emitted to (when received).
*/
Storm.prototype.emit = function(messageDetails, onTaskIds) {
//Every emit triggers a response - list of task ids to which the tuple was emitted. The task ids are accessible
//through the callback (will be called when the response arrives). The callback is stored in a list until the
//corresponding task id list arrives.
if (messageDetails.task) {
throw new Error('Illegal input - task. To emit to specific task use emit direct!');
}
if (!onTaskIds) {
throw new Error('You must pass a onTaskIds callback when using emit!')
}
this.taskIdsCallbacks.push(onTaskIds);
this.__emit(messageDetails);;
}
/**
* Emit message to specific task.
* @param messageDetails json with the emit details.
*
* For bolt, the json must contain the required fields:
* - tuple - the value to emit
* - anchorTupleId - the value of the anchor tuple (the input tuple that lead to this emit). Used to track the source
* tuple and return ack when all components successfully finished to process it.
* - task - indicate the task to send the tuple to.
* and may contain the optional fields:
* - stream (if empty - emit to default stream)
*
* For spout, the json must contain the required fields:
* - tuple - the value to emit
* - task - indicate the task to send the tuple to.
* and may contain the optional fields:
* - id - pass id for reliable emit (and receive ack/fail later).
* - stream - if empty - emit to default stream.
*
* @param onTaskIds function than will be called with list of task ids the message was emitted to (when received).
*/
Storm.prototype.emitDirect = function(commandDetails) {
if (!commandDetails.task) {
throw new Error("Emit direct must receive task id!")
}
this.__emit(commandDetails);
}
/**
* Initialize storm component according to the configuration received.
* @param conf configuration object accrding to storm protocol.
* @param context context object according to storm protocol.
* @param done callback. Call this method when finished initializing.
*/
Storm.prototype.initialize = function(conf, context, done) {
done();
}
Storm.prototype.run = function() {
process.stdout.setEncoding('utf8');
process.stdin.setEncoding('utf8');
this.startReadingInput();
}
function Tuple(id, component, stream, task, values) {
this.id = id;
this.component = component;
this.stream = stream;
this.task = task;
this.values = values;
}
/**
* Base class for storm bolt.
* To create a bolt implement 'process' method.
* You may also implement initialize method to
*/
function BasicBolt() {
Storm.call(this);
this.anchorTuple = null;
};
BasicBolt.prototype = Object.create(Storm.prototype);
BasicBolt.prototype.constructor = BasicBolt;
/**
* Emit message.
* @param commandDetails json with the required fields:
* - tuple - the value to emit
* - anchorTupleId - the value of the anchor tuple (the input tuple that lead to this emit). Used to track the source
* tuple and return ack when all components successfully finished to process it.
* and the optional fields:
* - stream (if empty - emit to default stream)
* - task (pass only to emit to specific task)
*/
BasicBolt.prototype.__emit = function(commandDetails) {
var self = this;
var message = {
command: "emit",
tuple: commandDetails.tuple,
stream: commandDetails.stream,
task: commandDetails.task,
anchors: [commandDetails.anchorTupleId]
};
this.sendMsgToParent(message);
}
BasicBolt.prototype.handleNewCommand = function(command) {
var self = this;
var tup = new Tuple(command["id"], command["comp"], command["stream"], command["task"], command["tuple"]);
if (tup.task === -1 && tup.stream === "__heartbeat") {
self.sync();
return;
}
var callback = function(err) {
if (err) {
self.fail(tup, err);
return;
}
self.ack(tup);
}
this.process(tup, callback);
}
/**
* Implement this method when creating a bolt. This is the main method that provides the logic of the bolt (what
* should it do?).
* @param tuple the input of the bolt - what to process.
* @param done call this method when done processing.
*/
BasicBolt.prototype.process = function(tuple, done) {};
BasicBolt.prototype.ack = function(tup) {
this.sendMsgToParent({"command": "ack", "id": tup.id});
}
BasicBolt.prototype.fail = function(tup, err) {
this.sendMsgToParent({"command": "fail", "id": tup.id});
}
function SimpleBolt() {
BasicBolt.call(this);
}
SimpleBolt.prototype = new BasicBolt();
SimpleBolt.prototype = Object.create(BasicBolt.prototype);
SimpleBolt.prototype.process = function(tup, done) {
this.emit({tuple: tup.values, anchorTupleId: tup.id}, function(){});
done();
};
SimpleBolt.prototype.initialize = function(conf, context, initializeDone) {
initializeDone();
};
new SimpleBolt().run();