mirror of
https://github.com/refraction-networking/uquic.git
synced 2025-04-03 20:27:35 +03:00
use IETF QUIC transport error codes
This commit is contained in:
parent
966b00dd3d
commit
9c09e84765
34 changed files with 156 additions and 333 deletions
|
@ -9,7 +9,6 @@ import (
|
|||
|
||||
"github.com/lucas-clemente/quic-go/internal/handshake"
|
||||
"github.com/lucas-clemente/quic-go/internal/protocol"
|
||||
"github.com/lucas-clemente/quic-go/internal/qerr"
|
||||
"github.com/lucas-clemente/quic-go/internal/utils"
|
||||
"github.com/lucas-clemente/quic-go/internal/wire"
|
||||
)
|
||||
|
@ -328,8 +327,8 @@ func (c *client) handleVersionNegotiationPacket(p *receivedPacket) {
|
|||
c.logger.Infof("Received a Version Negotiation packet. Supported Versions: %s", hdr.SupportedVersions)
|
||||
newVersion, ok := protocol.ChooseSupportedVersion(c.config.Versions, hdr.SupportedVersions)
|
||||
if !ok {
|
||||
c.session.destroy(qerr.InvalidVersion)
|
||||
c.logger.Debugf("No compatible version found.")
|
||||
c.session.destroy(fmt.Errorf("No compatible QUIC version found. We support %s, server offered %s", c.config.Versions, hdr.SupportedVersions))
|
||||
c.logger.Debugf("No compatible QUIC version found.")
|
||||
return
|
||||
}
|
||||
c.receivedVersionNegotiationPacket = true
|
||||
|
|
|
@ -12,7 +12,6 @@ import (
|
|||
"github.com/golang/mock/gomock"
|
||||
"github.com/lucas-clemente/quic-go/internal/handshake"
|
||||
"github.com/lucas-clemente/quic-go/internal/protocol"
|
||||
"github.com/lucas-clemente/quic-go/internal/qerr"
|
||||
"github.com/lucas-clemente/quic-go/internal/utils"
|
||||
"github.com/lucas-clemente/quic-go/internal/wire"
|
||||
|
||||
|
@ -561,17 +560,27 @@ var _ = Describe("Client", func() {
|
|||
It("errors if no matching version is found", func() {
|
||||
sess := NewMockQuicSession(mockCtrl)
|
||||
done := make(chan struct{})
|
||||
sess.EXPECT().destroy(qerr.InvalidVersion).Do(func(error) { close(done) })
|
||||
sess.EXPECT().destroy(gomock.Any()).Do(func(err error) {
|
||||
defer GinkgoRecover()
|
||||
Expect(err).To(HaveOccurred())
|
||||
Expect(err.Error()).To(ContainSubstring("No compatible QUIC version found."))
|
||||
close(done)
|
||||
})
|
||||
cl.session = sess
|
||||
cl.config = &Config{Versions: protocol.SupportedVersions}
|
||||
cl.handlePacket(composeVersionNegotiationPacket(connID, []protocol.VersionNumber{1}))
|
||||
cl.handlePacket(composeVersionNegotiationPacket(connID, []protocol.VersionNumber{1337}))
|
||||
Eventually(done).Should(BeClosed())
|
||||
})
|
||||
|
||||
It("errors if the version is supported by quic-go, but disabled by the quic.Config", func() {
|
||||
sess := NewMockQuicSession(mockCtrl)
|
||||
done := make(chan struct{})
|
||||
sess.EXPECT().destroy(qerr.InvalidVersion).Do(func(error) { close(done) })
|
||||
sess.EXPECT().destroy(gomock.Any()).Do(func(err error) {
|
||||
defer GinkgoRecover()
|
||||
Expect(err).To(HaveOccurred())
|
||||
Expect(err.Error()).To(ContainSubstring("No compatible QUIC version found."))
|
||||
close(done)
|
||||
})
|
||||
cl.session = sess
|
||||
v := protocol.VersionNumber(1234)
|
||||
Expect(v).ToNot(Equal(cl.version))
|
||||
|
|
|
@ -108,10 +108,10 @@ func (c *client) handleHeaderStream() {
|
|||
for err == nil {
|
||||
err = c.readResponse(h2framer, decoder)
|
||||
}
|
||||
if quicErr, ok := err.(*qerr.QuicError); !ok || quicErr.ErrorCode != qerr.PeerGoingAway {
|
||||
if quicErr, ok := err.(*qerr.QuicError); !ok || quicErr.ErrorCode != qerr.NoError {
|
||||
c.logger.Debugf("Error handling header stream: %s", err)
|
||||
}
|
||||
c.headerErr = qerr.Error(qerr.InvalidHeadersStreamData, err.Error())
|
||||
c.headerErr = qerr.Error(qerr.InternalError, err.Error())
|
||||
// stop all running request
|
||||
close(c.headerErrored)
|
||||
}
|
||||
|
|
|
@ -306,7 +306,7 @@ var _ = Describe("Client", func() {
|
|||
}()
|
||||
|
||||
Eventually(done).Should(BeClosed())
|
||||
Expect(client.headerErr.ErrorCode).To(Equal(qerr.InvalidHeadersStreamData))
|
||||
Expect(client.headerErr.ErrorCode).To(Equal(qerr.InternalError))
|
||||
Expect(client.session.(*mockSession).closedWithError).To(MatchError(client.headerErr))
|
||||
})
|
||||
|
||||
|
@ -315,7 +315,7 @@ var _ = Describe("Client", func() {
|
|||
headerStream.dataToRead.Write(bytes.Repeat([]byte{0}, 100))
|
||||
_, err := client.RoundTrip(request)
|
||||
Expect(err).To(BeAssignableToTypeOf(&qerr.QuicError{}))
|
||||
Expect(err.(*qerr.QuicError).ErrorCode).To(Equal(qerr.InvalidHeadersStreamData))
|
||||
Expect(err.(*qerr.QuicError).ErrorCode).To(Equal(qerr.InternalError))
|
||||
// now that the first request failed due to an error on the header stream, try another request
|
||||
_, nextErr := client.RoundTrip(request)
|
||||
Expect(nextErr).To(MatchError(err))
|
||||
|
@ -605,7 +605,7 @@ var _ = Describe("Client", func() {
|
|||
h2framer.WritePing(true, [8]byte{0, 0, 0, 0, 0, 0, 0, 0})
|
||||
client.handleHeaderStream()
|
||||
Eventually(client.headerErrored).Should(BeClosed())
|
||||
Expect(client.headerErr).To(MatchError(qerr.Error(qerr.InvalidHeadersStreamData, "not a headers frame")))
|
||||
Expect(client.headerErr).To(MatchError(qerr.Error(qerr.InternalError, "not a headers frame")))
|
||||
})
|
||||
|
||||
It("errors if it can't read the HPACK encoded header fields", func() {
|
||||
|
@ -616,7 +616,7 @@ var _ = Describe("Client", func() {
|
|||
})
|
||||
client.handleHeaderStream()
|
||||
Eventually(client.headerErrored).Should(BeClosed())
|
||||
Expect(client.headerErr.ErrorCode).To(Equal(qerr.InvalidHeadersStreamData))
|
||||
Expect(client.headerErr.ErrorCode).To(Equal(qerr.InternalError))
|
||||
Expect(client.headerErr.ErrorMessage).To(ContainSubstring("cannot read header fields"))
|
||||
})
|
||||
|
||||
|
@ -632,7 +632,7 @@ var _ = Describe("Client", func() {
|
|||
Expect(err).ToNot(HaveOccurred())
|
||||
client.handleHeaderStream()
|
||||
Eventually(client.headerErrored).Should(BeClosed())
|
||||
Expect(client.headerErr.ErrorCode).To(Equal(qerr.InvalidHeadersStreamData))
|
||||
Expect(client.headerErr.ErrorCode).To(Equal(qerr.InternalError))
|
||||
Expect(client.headerErr.ErrorMessage).To(ContainSubstring("response channel for stream 1337 not found"))
|
||||
})
|
||||
})
|
||||
|
|
|
@ -127,7 +127,7 @@ func (s *Server) serveImpl(tlsConfig *tls.Config, conn net.PacketConn) error {
|
|||
func (s *Server) handleHeaderStream(session streamCreator) {
|
||||
stream, err := session.AcceptStream()
|
||||
if err != nil {
|
||||
session.CloseWithError(quic.ErrorCode(qerr.InvalidHeadersStreamData), err)
|
||||
session.CloseWithError(quic.ErrorCode(qerr.InternalError), err)
|
||||
return
|
||||
}
|
||||
|
||||
|
@ -154,7 +154,7 @@ func (s *Server) handleHeaderStream(session streamCreator) {
|
|||
func (s *Server) handleRequest(session streamCreator, headerStream quic.Stream, headerStreamMutex *sync.Mutex, hpackDecoder *hpack.Decoder, h2framer *http2.Framer) error {
|
||||
h2frame, err := h2framer.ReadFrame()
|
||||
if err != nil {
|
||||
return qerr.Error(qerr.HeadersStreamDataDecompressFailure, "cannot read frame")
|
||||
return qerr.Error(qerr.InternalError, "cannot read frame")
|
||||
}
|
||||
var h2headersFrame *http2.HeadersFrame
|
||||
switch f := h2frame.(type) {
|
||||
|
@ -165,7 +165,7 @@ func (s *Server) handleRequest(session streamCreator, headerStream quic.Stream,
|
|||
case *http2.HeadersFrame:
|
||||
h2headersFrame = f
|
||||
default:
|
||||
return qerr.Error(qerr.InvalidHeadersStreamData, "expected a header frame")
|
||||
return qerr.Error(qerr.ProtocolViolation, "expected a header frame")
|
||||
}
|
||||
|
||||
if !h2headersFrame.HeadersEnded() {
|
||||
|
|
|
@ -286,7 +286,7 @@ var _ = Describe("H2 server", func() {
|
|||
'f', 'o', 'o', 'b', 'a', 'r',
|
||||
})
|
||||
err := s.handleRequest(session, headerStream, &sync.Mutex{}, hpackDecoder, h2framer)
|
||||
Expect(err).To(MatchError("InvalidHeadersStreamData: expected a header frame"))
|
||||
Expect(err).To(MatchError("ProtocolViolation: expected a header frame"))
|
||||
})
|
||||
|
||||
It("Cancels the request context when the datstream is closed", func() {
|
||||
|
@ -340,7 +340,7 @@ var _ = Describe("H2 server", func() {
|
|||
go s.handleHeaderStream(session)
|
||||
Consistently(func() bool { return handlerCalled }).Should(BeFalse())
|
||||
Eventually(func() bool { return session.closed }).Should(BeTrue())
|
||||
Expect(session.closedWithError).To(MatchError(qerr.Error(qerr.HeadersStreamDataDecompressFailure, "cannot read frame")))
|
||||
Expect(session.closedWithError).To(MatchError(qerr.Error(qerr.InternalError, "cannot read frame")))
|
||||
})
|
||||
|
||||
It("supports closing after first request", func() {
|
||||
|
|
|
@ -2,13 +2,13 @@ package self_test
|
|||
|
||||
import (
|
||||
"crypto/tls"
|
||||
"fmt"
|
||||
"net"
|
||||
"time"
|
||||
|
||||
quic "github.com/lucas-clemente/quic-go"
|
||||
quicproxy "github.com/lucas-clemente/quic-go/integrationtests/tools/proxy"
|
||||
"github.com/lucas-clemente/quic-go/internal/protocol"
|
||||
"github.com/lucas-clemente/quic-go/internal/qerr"
|
||||
|
||||
"github.com/lucas-clemente/quic-go/internal/testdata"
|
||||
. "github.com/onsi/ginkgo"
|
||||
|
@ -83,7 +83,8 @@ var _ = Describe("Handshake RTT tests", func() {
|
|||
}
|
||||
_, err := quic.DialAddr(proxy.LocalAddr().String(), nil, clientConfig)
|
||||
Expect(err).To(HaveOccurred())
|
||||
Expect(err.(qerr.ErrorCode)).To(Equal(qerr.InvalidVersion))
|
||||
fmt.Println(err)
|
||||
// Expect(err.(qerr.ErrorCode)).To(Equal(qerr.InvalidVersion))
|
||||
expectDurationInRTTs(1)
|
||||
})
|
||||
|
||||
|
@ -138,6 +139,6 @@ var _ = Describe("Handshake RTT tests", func() {
|
|||
clientConfig,
|
||||
)
|
||||
Expect(err).To(HaveOccurred())
|
||||
Expect(err.(*qerr.QuicError).ErrorCode).To(Equal(qerr.HandshakeTimeout))
|
||||
Expect(err.Error()).To(ContainSubstring("Handshake did not complete in time"))
|
||||
})
|
||||
})
|
||||
|
|
|
@ -184,8 +184,7 @@ var _ = Describe("Handshake tests", func() {
|
|||
|
||||
_, err := dial()
|
||||
Expect(err).To(HaveOccurred())
|
||||
// TODO(#1567): use the SERVER_BUSY error code
|
||||
Expect(err.(*qerr.QuicError).ErrorCode).To(Equal(qerr.PeerGoingAway))
|
||||
Expect(err.(*qerr.QuicError).ErrorCode).To(Equal(qerr.ServerBusy))
|
||||
|
||||
// now accept one session, freeing one spot in the queue
|
||||
_, err = server.Accept()
|
||||
|
@ -198,8 +197,7 @@ var _ = Describe("Handshake tests", func() {
|
|||
|
||||
_, err = dial()
|
||||
Expect(err).To(HaveOccurred())
|
||||
// TODO(#1567): use the SERVER_BUSY error code
|
||||
Expect(err.(*qerr.QuicError).ErrorCode).To(Equal(qerr.PeerGoingAway))
|
||||
Expect(err.(*qerr.QuicError).ErrorCode).To(Equal(qerr.ServerBusy))
|
||||
})
|
||||
|
||||
It("rejects new connection attempts if connections don't get accepted", func() {
|
||||
|
@ -215,8 +213,7 @@ var _ = Describe("Handshake tests", func() {
|
|||
|
||||
_, err = dial()
|
||||
Expect(err).To(HaveOccurred())
|
||||
// TODO(#1567): use the SERVER_BUSY error code
|
||||
Expect(err.(*qerr.QuicError).ErrorCode).To(Equal(qerr.PeerGoingAway))
|
||||
Expect(err.(*qerr.QuicError).ErrorCode).To(Equal(qerr.ServerBusy))
|
||||
|
||||
// Now close the one of the session that are waiting to be accepted.
|
||||
// This should free one spot in the queue.
|
||||
|
@ -229,8 +226,7 @@ var _ = Describe("Handshake tests", func() {
|
|||
time.Sleep(25 * time.Millisecond) // wait a bit for the session to be queued
|
||||
|
||||
_, err = dial()
|
||||
// TODO(#1567): use the SERVER_BUSY error code
|
||||
Expect(err.(*qerr.QuicError).ErrorCode).To(Equal(qerr.PeerGoingAway))
|
||||
Expect(err.(*qerr.QuicError).ErrorCode).To(Equal(qerr.ServerBusy))
|
||||
})
|
||||
|
||||
})
|
||||
|
|
|
@ -16,7 +16,7 @@ type receivedPacketHistory struct {
|
|||
lowestInReceivedPacketNumbers protocol.PacketNumber
|
||||
}
|
||||
|
||||
var errTooManyOutstandingReceivedAckRanges = qerr.Error(qerr.TooManyOutstandingReceivedPackets, "Too many outstanding received ACK ranges")
|
||||
var errTooManyOutstandingReceivedAckRanges = qerr.Error(qerr.InternalError, "Too many outstanding received ACK ranges")
|
||||
|
||||
// newReceivedPacketHistory creates a new received packet history
|
||||
func newReceivedPacketHistory() *receivedPacketHistory {
|
||||
|
|
|
@ -198,13 +198,13 @@ func (h *sentPacketHandler) ReceivedAck(ackFrame *wire.AckFrame, withPacketNumbe
|
|||
|
||||
largestAcked := ackFrame.LargestAcked()
|
||||
if largestAcked > pnSpace.largestSent {
|
||||
return qerr.Error(qerr.InvalidAckData, "Received ACK for an unsent packet")
|
||||
return qerr.Error(qerr.ProtocolViolation, "Received ACK for an unsent packet")
|
||||
}
|
||||
|
||||
pnSpace.largestAcked = utils.MaxPacketNumber(pnSpace.largestAcked, largestAcked)
|
||||
|
||||
if !pnSpace.pns.Validate(ackFrame) {
|
||||
return qerr.Error(qerr.InvalidAckData, "Received an ACK for a skipped packet number")
|
||||
return qerr.Error(qerr.ProtocolViolation, "Received an ACK for a skipped packet number")
|
||||
}
|
||||
|
||||
// maybe update the RTT
|
||||
|
|
|
@ -161,7 +161,7 @@ var _ = Describe("SentPacketHandler", func() {
|
|||
It("rejects ACKs with a too high LargestAcked packet number", func() {
|
||||
ack := &wire.AckFrame{AckRanges: []wire.AckRange{{Smallest: 0, Largest: 9999}}}
|
||||
err := handler.ReceivedAck(ack, 1, protocol.Encryption1RTT, time.Now())
|
||||
Expect(err).To(MatchError("InvalidAckData: Received ACK for an unsent packet"))
|
||||
Expect(err).To(MatchError("ProtocolViolation: Received ACK for an unsent packet"))
|
||||
Expect(handler.bytesInFlight).To(Equal(protocol.ByteCount(10)))
|
||||
})
|
||||
|
||||
|
@ -809,7 +809,7 @@ var _ = Describe("SentPacketHandler", func() {
|
|||
})
|
||||
ack := &wire.AckFrame{AckRanges: []wire.AckRange{{Smallest: 13, Largest: 13}}}
|
||||
err := handler.ReceivedAck(ack, 1, protocol.EncryptionHandshake, time.Now())
|
||||
Expect(err).To(MatchError("InvalidAckData: Received ACK for an unsent packet"))
|
||||
Expect(err).To(MatchError("ProtocolViolation: Received ACK for an unsent packet"))
|
||||
})
|
||||
|
||||
It("deletes crypto packets when the handshake completes", func() {
|
||||
|
|
|
@ -49,7 +49,7 @@ func (c *connectionFlowController) IncrementHighestReceived(increment protocol.B
|
|||
|
||||
c.highestReceived += increment
|
||||
if c.checkFlowControlViolation() {
|
||||
return qerr.Error(qerr.FlowControlReceivedTooMuchData, fmt.Sprintf("Received %d bytes for the connection, allowed %d bytes", c.highestReceived, c.receiveWindow))
|
||||
return qerr.Error(qerr.FlowControlError, fmt.Sprintf("Received %d bytes for the connection, allowed %d bytes", c.highestReceived, c.receiveWindow))
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -58,11 +58,11 @@ func (c *streamFlowController) UpdateHighestReceived(offset protocol.ByteCount,
|
|||
if c.receivedFinalOffset {
|
||||
// If we receive another final offset, check that it's the same.
|
||||
if final && offset != c.highestReceived {
|
||||
return qerr.Error(qerr.StreamDataAfterTermination, fmt.Sprintf("Received inconsistent final offset for stream %d (old: %#x, new: %#x bytes)", c.streamID, c.highestReceived, offset))
|
||||
return qerr.Error(qerr.FinalSizeError, fmt.Sprintf("Received inconsistent final offset for stream %d (old: %#x, new: %#x bytes)", c.streamID, c.highestReceived, offset))
|
||||
}
|
||||
// Check that the offset is below the final offset.
|
||||
if offset > c.highestReceived {
|
||||
return qerr.Error(qerr.StreamDataAfterTermination, fmt.Sprintf("Received offset %#x for stream %d. Final offset was already received at %#x", offset, c.streamID, c.highestReceived))
|
||||
return qerr.Error(qerr.FinalSizeError, fmt.Sprintf("Received offset %#x for stream %d. Final offset was already received at %#x", offset, c.streamID, c.highestReceived))
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -76,7 +76,7 @@ func (c *streamFlowController) UpdateHighestReceived(offset protocol.ByteCount,
|
|||
// This can happen due to reordering.
|
||||
if offset <= c.highestReceived {
|
||||
if final {
|
||||
return qerr.Error(qerr.StreamDataAfterTermination, fmt.Sprintf("Received final offset %#x for stream %d, but already received offset %#x before", offset, c.streamID, c.highestReceived))
|
||||
return qerr.Error(qerr.FinalSizeError, fmt.Sprintf("Received final offset %#x for stream %d, but already received offset %#x before", offset, c.streamID, c.highestReceived))
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
@ -84,7 +84,7 @@ func (c *streamFlowController) UpdateHighestReceived(offset protocol.ByteCount,
|
|||
increment := offset - c.highestReceived
|
||||
c.highestReceived = offset
|
||||
if c.checkFlowControlViolation() {
|
||||
return qerr.Error(qerr.FlowControlReceivedTooMuchData, fmt.Sprintf("Received %#x bytes on stream %d, allowed %#x bytes", offset, c.streamID, c.receiveWindow))
|
||||
return qerr.Error(qerr.FlowControlError, fmt.Sprintf("Received %#x bytes on stream %d, allowed %#x bytes", offset, c.streamID, c.receiveWindow))
|
||||
}
|
||||
return c.connection.IncrementHighestReceived(increment)
|
||||
}
|
||||
|
|
|
@ -98,7 +98,7 @@ var _ = Describe("Stream Flow controller", func() {
|
|||
})
|
||||
|
||||
It("detects a flow control violation", func() {
|
||||
Expect(controller.UpdateHighestReceived(receiveWindow+1, false)).To(MatchError("FlowControlReceivedTooMuchData: Received 0x10001 bytes on stream 10, allowed 0x10000 bytes"))
|
||||
Expect(controller.UpdateHighestReceived(receiveWindow+1, false)).To(MatchError("FlowControlError: Received 0x10001 bytes on stream 10, allowed 0x10000 bytes"))
|
||||
})
|
||||
|
||||
It("accepts a final offset higher than the highest received", func() {
|
||||
|
@ -109,7 +109,7 @@ var _ = Describe("Stream Flow controller", func() {
|
|||
|
||||
It("errors when receiving a final offset smaller than the highest offset received so far", func() {
|
||||
controller.UpdateHighestReceived(0x100, false)
|
||||
Expect(controller.UpdateHighestReceived(0xff, true)).To(MatchError("StreamDataAfterTermination: Received final offset 0xff for stream 10, but already received offset 0x100 before"))
|
||||
Expect(controller.UpdateHighestReceived(0xff, true)).To(MatchError("FinalSizeError: Received final offset 0xff for stream 10, but already received offset 0x100 before"))
|
||||
})
|
||||
|
||||
It("accepts delayed data after receiving a final offset", func() {
|
||||
|
@ -119,7 +119,7 @@ var _ = Describe("Stream Flow controller", func() {
|
|||
|
||||
It("errors when receiving a higher offset after receiving a final offset", func() {
|
||||
Expect(controller.UpdateHighestReceived(0x200, true)).To(Succeed())
|
||||
Expect(controller.UpdateHighestReceived(0x250, false)).To(MatchError("StreamDataAfterTermination: Received offset 0x250 for stream 10. Final offset was already received at 0x200"))
|
||||
Expect(controller.UpdateHighestReceived(0x250, false)).To(MatchError("FinalSizeError: Received offset 0x250 for stream 10. Final offset was already received at 0x200"))
|
||||
})
|
||||
|
||||
It("accepts duplicate final offsets", func() {
|
||||
|
@ -130,7 +130,7 @@ var _ = Describe("Stream Flow controller", func() {
|
|||
|
||||
It("errors when receiving inconsistent final offsets", func() {
|
||||
Expect(controller.UpdateHighestReceived(0x200, true)).To(Succeed())
|
||||
Expect(controller.UpdateHighestReceived(0x201, true)).To(MatchError("StreamDataAfterTermination: Received inconsistent final offset for stream 10 (old: 0x200, new: 0x201 bytes)"))
|
||||
Expect(controller.UpdateHighestReceived(0x201, true)).To(MatchError("FinalSizeError: Received inconsistent final offset for stream 10 (old: 0x200, new: 0x201 bytes)"))
|
||||
})
|
||||
|
||||
It("tells the connection flow controller when a stream is abandoned", func() {
|
||||
|
|
|
@ -461,7 +461,7 @@ func (h *cryptoSetup) WriteRecord(p []byte) (int, error) {
|
|||
}
|
||||
|
||||
func (h *cryptoSetup) SendAlert(alert uint8) {
|
||||
// TODO(#1567): send the correct IETF QUIC error code
|
||||
// TODO(#1804): send the correct IETF QUIC error code
|
||||
h.alertChan <- fmt.Errorf("TLS alert: %d", alert)
|
||||
}
|
||||
|
||||
|
|
|
@ -2,192 +2,22 @@ package qerr
|
|||
|
||||
// The error codes defined by QUIC
|
||||
// Remember to run `go generate ./...` whenever the error codes change.
|
||||
// This uses the Go stringer tool, which can be installed by running
|
||||
// go get -u golang.org/x/tools/cmd/stringer
|
||||
|
||||
//go:generate stringer -type=ErrorCode
|
||||
const (
|
||||
InternalError ErrorCode = 1
|
||||
// There were data frames after the a fin or reset.
|
||||
StreamDataAfterTermination ErrorCode = 2
|
||||
// Control frame is malformed.
|
||||
InvalidPacketHeader ErrorCode = 3
|
||||
// Frame data is malformed.
|
||||
InvalidFrameData ErrorCode = 4
|
||||
// The packet contained no payload.
|
||||
MissingPayload ErrorCode = 48
|
||||
// FEC data is malformed.
|
||||
InvalidFecData ErrorCode = 5
|
||||
// STREAM frame data is malformed.
|
||||
InvalidStreamData ErrorCode = 46
|
||||
// STREAM frame data overlaps with buffered data.
|
||||
OverlappingStreamData ErrorCode = 87
|
||||
// Received STREAM frame data is not encrypted.
|
||||
UnencryptedStreamData ErrorCode = 61
|
||||
// Attempt to send unencrypted STREAM frame.
|
||||
AttemptToSendUnencryptedStreamData ErrorCode = 88
|
||||
// FEC frame data is not encrypted.
|
||||
UnencryptedFecData ErrorCode = 77
|
||||
// RST_STREAM frame data is malformed.
|
||||
InvalidRstStreamData ErrorCode = 6
|
||||
// CONNECTION_CLOSE frame data is malformed.
|
||||
InvalidConnectionCloseData ErrorCode = 7
|
||||
// GOAWAY frame data is malformed.
|
||||
InvalidGoawayData ErrorCode = 8
|
||||
// WINDOW_UPDATE frame data is malformed.
|
||||
InvalidWindowUpdateData ErrorCode = 57
|
||||
// BLOCKED frame data is malformed.
|
||||
InvalidBlockedData ErrorCode = 58
|
||||
// STOP_WAITING frame data is malformed.
|
||||
InvalidStopWaitingData ErrorCode = 60
|
||||
// PATH_CLOSE frame data is malformed.
|
||||
InvalidPathCloseData ErrorCode = 78
|
||||
// ACK frame data is malformed.
|
||||
InvalidAckData ErrorCode = 9
|
||||
|
||||
// Version negotiation packet is malformed.
|
||||
InvalidVersionNegotiationPacket ErrorCode = 10
|
||||
// Public RST packet is malformed.
|
||||
InvalidPublicRstPacket ErrorCode = 11
|
||||
// There was an error decrypting.
|
||||
DecryptionFailure ErrorCode = 12
|
||||
// There was an error encrypting.
|
||||
EncryptionFailure ErrorCode = 13
|
||||
// The packet exceeded kMaxPacketSize.
|
||||
PacketTooLarge ErrorCode = 14
|
||||
// The peer is going away. May be a client or server.
|
||||
PeerGoingAway ErrorCode = 16
|
||||
// A stream ID was invalid.
|
||||
InvalidStreamID ErrorCode = 17
|
||||
// A priority was invalid.
|
||||
InvalidPriority ErrorCode = 49
|
||||
// Too many streams already open.
|
||||
TooManyOpenStreams ErrorCode = 18
|
||||
// The peer created too many available streams.
|
||||
TooManyAvailableStreams ErrorCode = 76
|
||||
// Received public reset for this connection.
|
||||
PublicReset ErrorCode = 19
|
||||
// Invalid protocol version.
|
||||
InvalidVersion ErrorCode = 20
|
||||
|
||||
// The Header ID for a stream was too far from the previous.
|
||||
InvalidHeaderID ErrorCode = 22
|
||||
// Negotiable parameter received during handshake had invalid value.
|
||||
InvalidNegotiatedValue ErrorCode = 23
|
||||
// There was an error decompressing data.
|
||||
DecompressionFailure ErrorCode = 24
|
||||
// The connection timed out due to no network activity.
|
||||
NetworkIdleTimeout ErrorCode = 25
|
||||
// The connection timed out waiting for the handshake to complete.
|
||||
HandshakeTimeout ErrorCode = 67
|
||||
// There was an error encountered migrating addresses.
|
||||
ErrorMigratingAddress ErrorCode = 26
|
||||
// There was an error encountered migrating port only.
|
||||
ErrorMigratingPort ErrorCode = 86
|
||||
// There was an error while writing to the socket.
|
||||
PacketWriteError ErrorCode = 27
|
||||
// There was an error while reading from the socket.
|
||||
PacketReadError ErrorCode = 51
|
||||
// We received a STREAM_FRAME with no data and no fin flag set.
|
||||
EmptyStreamFrameNoFin ErrorCode = 50
|
||||
// We received invalid data on the headers stream.
|
||||
InvalidHeadersStreamData ErrorCode = 56
|
||||
// Invalid data on the headers stream received because of decompression
|
||||
// failure.
|
||||
HeadersStreamDataDecompressFailure ErrorCode = 97
|
||||
// The peer received too much data, violating flow control.
|
||||
FlowControlReceivedTooMuchData ErrorCode = 59
|
||||
// The peer sent too much data, violating flow control.
|
||||
FlowControlSentTooMuchData ErrorCode = 63
|
||||
// The peer received an invalid flow control window.
|
||||
FlowControlInvalidWindow ErrorCode = 64
|
||||
// The connection has been IP pooled into an existing connection.
|
||||
ConnectionIPPooled ErrorCode = 62
|
||||
// The connection has too many outstanding sent packets.
|
||||
TooManyOutstandingSentPackets ErrorCode = 68
|
||||
// The connection has too many outstanding received packets.
|
||||
TooManyOutstandingReceivedPackets ErrorCode = 69
|
||||
// The quic connection has been cancelled.
|
||||
ConnectionCancelled ErrorCode = 70
|
||||
// Disabled QUIC because of high packet loss rate.
|
||||
BadPacketLossRate ErrorCode = 71
|
||||
// Disabled QUIC because of too many PUBLIC_RESETs post handshake.
|
||||
PublicResetsPostHandshake ErrorCode = 73
|
||||
// Disabled QUIC because of too many timeouts with streams open.
|
||||
TimeoutsWithOpenStreams ErrorCode = 74
|
||||
// Closed because we failed to serialize a packet.
|
||||
FailedToSerializePacket ErrorCode = 75
|
||||
// QUIC timed out after too many RTOs.
|
||||
TooManyRtos ErrorCode = 85
|
||||
|
||||
// Crypto errors.
|
||||
|
||||
// Hanshake failed.
|
||||
HandshakeFailed ErrorCode = 28
|
||||
// Handshake message contained out of order tags.
|
||||
CryptoTagsOutOfOrder ErrorCode = 29
|
||||
// Handshake message contained too many entries.
|
||||
CryptoTooManyEntries ErrorCode = 30
|
||||
// Handshake message contained an invalid value length.
|
||||
CryptoInvalidValueLength ErrorCode = 31
|
||||
// A crypto message was received after the handshake was complete.
|
||||
CryptoMessageAfterHandshakeComplete ErrorCode = 32
|
||||
// A crypto message was received with an illegal message tag.
|
||||
InvalidCryptoMessageType ErrorCode = 33
|
||||
// A crypto message was received with an illegal parameter.
|
||||
InvalidCryptoMessageParameter ErrorCode = 34
|
||||
// An invalid channel id signature was supplied.
|
||||
InvalidChannelIDSignature ErrorCode = 52
|
||||
// A crypto message was received with a mandatory parameter missing.
|
||||
CryptoMessageParameterNotFound ErrorCode = 35
|
||||
// A crypto message was received with a parameter that has no overlap
|
||||
// with the local parameter.
|
||||
CryptoMessageParameterNoOverlap ErrorCode = 36
|
||||
// A crypto message was received that contained a parameter with too few
|
||||
// values.
|
||||
CryptoMessageIndexNotFound ErrorCode = 37
|
||||
// An internal error occurred in crypto processing.
|
||||
CryptoInternalError ErrorCode = 38
|
||||
// A crypto handshake message specified an unsupported version.
|
||||
CryptoVersionNotSupported ErrorCode = 39
|
||||
// A crypto handshake message resulted in a stateless reject.
|
||||
CryptoHandshakeStatelessReject ErrorCode = 72
|
||||
// There was no intersection between the crypto primitives supported by the
|
||||
// peer and ourselves.
|
||||
CryptoNoSupport ErrorCode = 40
|
||||
// The server rejected our client hello messages too many times.
|
||||
CryptoTooManyRejects ErrorCode = 41
|
||||
// The client rejected the server's certificate chain or signature.
|
||||
ProofInvalid ErrorCode = 42
|
||||
// A crypto message was received with a duplicate tag.
|
||||
CryptoDuplicateTag ErrorCode = 43
|
||||
// A crypto message was received with the wrong encryption level (i.e. it
|
||||
// should have been encrypted but was not.)
|
||||
CryptoEncryptionLevelIncorrect ErrorCode = 44
|
||||
// The server config for a server has expired.
|
||||
CryptoServerConfigExpired ErrorCode = 45
|
||||
// We failed to setup the symmetric keys for a connection.
|
||||
CryptoSymmetricKeySetupFailed ErrorCode = 53
|
||||
// A handshake message arrived, but we are still validating the
|
||||
// previous handshake message.
|
||||
CryptoMessageWhileValidatingClientHello ErrorCode = 54
|
||||
// A server config update arrived before the handshake is complete.
|
||||
CryptoUpdateBeforeHandshakeComplete ErrorCode = 65
|
||||
// This connection involved a version negotiation which appears to have been
|
||||
// tampered with.
|
||||
VersionNegotiationMismatch ErrorCode = 55
|
||||
|
||||
// Multipath is not enabled, but a packet with multipath flag on is received.
|
||||
BadMultipathFlag ErrorCode = 79
|
||||
|
||||
// IP address changed causing connection close.
|
||||
IPAddressChanged ErrorCode = 80
|
||||
|
||||
// Connection migration errors.
|
||||
// Network changed, but connection had no migratable streams.
|
||||
ConnectionMigrationNoMigratableStreams ErrorCode = 81
|
||||
// Connection changed networks too many times.
|
||||
ConnectionMigrationTooManyChanges ErrorCode = 82
|
||||
// Connection migration was attempted, but there was no new network to
|
||||
// migrate to.
|
||||
ConnectionMigrationNoNewNetwork ErrorCode = 83
|
||||
// Network changed, but connection had one or more non-migratable streams.
|
||||
ConnectionMigrationNonMigratableStream ErrorCode = 84
|
||||
NoError ErrorCode = 0x0
|
||||
InternalError ErrorCode = 0x1
|
||||
ServerBusy ErrorCode = 0x2
|
||||
FlowControlError ErrorCode = 0x3
|
||||
StreamLimitError ErrorCode = 0x4
|
||||
StreamStateError ErrorCode = 0x5
|
||||
FinalSizeError ErrorCode = 0x6
|
||||
FrameEncodingError ErrorCode = 0x7
|
||||
TransportParameterError ErrorCode = 0x8
|
||||
VersionNegotiationError ErrorCode = 0x9
|
||||
ProtocolViolation ErrorCode = 0xa
|
||||
InvalidMigration ErrorCode = 0xc
|
||||
CryptoError ErrorCode = 0x100
|
||||
)
|
||||
|
|
|
@ -5,41 +5,23 @@ package qerr
|
|||
import "strconv"
|
||||
|
||||
const (
|
||||
_ErrorCode_name_0 = "InternalErrorStreamDataAfterTerminationInvalidPacketHeaderInvalidFrameDataInvalidFecDataInvalidRstStreamDataInvalidConnectionCloseDataInvalidGoawayDataInvalidAckDataInvalidVersionNegotiationPacketInvalidPublicRstPacketDecryptionFailureEncryptionFailurePacketTooLarge"
|
||||
_ErrorCode_name_1 = "PeerGoingAwayInvalidStreamIDTooManyOpenStreamsPublicResetInvalidVersion"
|
||||
_ErrorCode_name_2 = "InvalidHeaderIDInvalidNegotiatedValueDecompressionFailureNetworkIdleTimeoutErrorMigratingAddressPacketWriteErrorHandshakeFailedCryptoTagsOutOfOrderCryptoTooManyEntriesCryptoInvalidValueLengthCryptoMessageAfterHandshakeCompleteInvalidCryptoMessageTypeInvalidCryptoMessageParameterCryptoMessageParameterNotFoundCryptoMessageParameterNoOverlapCryptoMessageIndexNotFoundCryptoInternalErrorCryptoVersionNotSupportedCryptoNoSupportCryptoTooManyRejectsProofInvalidCryptoDuplicateTagCryptoEncryptionLevelIncorrectCryptoServerConfigExpiredInvalidStreamData"
|
||||
_ErrorCode_name_3 = "MissingPayloadInvalidPriorityEmptyStreamFrameNoFinPacketReadErrorInvalidChannelIDSignatureCryptoSymmetricKeySetupFailedCryptoMessageWhileValidatingClientHelloVersionNegotiationMismatchInvalidHeadersStreamDataInvalidWindowUpdateDataInvalidBlockedDataFlowControlReceivedTooMuchDataInvalidStopWaitingDataUnencryptedStreamDataConnectionIPPooledFlowControlSentTooMuchDataFlowControlInvalidWindowCryptoUpdateBeforeHandshakeComplete"
|
||||
_ErrorCode_name_4 = "HandshakeTimeoutTooManyOutstandingSentPacketsTooManyOutstandingReceivedPacketsConnectionCancelledBadPacketLossRateCryptoHandshakeStatelessRejectPublicResetsPostHandshakeTimeoutsWithOpenStreamsFailedToSerializePacketTooManyAvailableStreamsUnencryptedFecDataInvalidPathCloseDataBadMultipathFlagIPAddressChangedConnectionMigrationNoMigratableStreamsConnectionMigrationTooManyChangesConnectionMigrationNoNewNetworkConnectionMigrationNonMigratableStreamTooManyRtosErrorMigratingPortOverlappingStreamDataAttemptToSendUnencryptedStreamData"
|
||||
_ErrorCode_name_5 = "HeadersStreamDataDecompressFailure"
|
||||
_ErrorCode_name_0 = "NoErrorInternalErrorServerBusyFlowControlErrorStreamLimitErrorStreamStateErrorFinalSizeErrorFrameEncodingErrorTransportParameterErrorVersionNegotiationErrorProtocolViolation"
|
||||
_ErrorCode_name_1 = "InvalidMigration"
|
||||
_ErrorCode_name_2 = "CryptoError"
|
||||
)
|
||||
|
||||
var (
|
||||
_ErrorCode_index_0 = [...]uint16{0, 13, 39, 58, 74, 88, 108, 134, 151, 165, 196, 218, 235, 252, 266}
|
||||
_ErrorCode_index_1 = [...]uint8{0, 13, 28, 46, 57, 71}
|
||||
_ErrorCode_index_2 = [...]uint16{0, 15, 37, 57, 75, 96, 112, 127, 147, 167, 191, 226, 250, 279, 309, 340, 366, 385, 410, 425, 445, 457, 475, 505, 530, 547}
|
||||
_ErrorCode_index_3 = [...]uint16{0, 14, 29, 50, 65, 90, 119, 158, 184, 208, 231, 249, 279, 301, 322, 340, 366, 390, 425}
|
||||
_ErrorCode_index_4 = [...]uint16{0, 16, 45, 78, 97, 114, 144, 169, 192, 215, 238, 256, 276, 292, 308, 346, 379, 410, 448, 459, 477, 498, 532}
|
||||
_ErrorCode_index_0 = [...]uint8{0, 7, 20, 30, 46, 62, 78, 92, 110, 133, 156, 173}
|
||||
)
|
||||
|
||||
func (i ErrorCode) String() string {
|
||||
switch {
|
||||
case 1 <= i && i <= 14:
|
||||
i -= 1
|
||||
case 0 <= i && i <= 10:
|
||||
return _ErrorCode_name_0[_ErrorCode_index_0[i]:_ErrorCode_index_0[i+1]]
|
||||
case 16 <= i && i <= 20:
|
||||
i -= 16
|
||||
return _ErrorCode_name_1[_ErrorCode_index_1[i]:_ErrorCode_index_1[i+1]]
|
||||
case 22 <= i && i <= 46:
|
||||
i -= 22
|
||||
return _ErrorCode_name_2[_ErrorCode_index_2[i]:_ErrorCode_index_2[i+1]]
|
||||
case 48 <= i && i <= 65:
|
||||
i -= 48
|
||||
return _ErrorCode_name_3[_ErrorCode_index_3[i]:_ErrorCode_index_3[i+1]]
|
||||
case 67 <= i && i <= 88:
|
||||
i -= 67
|
||||
return _ErrorCode_name_4[_ErrorCode_index_4[i]:_ErrorCode_index_4[i+1]]
|
||||
case i == 97:
|
||||
return _ErrorCode_name_5
|
||||
case i == 12:
|
||||
return _ErrorCode_name_1
|
||||
case i == 256:
|
||||
return _ErrorCode_name_2
|
||||
default:
|
||||
return "ErrorCode(" + strconv.FormatInt(int64(i), 10) + ")"
|
||||
}
|
||||
|
|
|
@ -29,10 +29,9 @@ var _ = Describe("error codes", func() {
|
|||
for _, c := range constSpecs {
|
||||
name := c.(*ast.ValueSpec).Names[0].Name
|
||||
valString := c.(*ast.ValueSpec).Values[0].(*ast.BasicLit).Value
|
||||
val, err := strconv.Atoi(valString)
|
||||
val, err := strconv.ParseInt(valString, 0, 64)
|
||||
Expect(err).NotTo(HaveOccurred())
|
||||
Expect(ErrorCode(val).String()).To(Equal(name))
|
||||
}
|
||||
Expect(ErrorCode(0).String()).To(Equal("ErrorCode(0)"))
|
||||
})
|
||||
})
|
||||
|
|
|
@ -16,6 +16,7 @@ func (e ErrorCode) Error() string {
|
|||
type QuicError struct {
|
||||
ErrorCode ErrorCode
|
||||
ErrorMessage string
|
||||
isTimeout bool
|
||||
}
|
||||
|
||||
var _ net.Error = &QuicError{}
|
||||
|
@ -28,6 +29,14 @@ func Error(errorCode ErrorCode, errorMessage string) *QuicError {
|
|||
}
|
||||
}
|
||||
|
||||
// TimeoutError creates a new QuicError instance for a timeout error
|
||||
func TimeoutError(errorMessage string) *QuicError {
|
||||
return &QuicError{
|
||||
ErrorMessage: errorMessage,
|
||||
isTimeout: true,
|
||||
}
|
||||
}
|
||||
|
||||
func (e *QuicError) Error() string {
|
||||
return fmt.Sprintf("%s: %s", e.ErrorCode.String(), e.ErrorMessage)
|
||||
}
|
||||
|
@ -39,13 +48,7 @@ func (e *QuicError) Temporary() bool {
|
|||
|
||||
// Timeout says if this error is a timeout.
|
||||
func (e *QuicError) Timeout() bool {
|
||||
switch e.ErrorCode {
|
||||
case NetworkIdleTimeout,
|
||||
HandshakeTimeout,
|
||||
TimeoutsWithOpenStreams:
|
||||
return true
|
||||
}
|
||||
return false
|
||||
return e.isTimeout
|
||||
}
|
||||
|
||||
// ToQuicError converts an arbitrary error to a QuicError. It leaves QuicErrors
|
||||
|
|
|
@ -7,37 +7,30 @@ import (
|
|||
. "github.com/onsi/gomega"
|
||||
)
|
||||
|
||||
var _ = Describe("Quic error", func() {
|
||||
var _ = Describe("QUIC Transport Errors", func() {
|
||||
Context("QuicError", func() {
|
||||
It("has a string representation", func() {
|
||||
err := Error(DecryptionFailure, "foobar")
|
||||
Expect(err.Error()).To(Equal("DecryptionFailure: foobar"))
|
||||
err := Error(FlowControlError, "foobar")
|
||||
Expect(err.Error()).To(Equal("FlowControlError: foobar"))
|
||||
})
|
||||
})
|
||||
|
||||
Context("ErrorCode", func() {
|
||||
It("works as error", func() {
|
||||
var err error = DecryptionFailure
|
||||
Expect(err).To(MatchError("DecryptionFailure"))
|
||||
})
|
||||
})
|
||||
|
||||
Context("TimeoutError", func() {
|
||||
It("works as timeout error", func() {
|
||||
err := Error(HandshakeTimeout, "handshake timeout")
|
||||
Expect(err.Timeout()).Should(BeTrue())
|
||||
var err error = StreamStateError
|
||||
Expect(err).To(MatchError("StreamStateError"))
|
||||
})
|
||||
})
|
||||
|
||||
Context("ToQuicError", func() {
|
||||
It("leaves QuicError unchanged", func() {
|
||||
err := Error(DecryptionFailure, "foo")
|
||||
err := Error(TransportParameterError, "foo")
|
||||
Expect(ToQuicError(err)).To(Equal(err))
|
||||
})
|
||||
|
||||
It("wraps ErrorCode properly", func() {
|
||||
var err error = DecryptionFailure
|
||||
Expect(ToQuicError(err)).To(Equal(Error(DecryptionFailure, "")))
|
||||
var err error = FinalSizeError
|
||||
Expect(ToQuicError(err)).To(Equal(Error(FinalSizeError, "")))
|
||||
})
|
||||
|
||||
It("changes default errors to InternalError", func() {
|
||||
|
|
|
@ -40,7 +40,7 @@ func (p *frameParser) parseFrame(r *bytes.Reader, typeByte byte, encLevel protoc
|
|||
if typeByte&0xf8 == 0x8 {
|
||||
frame, err = parseStreamFrame(r, p.version)
|
||||
if err != nil {
|
||||
return nil, qerr.Error(qerr.InvalidFrameData, err.Error())
|
||||
return nil, qerr.Error(qerr.FrameEncodingError, err.Error())
|
||||
}
|
||||
return frame, nil
|
||||
}
|
||||
|
@ -87,7 +87,7 @@ func (p *frameParser) parseFrame(r *bytes.Reader, typeByte byte, encLevel protoc
|
|||
err = fmt.Errorf("unknown type byte 0x%x", typeByte)
|
||||
}
|
||||
if err != nil {
|
||||
return nil, qerr.Error(qerr.InvalidFrameData, err.Error())
|
||||
return nil, qerr.Error(qerr.FrameEncodingError, err.Error())
|
||||
}
|
||||
return frame, nil
|
||||
}
|
||||
|
|
|
@ -273,7 +273,7 @@ var _ = Describe("Frame parsing", func() {
|
|||
|
||||
It("errors on invalid type", func() {
|
||||
_, err := parser.ParseNext(bytes.NewReader([]byte{0x42}), protocol.Encryption1RTT)
|
||||
Expect(err).To(MatchError("InvalidFrameData: unknown type byte 0x42"))
|
||||
Expect(err).To(MatchError("FrameEncodingError: unknown type byte 0x42"))
|
||||
})
|
||||
|
||||
It("errors on invalid frames", func() {
|
||||
|
@ -285,6 +285,6 @@ var _ = Describe("Frame parsing", func() {
|
|||
f.Write(b, versionIETFFrames)
|
||||
_, err := parser.ParseNext(bytes.NewReader(b.Bytes()[:b.Len()-2]), protocol.Encryption1RTT)
|
||||
Expect(err).To(HaveOccurred())
|
||||
Expect(err.(*qerr.QuicError).ErrorCode).To(Equal(qerr.InvalidFrameData))
|
||||
Expect(err.(*qerr.QuicError).ErrorCode).To(Equal(qerr.FrameEncodingError))
|
||||
})
|
||||
})
|
||||
|
|
|
@ -7,7 +7,6 @@ import (
|
|||
"io"
|
||||
|
||||
"github.com/lucas-clemente/quic-go/internal/protocol"
|
||||
"github.com/lucas-clemente/quic-go/internal/qerr"
|
||||
"github.com/lucas-clemente/quic-go/internal/utils"
|
||||
)
|
||||
|
||||
|
@ -209,13 +208,16 @@ func (h *Header) parseLongHeader(b *bytes.Reader) error {
|
|||
|
||||
func (h *Header) parseVersionNegotiationPacket(b *bytes.Reader) error {
|
||||
if b.Len() == 0 {
|
||||
return qerr.Error(qerr.InvalidVersionNegotiationPacket, "empty version list")
|
||||
return errors.New("Version Negoation packet has empty version list")
|
||||
}
|
||||
if b.Len()%4 != 0 {
|
||||
return errors.New("Version Negotation packet has a version list with an invalid length")
|
||||
}
|
||||
h.SupportedVersions = make([]protocol.VersionNumber, b.Len()/4)
|
||||
for i := 0; b.Len() > 0; i++ {
|
||||
v, err := utils.BigEndian.ReadUint32(b)
|
||||
if err != nil {
|
||||
return qerr.InvalidVersionNegotiationPacket
|
||||
return err
|
||||
}
|
||||
h.SupportedVersions[i] = protocol.VersionNumber(v)
|
||||
}
|
||||
|
|
|
@ -6,7 +6,6 @@ import (
|
|||
"io"
|
||||
|
||||
"github.com/lucas-clemente/quic-go/internal/protocol"
|
||||
"github.com/lucas-clemente/quic-go/internal/qerr"
|
||||
. "github.com/onsi/ginkgo"
|
||||
. "github.com/onsi/gomega"
|
||||
)
|
||||
|
@ -138,7 +137,7 @@ var _ = Describe("Header Parsing", func() {
|
|||
data, err := ComposeVersionNegotiation(connID, connID, versions)
|
||||
Expect(err).ToNot(HaveOccurred())
|
||||
_, _, _, err = ParsePacket(data[:len(data)-2], 0)
|
||||
Expect(err).To(MatchError(qerr.InvalidVersionNegotiationPacket))
|
||||
Expect(err).To(MatchError("Version Negotation packet has a version list with an invalid length"))
|
||||
})
|
||||
|
||||
It("errors if the version list is empty", func() {
|
||||
|
@ -149,7 +148,7 @@ var _ = Describe("Header Parsing", func() {
|
|||
// remove 8 bytes (two versions), since ComposeVersionNegotiation also added a reserved version number
|
||||
data = data[:len(data)-8]
|
||||
_, _, _, err = ParsePacket(data, 0)
|
||||
Expect(err).To(MatchError("InvalidVersionNegotiationPacket: empty version list"))
|
||||
Expect(err).To(MatchError("Version Negoation packet has empty version list"))
|
||||
})
|
||||
})
|
||||
|
||||
|
|
|
@ -69,7 +69,7 @@ func parseStreamFrame(r *bytes.Reader, version protocol.VersionNumber) (*StreamF
|
|||
}
|
||||
}
|
||||
if frame.Offset+frame.DataLen() > protocol.MaxByteCount {
|
||||
return nil, qerr.Error(qerr.InvalidStreamData, "data overflows maximum offset")
|
||||
return nil, qerr.Error(qerr.FrameEncodingError, "stream data overflows maximum offset")
|
||||
}
|
||||
return frame, nil
|
||||
}
|
||||
|
|
|
@ -4,7 +4,6 @@ import (
|
|||
"bytes"
|
||||
|
||||
"github.com/lucas-clemente/quic-go/internal/protocol"
|
||||
"github.com/lucas-clemente/quic-go/internal/qerr"
|
||||
"github.com/lucas-clemente/quic-go/internal/utils"
|
||||
. "github.com/onsi/ginkgo"
|
||||
. "github.com/onsi/gomega"
|
||||
|
@ -76,7 +75,7 @@ var _ = Describe("STREAM frame", func() {
|
|||
data = append(data, []byte("foobar")...)
|
||||
r := bytes.NewReader(data)
|
||||
_, err := parseStreamFrame(r, versionIETFFrames)
|
||||
Expect(err).To(MatchError(qerr.Error(qerr.InvalidStreamData, "data overflows maximum offset")))
|
||||
Expect(err).To(MatchError("FrameEncodingError: stream data overflows maximum offset"))
|
||||
})
|
||||
|
||||
It("errors on EOFs", func() {
|
||||
|
|
|
@ -504,8 +504,7 @@ func (s *server) sendServerBusy(remoteAddr net.Addr, hdr *wire.Header) error {
|
|||
defer packetBuffer.Release()
|
||||
buf := bytes.NewBuffer(packetBuffer.Slice[:0])
|
||||
|
||||
// TODO(#1567): use the SERVER_BUSY error code
|
||||
ccf := &wire.ConnectionCloseFrame{ErrorCode: qerr.PeerGoingAway}
|
||||
ccf := &wire.ConnectionCloseFrame{ErrorCode: qerr.ServerBusy}
|
||||
|
||||
replyHdr := &wire.ExtendedHeader{}
|
||||
replyHdr.IsLongHeader = true
|
||||
|
|
25
session.go
25
session.go
|
@ -410,11 +410,11 @@ runLoop:
|
|||
}
|
||||
|
||||
if !s.handshakeComplete && now.Sub(s.sessionCreationTime) >= s.config.HandshakeTimeout {
|
||||
s.destroy(qerr.Error(qerr.HandshakeTimeout, "Crypto handshake did not complete in time."))
|
||||
s.destroy(qerr.TimeoutError("Handshake did not complete in time"))
|
||||
continue
|
||||
}
|
||||
if s.handshakeComplete && now.Sub(s.lastNetworkActivityTime) >= s.config.IdleTimeout {
|
||||
s.destroy(qerr.Error(qerr.NetworkIdleTimeout, "No recent network activity."))
|
||||
s.destroy(qerr.TimeoutError("No recent network activity"))
|
||||
continue
|
||||
}
|
||||
|
||||
|
@ -613,7 +613,7 @@ func (s *session) handleRetryPacket(p *receivedPacket, hdr *wire.Header) bool /*
|
|||
|
||||
func (s *session) handleUnpackedPacket(packet *unpackedPacket, rcvTime time.Time) error {
|
||||
if len(packet.data) == 0 {
|
||||
return qerr.MissingPayload
|
||||
return qerr.Error(qerr.ProtocolViolation, "empty packet")
|
||||
}
|
||||
|
||||
// The server can change the source connection ID with the first Handshake packet.
|
||||
|
@ -745,8 +745,9 @@ func (s *session) handleCryptoFrame(frame *wire.CryptoFrame, encLevel protocol.E
|
|||
}
|
||||
|
||||
func (s *session) handleStreamFrame(frame *wire.StreamFrame, encLevel protocol.EncryptionLevel) error {
|
||||
// TODO(#1261): implement strict rules for frames types in unencrypted packets
|
||||
if encLevel < protocol.Encryption1RTT {
|
||||
return qerr.Error(qerr.UnencryptedStreamData, fmt.Sprintf("received unencrypted stream data on stream %d", frame.StreamID))
|
||||
return qerr.Error(qerr.ProtocolViolation, fmt.Sprintf("received unencrypted stream data on stream %d", frame.StreamID))
|
||||
}
|
||||
str, err := s.streamsMap.GetOrOpenReceiveStream(frame.StreamID)
|
||||
if err != nil {
|
||||
|
@ -851,7 +852,7 @@ func (s *session) closeRemote(e error) {
|
|||
})
|
||||
}
|
||||
|
||||
// Close the connection. It sends a qerr.PeerGoingAway.
|
||||
// Close the connection. It sends a qerr.NoError.
|
||||
// It waits until the run loop has stopped before returning
|
||||
func (s *session) Close() error {
|
||||
s.closeLocal(nil)
|
||||
|
@ -867,7 +868,7 @@ func (s *session) CloseWithError(code protocol.ApplicationErrorCode, e error) er
|
|||
|
||||
func (s *session) handleCloseError(closeErr closeError) error {
|
||||
if closeErr.err == nil {
|
||||
closeErr.err = qerr.PeerGoingAway
|
||||
closeErr.err = qerr.NoError
|
||||
}
|
||||
|
||||
var quicErr *qerr.QuicError
|
||||
|
@ -875,8 +876,8 @@ func (s *session) handleCloseError(closeErr closeError) error {
|
|||
if quicErr, ok = closeErr.err.(*qerr.QuicError); !ok {
|
||||
quicErr = qerr.ToQuicError(closeErr.err)
|
||||
}
|
||||
// Don't log 'normal' reasons
|
||||
if quicErr.ErrorCode == qerr.PeerGoingAway || quicErr.ErrorCode == qerr.NetworkIdleTimeout {
|
||||
// Don't log timeout errors
|
||||
if quicErr.Timeout() {
|
||||
s.logger.Infof("Closing connection %s.", s.srcConnID)
|
||||
} else {
|
||||
s.logger.Errorf("Closing session with error: %s", closeErr.err.Error())
|
||||
|
@ -924,17 +925,17 @@ func (s *session) processTransportParametersForClient(data []byte) (*handshake.T
|
|||
}
|
||||
// check that the negotiated_version is the current version
|
||||
if eetp.NegotiatedVersion != s.version {
|
||||
return nil, qerr.Error(qerr.VersionNegotiationMismatch, "current version doesn't match negotiated_version")
|
||||
return nil, qerr.Error(qerr.VersionNegotiationError, "current version doesn't match negotiated_version")
|
||||
}
|
||||
// check that the current version is included in the supported versions
|
||||
if !protocol.IsSupportedVersion(eetp.SupportedVersions, s.version) {
|
||||
return nil, qerr.Error(qerr.VersionNegotiationMismatch, "current version not included in the supported versions")
|
||||
return nil, qerr.Error(qerr.VersionNegotiationError, "current version not included in the supported versions")
|
||||
}
|
||||
// if version negotiation was performed, check that we would have selected the current version based on the supported versions sent by the server
|
||||
if s.version != s.initialVersion {
|
||||
negotiatedVersion, ok := protocol.ChooseSupportedVersion(s.config.Versions, eetp.SupportedVersions)
|
||||
if !ok || s.version != negotiatedVersion {
|
||||
return nil, qerr.Error(qerr.VersionNegotiationMismatch, "would have picked a different version")
|
||||
return nil, qerr.Error(qerr.VersionNegotiationError, "would have picked a different version")
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -960,7 +961,7 @@ func (s *session) processTransportParametersForServer(data []byte) (*handshake.T
|
|||
// make sure that we would have sent a Version Negotiation Packet if the client offered the initial version
|
||||
// this is the case if and only if the initial version is not contained in the supported versions
|
||||
if chtp.InitialVersion != s.version && protocol.IsSupportedVersion(s.config.Versions, chtp.InitialVersion) {
|
||||
return nil, qerr.Error(qerr.VersionNegotiationMismatch, "Client should have used the initial version")
|
||||
return nil, qerr.Error(qerr.VersionNegotiationError, "Client should have used the initial version")
|
||||
}
|
||||
return &chtp.Parameters, nil
|
||||
}
|
||||
|
|
|
@ -146,7 +146,7 @@ var _ = Describe("Session", func() {
|
|||
StreamID: 3,
|
||||
Data: []byte("foobar"),
|
||||
}, protocol.EncryptionHandshake)
|
||||
Expect(err).To(MatchError(qerr.Error(qerr.UnencryptedStreamData, "received unencrypted stream data on stream 3")))
|
||||
Expect(err).To(MatchError(qerr.Error(qerr.ProtocolViolation, "received unencrypted stream data on stream 3")))
|
||||
})
|
||||
})
|
||||
|
||||
|
@ -320,7 +320,7 @@ var _ = Describe("Session", func() {
|
|||
})
|
||||
|
||||
It("handles CONNECTION_CLOSE frames", func() {
|
||||
testErr := qerr.Error(qerr.ProofInvalid, "foobar")
|
||||
testErr := qerr.Error(qerr.StreamLimitError, "foobar")
|
||||
streamManager.EXPECT().CloseWithError(testErr)
|
||||
sessionRunner.EXPECT().removeConnectionID(gomock.Any())
|
||||
cryptoSetup.EXPECT().Close()
|
||||
|
@ -328,11 +328,13 @@ var _ = Describe("Session", func() {
|
|||
go func() {
|
||||
defer GinkgoRecover()
|
||||
cryptoSetup.EXPECT().RunHandshake().Do(func() { <-sess.Context().Done() })
|
||||
err := sess.run()
|
||||
Expect(err).To(MatchError(testErr))
|
||||
Expect(sess.run()).To(MatchError(testErr))
|
||||
}()
|
||||
err := sess.handleFrame(&wire.ConnectionCloseFrame{ErrorCode: qerr.ProofInvalid, ReasonPhrase: "foobar"}, 0, protocol.EncryptionUnspecified)
|
||||
Expect(err).NotTo(HaveOccurred())
|
||||
ccf := &wire.ConnectionCloseFrame{
|
||||
ErrorCode: qerr.StreamLimitError,
|
||||
ReasonPhrase: "foobar",
|
||||
}
|
||||
Expect(sess.handleFrame(ccf, 0, protocol.EncryptionUnspecified)).To(Succeed())
|
||||
Eventually(sess.Context().Done()).Should(BeClosed())
|
||||
})
|
||||
})
|
||||
|
@ -374,7 +376,7 @@ var _ = Describe("Session", func() {
|
|||
})
|
||||
|
||||
It("shuts down without error", func() {
|
||||
streamManager.EXPECT().CloseWithError(qerr.Error(qerr.PeerGoingAway, ""))
|
||||
streamManager.EXPECT().CloseWithError(qerr.Error(qerr.NoError, ""))
|
||||
sessionRunner.EXPECT().retireConnectionID(gomock.Any())
|
||||
cryptoSetup.EXPECT().Close()
|
||||
packer.EXPECT().PackConnectionClose(gomock.Any()).Return(&packedPacket{raw: []byte("connection close")}, nil)
|
||||
|
@ -386,7 +388,7 @@ var _ = Describe("Session", func() {
|
|||
})
|
||||
|
||||
It("only closes once", func() {
|
||||
streamManager.EXPECT().CloseWithError(qerr.Error(qerr.PeerGoingAway, ""))
|
||||
streamManager.EXPECT().CloseWithError(qerr.Error(qerr.NoError, ""))
|
||||
sessionRunner.EXPECT().retireConnectionID(gomock.Any())
|
||||
cryptoSetup.EXPECT().Close()
|
||||
packer.EXPECT().PackConnectionClose(gomock.Any()).Return(&packedPacket{}, nil)
|
||||
|
@ -569,7 +571,7 @@ var _ = Describe("Session", func() {
|
|||
defer GinkgoRecover()
|
||||
cryptoSetup.EXPECT().RunHandshake().Do(func() { <-sess.Context().Done() })
|
||||
err := sess.run()
|
||||
Expect(err).To(MatchError(qerr.MissingPayload))
|
||||
Expect(err).To(MatchError("ProtocolViolation: empty packet"))
|
||||
close(done)
|
||||
}()
|
||||
sessionRunner.EXPECT().retireConnectionID(gomock.Any())
|
||||
|
@ -1217,7 +1219,7 @@ var _ = Describe("Session", func() {
|
|||
InitialVersion: 13, // this must be a supported version
|
||||
}
|
||||
_, err := sess.processTransportParametersForServer(chtp.Marshal())
|
||||
Expect(err).To(MatchError("VersionNegotiationMismatch: Client should have used the initial version"))
|
||||
Expect(err).To(MatchError("VersionNegotiationError: Client should have used the initial version"))
|
||||
})
|
||||
})
|
||||
|
||||
|
@ -1314,7 +1316,10 @@ var _ = Describe("Session", func() {
|
|||
defer GinkgoRecover()
|
||||
cryptoSetup.EXPECT().RunHandshake().Do(func() { <-sess.Context().Done() })
|
||||
err := sess.run()
|
||||
Expect(err.(*qerr.QuicError).ErrorCode).To(Equal(qerr.NetworkIdleTimeout))
|
||||
nerr, ok := err.(net.Error)
|
||||
Expect(ok).To(BeTrue())
|
||||
Expect(nerr.Timeout()).To(BeTrue())
|
||||
Expect(err.Error()).To(ContainSubstring("No recent network activity"))
|
||||
close(done)
|
||||
}()
|
||||
Eventually(done).Should(BeClosed())
|
||||
|
@ -1329,7 +1334,10 @@ var _ = Describe("Session", func() {
|
|||
defer GinkgoRecover()
|
||||
cryptoSetup.EXPECT().RunHandshake().Do(func() { <-sess.Context().Done() })
|
||||
err := sess.run()
|
||||
Expect(err.(*qerr.QuicError).ErrorCode).To(Equal(qerr.HandshakeTimeout))
|
||||
nerr, ok := err.(net.Error)
|
||||
Expect(ok).To(BeTrue())
|
||||
Expect(nerr.Timeout()).To(BeTrue())
|
||||
Expect(err.Error()).To(ContainSubstring("Handshake did not complete in time"))
|
||||
close(done)
|
||||
}()
|
||||
Eventually(done).Should(BeClosed())
|
||||
|
@ -1339,7 +1347,7 @@ var _ = Describe("Session", func() {
|
|||
sess.config.IdleTimeout = 9999 * time.Second
|
||||
sess.lastNetworkActivityTime = time.Now().Add(-time.Minute)
|
||||
packer.EXPECT().PackConnectionClose(gomock.Any()).DoAndReturn(func(f *wire.ConnectionCloseFrame) (*packedPacket, error) {
|
||||
Expect(f.ErrorCode).To(Equal(qerr.PeerGoingAway))
|
||||
Expect(f.ErrorCode).To(Equal(qerr.NoError))
|
||||
return &packedPacket{}, nil
|
||||
})
|
||||
// the handshake timeout is irrelevant here, since it depends on the time the session was created,
|
||||
|
@ -1368,7 +1376,10 @@ var _ = Describe("Session", func() {
|
|||
sessionRunner.EXPECT().onHandshakeComplete(sess)
|
||||
cryptoSetup.EXPECT().RunHandshake()
|
||||
err := sess.run()
|
||||
Expect(err.(*qerr.QuicError).ErrorCode).To(Equal(qerr.NetworkIdleTimeout))
|
||||
nerr, ok := err.(net.Error)
|
||||
Expect(ok).To(BeTrue())
|
||||
Expect(nerr.Timeout()).To(BeTrue())
|
||||
Expect(err.Error()).To(ContainSubstring("No recent network activity"))
|
||||
close(done)
|
||||
}()
|
||||
Eventually(done).Should(BeClosed())
|
||||
|
@ -1685,7 +1696,7 @@ var _ = Describe("Client Session", func() {
|
|||
Parameters: params,
|
||||
}
|
||||
_, err := sess.processTransportParametersForClient(eetp.Marshal())
|
||||
Expect(err).To(MatchError("VersionNegotiationMismatch: current version doesn't match negotiated_version"))
|
||||
Expect(err).To(MatchError("VersionNegotiationError: current version doesn't match negotiated_version"))
|
||||
})
|
||||
|
||||
It("errors if the current version is not contained in the server's supported versions", func() {
|
||||
|
@ -1696,7 +1707,7 @@ var _ = Describe("Client Session", func() {
|
|||
Parameters: params,
|
||||
}
|
||||
_, err := sess.processTransportParametersForClient(eetp.Marshal())
|
||||
Expect(err).To(MatchError("VersionNegotiationMismatch: current version not included in the supported versions"))
|
||||
Expect(err).To(MatchError("VersionNegotiationError: current version not included in the supported versions"))
|
||||
})
|
||||
|
||||
It("errors if version negotiation was performed, but would have picked a different version based on the supported version list", func() {
|
||||
|
@ -1714,7 +1725,7 @@ var _ = Describe("Client Session", func() {
|
|||
Parameters: params,
|
||||
}
|
||||
_, err := sess.processTransportParametersForClient(eetp.Marshal())
|
||||
Expect(err).To(MatchError("VersionNegotiationMismatch: would have picked a different version"))
|
||||
Expect(err).To(MatchError("VersionNegotiationError: would have picked a different version"))
|
||||
})
|
||||
|
||||
It("doesn't error if it would have picked a different version based on the supported version list, if no version negotiation was performed", func() {
|
||||
|
|
|
@ -107,7 +107,7 @@ func (m *outgoingBidiStreamsMap) GetStream(id protocol.StreamID) (streamI, error
|
|||
m.mutex.RLock()
|
||||
if id >= m.nextStream {
|
||||
m.mutex.RUnlock()
|
||||
return nil, qerr.Error(qerr.InvalidStreamID, fmt.Sprintf("peer attempted to open stream %d", id))
|
||||
return nil, qerr.Error(qerr.StreamStateError, fmt.Sprintf("peer attempted to open stream %d", id))
|
||||
}
|
||||
s := m.streams[id]
|
||||
m.mutex.RUnlock()
|
||||
|
|
|
@ -105,7 +105,7 @@ func (m *outgoingItemsMap) GetStream(id protocol.StreamID) (item, error) {
|
|||
m.mutex.RLock()
|
||||
if id >= m.nextStream {
|
||||
m.mutex.RUnlock()
|
||||
return nil, qerr.Error(qerr.InvalidStreamID, fmt.Sprintf("peer attempted to open stream %d", id))
|
||||
return nil, qerr.Error(qerr.StreamStateError, fmt.Sprintf("peer attempted to open stream %d", id))
|
||||
}
|
||||
s := m.streams[id]
|
||||
m.mutex.RUnlock()
|
||||
|
|
|
@ -59,7 +59,7 @@ var _ = Describe("Streams Map (outgoing)", func() {
|
|||
|
||||
It("errors when trying to get a stream that has not yet been opened", func() {
|
||||
_, err := m.GetStream(firstNewStream)
|
||||
Expect(err).To(MatchError(qerr.Error(qerr.InvalidStreamID, "peer attempted to open stream 3")))
|
||||
Expect(err).To(MatchError(qerr.Error(qerr.StreamStateError, "peer attempted to open stream 3")))
|
||||
})
|
||||
|
||||
It("deletes streams", func() {
|
||||
|
|
|
@ -107,7 +107,7 @@ func (m *outgoingUniStreamsMap) GetStream(id protocol.StreamID) (sendStreamI, er
|
|||
m.mutex.RLock()
|
||||
if id >= m.nextStream {
|
||||
m.mutex.RUnlock()
|
||||
return nil, qerr.Error(qerr.InvalidStreamID, fmt.Sprintf("peer attempted to open stream %d", id))
|
||||
return nil, qerr.Error(qerr.StreamStateError, fmt.Sprintf("peer attempted to open stream %d", id))
|
||||
}
|
||||
s := m.streams[id]
|
||||
m.mutex.RUnlock()
|
||||
|
|
|
@ -222,7 +222,7 @@ var _ = Describe("Streams Map", func() {
|
|||
It("errors when the peer tries to open a higher outgoing bidirectional stream", func() {
|
||||
id := ids.firstOutgoingBidiStream + 5*4
|
||||
_, err := m.GetOrOpenSendStream(id)
|
||||
Expect(err).To(MatchError(qerr.Error(qerr.InvalidStreamID, fmt.Sprintf("peer attempted to open stream %d", id))))
|
||||
Expect(err).To(MatchError(qerr.Error(qerr.StreamStateError, fmt.Sprintf("peer attempted to open stream %d", id))))
|
||||
})
|
||||
|
||||
It("gets an outgoing unidirectional stream", func() {
|
||||
|
@ -238,7 +238,7 @@ var _ = Describe("Streams Map", func() {
|
|||
It("errors when the peer tries to open a higher outgoing bidirectional stream", func() {
|
||||
id := ids.firstOutgoingUniStream + 5*4
|
||||
_, err := m.GetOrOpenSendStream(id)
|
||||
Expect(err).To(MatchError(qerr.Error(qerr.InvalidStreamID, fmt.Sprintf("peer attempted to open stream %d", id))))
|
||||
Expect(err).To(MatchError(qerr.Error(qerr.StreamStateError, fmt.Sprintf("peer attempted to open stream %d", id))))
|
||||
})
|
||||
|
||||
It("gets an incoming bidirectional stream", func() {
|
||||
|
@ -269,7 +269,7 @@ var _ = Describe("Streams Map", func() {
|
|||
It("errors when the peer tries to open a higher outgoing bidirectional stream", func() {
|
||||
id := ids.firstOutgoingBidiStream + 5*4
|
||||
_, err := m.GetOrOpenReceiveStream(id)
|
||||
Expect(err).To(MatchError(qerr.Error(qerr.InvalidStreamID, fmt.Sprintf("peer attempted to open stream %d", id))))
|
||||
Expect(err).To(MatchError(qerr.Error(qerr.StreamStateError, fmt.Sprintf("peer attempted to open stream %d", id))))
|
||||
})
|
||||
|
||||
It("gets an incoming bidirectional stream", func() {
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue