Copyright (C) 1983, 1985 by Xerox Corporation. All rights reserved. The following program was created in 1983 but has not been published within the meaning of the copyright law, is furnished under license, and may not be used, copied and/or disclosed except in accordance with the terms of said license.
TCPReceivingImpl.mesa
Last Edited by: Nichols, September 1, 1983 4:25 pm
Last Edited by: Taft, January 4, 1984 11:13 am
Last Edited by: HGM, February 23, 1985 11:32:59 pm PST
Hal Murray July 16, 1985 4:52:04 pm PDT
John Larson, June 16, 1986 9:10:34 pm PDT
DIRECTORY
List USING [Nconc1],
IPDefs USING [Datagram, DByte],
TCPLogging USING [Direction, PrintMessage, PrintStateChange, PrintTCPPacket],
TCPOps USING [ChecksumsMatch, Flip, pktsDuplicate, pktsFromFuture, pktsFromPast, pktsRcvd, pktsWithNoConnection, pktsWithBadChecksum, repacketizing, sendBufferLength, SetTimeout, TCPChecksum, TCPHandle, TCPHeaderP, TCPRcvBuffer, tcpSegmentLife],
TCPReceiving USING [],
TCPStates USING [CloseConnection, FindHandle, GetInitialSequenceNumber],
TCPTransmit USING [RemoveAckedSegments, SendReset, SendSYN, TryToSend, TryToSendData];
TCPReceivingImpl: CEDAR MONITOR LOCKS handle USING handle: TCPHandle
IMPORTS List, TCPLogging, TCPOps, TCPStates, TCPTransmit
EXPORTS TCPReceiving =
BEGIN OPEN TCPOps, TCPReceiving;
SeqCode: TYPE = {pastSeq, okSeq, futureSeq};
ProcessRcvdSegment: PUBLIC PROC [rcvdDatagram: IPDefs.Datagram] = TRUSTED {
This procedure does the processing of a datagram received from the network:
It checks the checksum and if it is bad disposes of the packet received.
It checks for a matching handle and if none is found, it sends a reset and disposes of the packet received.
If it finds a handle, it calls ContinueProcessingRcvdSegment to process the segment with the handle lock held.
check checksum
tcpHdrPtr: TCPHeaderP;
nSeqBytes: INT;
handle: TCPHandle;
checksum: IPDefs.DByte ← TCPOps.TCPChecksum[rcvdDatagram];
pktsRcvd ← pktsRcvd + 1;
tcpHdrPtr ← LOOPHOLE[@rcvdDatagram.data];
IF ~TCPOps.ChecksumsMatch[checksum, tcpHdrPtr.checksum] THEN {
pktsWithBadChecksum ← pktsWithBadChecksum + 1;
TCPLogging.PrintMessage["Bad checksum"];
RETURN; };
nSeqBytes ← rcvdDatagram.dataLength - tcpHdrPtr.dataWordOffset * 4;
IF tcpHdrPtr.syn THEN nSeqBytes ← nSeqBytes + 1;
IF tcpHdrPtr.fin THEN nSeqBytes ← nSeqBytes + 1;
find matching connection
handle ← TCPStates.FindHandle[rcvdDatagram];
TCPLogging.PrintTCPPacket[handle, rcvdDatagram, fromNet];
IF handle = NIL THEN {
IF ~tcpHdrPtr.rst THEN -- if no matching TCB
IF tcpHdrPtr.ack THEN -- if not a reset, then send a reset
TCPTransmit.SendReset[NIL, tcpHdrPtr.dstnPort, tcpHdrPtr.sourcePort, rcvdDatagram.inHdr.source, Flip[tcpHdrPtr.ackNumber], 0] -- with no ack
ELSE
TCPTransmit.SendReset[NIL, tcpHdrPtr.dstnPort, tcpHdrPtr.sourcePort, rcvdDatagram.inHdr.source, 0, Flip[tcpHdrPtr.seqNumber] + nSeqBytes]; -- with ack
pktsWithNoConnection ← pktsWithNoConnection + 1;
TCPLogging.PrintMessage["No connection"];
RETURN; };
ContinueProcessingRcvdSegment[handle, rcvdDatagram]; };
ContinueProcessingRcvdSegment: ENTRY PROC [handle: TCPHandle, rcvdDatagram: IPDefs.Datagram] = {
This procedure finishes the processing of a datagram received from the network:
It checks the sequence number and if it is outside the window,
it sends an ack and disposes of the packet received.
It processes resets; if the packet contains a reset, processing is finished.
It processes acks.
It processes urgents.
It queues the segment on the TCB FromNet queue in sequence number order.
It processes packets on the FromNet Queue until the next expected sequence number is not present:
It processes SYNs.
It processes data and sends it to the user.
It processes FINs.
ENABLE UNWIND => NULL;
tcpRcvBufferPtr: REF TCPRcvBuffer; -- info block for rcvd packet
tcpHdrPtr: TCPHeaderP; -- pointer to tcp header
ackNumber, seqNumber: INT; -- from tcpHdrPtr, Flipped for us
seqResult: SeqCode; -- result of seq number check
nSeqBytes: INT; -- number of data bytes + syn, fin
continueProcessing: BOOL; -- true if more processing to do
freeSegment: BOOL; -- true to dispose of segment
finFlag: BOOL; -- true if segment contains fin
sendAck: BOOL; -- set to send ack
sendData: BOOL; -- set to send data
connectionClosed: BOOL; -- set when close connection so that its not referenced again
DisposeRcvdBuffer: PROC = { -- dispose of pkts on fromNet queue
IF handle.fromNetQueue # NIL THEN
handle.fromNetQueue ← handle.fromNetQueue.rest; };
ProcessReset: PROC = {
SELECT handle.state FROM
listen => freeSegment ← TRUE; -- ignore the reset
synSent => TRUSTED { -- if segment has acceptable ack, close connection
IF tcpHdrPtr.ack AND ackNumber-handle.iss > 0 AND ackNumber-handle.sndNxt <= 0 THEN {
TCPStates.CloseConnection[handle, remoteAbort];
connectionClosed ← TRUE; }; };
synRcvd => -- if were listening, return to listen else reset
IF ~handle.active THEN {
TCPLogging.PrintStateChange[handle, listen];
handle.state ← listen; }
ELSE {
TCPStates.CloseConnection[handle, remoteAbort];
connectionClosed ← TRUE; };
established, finWait1, finWait2, closeWait, closing, lastAck, timeWait => {
TCPStates.CloseConnection[handle, remoteAbort];
connectionClosed ← TRUE; };
ENDCASE; };
SignalUserUrgent: INTERNAL PROC = {
NOTIFY handle.urgentAvailable; };
ProcessUrgent: INTERNAL PROC = TRUSTED {
IF ~handle.urgentMode THEN {
not in urgent mode yet; set urgent ptr and signal user
handle.urgentMode ← TRUE;
handle.rcvUp ← tcpHdrPtr.urgentPtr + seqNumber;
IF handle.state = established OR handle.state = finWait1 OR handle.state = finWait2 THEN
SignalUserUrgent[] }
ELSE -- else update urgent pointer
IF handle.rcvUp < tcpHdrPtr.urgentPtr + seqNumber THEN
handle.rcvUp ← tcpHdrPtr.urgentPtr + seqNumber; };
ProcessAck: INTERNAL PROC = {
nDataBytesAcked: INT;
SELECT handle.state FROM
listen => {-- no acks are acceptable, send reset
TRUSTED {TCPTransmit.SendReset[handle, handle.localPort, tcpHdrPtr.sourcePort, rcvdDatagram.inHdr.source, ackNumber, 0]};
freeSegment ← TRUE;
continueProcessing ← FALSE;
RETURN;
};
synRcvd, synSent =>
IF ackNumber - handle.iss <= 0 OR ackNumber - handle.sndNxt > 0 THEN { -- if unacceptable ack, then send reset
TRUSTED {TCPTransmit.SendReset[handle, handle.localPort, handle.foreignPort, handle.foreignAddr, ackNumber, 0]};
freeSegment ← TRUE;
continueProcessing ← FALSE;
RETURN;
}
ENDCASE;
IF ackNumber - handle.sndNxt > 0 THEN { -- ack from future send an ack and discard the packet
sendAck ← TRUE;
freeSegment ← TRUE;
continueProcessing ← FALSE;
RETURN;
};
acceptable ack
IF (handle.sndWL1 - seqNumber < 0 OR handle.sndWL1 = seqNumber AND handle.sndWL2 - ackNumber <= 0) OR (handle.sndWL1 = 0 AND handle.sndWL2 = 0) THEN {
if packet has new sequence number or most recent sequence number and new ack then update the window (also update the window if we've never set it before. It it fairly unlikely that sndWL1 and sndWL2 will both go to zero at the same time and even if they do it won't cause much harm.).
TRUSTED {handle.sndWnd ← MIN[tcpHdrPtr.window, TCPOps.sendBufferLength]};
handle.sndWL1 ← seqNumber;
handle.sndWL2 ← ackNumber;
};
IF ackNumber - handle.sndUna <= 0 THEN -- duplicate ack
RETURN; -- ignore ack
calculate number of data bytes acked, i.e. not including SYN and FIN, and number of unacked bytes to send or rexmit
nDataBytesAcked ← ackNumber - handle.sndUna;
IF handle.sndUna = handle.iss AND nDataBytesAcked # 0 THEN
nDataBytesAcked ← nDataBytesAcked - 1;
IF (SELECT handle.state FROM
finWait1, finWait2, closing, lastAck, timeWait => TRUE
ENDCASE => FALSE) AND
ackNumber - handle.finSequence >= 0 AND
nDataBytesAcked # 0 THEN
nDataBytesAcked ← nDataBytesAcked - 1;
IF handle.nBytesToSend >= nDataBytesAcked THEN -- Occasional BoundsFault
handle.nBytesToSend ← handle.nBytesToSend - nDataBytesAcked;
IF repacketizing THEN -- if repacketizing, update send ptr in TCP buffer
handle.sendSlot ← (handle.sendSlot + nDataBytesAcked) MOD sendBufferLength;
TRUSTED {handle.sndUna ← Flip[tcpHdrPtr.ackNumber]}; -- update ack info in handle
TCPTransmit.RemoveAckedSegments[handle]; -- dispose of acked segments or return to user
NOTIFY handle.windowAvailable; -- tell waiters that there may be some room now
SELECT handle.state FROM
synRcvd => {
TCPLogging.PrintStateChange[handle, established];
handle.state ← established;
IF handle.urgentMode THEN -- if rcvd urgent when in synrcvd state
SignalUserUrgent[]; -- then tell user now
sendData ← TRUE;
};
established => sendData ← TRUE;
finWait1 =>
IF ackNumber - handle.finSequence > 0 THEN {-- if our fin acked, then state is finwait2
TCPLogging.PrintStateChange[handle, finWait2];
handle.state ← finWait2;
};
no data expected in following states, finished processing
closeWait => {
continueProcessing ← FALSE;
freeSegment ← TRUE;
sendData ← TRUE; -- try to send more data
};
closing => {
IF ackNumber > handle.finSequence THEN { -- if fin acked, state is timewait
TCPLogging.PrintStateChange[handle, timeWait];
handle.state ← timeWait;
handle.timeWaitTime ← SetTimeout[tcpSegmentLife*1000*2]; -- tcpSegmentLife is in seconds; SetTimeout takes milliseconds; *2 is for round-trip
};
continueProcessing ← FALSE;
freeSegment ← TRUE;
};
lastAck => {
IF ackNumber > handle.finSequence THEN { -- close connection if fin acked
TCPStates.CloseConnection[handle, handle.reason];
connectionClosed ← TRUE
}
ELSE
freeSegment ← TRUE;
continueProcessing ← FALSE
}
ENDCASE
}; -- ProcessAck
ProcessSyn: INTERNAL PROC = {
SELECT handle.state FROM
listen => {
TCPLogging.PrintStateChange[handle, synRcvd];
handle.state ← synRcvd;
handle.irs ← seqNumber;
handle.rcvNxt ← seqNumber + 1;
handle.iss ← TCPStates.GetInitialSequenceNumber[];
handle.sndUna ← handle.iss;
NOTIFY handle.notListening;
handle.matchForeignAddr ← TRUE;
handle.matchForeignPort ← TRUE;
handle.foreignAddr ← tcpRcvBufferPtr.datagramPtr.inHdr.source;
TRUSTED {handle.foreignPort ← tcpHdrPtr.sourcePort};
TCPTransmit.SendSYN[handle]; -- send a syn and ack
tcpRcvBufferPtr.dataByteCount ← tcpRcvBufferPtr.dataByteCount - 1;
tcpRcvBufferPtr.offsetSeqNo ← tcpRcvBufferPtr.offsetSeqNo + 1;
TRUSTED {tcpHdrPtr.syn ← FALSE}; -- don't reprocess syn
continueProcessing ← FALSE; -- don't process data or fin until in established state
};
synSent => {
handle.irs ← seqNumber; -- initial rcv sequence
handle.rcvNxt ← seqNumber + 1;
tcpRcvBufferPtr.dataByteCount ← tcpRcvBufferPtr.dataByteCount - 1;
tcpRcvBufferPtr.offsetSeqNo ← tcpRcvBufferPtr.offsetSeqNo + 1;
IF handle.sndUna > handle.iss THEN { -- if our syn was acked then conn. established
TCPLogging.PrintStateChange[handle, established];
handle.state ← established;
}
ELSE { -- if our syn not acked, state becomes synRcvd
TCPLogging.PrintStateChange[handle, synRcvd];
handle.state ← synRcvd;
continueProcessing ← FALSE; -- wait till in established state
};
TRUSTED {tcpHdrPtr.syn ← FALSE}; -- don't reprocess syn
sendAck ← TRUE; -- ack the syn received
};
synRcvd => {
TCPLogging.PrintStateChange[handle, listen];
handle.state ← listen; -- return to listen (why? -DN)
continueProcessing ← FALSE; -- stop processing
freeSegment ← TRUE; -- dispose of buffer
}
ENDCASE => { -- in any other state, reset connection
TCPStates.CloseConnection[handle, protocolViolation];
connectionClosed ← TRUE;
}
}; -- ProcessSyn
ProcessFin: INTERNAL PROC = {
SELECT handle.state FROM
synRcvd, established => {
TCPLogging.PrintStateChange[handle, closeWait];
handle.state ← closeWait;
handle.reason ← remoteClose;
};
finWait1 => {
TCPLogging.PrintStateChange[handle, closing];
handle.state ← closing;
};
finWait2 => {
TCPLogging.PrintStateChange[handle, timeWait];
handle.state ← timeWait;
handle.timeWaitTime ← SetTimeout[tcpSegmentLife*1000*2]; -- tcpSegmentLife is in seconds; SetTimeout takes milliseconds; *2 is for round-trip
};
ENDCASE;
NOTIFY handle.dataAvailable; -- tell the user that that data is never coming now.
}; -- ProcessFin
begin ContinueProcessingRcvdSegment
TRUSTED {
tcpHdrPtr ← LOOPHOLE[@rcvdDatagram.data];
ackNumber ← Flip[tcpHdrPtr.ackNumber];
seqNumber ← Flip[tcpHdrPtr.seqNumber];
nSeqBytes ← rcvdDatagram.dataLength - tcpHdrPtr.dataWordOffset * 4;
IF ~(handle.state = listen OR handle.state = synSent) THEN {
seqResult ← CheckSequenceNumber[seqNumber, nSeqBytes, handle.rcvNxt, handle.rcvWnd];
IF seqResult # okSeq THEN {
SELECT seqResult FROM
pastSeq => {
TCPLogging.PrintMessage["Duplicate packet"];
pktsFromPast ← pktsFromPast + 1; };
futureSeq => {
TCPLogging.PrintMessage["Future packet"];
pktsFromFuture ← pktsFromFuture + 1; };
ENDCASE => ERROR;
IF ~tcpHdrPtr.rst THEN TCPTransmit.TryToSend[handle]; -- ~reset => send an ack
RETURN; }; };
IF tcpHdrPtr.syn THEN nSeqBytes ← nSeqBytes + 1;
IF tcpHdrPtr.fin THEN nSeqBytes ← nSeqBytes + 1;
initialize flags used in processing segment
continueProcessing ← TRUE; -- more info in packet to process
freeSegment ← FALSE; -- don't dispose of packet
sendAck ← FALSE; -- set to send an ack
sendData ← FALSE; -- set to try to send more data
connectionClosed ← FALSE; -- set if the connection closes
IF tcpHdrPtr.rst THEN { -- if its a reset, process it and exit
ProcessReset[];
RETURN; };
IF tcpHdrPtr.ack THEN { -- if its an ack, process it
ProcessAck[];
IF connectionClosed THEN RETURN;
IF freeSegment OR (nSeqBytes = 0 AND ~tcpHdrPtr.urg) OR ~continueProcessing THEN GOTO SendAndExit; };
IF tcpHdrPtr.urg THEN -- if in state where data is acceptable, process urgent
SELECT handle.state FROM
synRcvd, established, finWait1, finWait2 => ProcessUrgent[];
ENDCASE => NULL;
IF nSeqBytes > handle.rcvWnd THEN {
SU-AI used to do something that provoked a problem way downstream.
Truncating is slightly bogus if SendDataToUser is going to chop off part of the front
SIGNAL PacketTooBig; -- HGM want's to look at one
TCPLogging.PrintMessage["Packet too big"];
nSeqBytes ← handle.rcvWnd; };
put segment in sequence number order on queue of segments to be processed
tcpRcvBufferPtr ← NEW[TCPRcvBuffer]; -- set up receive info block
tcpRcvBufferPtr.datagramPtr ← rcvdDatagram;
tcpRcvBufferPtr.dataByteCount ← nSeqBytes;
tcpRcvBufferPtr.dataOffset ← tcpHdrPtr.dataWordOffset * 4;
tcpRcvBufferPtr.offsetSeqNo ← Flip[tcpHdrPtr.seqNumber];
tcpRcvBufferPtr.tcpHdrPtr ← tcpHdrPtr;
QueueRcvdSegment[handle, tcpRcvBufferPtr]; -- put on handle fromNet queue
DO
process SYN, data, and FIN, repeat until FromNetQueue is empty or next expected sequence number has not been received
IF handle.fromNetQueue = NIL THEN GOTO SendAndExit;
tcpRcvBufferPtr ← NARROW[handle.fromNetQueue.first];
IF ~(handle.state = listen OR handle.state = synSent) THEN -- if in synced state and
IF tcpRcvBufferPtr.offsetSeqNo # handle.rcvNxt THEN -- dont have next seq
GOTO SendAndExit ;
tcpHdrPtr ← tcpRcvBufferPtr.tcpHdrPtr;
ackNumber ← Flip[tcpHdrPtr.ackNumber];
seqNumber ← Flip[tcpHdrPtr.seqNumber];
process SYN
IF tcpHdrPtr.syn THEN {
ProcessSyn[];
IF connectionClosed THEN RETURN;
IF freeSegment OR tcpRcvBufferPtr.dataByteCount = 0 THEN {
DisposeRcvdBuffer[];
IF continueProcessing THEN -- dispose of processed or empty packets
LOOP; -- get next packet to process
};
IF ~continueProcessing THEN GOTO SendAndExit; };
IF handle.state = listen OR handle.state = synSent THEN {-- can't do anything more in these states
DisposeRcvdBuffer[];
GOTO SendAndExit; };
IF ~tcpHdrPtr.ack THEN { -- ignore segments without acks (why? -DN)
DisposeRcvdBuffer[];
LOOP; }; -- get next packet to process
process data, send buffer to user, dispose of buffer if no data
handle.rcvNxt ← handle.rcvNxt + tcpRcvBufferPtr.dataByteCount;
IF tcpHdrPtr.fin THEN { -- remember if fin is set as buffer is sent to user
finFlag ← TRUE;
tcpRcvBufferPtr.dataByteCount ← tcpRcvBufferPtr.dataByteCount - 1; }
ELSE finFlag ← FALSE;
IF tcpRcvBufferPtr.dataByteCount > 0 THEN { -- if there's data then send it to the user
handle.fromNetQueue ← handle.fromNetQueue.rest;
SendDataToUser[handle, tcpRcvBufferPtr]; -- send to user and update window
sendAck ← TRUE; } -- remember to send an ack
ELSE {
DisposeRcvdBuffer[]; -- if no data dispose of buffer
IF finFlag THEN sendAck ← TRUE; };
process fin
IF finFlag THEN ProcessFin[];
ENDLOOP;
EXITS -- repeat processing each segment received
SendAndExit =>
SELECT TRUE FROM
sendAck => TCPTransmit.TryToSend[handle] ;-- send ack and data
sendData => TCPTransmit.TryToSendData[handle]; -- send only if data to send
ENDCASE;
};
}; -- ProcessRcvdSegment
CheckSequenceNumber: INTERNAL PROC [seq, segLen, rcvNxt, rcvWnd: INT] RETURNS [result: SeqCode] = {
Determine whether the packet is from the past or the future or whether it is in the current window (from next expected sequence number to next expected sequence number + window).
rcvEnd: INT ← rcvWnd + rcvNxt; -- rcvwnd is rcvNxt to rcvEnd-1 inclusive
segEnd: INT ← seq + segLen - 1; -- segment is seq to segEnd inclusive
IF segLen = 0 AND rcvWnd = 0 THEN
IF seq = rcvNxt THEN result ← okSeq
ELSE
IF seq - rcvNxt > 0 THEN result ← futureSeq
ELSE result ← pastSeq
ELSE
IF segLen = 0 AND rcvWnd > 0 THEN
IF seq - rcvNxt < 0 THEN result ← pastSeq
ELSE
IF seq - rcvEnd >= 0 THEN result ← futureSeq
ELSE result ← okSeq
ELSE
IF rcvWnd = 0 THEN
IF seq >= rcvNxt THEN result ← futureSeq
ELSE result ← pastSeq
ELSE
IF segEnd < rcvNxt THEN result ← pastSeq
ELSE
IF seq >= rcvEnd THEN result ← futureSeq
ELSE result ← okSeq
}; -- CheckSequenceNumber
QueueRcvdSegment: INTERNAL PROC [handle: TCPHandle, newBufferPtr: REF TCPRcvBuffer] = {
Called when packet received from net contains SYN, FIN or data and has an acceptable sequence number (within the receive window). Packets are queued in sequence number order on the TCB FromNet queue. If packets overlap, then the DataPtr and DataByteCount fields in the TCPRcvBuffer are adjusted to "trim" the packet to contain only the non-overlapping data. Packets that are duplicates are discarded.
newStart: INT; -- first seq number in new segment
newEnd: INT; -- last seq number in new segment
oldStart: INT; -- first seq number in segment on queue
oldEnd: INT; -- last seq number in segment on queue
oldBufferPtr: REF TCPRcvBuffer; -- buffer on from net queue
l, prevL: LIST OF REF ANY; -- point to the cons cells as we walk the list
IF ~(handle.state = listen OR handle.state = synSent) THEN -- if in synced state
IF newBufferPtr.offsetSeqNo < handle.rcvNxt THEN {
buffer overlaps start of window trim buffer to start of window
newBufferPtr.dataByteCount ← newBufferPtr.dataByteCount - (handle.rcvNxt - newBufferPtr.offsetSeqNo);
newBufferPtr.dataOffset ← newBufferPtr.dataOffset + (handle.rcvNxt - newBufferPtr.offsetSeqNo);
newBufferPtr.offsetSeqNo ← handle.rcvNxt; };
IF handle.fromNetQueue = NIL THEN {
queue is empty => put new segment at head
handle.fromNetQueue ← LIST[newBufferPtr];
RETURN; };
prevL ← NIL;
l ← handle.fromNetQueue;
newStart ← newBufferPtr.offsetSeqNo;
newEnd ← newStart + newBufferPtr.dataByteCount - 1;
WHILE l # NIL DO
oldBufferPtr ← NARROW[l.first];
oldStart ← oldBufferPtr.offsetSeqNo;
oldEnd ← oldStart + oldBufferPtr.dataByteCount - 1;
SELECT TRUE FROM
newStart < oldStart AND newEnd < oldStart => { -- new segment is before old one
IF prevL = NIL THEN
handle.fromNetQueue ← CONS[newBufferPtr, handle.fromNetQueue]
ELSE prevL.rest ← CONS[newBufferPtr, prevL.rest];
RETURN; };
newStart > oldEnd => NULL; -- new segment is after old one, keep going
newStart >= oldStart AND newEnd <= oldEnd => {
new segment is duplicate
pktsDuplicate ← pktsDuplicate + 1;
RETURN; };
newStart <= oldStart AND newEnd >= oldEnd => {
old segment is duplicate discard old segment
pktsDuplicate ← pktsDuplicate + 1;
oldBufferPtr.datagramPtr ← NIL;
IF prevL = NIL THEN handle.fromNetQueue ← l ← l.rest
ELSE prevL.rest ← l ← l.rest;
IF l = NIL THEN { -- if queue is now empty, just enqueue new buffer
IF prevL = NIL THEN handle.fromNetQueue ← LIST[newBufferPtr]
ELSE prevL.rest ← LIST[newBufferPtr];
RETURN; }
ELSE LOOP; }; -- else continue searching queue
newEnd < oldEnd => { -- tail of new overlaps old
use first part of new segment
newBufferPtr.dataByteCount ← oldStart - newStart;
IF prevL = NIL THEN
handle.fromNetQueue ← CONS[newBufferPtr, handle.fromNetQueue]
ELSE prevL.rest ← CONS[newBufferPtr, prevL.rest];
RETURN };
ENDCASE => { -- head of new overlaps old
continue with last part of new segment
newStart ← oldEnd + 1;
newBufferPtr.offsetSeqNo ← newStart;
newBufferPtr.dataByteCount ← newEnd - oldEnd;
newBufferPtr.dataOffset ← newBufferPtr.dataOffset + oldEnd - newStart + 1; };
prevL ← l;
l ← l.rest;
ENDLOOP; -- check for more overlapping segments --
IF prevL = NIL THEN handle.fromNetQueue ← LIST[newBufferPtr]
ELSE prevL.rest ← LIST[newBufferPtr]; }; -- QueueRcvdSegment
Debugging crap
PacketTooBig: SIGNAL = CODE;
NegativeWindow: SIGNAL = CODE;
SendDataToUser: INTERNAL PROC [handle: TCPOps.TCPHandle, tcpRcvBufferPtr: REF TCPRcvBuffer] = TRUSTED {
Queues the datagram on the readyToRead queue and lets any waiting processes know that it's there.
IF tcpRcvBufferPtr.tcpHdrPtr.urg THEN {
tcpRcvBufferPtr.urg ← TRUE;
tcpRcvBufferPtr.endUrgentData ← handle.rcvUp - tcpRcvBufferPtr.offsetSeqNo;
handle.urgentMode ← FALSE}
ELSE tcpRcvBufferPtr.urg ← FALSE;
IF tcpRcvBufferPtr.dataByteCount > handle.rcvWnd THEN {
LLL-MFE used to do something that provoked a problem here.
Truncating is slightly bogus if SendDataToUser is going to chop off part of the front
SIGNAL PacketTooBig; -- HGM want's to look at one
TCPLogging.PrintMessage["Packet too big."];
tcpRcvBufferPtr.dataByteCount ← handle.rcvWnd; };
handle.rcvWnd ← handle.rcvWnd - tcpRcvBufferPtr.dataByteCount;
IF handle.rcvWnd < 0 THEN SIGNAL NegativeWindow;
handle.readyToReadQueue ← List.Nconc1[handle.readyToReadQueue, tcpRcvBufferPtr];
NOTIFY handle.dataAvailable; };
END.