use Optional<T> instead of T? to workaround SR-11777 (#1252)

Motivation:

In Swift, writing

    var something: T?

    init() {
        self.something = someValue
    }

means that the compiler will first set `self.something` to `nil` and
then in the init override it with `self.someValue`
(https://bugs.swift.org/browse/SR-11777). Unfortunately, because of
https://bugs.swift.org/browse/SR-11768 , stored property initialisation
cannot be made `@inlinable` (short of using `@frozen` which isn't
available in Swift 5.0).

The combination of SR-11768 and SR-11777 leads to `var something: T?`
having much worse code than `var something: Optional<T>` iff the `init`
is `public` and `@inlinable`.

Modifications:

Change all `var something: T?` to `var something: Optional<T>`

Result:

Faster code, sad NIO developers.
This commit is contained in:
Johannes Weiss 2019-11-27 18:37:38 +00:00 committed by GitHub
parent 732a6e0ef4
commit 74944a937d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 68 additions and 36 deletions

View File

@ -227,7 +227,7 @@ class BaseSocketChannel<SocketType: BaseSocketProtocol>: SelectableChannel, Chan
}
var readPending = false
var pendingConnect: EventLoopPromise<Void>?
var pendingConnect: Optional<EventLoopPromise<Void>>
var recvAllocator: RecvByteBufferAllocator
var maxMessagesPerRead: UInt = 4
@ -390,6 +390,7 @@ class BaseSocketChannel<SocketType: BaseSocketProtocol>: SelectableChannel, Chan
// As the socket may already be connected we should ensure we start with the correct addresses cached.
self.addressesCached.store(Box((local: try? socket.localAddress(), remote: try? socket.remoteAddress())))
self.socketDescription = socket.description
self.pendingConnect = nil
self._pipeline = ChannelPipeline(channel: self)
}

View File

@ -13,7 +13,7 @@
//===----------------------------------------------------------------------===//
class BaseStreamSocketChannel<Socket: SocketProtocol>: BaseSocketChannel<Socket> {
internal var connectTimeoutScheduled: Scheduled<Void>?
internal var connectTimeoutScheduled: Optional<Scheduled<Void>>
private var allowRemoteHalfClosure: Bool = false
private var inputShutdown: Bool = false
private var outputShutdown: Bool = false
@ -24,6 +24,7 @@ class BaseStreamSocketChannel<Socket: SocketProtocol>: BaseSocketChannel<Socket>
eventLoop: SelectableEventLoop,
recvAllocator: RecvByteBufferAllocator) throws {
self.pendingWrites = PendingStreamWritesManager(iovecs: eventLoop.iovecs, storageRefs: eventLoop.storageRefs)
self.connectTimeoutScheduled = nil
try super.init(socket: socket, parent: parent, eventLoop: eventLoop, recvAllocator: recvAllocator)
}

View File

@ -12,6 +12,9 @@
//
//===----------------------------------------------------------------------===//
/// The type of all `channelInitializer` callbacks.
internal typealias ChannelInitializerCallback = (Channel) -> EventLoopFuture<Void>
/// A `ServerBootstrap` is an easy way to bootstrap a `ServerSocketChannel` when creating network servers.
///
/// Example:
@ -56,8 +59,8 @@ public final class ServerBootstrap {
private let group: EventLoopGroup
private let childGroup: EventLoopGroup
private var serverChannelInit: ((Channel) -> EventLoopFuture<Void>)?
private var childChannelInit: ((Channel) -> EventLoopFuture<Void>)?
private var serverChannelInit: Optional<ChannelInitializerCallback>
private var childChannelInit: Optional<ChannelInitializerCallback>
@usableFromInline
internal var _serverChannelOptions: ChannelOptions.Storage
@usableFromInline
@ -81,6 +84,8 @@ public final class ServerBootstrap {
self.childGroup = childGroup
self._serverChannelOptions = ChannelOptions.Storage()
self._childChannelOptions = ChannelOptions.Storage()
self.serverChannelInit = nil
self.childChannelInit = nil
self._serverChannelOptions.append(key: ChannelOptions.socket(IPPROTO_TCP, TCP_NODELAY), value: 1)
}
@ -354,11 +359,11 @@ private extension Channel {
public final class ClientBootstrap {
private let group: EventLoopGroup
private var channelInitializer: ((Channel) -> EventLoopFuture<Void>)?
private var channelInitializer: Optional<ChannelInitializerCallback>
@usableFromInline
internal var _channelOptions: ChannelOptions.Storage
private var connectTimeout: TimeAmount = TimeAmount.seconds(10)
private var resolver: Resolver?
private var resolver: Optional<Resolver>
/// Create a `ClientBootstrap` on the `EventLoopGroup` `group`.
///
@ -368,6 +373,8 @@ public final class ClientBootstrap {
self.group = group
self._channelOptions = ChannelOptions.Storage()
self._channelOptions.append(key: ChannelOptions.socket(IPPROTO_TCP, TCP_NODELAY), value: 1)
self.channelInitializer = nil
self.resolver = nil
}
/// Initialize the connected `SocketChannel` with `initializer`. The most common task in initializer is to add
@ -579,7 +586,7 @@ public final class ClientBootstrap {
public final class DatagramBootstrap {
private let group: EventLoopGroup
private var channelInitializer: ((Channel) -> EventLoopFuture<Void>)?
private var channelInitializer: Optional<ChannelInitializerCallback>
@usableFromInline
internal var _channelOptions: ChannelOptions.Storage
@ -590,6 +597,7 @@ public final class DatagramBootstrap {
public init(group: EventLoopGroup) {
self._channelOptions = ChannelOptions.Storage()
self.group = group
self.channelInitializer = nil
}
/// Initialize the bound `DatagramChannel` with `initializer`. The most common task in initializer is to add
@ -725,7 +733,7 @@ public final class DatagramBootstrap {
///
public final class NIOPipeBootstrap {
private let group: EventLoopGroup
private var channelInitializer: ((Channel) -> EventLoopFuture<Void>)?
private var channelInitializer: Optional<ChannelInitializerCallback>
@usableFromInline
internal var _channelOptions: ChannelOptions.Storage
@ -736,6 +744,7 @@ public final class NIOPipeBootstrap {
public init(group: EventLoopGroup) {
self._channelOptions = ChannelOptions.Storage()
self.group = group
self.channelInitializer = nil
}
/// Initialize the connected `PipeChannel` with `initializer`. The most common task in initializer is to add

View File

@ -22,9 +22,9 @@ public final class AcceptBackoffHandler: ChannelDuplexHandler, RemovableChannelH
public typealias InboundIn = Channel
public typealias OutboundIn = Channel
private var nextReadDeadlineNS: NIODeadline?
private var nextReadDeadlineNS: Optional<NIODeadline>
private let backoffProvider: (IOError) -> TimeAmount?
private var scheduledRead: Scheduled<Void>?
private var scheduledRead: Optional<Scheduled<Void>>
/// Default implementation used as `backoffProvider` which delays accept by 1 second.
public static func defaultBackoffProvider(error: IOError) -> TimeAmount? {
@ -37,6 +37,8 @@ public final class AcceptBackoffHandler: ChannelDuplexHandler, RemovableChannelH
/// - backoffProvider: returns a `TimeAmount` which will be the amount of time to wait before attempting another `read`.
public init(backoffProvider: @escaping (IOError) -> TimeAmount? = AcceptBackoffHandler.defaultBackoffProvider) {
self.backoffProvider = backoffProvider
self.nextReadDeadlineNS = nil
self.scheduledRead = nil
}
public func read(context: ChannelHandlerContext) {
@ -174,14 +176,17 @@ public final class IdleStateHandler: ChannelDuplexHandler, RemovableChannelHandl
private var reading = false
private var lastReadTime: NIODeadline = .distantPast
private var lastWriteCompleteTime: NIODeadline = .distantPast
private var scheduledReaderTask: Scheduled<Void>?
private var scheduledWriterTask: Scheduled<Void>?
private var scheduledAllTask: Scheduled<Void>?
private var scheduledReaderTask: Optional<Scheduled<Void>>
private var scheduledWriterTask: Optional<Scheduled<Void>>
private var scheduledAllTask: Optional<Scheduled<Void>>
public init(readTimeout: TimeAmount? = nil, writeTimeout: TimeAmount? = nil, allTimeout: TimeAmount? = nil) {
self.readTimeout = readTimeout
self.writeTimeout = writeTimeout
self.allTimeout = allTimeout
self.scheduledAllTask = nil
self.scheduledReaderTask = nil
self.scheduledWriterTask = nil
}
public func handlerAdded(context: ChannelHandlerContext) {

View File

@ -134,8 +134,8 @@
///
/// A `ChannelHandler` can be added or removed at any time because a `ChannelPipeline` is thread safe.
public final class ChannelPipeline: ChannelInvoker {
private var head: ChannelHandlerContext?
private var tail: ChannelHandlerContext?
private var head: Optional<ChannelHandlerContext>
private var tail: Optional<ChannelHandlerContext>
private var idx: Int = 0
internal private(set) var destroyed: Bool = false
@ -146,7 +146,7 @@ public final class ChannelPipeline: ChannelInvoker {
/// The `Channel` that this `ChannelPipeline` belongs to.
///
/// - note: This will be nil after the channel has closed
private var _channel: Channel?
private var _channel: Optional<Channel>
/// The `Channel` that this `ChannelPipeline` belongs to.
internal var channel: Channel {
@ -868,6 +868,8 @@ public final class ChannelPipeline: ChannelInvoker {
public init(channel: Channel) {
self._channel = channel
self.eventLoop = channel.eventLoop
self.head = nil // we need to initialise these to `nil` so we can use `self` in the lines below
self.tail = nil // we need to initialise these to `nil` so we can use `self` in the lines below
self.head = ChannelHandlerContext(name: HeadChannelHandler.name, handler: HeadChannelHandler.sharedInstance, pipeline: self)
self.tail = ChannelHandlerContext(name: TailChannelHandler.name, handler: TailChannelHandler.sharedInstance, pipeline: self)
@ -1057,8 +1059,8 @@ public enum ChannelPipelineError: Error {
/// `ChannelHandler`.
public final class ChannelHandlerContext: ChannelInvoker {
// visible for ChannelPipeline to modify
fileprivate var next: ChannelHandlerContext?
fileprivate var prev: ChannelHandlerContext?
fileprivate var next: Optional<ChannelHandlerContext>
fileprivate var prev: Optional<ChannelHandlerContext>
public let pipeline: ChannelPipeline
@ -1112,6 +1114,8 @@ public final class ChannelHandlerContext: ChannelInvoker {
self.pipeline = pipeline
self.inboundHandler = handler as? _ChannelInboundHandler
self.outboundHandler = handler as? _ChannelOutboundHandler
self.next = nil
self.prev = nil
precondition(self.inboundHandler != nil || self.outboundHandler != nil, "ChannelHandlers need to either be inbound or outbound")
}

View File

@ -156,7 +156,7 @@ class EmbeddedChannelCore: ChannelCore {
var eventLoop: EventLoop
var closePromise: EventLoopPromise<Void>
var error: Error?
var error: Optional<Error>
private let pipeline: ChannelPipeline
@ -164,6 +164,7 @@ class EmbeddedChannelCore: ChannelCore {
closePromise = eventLoop.makePromise()
self.pipeline = pipeline
self.eventLoop = eventLoop
self.error = nil
}
deinit {

View File

@ -58,14 +58,15 @@ public final class RepeatedTask {
private let delay: TimeAmount
private let eventLoop: EventLoop
private let cancellationPromise: EventLoopPromise<Void>?
private var scheduled: Scheduled<EventLoopFuture<Void>>?
private var task: ((RepeatedTask) -> EventLoopFuture<Void>)?
private var scheduled: Optional<Scheduled<EventLoopFuture<Void>>>
private var task: Optional<(RepeatedTask) -> EventLoopFuture<Void>>
internal init(interval: TimeAmount, eventLoop: EventLoop, cancellationPromise: EventLoopPromise<Void>? = nil, task: @escaping (RepeatedTask) -> EventLoopFuture<Void>) {
self.delay = interval
self.eventLoop = eventLoop
self.cancellationPromise = cancellationPromise
self.task = task
self.scheduled = nil
}
internal func begin(in delay: TimeAmount) {

View File

@ -26,9 +26,9 @@ internal struct CallbackList {
@usableFromInline
internal typealias Element = () -> CallbackList
@usableFromInline
internal var firstCallback: Element?
internal var firstCallback: Optional<Element>
@usableFromInline
internal var furtherCallbacks: [Element]?
internal var furtherCallbacks: Optional<[Element]>
@inlinable
internal init() {
@ -361,7 +361,7 @@ public struct EventLoopPromise<Value> {
public final class EventLoopFuture<Value> {
// TODO: Provide a tracing facility. It would be nice to be able to set '.debugTrace = true' on any EventLoopFuture or EventLoopPromise and have every subsequent chained EventLoopFuture report the success result or failure error. That would simplify some debugging scenarios.
@usableFromInline
internal var _value: Result<Value, Error>?
internal var _value: Optional<Result<Value, Error>>
/// The `EventLoop` which is tied to the `EventLoopFuture` and is used to notify all registered callbacks.
public let eventLoop: EventLoop

View File

@ -230,7 +230,7 @@ internal class HappyEyeballsConnector {
/// A reference to the task that will execute after the resolution delay expires, if
/// one is scheduled. This is held to ensure that we can cancel this task if the AAAA
/// response comes in before the resolution delay expires.
private var resolutionTask: Scheduled<Void>?
private var resolutionTask: Optional<Scheduled<Void>>
/// The amount of time to wait for a connection to succeed before beginning a new connection
/// attempt. By default this is 250ms.
@ -239,13 +239,13 @@ internal class HappyEyeballsConnector {
/// A reference to the task that will execute after the connection delay expires, if one
/// is scheduled. This is held to ensure that we can cancel this task if a connection
/// succeeds before the connection delay expires.
private var connectionTask: Scheduled<Void>?
private var connectionTask: Optional<Scheduled<Void>>
/// The amount of time to allow for the overall connection process before timing it out.
private let connectTimeout: TimeAmount
/// A reference to the task that will time us out.
private var timeoutTask: Scheduled<Void>?
private var timeoutTask: Optional<Scheduled<Void>>
/// The promise that will hold the final connected channel.
private let resolutionPromise: EventLoopPromise<Channel>
@ -286,6 +286,9 @@ internal class HappyEyeballsConnector {
self.port = port
self.connectTimeout = connectTimeout
self.channelBuilderCallback = channelBuilderCallback
self.resolutionTask = nil
self.connectionTask = nil
self.timeoutTask = nil
self.state = .idle
self.resolutionPromise = self.loop.makePromise()

View File

@ -15,7 +15,7 @@ import NIOConcurrencyHelpers
private struct PendingDatagramWrite {
var data: ByteBuffer
var promise: EventLoopPromise<Void>?
var promise: Optional<EventLoopPromise<Void>>
let address: SocketAddress
/// A helper function that copies the underlying sockaddr structure into temporary storage,

View File

@ -15,7 +15,7 @@ import NIOConcurrencyHelpers
private struct PendingStreamWrite {
var data: IOData
var promise: EventLoopPromise<Void>?
var promise: Optional<EventLoopPromise<Void>>
}
/// Does the setup required to issue a writev.

View File

@ -326,7 +326,7 @@ final class DatagramChannel: BaseSocketChannel<Socket> {
private let pendingWrites: PendingDatagramWritesManager
/// Support for vector reads, if enabled.
private var vectorReadManager: DatagramVectorReadManager?
private var vectorReadManager: Optional<DatagramVectorReadManager>
// This is `Channel` API so must be thread-safe.
override public var isWritable: Bool {
@ -357,6 +357,7 @@ final class DatagramChannel: BaseSocketChannel<Socket> {
}
init(eventLoop: SelectableEventLoop, protocolFamily: Int32) throws {
self.vectorReadManager = nil
let socket = try Socket(protocolFamily: protocolFamily, type: Posix.SOCK_DGRAM)
do {
try socket.setNonBlocking()
@ -377,6 +378,7 @@ final class DatagramChannel: BaseSocketChannel<Socket> {
}
init(socket: Socket, parent: Channel? = nil, eventLoop: SelectableEventLoop) throws {
self.vectorReadManager = nil
try socket.setNonBlocking()
self.pendingWrites = PendingDatagramWritesManager(msgs: eventLoop.msgs,
iovecs: eventLoop.iovecs,

View File

@ -64,6 +64,9 @@ public final class HTTPServerPipelineHandler: ChannelDuplexHandler, RemovableCha
public typealias OutboundOut = HTTPServerResponsePart
public init() {
self.nextExpectedInboundMessage = nil
self.nextExpectedOutboundMessage = nil
debugOnly {
self.nextExpectedInboundMessage = .head
self.nextExpectedOutboundMessage = .head
@ -172,9 +175,9 @@ public final class HTTPServerPipelineHandler: ChannelDuplexHandler, RemovableCha
private var lifecycleState: LifecycleState = .acceptingEvents
// always `nil` in release builds, never `nil` in debug builds
private var nextExpectedInboundMessage: NextExpectedMessageType?
private var nextExpectedInboundMessage: Optional<NextExpectedMessageType>
// always `nil` in release builds, never `nil` in debug builds
private var nextExpectedOutboundMessage: NextExpectedMessageType?
private var nextExpectedOutboundMessage: Optional<NextExpectedMessageType>
public func channelRead(context: ChannelHandlerContext, data: NIOAny) {
switch self.lifecycleState {

View File

@ -22,10 +22,11 @@ final class WebSocketFrameEncoderBenchmark {
private let runCount: Int
private let dataStrategy: DataStrategy
private let cowStrategy: CoWStrategy
private var maskingKey: WebSocketMaskingKey?
private var frame: WebSocketFrame?
private var maskingKey: Optional<WebSocketMaskingKey>
private var frame: Optional<WebSocketFrame>
init(dataSize: Int, runCount: Int, dataStrategy: DataStrategy, cowStrategy: CoWStrategy, maskingKeyStrategy: MaskingKeyStrategy) {
self.frame = nil
self.channel = EmbeddedChannel()
self.dataSize = dataSize
self.runCount = runCount

View File

@ -95,7 +95,7 @@ private extension Sequence where Element == UInt8 {
/// be done asynchronously, providing more flexibility about how the user configures the
/// pipeline.
public final class SNIHandler: ByteToMessageDecoder {
public var cumulationBuffer: ByteBuffer?
public var cumulationBuffer: Optional<ByteBuffer>
public typealias InboundIn = ByteBuffer
public typealias InboundOut = ByteBuffer
@ -103,6 +103,7 @@ public final class SNIHandler: ByteToMessageDecoder {
private var waitingForUser: Bool
public init(sniCompleteHandler: @escaping (SNIResult) -> EventLoopFuture<Void>) {
self.cumulationBuffer = nil
self.completionHandler = sniCompleteHandler
self.waitingForUser = false
}

View File

@ -306,7 +306,7 @@ extension WebSocketFrame: Equatable {}
extension WebSocketFrame {
fileprivate class _Storage {
var data: ByteBuffer
var extensionData: ByteBuffer?
var extensionData: Optional<ByteBuffer>
fileprivate init(data: ByteBuffer, extensionData: ByteBuffer?) {
self.data = data