Browse Source

node: allow multiple arguments passed to nextTick

PR-URL: https://github.com/iojs/io.js/pull/1077
Reviewed-by: Colin Ihrig <cjihrig@gmail.com>
v1.8.0-commit
Trevor Norris 10 years ago
parent
commit
10e31ba56c
  1. 2
      doc/api/process.markdown
  2. 4
      lib/_debugger.js
  3. 26
      lib/_http_client.js
  4. 25
      lib/_http_outgoing.js
  5. 6
      lib/_stream_duplex.js
  6. 31
      lib/_stream_readable.js
  7. 22
      lib/_stream_writable.js
  8. 37
      lib/_tls_legacy.js
  9. 12
      lib/child_process.js
  10. 20
      lib/cluster.js
  11. 20
      lib/dgram.js
  12. 10
      lib/dns.js
  13. 8
      lib/fs.js
  14. 34
      lib/net.js
  15. 9
      lib/zlib.js
  16. 18
      src/node.js
  17. 8
      test/message/stdin_messages.out
  18. 7
      test/parallel/test-next-tick.js

2
doc/api/process.markdown

@ -687,7 +687,7 @@ This will generate:
`heapTotal` and `heapUsed` refer to V8's memory usage. `heapTotal` and `heapUsed` refer to V8's memory usage.
## process.nextTick(callback) ## process.nextTick(callback[, arg][, ...])
* `callback` {Function} * `callback` {Function}

4
lib/_debugger.js

@ -587,9 +587,7 @@ Client.prototype.mirrorObject = function(handle, depth, cb) {
} else { } else {
val = handle; val = handle;
} }
process.nextTick(function() { process.nextTick(cb, null, val);
cb(null, val);
});
}; };

26
lib/_http_client.js

@ -166,11 +166,8 @@ ClientRequest.prototype._implicitHeader = function() {
}; };
ClientRequest.prototype.abort = function() { ClientRequest.prototype.abort = function() {
var self = this;
if (this.aborted === undefined) { if (this.aborted === undefined) {
process.nextTick(function() { process.nextTick(emitAbortNT, this);
self.emit('abort');
});
} }
// Mark as aborting so we can avoid sending queued request data // Mark as aborting so we can avoid sending queued request data
// This is used as a truthy flag elsewhere. The use of Date.now is for // This is used as a truthy flag elsewhere. The use of Date.now is for
@ -194,6 +191,11 @@ ClientRequest.prototype.abort = function() {
}; };
function emitAbortNT(self) {
self.emit('abort');
}
function createHangUpError() { function createHangUpError() {
var error = new Error('socket hang up'); var error = new Error('socket hang up');
error.code = 'ECONNRESET'; error.code = 'ECONNRESET';
@ -440,12 +442,14 @@ function responseOnEnd() {
socket.removeListener('error', socketErrorListener); socket.removeListener('error', socketErrorListener);
// Mark this socket as available, AFTER user-added end // Mark this socket as available, AFTER user-added end
// handlers have a chance to run. // handlers have a chance to run.
process.nextTick(function() { process.nextTick(emitFreeNT, socket);
socket.emit('free');
});
} }
} }
function emitFreeNT(socket) {
socket.emit('free');
}
function tickOnSocket(req, socket) { function tickOnSocket(req, socket) {
var parser = parsers.alloc(); var parser = parsers.alloc();
req.socket = socket; req.socket = socket;
@ -478,17 +482,17 @@ function tickOnSocket(req, socket) {
} }
ClientRequest.prototype.onSocket = function(socket) { ClientRequest.prototype.onSocket = function(socket) {
var req = this; process.nextTick(onSocketNT, this, socket);
};
process.nextTick(function() { function onSocketNT(req, socket) {
if (req.aborted) { if (req.aborted) {
// If we were aborted while waiting for a socket, skip the whole thing. // If we were aborted while waiting for a socket, skip the whole thing.
socket.emit('free'); socket.emit('free');
} else { } else {
tickOnSocket(req, socket); tickOnSocket(req, socket);
} }
}); }
};
ClientRequest.prototype._deferToConnect = function(method, arguments_, cb) { ClientRequest.prototype._deferToConnect = function(method, arguments_, cb) {
// This function is for calls that need to happen once the socket is // This function is for calls that need to happen once the socket is

25
lib/_http_outgoing.js

@ -406,14 +406,9 @@ Object.defineProperty(OutgoingMessage.prototype, 'headersSent', {
OutgoingMessage.prototype.write = function(chunk, encoding, callback) { OutgoingMessage.prototype.write = function(chunk, encoding, callback) {
var self = this;
if (this.finished) { if (this.finished) {
var err = new Error('write after end'); var err = new Error('write after end');
process.nextTick(function() { process.nextTick(writeAfterEndNT, this, err, callback);
self.emit('error', err);
if (callback) callback(err);
});
return true; return true;
} }
@ -455,11 +450,7 @@ OutgoingMessage.prototype.write = function(chunk, encoding, callback) {
if (this.connection && !this.connection.corked) { if (this.connection && !this.connection.corked) {
this.connection.cork(); this.connection.cork();
var conn = this.connection; process.nextTick(connectionCorkNT, this.connection);
process.nextTick(function connectionCork() {
if (conn)
conn.uncork();
});
} }
this._send(len.toString(16), 'binary', null); this._send(len.toString(16), 'binary', null);
this._send(crlf_buf, null, null); this._send(crlf_buf, null, null);
@ -475,6 +466,18 @@ OutgoingMessage.prototype.write = function(chunk, encoding, callback) {
}; };
function writeAfterEndNT(self, err, callback) {
self.emit('error', err);
if (callback) callback(err);
}
function connectionCorkNT(conn) {
if (conn)
conn.uncork();
}
OutgoingMessage.prototype.addTrailers = function(headers) { OutgoingMessage.prototype.addTrailers = function(headers) {
this._trailer = ''; this._trailer = '';
var keys = Object.keys(headers); var keys = Object.keys(headers);

6
lib/_stream_duplex.js

@ -49,5 +49,9 @@ function onend() {
// no more data can be written. // no more data can be written.
// But allow more writes to happen in this tick. // But allow more writes to happen in this tick.
process.nextTick(this.end.bind(this)); process.nextTick(onEndNT, this);
}
function onEndNT(self) {
self.end();
} }

31
lib/_stream_readable.js

@ -395,9 +395,7 @@ function emitReadable(stream) {
debug('emitReadable', state.flowing); debug('emitReadable', state.flowing);
state.emittedReadable = true; state.emittedReadable = true;
if (state.sync) if (state.sync)
process.nextTick(function() { process.nextTick(emitReadable_, stream);
emitReadable_(stream);
});
else else
emitReadable_(stream); emitReadable_(stream);
} }
@ -419,9 +417,7 @@ function emitReadable_(stream) {
function maybeReadMore(stream, state) { function maybeReadMore(stream, state) {
if (!state.readingMore) { if (!state.readingMore) {
state.readingMore = true; state.readingMore = true;
process.nextTick(function() { process.nextTick(maybeReadMore_, stream, state);
maybeReadMore_(stream, state);
});
} }
} }
@ -667,11 +663,7 @@ Readable.prototype.on = function(ev, fn) {
state.emittedReadable = false; state.emittedReadable = false;
state.needReadable = true; state.needReadable = true;
if (!state.reading) { if (!state.reading) {
var self = this; process.nextTick(nReadingNextTick, this);
process.nextTick(function() {
debug('readable nexttick read 0');
self.read(0);
});
} else if (state.length) { } else if (state.length) {
emitReadable(this, state); emitReadable(this, state);
} }
@ -682,6 +674,11 @@ Readable.prototype.on = function(ev, fn) {
}; };
Readable.prototype.addListener = Readable.prototype.on; Readable.prototype.addListener = Readable.prototype.on;
function nReadingNextTick(self) {
debug('readable nexttick read 0');
self.read(0);
}
// pause() and resume() are remnants of the legacy readable stream API // pause() and resume() are remnants of the legacy readable stream API
// If the user uses them, then switch into old mode. // If the user uses them, then switch into old mode.
Readable.prototype.resume = function() { Readable.prototype.resume = function() {
@ -697,9 +694,7 @@ Readable.prototype.resume = function() {
function resume(stream, state) { function resume(stream, state) {
if (!state.resumeScheduled) { if (!state.resumeScheduled) {
state.resumeScheduled = true; state.resumeScheduled = true;
process.nextTick(function() { process.nextTick(resume_, stream, state);
resume_(stream, state);
});
} }
} }
@ -883,13 +878,15 @@ function endReadable(stream) {
if (!state.endEmitted) { if (!state.endEmitted) {
state.ended = true; state.ended = true;
process.nextTick(function() { process.nextTick(endReadableNT, state, stream);
}
}
function endReadableNT(state, stream) {
// Check that we didn't get one last unshift. // Check that we didn't get one last unshift.
if (!state.endEmitted && state.length === 0) { if (!state.endEmitted && state.length === 0) {
state.endEmitted = true; state.endEmitted = true;
stream.readable = false; stream.readable = false;
stream.emit('end'); stream.emit('end');
} }
});
}
} }

22
lib/_stream_writable.js

@ -158,9 +158,7 @@ function writeAfterEnd(stream, cb) {
var er = new Error('write after end'); var er = new Error('write after end');
// TODO: defer error events consistently everywhere, not just the cb // TODO: defer error events consistently everywhere, not just the cb
stream.emit('error', er); stream.emit('error', er);
process.nextTick(function() { process.nextTick(cb, er);
cb(er);
});
} }
// If we get something that is not a buffer, string, null, or undefined, // If we get something that is not a buffer, string, null, or undefined,
@ -178,9 +176,7 @@ function validChunk(stream, state, chunk, cb) {
!state.objectMode) { !state.objectMode) {
var er = new TypeError('Invalid non-string/buffer chunk'); var er = new TypeError('Invalid non-string/buffer chunk');
stream.emit('error', er); stream.emit('error', er);
process.nextTick(function() { process.nextTick(cb, er);
cb(er);
});
valid = false; valid = false;
} }
return valid; return valid;
@ -298,10 +294,7 @@ function doWrite(stream, state, writev, len, chunk, encoding, cb) {
function onwriteError(stream, state, sync, er, cb) { function onwriteError(stream, state, sync, er, cb) {
if (sync) if (sync)
process.nextTick(function() { process.nextTick(onwriteErrorNT, state, cb, er);
state.pendingcb--;
cb(er);
});
else { else {
state.pendingcb--; state.pendingcb--;
cb(er); cb(er);
@ -311,6 +304,11 @@ function onwriteError(stream, state, sync, er, cb) {
stream.emit('error', er); stream.emit('error', er);
} }
function onwriteErrorNT(state, cb, er) {
state.pendingcb--;
cb(er);
}
function onwriteStateUpdate(state) { function onwriteStateUpdate(state) {
state.writing = false; state.writing = false;
state.writecb = null; state.writecb = null;
@ -339,9 +337,7 @@ function onwrite(stream, er) {
} }
if (sync) { if (sync) {
process.nextTick(function() { process.nextTick(afterWrite, stream, state, finished, cb);
afterWrite(stream, state, finished, cb);
});
} else { } else {
afterWrite(stream, state, finished, cb); afterWrite(stream, state, finished, cb);
} }

37
lib/_tls_legacy.js

@ -448,15 +448,17 @@ CryptoStream.prototype.destroy = function(err) {
} }
this._opposite.destroy(); this._opposite.destroy();
var self = this; process.nextTick(destroyNT, this, err);
process.nextTick(function() { };
function destroyNT(self, err) {
// Force EOF // Force EOF
self.push(null); self.push(null);
// Emit 'close' event // Emit 'close' event
self.emit('close', err ? true : false); self.emit('close', err ? true : false);
}); }
};
CryptoStream.prototype._done = function() { CryptoStream.prototype._done = function() {
@ -667,8 +669,6 @@ function SecurePair(context, isServer, requestCert, rejectUnauthorized,
options); options);
} }
var self = this;
options || (options = {}); options || (options = {});
events.EventEmitter.call(this); events.EventEmitter.call(this);
@ -737,7 +737,12 @@ function SecurePair(context, isServer, requestCert, rejectUnauthorized,
this.cleartext.init(); this.cleartext.init();
this.encrypted.init(); this.encrypted.init();
process.nextTick(function() { process.nextTick(securePairNT, this, options);
}
util.inherits(SecurePair, events.EventEmitter);
function securePairNT(self, options) {
/* The Connection may be destroyed by an abort call */ /* The Connection may be destroyed by an abort call */
if (self.ssl) { if (self.ssl) {
self.ssl.start(); self.ssl.start();
@ -749,11 +754,8 @@ function SecurePair(context, isServer, requestCert, rejectUnauthorized,
if (self.ssl && self.ssl.error) if (self.ssl && self.ssl.error)
self.error(); self.error();
} }
});
} }
util.inherits(SecurePair, events.EventEmitter);
exports.createSecurePair = function(context, exports.createSecurePair = function(context,
isServer, isServer,
@ -835,12 +837,7 @@ exports.pipe = function pipe(pair, socket) {
socket.pipe(pair.encrypted); socket.pipe(pair.encrypted);
pair.encrypted.on('close', function() { pair.encrypted.on('close', function() {
process.nextTick(function() { process.nextTick(pipeCloseNT, pair, socket);
// Encrypted should be unpiped from socket to prevent possible
// write after destroy.
pair.encrypted.unpipe(socket);
socket.destroySoon();
});
}); });
pair.fd = socket.fd; pair.fd = socket.fd;
@ -886,3 +883,11 @@ exports.pipe = function pipe(pair, socket) {
return cleartext; return cleartext;
}; };
function pipeCloseNT(pair, socket) {
// Encrypted should be unpiped from socket to prevent possible
// write after destroy.
pair.encrypted.unpipe(socket);
socket.destroySoon();
}

12
lib/child_process.js

@ -1015,9 +1015,7 @@ function ChildProcess() {
// Do it on nextTick so that the user has one last chance // Do it on nextTick so that the user has one last chance
// to consume the output, if for example they only want to // to consume the output, if for example they only want to
// start reading the data once the process exits. // start reading the data once the process exits.
process.nextTick(function() { process.nextTick(flushStdio, self);
flushStdio(self);
});
maybeClose(self); maybeClose(self);
}; };
@ -1075,9 +1073,7 @@ ChildProcess.prototype.spawn = function(options) {
err === uv.UV_EMFILE || err === uv.UV_EMFILE ||
err === uv.UV_ENFILE || err === uv.UV_ENFILE ||
err === uv.UV_ENOENT) { err === uv.UV_ENOENT) {
process.nextTick(function() { process.nextTick(onErrorNT, self, err);
self._handle.onexit(err);
});
// There is no point in continuing when we've hit EMFILE or ENFILE // There is no point in continuing when we've hit EMFILE or ENFILE
// because we won't be able to set up the stdio file descriptors. // because we won't be able to set up the stdio file descriptors.
// It's kind of silly that the de facto spec for ENOENT (the test suite) // It's kind of silly that the de facto spec for ENOENT (the test suite)
@ -1138,6 +1134,10 @@ ChildProcess.prototype.spawn = function(options) {
return err; return err;
}; };
function onErrorNT(self, err) {
self._handle.onexit(err);
}
ChildProcess.prototype.kill = function(sig) { ChildProcess.prototype.kill = function(sig) {
var signal; var signal;

20
lib/cluster.js

@ -241,9 +241,7 @@ function masterInit() {
} }
cluster.settings = settings; cluster.settings = settings;
if (initialized === true) if (initialized === true)
return process.nextTick(function() { return process.nextTick(setupSettingsNT, settings);
cluster.emit('setup', settings);
});
initialized = true; initialized = true;
schedulingPolicy = cluster.schedulingPolicy; // Freeze policy. schedulingPolicy = cluster.schedulingPolicy; // Freeze policy.
assert(schedulingPolicy === SCHED_NONE || schedulingPolicy === SCHED_RR, assert(schedulingPolicy === SCHED_NONE || schedulingPolicy === SCHED_RR,
@ -253,9 +251,7 @@ function masterInit() {
return /^(--debug|--debug-brk)(=\d+)?$/.test(argv); return /^(--debug|--debug-brk)(=\d+)?$/.test(argv);
}); });
process.nextTick(function() { process.nextTick(setupSettingsNT, settings);
cluster.emit('setup', settings);
});
// Send debug signal only if not started in debug mode, this helps a lot // Send debug signal only if not started in debug mode, this helps a lot
// on windows, because RegisterDebugHandler is not called when node starts // on windows, because RegisterDebugHandler is not called when node starts
@ -279,6 +275,10 @@ function masterInit() {
}); });
}; };
function setupSettingsNT(settings) {
cluster.emit('setup', settings);
}
function createWorkerProcess(id, env) { function createWorkerProcess(id, env) {
var workerEnv = util._extend({}, process.env); var workerEnv = util._extend({}, process.env);
var execArgv = cluster.settings.execArgv.slice(); var execArgv = cluster.settings.execArgv.slice();
@ -376,13 +376,15 @@ function masterInit() {
}); });
worker.process.on('internalMessage', internal(worker, onmessage)); worker.process.on('internalMessage', internal(worker, onmessage));
process.nextTick(function() { process.nextTick(emitForkNT, worker);
cluster.emit('fork', worker);
});
cluster.workers[worker.id] = worker; cluster.workers[worker.id] = worker;
return worker; return worker;
}; };
function emitForkNT(worker) {
cluster.emit('fork', worker);
}
cluster.disconnect = function(cb) { cluster.disconnect = function(cb) {
var workers = Object.keys(cluster.workers); var workers = Object.keys(cluster.workers);
if (workers.length === 0) { if (workers.length === 0) {

20
lib/dgram.js

@ -321,16 +321,19 @@ Socket.prototype.send = function(buffer,
!!callback); !!callback);
if (err && callback) { if (err && callback) {
// don't emit as error, dgram_legacy.js compatibility // don't emit as error, dgram_legacy.js compatibility
process.nextTick(function() { process.nextTick(sendEmitErrorNT, err, address, port, callback);
var ex = exceptionWithHostPort(err, 'send', address, port);
callback(ex);
});
} }
} }
}); });
}; };
function sendEmitErrorNT(err, address, port, callback) {
var ex = exceptionWithHostPort(err, 'send', address, port);
callback(ex);
}
function afterSend(err) { function afterSend(err) {
if (err) { if (err) {
err = exceptionWithHostPort(err, 'send', this.address, this.port); err = exceptionWithHostPort(err, 'send', this.address, this.port);
@ -347,14 +350,17 @@ Socket.prototype.close = function(callback) {
this._handle.close(); this._handle.close();
this._handle = null; this._handle = null;
var self = this; var self = this;
process.nextTick(function() { process.nextTick(socketCloseNT, self);
self.emit('close');
});
return this; return this;
}; };
function socketCloseNT(self) {
self.emit('close');
}
Socket.prototype.address = function() { Socket.prototype.address = function() {
this._healthCheck(); this._healthCheck();

10
lib/dns.js

@ -61,15 +61,17 @@ function makeAsync(callback) {
// The API already returned, we can invoke the callback immediately. // The API already returned, we can invoke the callback immediately.
callback.apply(null, arguments); callback.apply(null, arguments);
} else { } else {
var args = arguments; process.nextTick(callMakeAsyncCbNT, callback, arguments);
process.nextTick(function() {
callback.apply(null, args);
});
} }
}; };
} }
function callMakeAsyncCbNT(callback, args) {
callback.apply(null, args);
}
function onlookup(err, addresses) { function onlookup(err, addresses) {
if (err) { if (err) {
return this.callback(errnoException(err, 'getaddrinfo', this.hostname)); return this.callback(errnoException(err, 'getaddrinfo', this.hostname));

8
lib/fs.js

@ -91,14 +91,16 @@ function nullCheck(path, callback) {
er.code = 'ENOENT'; er.code = 'ENOENT';
if (typeof callback !== 'function') if (typeof callback !== 'function')
throw er; throw er;
process.nextTick(function() { process.nextTick(nullCheckCallNT, callback, er);
callback(er);
});
return false; return false;
} }
return true; return true;
} }
function nullCheckCallNT(callback, er) {
callback(er);
}
// Static method to set the stats properties on a Stats object. // Static method to set the stats properties on a Stats object.
fs.Stats = function( fs.Stats = function(
dev, dev,

34
lib/net.js

@ -267,12 +267,14 @@ function writeAfterFIN(chunk, encoding, cb) {
// TODO: defer error events consistently everywhere, not just the cb // TODO: defer error events consistently everywhere, not just the cb
self.emit('error', er); self.emit('error', er);
if (typeof cb === 'function') { if (typeof cb === 'function') {
process.nextTick(function() { process.nextTick(writeAfterFINNT, cb, er);
cb(er);
});
} }
} }
function writeAfterFINNT(cb, er) {
cb(er);
}
exports.Socket = Socket; exports.Socket = Socket;
exports.Stream = Socket; // Legacy naming. exports.Stream = Socket; // Legacy naming.
@ -923,13 +925,7 @@ Socket.prototype.connect = function(options, cb) {
// immediately calls net.Socket.connect() on it (that's us). // immediately calls net.Socket.connect() on it (that's us).
// There are no event listeners registered yet so defer the // There are no event listeners registered yet so defer the
// error event to the next tick. // error event to the next tick.
process.nextTick(function() { process.nextTick(connectErrorNT, self, err, options);
err.host = options.host;
err.port = options.port;
err.message = err.message + ' ' + options.host + ':' + options.port;
self.emit('error', err);
self._destroy();
});
} else { } else {
self._unrefTimer(); self._unrefTimer();
connect(self, connect(self,
@ -945,6 +941,15 @@ Socket.prototype.connect = function(options, cb) {
}; };
function connectErrorNT(self, err, options) {
err.host = options.host;
err.port = options.port;
err.message = err.message + ' ' + options.host + ':' + options.port;
self.emit('error', err);
self._destroy();
}
Socket.prototype.ref = function() { Socket.prototype.ref = function() {
if (this._handle) if (this._handle)
this._handle.ref(); this._handle.ref();
@ -1183,12 +1188,15 @@ Server.prototype._listen2 = function(address, port, addressType, backlog, fd) {
if (this._unref) if (this._unref)
this.unref(); this.unref();
process.nextTick(function() { process.nextTick(emitListeningNT, self);
};
function emitListeningNT(self) {
// ensure handle hasn't closed // ensure handle hasn't closed
if (self._handle) if (self._handle)
self.emit('listening'); self.emit('listening');
}); }
};
function listen(self, address, port, addressType, backlog, fd, exclusive) { function listen(self, address, port, addressType, backlog, fd, exclusive) {

9
lib/zlib.js

@ -455,12 +455,13 @@ Zlib.prototype.close = function(callback) {
this._handle.close(); this._handle.close();
var self = this; process.nextTick(emitCloseNT, this);
process.nextTick(function() {
self.emit('close');
});
}; };
function emitCloseNT(self) {
self.emit('close');
}
Zlib.prototype._transform = function(chunk, encoding, cb) { Zlib.prototype._transform = function(chunk, encoding, cb) {
var flushFlag; var flushFlag;
var ws = this._writableState; var ws = this._writableState;

18
src/node.js

@ -337,7 +337,10 @@
callback = tock.callback; callback = tock.callback;
threw = true; threw = true;
try { try {
if (tock.args === undefined)
callback(); callback();
else
callback.apply(null, tock.args);
threw = false; threw = false;
} finally { } finally {
if (threw) if (threw)
@ -364,7 +367,10 @@
domain.enter(); domain.enter();
threw = true; threw = true;
try { try {
if (tock.args === undefined)
callback(); callback();
else
callback.apply(null, tock.args);
threw = false; threw = false;
} finally { } finally {
if (threw) if (threw)
@ -381,9 +387,10 @@
} while (tickInfo[kLength] !== 0); } while (tickInfo[kLength] !== 0);
} }
function TickObject(c) { function TickObject(c, args) {
this.callback = c; this.callback = c;
this.domain = process.domain || null; this.domain = process.domain || null;
this.args = args;
} }
function nextTick(callback) { function nextTick(callback) {
@ -391,7 +398,14 @@
if (process._exiting) if (process._exiting)
return; return;
nextTickQueue.push(new TickObject(callback)); var args = undefined;
if (arguments.length > 1) {
args = [];
for (var i = 1; i < arguments.length; i++)
args.push(arguments[i]);
}
nextTickQueue.push(new TickObject(callback, args));
tickInfo[kLength]++; tickInfo[kLength]++;
} }

8
test/message/stdin_messages.out

@ -11,7 +11,7 @@ SyntaxError: Strict mode code may not include a with statement
at Socket.<anonymous> (node.js:*:*) at Socket.<anonymous> (node.js:*:*)
at emitNone (events.js:*:*) at emitNone (events.js:*:*)
at Socket.emit (events.js:*:*) at Socket.emit (events.js:*:*)
at _stream_readable.js:*:* at endReadableNT (_stream_readable.js:*:*)
at process._tickCallback (node.js:*:*) at process._tickCallback (node.js:*:*)
42 42
42 42
@ -28,7 +28,7 @@ Error: hello
at Socket.<anonymous> (node.js:*:*) at Socket.<anonymous> (node.js:*:*)
at emitNone (events.js:*:*) at emitNone (events.js:*:*)
at Socket.emit (events.js:*:*) at Socket.emit (events.js:*:*)
at _stream_readable.js:*:* at endReadableNT (_stream_readable.js:*:*)
at process._tickCallback (node.js:*:*) at process._tickCallback (node.js:*:*)
[stdin]:1 [stdin]:1
@ -43,7 +43,7 @@ Error: hello
at Socket.<anonymous> (node.js:*:*) at Socket.<anonymous> (node.js:*:*)
at emitNone (events.js:*:*) at emitNone (events.js:*:*)
at Socket.emit (events.js:*:*) at Socket.emit (events.js:*:*)
at _stream_readable.js:*:* at endReadableNT (_stream_readable.js:*:*)
at process._tickCallback (node.js:*:*) at process._tickCallback (node.js:*:*)
100 100
@ -59,7 +59,7 @@ ReferenceError: y is not defined
at Socket.<anonymous> (node.js:*:*) at Socket.<anonymous> (node.js:*:*)
at emitNone (events.js:*:*) at emitNone (events.js:*:*)
at Socket.emit (events.js:*:*) at Socket.emit (events.js:*:*)
at _stream_readable.js:*:* at endReadableNT (_stream_readable.js:*:*)
at process._tickCallback (node.js:*:*) at process._tickCallback (node.js:*:*)
[stdin]:1 [stdin]:1

7
test/parallel/test-next-tick.js

@ -23,6 +23,13 @@ process.nextTick(function() {
complete++; complete++;
}); });
var obj = {};
process.nextTick(function(a, b) {
assert.equal(a, 42);
assert.equal(b, obj);
}, 42, obj);
process.on('exit', function() { process.on('exit', function() {
assert.equal(5, complete); assert.equal(5, complete);
process.nextTick(function() { process.nextTick(function() {

Loading…
Cancel
Save