mirror of
https://github.com/square/okhttp.git
synced 2026-01-12 10:23:16 +03:00
Split RecordingEventListener in two (#9180)
The first half is EventListenerAdapter, which mechanically converts event function calls and emits event objects. The second half is EventRecorder, which validates that the right events are published. This is in preparation for a follow-up where I intend to dynamically add EventListeners to a Call. Co-authored-by: Jesse Wilson <jwilson@squareup.com>
This commit is contained in:
@@ -72,13 +72,13 @@ import okhttp3.Connection
|
||||
import okhttp3.DelegatingSSLSocket
|
||||
import okhttp3.DelegatingSSLSocketFactory
|
||||
import okhttp3.EventListener
|
||||
import okhttp3.EventRecorder
|
||||
import okhttp3.Gzip
|
||||
import okhttp3.Headers
|
||||
import okhttp3.HttpUrl.Companion.toHttpUrl
|
||||
import okhttp3.OkHttpClient
|
||||
import okhttp3.OkHttpClientTestRule
|
||||
import okhttp3.Protocol
|
||||
import okhttp3.RecordingEventListener
|
||||
import okhttp3.Request
|
||||
import okhttp3.TlsVersion
|
||||
import okhttp3.brotli.Brotli
|
||||
@@ -574,11 +574,15 @@ class OkHttpTest {
|
||||
|
||||
@Test
|
||||
fun testEventListener() {
|
||||
val eventListener = RecordingEventListener()
|
||||
val eventRecorder = EventRecorder()
|
||||
|
||||
enableTls()
|
||||
|
||||
client = client.newBuilder().eventListenerFactory(clientTestRule.wrap(eventListener)).build()
|
||||
client =
|
||||
client
|
||||
.newBuilder()
|
||||
.eventListenerFactory(clientTestRule.wrap(eventRecorder))
|
||||
.build()
|
||||
|
||||
server.enqueue(MockResponse(body = "abc1"))
|
||||
server.enqueue(MockResponse(body = "abc2"))
|
||||
@@ -611,10 +615,10 @@ class OkHttpTest {
|
||||
ConnectionReleased::class,
|
||||
CallEnd::class,
|
||||
),
|
||||
eventListener.recordedEventTypes(),
|
||||
eventRecorder.recordedEventTypes(),
|
||||
)
|
||||
|
||||
eventListener.clearAllEvents()
|
||||
eventRecorder.clearAllEvents()
|
||||
|
||||
client.newCall(request).execute().use { response ->
|
||||
assertEquals(200, response.code)
|
||||
@@ -634,7 +638,7 @@ class OkHttpTest {
|
||||
ConnectionReleased::class,
|
||||
CallEnd::class,
|
||||
),
|
||||
eventListener.recordedEventTypes(),
|
||||
eventRecorder.recordedEventTypes(),
|
||||
)
|
||||
}
|
||||
|
||||
|
||||
@@ -38,9 +38,9 @@ import okhttp3.CallEvent
|
||||
import okhttp3.CallEvent.CacheHit
|
||||
import okhttp3.CallEvent.CacheMiss
|
||||
import okhttp3.Dns
|
||||
import okhttp3.EventRecorder
|
||||
import okhttp3.OkHttpClient
|
||||
import okhttp3.Protocol
|
||||
import okhttp3.RecordingEventListener
|
||||
import okhttp3.testing.PlatformRule
|
||||
import okio.Buffer
|
||||
import okio.ByteString.Companion.decodeHex
|
||||
@@ -62,12 +62,12 @@ class DnsOverHttpsTest {
|
||||
|
||||
private lateinit var dns: Dns
|
||||
private val cacheFs = FakeFileSystem()
|
||||
private val eventListener = RecordingEventListener()
|
||||
private val eventRecorder = EventRecorder()
|
||||
private val bootstrapClient =
|
||||
OkHttpClient
|
||||
.Builder()
|
||||
.protocols(listOf(Protocol.HTTP_2, Protocol.HTTP_1_1))
|
||||
.eventListener(eventListener)
|
||||
.eventListener(eventRecorder.eventListener)
|
||||
.build()
|
||||
|
||||
@BeforeEach
|
||||
@@ -309,10 +309,10 @@ class DnsOverHttpsTest {
|
||||
}
|
||||
|
||||
private fun cacheEvents(): List<KClass<out CallEvent>> =
|
||||
eventListener
|
||||
eventRecorder
|
||||
.recordedEventTypes()
|
||||
.filter { "Cache" in it.simpleName!! }
|
||||
.also { eventListener.clearAllEvents() }
|
||||
.also { eventRecorder.clearAllEvents() }
|
||||
|
||||
private fun dnsResponse(s: String): MockResponse =
|
||||
MockResponse
|
||||
|
||||
@@ -39,9 +39,9 @@ import okhttp3.CallEvent.ResponseBodyEnd
|
||||
import okhttp3.CallEvent.ResponseBodyStart
|
||||
import okhttp3.CallEvent.ResponseHeadersEnd
|
||||
import okhttp3.CallEvent.ResponseHeadersStart
|
||||
import okhttp3.EventRecorder
|
||||
import okhttp3.Headers
|
||||
import okhttp3.OkHttpClientTestRule
|
||||
import okhttp3.RecordingEventListener
|
||||
import okhttp3.Request
|
||||
import okhttp3.sse.EventSource
|
||||
import okhttp3.sse.EventSources.createFactory
|
||||
@@ -62,12 +62,12 @@ class EventSourceHttpTest {
|
||||
|
||||
@RegisterExtension
|
||||
val clientTestRule = OkHttpClientTestRule()
|
||||
private val eventListener = RecordingEventListener()
|
||||
private val eventRecorder = EventRecorder()
|
||||
private val listener = EventSourceRecorder()
|
||||
private var client =
|
||||
clientTestRule
|
||||
.newClientBuilder()
|
||||
.eventListenerFactory(clientTestRule.wrap(eventListener))
|
||||
.eventListenerFactory(clientTestRule.wrap(eventRecorder))
|
||||
.build()
|
||||
|
||||
@AfterEach
|
||||
@@ -256,7 +256,7 @@ class EventSourceHttpTest {
|
||||
listener.assertOpen()
|
||||
listener.assertEvent(null, null, "hey")
|
||||
listener.assertClose()
|
||||
assertThat(eventListener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
|
||||
@@ -0,0 +1,255 @@
|
||||
/*
|
||||
* Copyright (C) 2025 Square, Inc.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package okhttp3
|
||||
|
||||
import java.io.IOException
|
||||
import java.net.InetAddress
|
||||
import java.net.InetSocketAddress
|
||||
import java.net.Proxy
|
||||
import okhttp3.CallEvent.CacheConditionalHit
|
||||
import okhttp3.CallEvent.CacheHit
|
||||
import okhttp3.CallEvent.CacheMiss
|
||||
import okhttp3.CallEvent.CallEnd
|
||||
import okhttp3.CallEvent.CallFailed
|
||||
import okhttp3.CallEvent.CallStart
|
||||
import okhttp3.CallEvent.Canceled
|
||||
import okhttp3.CallEvent.ConnectEnd
|
||||
import okhttp3.CallEvent.ConnectFailed
|
||||
import okhttp3.CallEvent.ConnectStart
|
||||
import okhttp3.CallEvent.ConnectionAcquired
|
||||
import okhttp3.CallEvent.ConnectionReleased
|
||||
import okhttp3.CallEvent.DispatcherQueueEnd
|
||||
import okhttp3.CallEvent.DispatcherQueueStart
|
||||
import okhttp3.CallEvent.DnsEnd
|
||||
import okhttp3.CallEvent.DnsStart
|
||||
import okhttp3.CallEvent.FollowUpDecision
|
||||
import okhttp3.CallEvent.ProxySelectEnd
|
||||
import okhttp3.CallEvent.ProxySelectStart
|
||||
import okhttp3.CallEvent.RequestBodyEnd
|
||||
import okhttp3.CallEvent.RequestBodyStart
|
||||
import okhttp3.CallEvent.RequestFailed
|
||||
import okhttp3.CallEvent.RequestHeadersEnd
|
||||
import okhttp3.CallEvent.RequestHeadersStart
|
||||
import okhttp3.CallEvent.ResponseBodyEnd
|
||||
import okhttp3.CallEvent.ResponseBodyStart
|
||||
import okhttp3.CallEvent.ResponseFailed
|
||||
import okhttp3.CallEvent.ResponseHeadersEnd
|
||||
import okhttp3.CallEvent.ResponseHeadersStart
|
||||
import okhttp3.CallEvent.RetryDecision
|
||||
import okhttp3.CallEvent.SatisfactionFailure
|
||||
import okhttp3.CallEvent.SecureConnectEnd
|
||||
import okhttp3.CallEvent.SecureConnectStart
|
||||
|
||||
/**
|
||||
* This accepts events as function calls on [EventListener], and publishes them as subtypes of
|
||||
* [CallEvent].
|
||||
*/
|
||||
class EventListenerAdapter : EventListener() {
|
||||
var listeners = listOf<(CallEvent) -> Unit>()
|
||||
|
||||
private fun onEvent(listener: CallEvent) {
|
||||
for (function in listeners) {
|
||||
function(listener)
|
||||
}
|
||||
}
|
||||
|
||||
override fun dispatcherQueueStart(
|
||||
call: Call,
|
||||
dispatcher: Dispatcher,
|
||||
) = onEvent(DispatcherQueueStart(System.nanoTime(), call, dispatcher))
|
||||
|
||||
override fun dispatcherQueueEnd(
|
||||
call: Call,
|
||||
dispatcher: Dispatcher,
|
||||
) = onEvent(DispatcherQueueEnd(System.nanoTime(), call, dispatcher))
|
||||
|
||||
override fun proxySelectStart(
|
||||
call: Call,
|
||||
url: HttpUrl,
|
||||
) = onEvent(ProxySelectStart(System.nanoTime(), call, url))
|
||||
|
||||
override fun proxySelectEnd(
|
||||
call: Call,
|
||||
url: HttpUrl,
|
||||
proxies: List<Proxy>,
|
||||
) = onEvent(ProxySelectEnd(System.nanoTime(), call, url, proxies))
|
||||
|
||||
override fun dnsStart(
|
||||
call: Call,
|
||||
domainName: String,
|
||||
) = onEvent(DnsStart(System.nanoTime(), call, domainName))
|
||||
|
||||
override fun dnsEnd(
|
||||
call: Call,
|
||||
domainName: String,
|
||||
inetAddressList: List<InetAddress>,
|
||||
) = onEvent(DnsEnd(System.nanoTime(), call, domainName, inetAddressList))
|
||||
|
||||
override fun connectStart(
|
||||
call: Call,
|
||||
inetSocketAddress: InetSocketAddress,
|
||||
proxy: Proxy,
|
||||
) = onEvent(ConnectStart(System.nanoTime(), call, inetSocketAddress, proxy))
|
||||
|
||||
override fun secureConnectStart(call: Call) =
|
||||
onEvent(
|
||||
SecureConnectStart(
|
||||
System.nanoTime(),
|
||||
call,
|
||||
),
|
||||
)
|
||||
|
||||
override fun secureConnectEnd(
|
||||
call: Call,
|
||||
handshake: Handshake?,
|
||||
) = onEvent(SecureConnectEnd(System.nanoTime(), call, handshake))
|
||||
|
||||
override fun connectEnd(
|
||||
call: Call,
|
||||
inetSocketAddress: InetSocketAddress,
|
||||
proxy: Proxy,
|
||||
protocol: Protocol?,
|
||||
) = onEvent(ConnectEnd(System.nanoTime(), call, inetSocketAddress, proxy, protocol))
|
||||
|
||||
override fun connectFailed(
|
||||
call: Call,
|
||||
inetSocketAddress: InetSocketAddress,
|
||||
proxy: Proxy,
|
||||
protocol: Protocol?,
|
||||
ioe: IOException,
|
||||
) = onEvent(
|
||||
ConnectFailed(
|
||||
System.nanoTime(),
|
||||
call,
|
||||
inetSocketAddress,
|
||||
proxy,
|
||||
protocol,
|
||||
ioe,
|
||||
),
|
||||
)
|
||||
|
||||
override fun connectionAcquired(
|
||||
call: Call,
|
||||
connection: Connection,
|
||||
) = onEvent(ConnectionAcquired(System.nanoTime(), call, connection))
|
||||
|
||||
override fun connectionReleased(
|
||||
call: Call,
|
||||
connection: Connection,
|
||||
) = onEvent(ConnectionReleased(System.nanoTime(), call, connection))
|
||||
|
||||
override fun callStart(call: Call) = onEvent(CallStart(System.nanoTime(), call))
|
||||
|
||||
override fun requestHeadersStart(call: Call) =
|
||||
onEvent(
|
||||
RequestHeadersStart(
|
||||
System.nanoTime(),
|
||||
call,
|
||||
),
|
||||
)
|
||||
|
||||
override fun requestHeadersEnd(
|
||||
call: Call,
|
||||
request: Request,
|
||||
) = onEvent(RequestHeadersEnd(System.nanoTime(), call, request.headers.byteCount()))
|
||||
|
||||
override fun requestBodyStart(call: Call) =
|
||||
onEvent(
|
||||
RequestBodyStart(
|
||||
System.nanoTime(),
|
||||
call,
|
||||
),
|
||||
)
|
||||
|
||||
override fun requestBodyEnd(
|
||||
call: Call,
|
||||
byteCount: Long,
|
||||
) = onEvent(RequestBodyEnd(System.nanoTime(), call, byteCount))
|
||||
|
||||
override fun requestFailed(
|
||||
call: Call,
|
||||
ioe: IOException,
|
||||
) = onEvent(RequestFailed(System.nanoTime(), call, ioe))
|
||||
|
||||
override fun responseHeadersStart(call: Call) =
|
||||
onEvent(
|
||||
ResponseHeadersStart(
|
||||
System.nanoTime(),
|
||||
call,
|
||||
),
|
||||
)
|
||||
|
||||
override fun responseHeadersEnd(
|
||||
call: Call,
|
||||
response: Response,
|
||||
) = onEvent(ResponseHeadersEnd(System.nanoTime(), call, response.headers.byteCount()))
|
||||
|
||||
override fun responseBodyStart(call: Call) =
|
||||
onEvent(
|
||||
ResponseBodyStart(
|
||||
System.nanoTime(),
|
||||
call,
|
||||
),
|
||||
)
|
||||
|
||||
override fun responseBodyEnd(
|
||||
call: Call,
|
||||
byteCount: Long,
|
||||
) = onEvent(ResponseBodyEnd(System.nanoTime(), call, byteCount))
|
||||
|
||||
override fun responseFailed(
|
||||
call: Call,
|
||||
ioe: IOException,
|
||||
) = onEvent(ResponseFailed(System.nanoTime(), call, ioe))
|
||||
|
||||
override fun callEnd(call: Call) = onEvent(CallEnd(System.nanoTime(), call))
|
||||
|
||||
override fun callFailed(
|
||||
call: Call,
|
||||
ioe: IOException,
|
||||
) = onEvent(CallFailed(System.nanoTime(), call, ioe))
|
||||
|
||||
override fun canceled(call: Call) = onEvent(Canceled(System.nanoTime(), call))
|
||||
|
||||
override fun satisfactionFailure(
|
||||
call: Call,
|
||||
response: Response,
|
||||
) = onEvent(SatisfactionFailure(System.nanoTime(), call))
|
||||
|
||||
override fun cacheMiss(call: Call) = onEvent(CacheMiss(System.nanoTime(), call))
|
||||
|
||||
override fun cacheHit(
|
||||
call: Call,
|
||||
response: Response,
|
||||
) = onEvent(CacheHit(System.nanoTime(), call))
|
||||
|
||||
override fun cacheConditionalHit(
|
||||
call: Call,
|
||||
cachedResponse: Response,
|
||||
) = onEvent(CacheConditionalHit(System.nanoTime(), call))
|
||||
|
||||
override fun retryDecision(
|
||||
call: Call,
|
||||
exception: IOException,
|
||||
retry: Boolean,
|
||||
) = onEvent(RetryDecision(System.nanoTime(), call, exception, retry))
|
||||
|
||||
override fun followUpDecision(
|
||||
call: Call,
|
||||
networkResponse: Response,
|
||||
nextRequest: Request?,
|
||||
) = onEvent(FollowUpDecision(System.nanoTime(), call, networkResponse, nextRequest))
|
||||
}
|
||||
150
okhttp-testing-support/src/main/kotlin/okhttp3/EventRecorder.kt
Normal file
150
okhttp-testing-support/src/main/kotlin/okhttp3/EventRecorder.kt
Normal file
@@ -0,0 +1,150 @@
|
||||
/*
|
||||
* Copyright (C) 2017 Square, Inc.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package okhttp3
|
||||
|
||||
import assertk.assertThat
|
||||
import assertk.assertions.isCloseTo
|
||||
import assertk.assertions.isFalse
|
||||
import assertk.assertions.isInstanceOf
|
||||
import assertk.assertions.matchesPredicate
|
||||
import java.util.Deque
|
||||
import java.util.concurrent.ConcurrentLinkedDeque
|
||||
import java.util.concurrent.TimeUnit
|
||||
import okhttp3.CallEvent.CallStart
|
||||
import okhttp3.CallEvent.Canceled
|
||||
import org.junit.jupiter.api.Assertions.fail
|
||||
|
||||
open class EventRecorder(
|
||||
/**
|
||||
* An override to ignore the normal order that is enforced.
|
||||
* EventListeners added by Interceptors will not see all events.
|
||||
*/
|
||||
private val enforceOrder: Boolean = true,
|
||||
) {
|
||||
private val eventListenerAdapter =
|
||||
EventListenerAdapter()
|
||||
.apply {
|
||||
listeners += ::logEvent
|
||||
}
|
||||
|
||||
val eventListener: EventListener
|
||||
get() = eventListenerAdapter
|
||||
|
||||
/** Events that haven't yet been removed. */
|
||||
val eventSequence: Deque<CallEvent> = ConcurrentLinkedDeque()
|
||||
|
||||
/** The full set of events, used to match starts with ends. */
|
||||
private val eventsForMatching = ConcurrentLinkedDeque<CallEvent>()
|
||||
|
||||
private val forbiddenLocks = mutableListOf<Any>()
|
||||
|
||||
/** The timestamp of the last taken event, used to measure elapsed time between events. */
|
||||
private var lastTimestampNs: Long? = null
|
||||
|
||||
/** Confirm that the thread does not hold a lock on `lock` during the callback. */
|
||||
fun forbidLock(lock: Any) {
|
||||
forbiddenLocks.add(lock)
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes recorded events up to (and including) an event is found whose class equals [eventClass]
|
||||
* and returns it.
|
||||
*/
|
||||
fun <T : CallEvent> removeUpToEvent(eventClass: Class<T>): T {
|
||||
val fullEventSequence = eventSequence.toList()
|
||||
try {
|
||||
while (true) {
|
||||
val event = takeEvent()
|
||||
if (eventClass.isInstance(event)) {
|
||||
return eventClass.cast(event)
|
||||
}
|
||||
}
|
||||
} catch (e: NoSuchElementException) {
|
||||
throw AssertionError("full event sequence: $fullEventSequence", e)
|
||||
}
|
||||
}
|
||||
|
||||
inline fun <reified T : CallEvent> removeUpToEvent(): T = removeUpToEvent(T::class.java)
|
||||
|
||||
inline fun <reified T : CallEvent> findEvent(): T = eventSequence.first { it is T } as T
|
||||
|
||||
/**
|
||||
* Remove and return the next event from the recorded sequence.
|
||||
*
|
||||
* @param eventClass a class to assert that the returned event is an instance of, or null to
|
||||
* take any event class.
|
||||
* @param elapsedMs the time in milliseconds elapsed since the immediately-preceding event, or
|
||||
* -1L to take any duration.
|
||||
*/
|
||||
fun takeEvent(
|
||||
eventClass: Class<out CallEvent>? = null,
|
||||
elapsedMs: Long = -1L,
|
||||
): CallEvent {
|
||||
val result = eventSequence.remove()
|
||||
val actualElapsedNs = result.timestampNs - (lastTimestampNs ?: result.timestampNs)
|
||||
lastTimestampNs = result.timestampNs
|
||||
|
||||
if (eventClass != null) {
|
||||
assertThat(result).isInstanceOf(eventClass)
|
||||
}
|
||||
|
||||
if (elapsedMs != -1L) {
|
||||
assertThat(
|
||||
TimeUnit.NANOSECONDS
|
||||
.toMillis(actualElapsedNs)
|
||||
.toDouble(),
|
||||
).isCloseTo(elapsedMs.toDouble(), 100.0)
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
fun recordedEventTypes() = eventSequence.map { it::class }
|
||||
|
||||
fun clearAllEvents() {
|
||||
while (eventSequence.isNotEmpty()) {
|
||||
takeEvent()
|
||||
}
|
||||
}
|
||||
|
||||
private fun logEvent(e: CallEvent) {
|
||||
for (lock in forbiddenLocks) {
|
||||
assertThat(Thread.holdsLock(lock), lock.toString()).isFalse()
|
||||
}
|
||||
|
||||
if (enforceOrder) {
|
||||
checkForStartEvent(e)
|
||||
}
|
||||
|
||||
eventsForMatching.offer(e)
|
||||
eventSequence.offer(e)
|
||||
}
|
||||
|
||||
private fun checkForStartEvent(e: CallEvent) {
|
||||
if (eventsForMatching.isEmpty()) {
|
||||
assertThat(e).matchesPredicate { it is CallStart || it is Canceled }
|
||||
} else {
|
||||
eventsForMatching.forEach loop@{
|
||||
when (e.closes(it)) {
|
||||
null -> return // no open event
|
||||
true -> return // found open event
|
||||
false -> return@loop // this is not the open event so continue
|
||||
}
|
||||
}
|
||||
fail<Any>("event $e without matching start event")
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -137,6 +137,8 @@ class OkHttpClientTestRule :
|
||||
ClientRuleEventListener(::addEvent) + eventListener
|
||||
}
|
||||
|
||||
fun wrap(eventRecorder: EventRecorder) = wrap(eventRecorder.eventListener)
|
||||
|
||||
fun wrap(eventListenerFactory: EventListener.Factory) =
|
||||
EventListener.Factory { call ->
|
||||
ClientRuleEventListener(::addEvent) + eventListenerFactory.create(call)
|
||||
|
||||
@@ -1,324 +0,0 @@
|
||||
/*
|
||||
* Copyright (C) 2017 Square, Inc.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package okhttp3
|
||||
|
||||
import assertk.assertThat
|
||||
import assertk.assertions.isCloseTo
|
||||
import assertk.assertions.isFalse
|
||||
import assertk.assertions.isInstanceOf
|
||||
import assertk.assertions.matchesPredicate
|
||||
import java.io.IOException
|
||||
import java.net.InetAddress
|
||||
import java.net.InetSocketAddress
|
||||
import java.net.Proxy
|
||||
import java.util.Deque
|
||||
import java.util.concurrent.ConcurrentLinkedDeque
|
||||
import java.util.concurrent.TimeUnit
|
||||
import okhttp3.CallEvent.CacheConditionalHit
|
||||
import okhttp3.CallEvent.CacheHit
|
||||
import okhttp3.CallEvent.CacheMiss
|
||||
import okhttp3.CallEvent.CallEnd
|
||||
import okhttp3.CallEvent.CallFailed
|
||||
import okhttp3.CallEvent.CallStart
|
||||
import okhttp3.CallEvent.Canceled
|
||||
import okhttp3.CallEvent.ConnectEnd
|
||||
import okhttp3.CallEvent.ConnectFailed
|
||||
import okhttp3.CallEvent.ConnectStart
|
||||
import okhttp3.CallEvent.ConnectionAcquired
|
||||
import okhttp3.CallEvent.ConnectionReleased
|
||||
import okhttp3.CallEvent.DispatcherQueueEnd
|
||||
import okhttp3.CallEvent.DispatcherQueueStart
|
||||
import okhttp3.CallEvent.DnsEnd
|
||||
import okhttp3.CallEvent.DnsStart
|
||||
import okhttp3.CallEvent.FollowUpDecision
|
||||
import okhttp3.CallEvent.ProxySelectEnd
|
||||
import okhttp3.CallEvent.ProxySelectStart
|
||||
import okhttp3.CallEvent.RequestBodyEnd
|
||||
import okhttp3.CallEvent.RequestBodyStart
|
||||
import okhttp3.CallEvent.RequestFailed
|
||||
import okhttp3.CallEvent.RequestHeadersEnd
|
||||
import okhttp3.CallEvent.RequestHeadersStart
|
||||
import okhttp3.CallEvent.ResponseBodyEnd
|
||||
import okhttp3.CallEvent.ResponseBodyStart
|
||||
import okhttp3.CallEvent.ResponseFailed
|
||||
import okhttp3.CallEvent.ResponseHeadersEnd
|
||||
import okhttp3.CallEvent.ResponseHeadersStart
|
||||
import okhttp3.CallEvent.RetryDecision
|
||||
import okhttp3.CallEvent.SatisfactionFailure
|
||||
import okhttp3.CallEvent.SecureConnectEnd
|
||||
import okhttp3.CallEvent.SecureConnectStart
|
||||
import org.junit.jupiter.api.Assertions.fail
|
||||
|
||||
open class RecordingEventListener(
|
||||
/**
|
||||
* An override to ignore the normal order that is enforced.
|
||||
* EventListeners added by Interceptors will not see all events.
|
||||
*/
|
||||
private val enforceOrder: Boolean = true,
|
||||
) : EventListener() {
|
||||
/** Events that haven't yet been removed. */
|
||||
val eventSequence: Deque<CallEvent> = ConcurrentLinkedDeque()
|
||||
|
||||
/** The full set of events, used to match starts with ends. */
|
||||
private val eventsForMatching = ConcurrentLinkedDeque<CallEvent>()
|
||||
|
||||
private val forbiddenLocks = mutableListOf<Any>()
|
||||
|
||||
/** The timestamp of the last taken event, used to measure elapsed time between events. */
|
||||
private var lastTimestampNs: Long? = null
|
||||
|
||||
/** Confirm that the thread does not hold a lock on `lock` during the callback. */
|
||||
fun forbidLock(lock: Any) {
|
||||
forbiddenLocks.add(lock)
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes recorded events up to (and including) an event is found whose class equals [eventClass]
|
||||
* and returns it.
|
||||
*/
|
||||
fun <T : CallEvent> removeUpToEvent(eventClass: Class<T>): T {
|
||||
val fullEventSequence = eventSequence.toList()
|
||||
try {
|
||||
while (true) {
|
||||
val event = takeEvent()
|
||||
if (eventClass.isInstance(event)) {
|
||||
return eventClass.cast(event)
|
||||
}
|
||||
}
|
||||
} catch (e: NoSuchElementException) {
|
||||
throw AssertionError("full event sequence: $fullEventSequence", e)
|
||||
}
|
||||
}
|
||||
|
||||
inline fun <reified T : CallEvent> removeUpToEvent(): T = removeUpToEvent(T::class.java)
|
||||
|
||||
inline fun <reified T : CallEvent> findEvent(): T = eventSequence.first { it is T } as T
|
||||
|
||||
/**
|
||||
* Remove and return the next event from the recorded sequence.
|
||||
*
|
||||
* @param eventClass a class to assert that the returned event is an instance of, or null to
|
||||
* take any event class.
|
||||
* @param elapsedMs the time in milliseconds elapsed since the immediately-preceding event, or
|
||||
* -1L to take any duration.
|
||||
*/
|
||||
fun takeEvent(
|
||||
eventClass: Class<out CallEvent>? = null,
|
||||
elapsedMs: Long = -1L,
|
||||
): CallEvent {
|
||||
val result = eventSequence.remove()
|
||||
val actualElapsedNs = result.timestampNs - (lastTimestampNs ?: result.timestampNs)
|
||||
lastTimestampNs = result.timestampNs
|
||||
|
||||
if (eventClass != null) {
|
||||
assertThat(result).isInstanceOf(eventClass)
|
||||
}
|
||||
|
||||
if (elapsedMs != -1L) {
|
||||
assertThat(
|
||||
TimeUnit.NANOSECONDS
|
||||
.toMillis(actualElapsedNs)
|
||||
.toDouble(),
|
||||
).isCloseTo(elapsedMs.toDouble(), 100.0)
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
fun recordedEventTypes() = eventSequence.map { it::class }
|
||||
|
||||
fun clearAllEvents() {
|
||||
while (eventSequence.isNotEmpty()) {
|
||||
takeEvent()
|
||||
}
|
||||
}
|
||||
|
||||
private fun logEvent(e: CallEvent) {
|
||||
for (lock in forbiddenLocks) {
|
||||
assertThat(Thread.holdsLock(lock), lock.toString()).isFalse()
|
||||
}
|
||||
|
||||
if (enforceOrder) {
|
||||
checkForStartEvent(e)
|
||||
}
|
||||
|
||||
eventsForMatching.offer(e)
|
||||
eventSequence.offer(e)
|
||||
}
|
||||
|
||||
private fun checkForStartEvent(e: CallEvent) {
|
||||
if (eventsForMatching.isEmpty()) {
|
||||
assertThat(e).matchesPredicate { it is CallStart || it is Canceled }
|
||||
} else {
|
||||
eventsForMatching.forEach loop@{
|
||||
when (e.closes(it)) {
|
||||
null -> return // no open event
|
||||
true -> return // found open event
|
||||
false -> return@loop // this is not the open event so continue
|
||||
}
|
||||
}
|
||||
fail<Any>("event $e without matching start event")
|
||||
}
|
||||
}
|
||||
|
||||
override fun dispatcherQueueStart(
|
||||
call: Call,
|
||||
dispatcher: Dispatcher,
|
||||
) = logEvent(DispatcherQueueStart(System.nanoTime(), call, dispatcher))
|
||||
|
||||
override fun dispatcherQueueEnd(
|
||||
call: Call,
|
||||
dispatcher: Dispatcher,
|
||||
) = logEvent(DispatcherQueueEnd(System.nanoTime(), call, dispatcher))
|
||||
|
||||
override fun proxySelectStart(
|
||||
call: Call,
|
||||
url: HttpUrl,
|
||||
) = logEvent(ProxySelectStart(System.nanoTime(), call, url))
|
||||
|
||||
override fun proxySelectEnd(
|
||||
call: Call,
|
||||
url: HttpUrl,
|
||||
proxies: List<Proxy>,
|
||||
) = logEvent(ProxySelectEnd(System.nanoTime(), call, url, proxies))
|
||||
|
||||
override fun dnsStart(
|
||||
call: Call,
|
||||
domainName: String,
|
||||
) = logEvent(DnsStart(System.nanoTime(), call, domainName))
|
||||
|
||||
override fun dnsEnd(
|
||||
call: Call,
|
||||
domainName: String,
|
||||
inetAddressList: List<InetAddress>,
|
||||
) = logEvent(DnsEnd(System.nanoTime(), call, domainName, inetAddressList))
|
||||
|
||||
override fun connectStart(
|
||||
call: Call,
|
||||
inetSocketAddress: InetSocketAddress,
|
||||
proxy: Proxy,
|
||||
) = logEvent(ConnectStart(System.nanoTime(), call, inetSocketAddress, proxy))
|
||||
|
||||
override fun secureConnectStart(call: Call) = logEvent(SecureConnectStart(System.nanoTime(), call))
|
||||
|
||||
override fun secureConnectEnd(
|
||||
call: Call,
|
||||
handshake: Handshake?,
|
||||
) = logEvent(SecureConnectEnd(System.nanoTime(), call, handshake))
|
||||
|
||||
override fun connectEnd(
|
||||
call: Call,
|
||||
inetSocketAddress: InetSocketAddress,
|
||||
proxy: Proxy,
|
||||
protocol: Protocol?,
|
||||
) = logEvent(ConnectEnd(System.nanoTime(), call, inetSocketAddress, proxy, protocol))
|
||||
|
||||
override fun connectFailed(
|
||||
call: Call,
|
||||
inetSocketAddress: InetSocketAddress,
|
||||
proxy: Proxy,
|
||||
protocol: Protocol?,
|
||||
ioe: IOException,
|
||||
) = logEvent(ConnectFailed(System.nanoTime(), call, inetSocketAddress, proxy, protocol, ioe))
|
||||
|
||||
override fun connectionAcquired(
|
||||
call: Call,
|
||||
connection: Connection,
|
||||
) = logEvent(ConnectionAcquired(System.nanoTime(), call, connection))
|
||||
|
||||
override fun connectionReleased(
|
||||
call: Call,
|
||||
connection: Connection,
|
||||
) = logEvent(ConnectionReleased(System.nanoTime(), call, connection))
|
||||
|
||||
override fun callStart(call: Call) = logEvent(CallStart(System.nanoTime(), call))
|
||||
|
||||
override fun requestHeadersStart(call: Call) = logEvent(RequestHeadersStart(System.nanoTime(), call))
|
||||
|
||||
override fun requestHeadersEnd(
|
||||
call: Call,
|
||||
request: Request,
|
||||
) = logEvent(RequestHeadersEnd(System.nanoTime(), call, request.headers.byteCount()))
|
||||
|
||||
override fun requestBodyStart(call: Call) = logEvent(RequestBodyStart(System.nanoTime(), call))
|
||||
|
||||
override fun requestBodyEnd(
|
||||
call: Call,
|
||||
byteCount: Long,
|
||||
) = logEvent(RequestBodyEnd(System.nanoTime(), call, byteCount))
|
||||
|
||||
override fun requestFailed(
|
||||
call: Call,
|
||||
ioe: IOException,
|
||||
) = logEvent(RequestFailed(System.nanoTime(), call, ioe))
|
||||
|
||||
override fun responseHeadersStart(call: Call) = logEvent(ResponseHeadersStart(System.nanoTime(), call))
|
||||
|
||||
override fun responseHeadersEnd(
|
||||
call: Call,
|
||||
response: Response,
|
||||
) = logEvent(ResponseHeadersEnd(System.nanoTime(), call, response.headers.byteCount()))
|
||||
|
||||
override fun responseBodyStart(call: Call) = logEvent(ResponseBodyStart(System.nanoTime(), call))
|
||||
|
||||
override fun responseBodyEnd(
|
||||
call: Call,
|
||||
byteCount: Long,
|
||||
) = logEvent(ResponseBodyEnd(System.nanoTime(), call, byteCount))
|
||||
|
||||
override fun responseFailed(
|
||||
call: Call,
|
||||
ioe: IOException,
|
||||
) = logEvent(ResponseFailed(System.nanoTime(), call, ioe))
|
||||
|
||||
override fun callEnd(call: Call) = logEvent(CallEnd(System.nanoTime(), call))
|
||||
|
||||
override fun callFailed(
|
||||
call: Call,
|
||||
ioe: IOException,
|
||||
) = logEvent(CallFailed(System.nanoTime(), call, ioe))
|
||||
|
||||
override fun canceled(call: Call) = logEvent(Canceled(System.nanoTime(), call))
|
||||
|
||||
override fun satisfactionFailure(
|
||||
call: Call,
|
||||
response: Response,
|
||||
) = logEvent(SatisfactionFailure(System.nanoTime(), call))
|
||||
|
||||
override fun cacheMiss(call: Call) = logEvent(CacheMiss(System.nanoTime(), call))
|
||||
|
||||
override fun cacheHit(
|
||||
call: Call,
|
||||
response: Response,
|
||||
) = logEvent(CacheHit(System.nanoTime(), call))
|
||||
|
||||
override fun cacheConditionalHit(
|
||||
call: Call,
|
||||
cachedResponse: Response,
|
||||
) = logEvent(CacheConditionalHit(System.nanoTime(), call))
|
||||
|
||||
override fun retryDecision(
|
||||
call: Call,
|
||||
exception: IOException,
|
||||
retry: Boolean,
|
||||
) = logEvent(RetryDecision(System.nanoTime(), call, exception, retry))
|
||||
|
||||
override fun followUpDecision(
|
||||
call: Call,
|
||||
networkResponse: Response,
|
||||
nextRequest: Request?,
|
||||
) = logEvent(FollowUpDecision(System.nanoTime(), call, networkResponse, nextRequest))
|
||||
}
|
||||
@@ -159,12 +159,12 @@ open class CallTest {
|
||||
@StartStop
|
||||
private val server2 = MockWebServer()
|
||||
|
||||
private var listener = RecordingEventListener()
|
||||
private var eventRecorder = EventRecorder()
|
||||
private val handshakeCertificates = platform.localhostHandshakeCertificates()
|
||||
private var client =
|
||||
clientTestRule
|
||||
.newClientBuilder()
|
||||
.eventListenerFactory(clientTestRule.wrap(listener))
|
||||
.eventListenerFactory(clientTestRule.wrap(eventRecorder))
|
||||
.build()
|
||||
private val callback = RecordingCallback()
|
||||
private val cache =
|
||||
@@ -1277,36 +1277,36 @@ open class CallTest {
|
||||
dispatcher.enqueue(MockResponse.Builder().onResponseStart(CloseSocket()).build())
|
||||
dispatcher.enqueue(MockResponse(body = "retry success"))
|
||||
server.dispatcher = dispatcher
|
||||
listener =
|
||||
object : RecordingEventListener() {
|
||||
val requestFinishedListener =
|
||||
object : EventListener() {
|
||||
override fun requestHeadersEnd(
|
||||
call: Call,
|
||||
request: Request,
|
||||
) {
|
||||
requestFinished.countDown()
|
||||
super.responseHeadersStart(call)
|
||||
}
|
||||
}
|
||||
client =
|
||||
client
|
||||
.newBuilder()
|
||||
.dns(DoubleInetAddressDns())
|
||||
.eventListenerFactory(clientTestRule.wrap(listener))
|
||||
.build()
|
||||
.eventListenerFactory(
|
||||
clientTestRule.wrap(eventRecorder.eventListener + requestFinishedListener),
|
||||
).build()
|
||||
assertThat(client.retryOnConnectionFailure).isTrue()
|
||||
executeSynchronously("/").assertBody("seed connection pool")
|
||||
executeSynchronously("/").assertBody("retry success")
|
||||
|
||||
// The call that seeds the connection pool.
|
||||
listener.removeUpToEvent(CallEnd::class.java)
|
||||
eventRecorder.removeUpToEvent(CallEnd::class.java)
|
||||
|
||||
// The ResponseFailed event is not necessarily fatal!
|
||||
listener.removeUpToEvent(ConnectionAcquired::class.java)
|
||||
listener.removeUpToEvent(ResponseFailed::class.java)
|
||||
listener.removeUpToEvent(ConnectionReleased::class.java)
|
||||
listener.removeUpToEvent(ConnectionAcquired::class.java)
|
||||
listener.removeUpToEvent(ConnectionReleased::class.java)
|
||||
listener.removeUpToEvent(CallEnd::class.java)
|
||||
eventRecorder.removeUpToEvent(ConnectionAcquired::class.java)
|
||||
eventRecorder.removeUpToEvent(ResponseFailed::class.java)
|
||||
eventRecorder.removeUpToEvent(ConnectionReleased::class.java)
|
||||
eventRecorder.removeUpToEvent(ConnectionAcquired::class.java)
|
||||
eventRecorder.removeUpToEvent(ConnectionReleased::class.java)
|
||||
eventRecorder.removeUpToEvent(CallEnd::class.java)
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -1718,7 +1718,7 @@ open class CallTest {
|
||||
val response1 = client.newCall(request1).execute()
|
||||
assertThat(response1.body.string()).isEqualTo("abc")
|
||||
|
||||
listener.clearAllEvents()
|
||||
eventRecorder.clearAllEvents()
|
||||
|
||||
val request2 =
|
||||
Request(
|
||||
@@ -1731,7 +1731,7 @@ open class CallTest {
|
||||
assertThat(expected.message!!).startsWith("unexpected end of stream on http://")
|
||||
}
|
||||
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ConnectionAcquired::class,
|
||||
RequestHeadersStart::class,
|
||||
@@ -1743,15 +1743,15 @@ open class CallTest {
|
||||
ConnectionReleased::class,
|
||||
CallFailed::class,
|
||||
)
|
||||
assertThat(listener.findEvent<RetryDecision>()).all {
|
||||
assertThat(eventRecorder.findEvent<RetryDecision>()).all {
|
||||
prop(RetryDecision::retry).isFalse()
|
||||
}
|
||||
listener.clearAllEvents()
|
||||
eventRecorder.clearAllEvents()
|
||||
|
||||
val response3 = client.newCall(request1).execute()
|
||||
assertThat(response3.body.string()).isEqualTo("abc")
|
||||
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -3648,7 +3648,7 @@ open class CallTest {
|
||||
}
|
||||
if (!platform.isJdk8()) {
|
||||
val connectCount =
|
||||
listener.eventSequence
|
||||
eventRecorder.eventSequence
|
||||
.stream()
|
||||
.filter { event: CallEvent? -> event is ConnectStart }
|
||||
.count()
|
||||
|
||||
@@ -51,20 +51,20 @@ class DispatcherTest {
|
||||
object : WebSocketListener() {
|
||||
}
|
||||
val dispatcher = Dispatcher(executor)
|
||||
val listener = RecordingEventListener()
|
||||
val eventRecorder = EventRecorder()
|
||||
var client =
|
||||
clientTestRule
|
||||
.newClientBuilder()
|
||||
.dns { throw UnknownHostException() }
|
||||
.dispatcher(dispatcher)
|
||||
.eventListenerFactory(clientTestRule.wrap(listener))
|
||||
.eventListenerFactory(clientTestRule.wrap(eventRecorder))
|
||||
.build()
|
||||
|
||||
@BeforeEach
|
||||
fun setUp() {
|
||||
dispatcher.maxRequests = 20
|
||||
dispatcher.maxRequestsPerHost = 10
|
||||
listener.forbidLock(dispatcher)
|
||||
eventRecorder.forbidLock(dispatcher)
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -86,8 +86,8 @@ class DispatcherTest {
|
||||
client.newCall(newRequest("http://a/1")).enqueue(callback)
|
||||
executor.assertJobs("http://a/1")
|
||||
|
||||
assertThat(listener.eventSequence).none { it.isInstanceOf<DispatcherQueueStart>() }
|
||||
assertThat(listener.eventSequence).none { it.isInstanceOf<DispatcherQueueEnd>() }
|
||||
assertThat(eventRecorder.eventSequence).none { it.isInstanceOf<DispatcherQueueStart>() }
|
||||
assertThat(eventRecorder.eventSequence).none { it.isInstanceOf<DispatcherQueueEnd>() }
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -99,9 +99,9 @@ class DispatcherTest {
|
||||
client.newCall(newRequest("http://b/2")).enqueue(callback)
|
||||
executor.assertJobs("http://a/1", "http://a/2", "http://b/1")
|
||||
|
||||
val dispatcherQueueStart = listener.removeUpToEvent<DispatcherQueueStart>()
|
||||
val dispatcherQueueStart = eventRecorder.removeUpToEvent<DispatcherQueueStart>()
|
||||
assertThat(dispatcherQueueStart.call.request().url).isEqualTo("http://b/2".toHttpUrl())
|
||||
assertThat(listener.eventSequence).none { it.isInstanceOf<DispatcherQueueEnd>() }
|
||||
assertThat(eventRecorder.eventSequence).none { it.isInstanceOf<DispatcherQueueEnd>() }
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -112,9 +112,9 @@ class DispatcherTest {
|
||||
client.newCall(newRequest("http://a/3")).enqueue(callback)
|
||||
executor.assertJobs("http://a/1", "http://a/2")
|
||||
|
||||
val dispatcherQueueStart = listener.removeUpToEvent<DispatcherQueueStart>()
|
||||
val dispatcherQueueStart = eventRecorder.removeUpToEvent<DispatcherQueueStart>()
|
||||
assertThat(dispatcherQueueStart.call.request().url).isEqualTo("http://a/3".toHttpUrl())
|
||||
assertThat(listener.eventSequence).none { it.isInstanceOf<DispatcherQueueEnd>() }
|
||||
assertThat(eventRecorder.eventSequence).none { it.isInstanceOf<DispatcherQueueEnd>() }
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -135,22 +135,22 @@ class DispatcherTest {
|
||||
client.newCall(newRequest("http://a/2")).enqueue(callback)
|
||||
client.newCall(newRequest("http://b/2")).enqueue(callback)
|
||||
|
||||
val dispatcherQueueStartC1 = listener.removeUpToEvent<DispatcherQueueStart>()
|
||||
val dispatcherQueueStartC1 = eventRecorder.removeUpToEvent<DispatcherQueueStart>()
|
||||
assertThat(dispatcherQueueStartC1.call.request().url).isEqualTo("http://c/1".toHttpUrl())
|
||||
val dispatcherQueueStartA2 = listener.removeUpToEvent<DispatcherQueueStart>()
|
||||
val dispatcherQueueStartA2 = eventRecorder.removeUpToEvent<DispatcherQueueStart>()
|
||||
assertThat(dispatcherQueueStartA2.call.request().url).isEqualTo("http://a/2".toHttpUrl())
|
||||
val dispatcherQueueStartB2 = listener.removeUpToEvent<DispatcherQueueStart>()
|
||||
val dispatcherQueueStartB2 = eventRecorder.removeUpToEvent<DispatcherQueueStart>()
|
||||
assertThat(dispatcherQueueStartB2.call.request().url).isEqualTo("http://b/2".toHttpUrl())
|
||||
assertThat(listener.eventSequence).none { it.isInstanceOf<DispatcherQueueEnd>() }
|
||||
assertThat(eventRecorder.eventSequence).none { it.isInstanceOf<DispatcherQueueEnd>() }
|
||||
|
||||
dispatcher.maxRequests = 4
|
||||
executor.assertJobs("http://a/1", "http://b/1", "http://c/1", "http://a/2")
|
||||
|
||||
val dispatcherQueueEndC1 = listener.removeUpToEvent<DispatcherQueueEnd>()
|
||||
val dispatcherQueueEndC1 = eventRecorder.removeUpToEvent<DispatcherQueueEnd>()
|
||||
assertThat(dispatcherQueueEndC1.call.request().url).isEqualTo("http://c/1".toHttpUrl())
|
||||
val dispatcherQueueEndA2 = listener.removeUpToEvent<DispatcherQueueEnd>()
|
||||
val dispatcherQueueEndA2 = eventRecorder.removeUpToEvent<DispatcherQueueEnd>()
|
||||
assertThat(dispatcherQueueEndA2.call.request().url).isEqualTo("http://a/2".toHttpUrl())
|
||||
assertThat(listener.eventSequence).none { it.isInstanceOf<DispatcherQueueEnd>() }
|
||||
assertThat(eventRecorder.eventSequence).none { it.isInstanceOf<DispatcherQueueEnd>() }
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -162,22 +162,22 @@ class DispatcherTest {
|
||||
client.newCall(newRequest("http://a/4")).enqueue(callback)
|
||||
client.newCall(newRequest("http://a/5")).enqueue(callback)
|
||||
|
||||
val dispatcherQueueStartA3 = listener.removeUpToEvent<DispatcherQueueStart>()
|
||||
val dispatcherQueueStartA3 = eventRecorder.removeUpToEvent<DispatcherQueueStart>()
|
||||
assertThat(dispatcherQueueStartA3.call.request().url).isEqualTo("http://a/3".toHttpUrl())
|
||||
val dispatcherQueueStartA4 = listener.removeUpToEvent<DispatcherQueueStart>()
|
||||
val dispatcherQueueStartA4 = eventRecorder.removeUpToEvent<DispatcherQueueStart>()
|
||||
assertThat(dispatcherQueueStartA4.call.request().url).isEqualTo("http://a/4".toHttpUrl())
|
||||
val dispatcherQueueStartA5 = listener.removeUpToEvent<DispatcherQueueStart>()
|
||||
val dispatcherQueueStartA5 = eventRecorder.removeUpToEvent<DispatcherQueueStart>()
|
||||
assertThat(dispatcherQueueStartA5.call.request().url).isEqualTo("http://a/5".toHttpUrl())
|
||||
assertThat(listener.eventSequence).none { it.isInstanceOf<DispatcherQueueEnd>() }
|
||||
assertThat(eventRecorder.eventSequence).none { it.isInstanceOf<DispatcherQueueEnd>() }
|
||||
|
||||
dispatcher.maxRequestsPerHost = 4
|
||||
executor.assertJobs("http://a/1", "http://a/2", "http://a/3", "http://a/4")
|
||||
|
||||
val dispatcherQueueEndA3 = listener.removeUpToEvent<DispatcherQueueEnd>()
|
||||
val dispatcherQueueEndA3 = eventRecorder.removeUpToEvent<DispatcherQueueEnd>()
|
||||
assertThat(dispatcherQueueEndA3.call.request().url).isEqualTo("http://a/3".toHttpUrl())
|
||||
val dispatcherQueueEndA4 = listener.removeUpToEvent<DispatcherQueueEnd>()
|
||||
val dispatcherQueueEndA4 = eventRecorder.removeUpToEvent<DispatcherQueueEnd>()
|
||||
assertThat(dispatcherQueueEndA4.call.request().url).isEqualTo("http://a/4".toHttpUrl())
|
||||
assertThat(listener.eventSequence).none { it.isInstanceOf<DispatcherQueueEnd>() }
|
||||
assertThat(eventRecorder.eventSequence).none { it.isInstanceOf<DispatcherQueueEnd>() }
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -357,7 +357,7 @@ class DispatcherTest {
|
||||
executor.shutdown()
|
||||
client.newCall(request).enqueue(callback)
|
||||
callback.await(request.url).assertFailure(InterruptedIOException::class.java)
|
||||
assertThat(listener.recordedEventTypes())
|
||||
assertThat(eventRecorder.recordedEventTypes())
|
||||
.containsExactly(CallStart::class, CallFailed::class)
|
||||
}
|
||||
|
||||
@@ -371,7 +371,7 @@ class DispatcherTest {
|
||||
client.newCall(request2).enqueue(callback)
|
||||
dispatcher.maxRequests = 2 // Trigger promotion.
|
||||
callback.await(request2.url).assertFailure(InterruptedIOException::class.java)
|
||||
assertThat(listener.recordedEventTypes())
|
||||
assertThat(eventRecorder.recordedEventTypes())
|
||||
.containsExactly(CallStart::class, CallStart::class, CallFailed::class)
|
||||
}
|
||||
|
||||
@@ -385,7 +385,7 @@ class DispatcherTest {
|
||||
client.newCall(request2).enqueue(callback)
|
||||
dispatcher.maxRequestsPerHost = 2 // Trigger promotion.
|
||||
callback.await(request2.url).assertFailure(InterruptedIOException::class.java)
|
||||
assertThat(listener.recordedEventTypes())
|
||||
assertThat(eventRecorder.recordedEventTypes())
|
||||
.containsExactly(CallStart::class, CallStart::class, CallFailed::class)
|
||||
}
|
||||
|
||||
@@ -399,7 +399,7 @@ class DispatcherTest {
|
||||
client.newCall(request2).enqueue(callback)
|
||||
executor.finishJob("http://a/1") // Trigger promotion.
|
||||
callback.await(request2.url).assertFailure(InterruptedIOException::class.java)
|
||||
assertThat(listener.recordedEventTypes())
|
||||
assertThat(eventRecorder.recordedEventTypes())
|
||||
.containsExactly(CallStart::class, CallStart::class, CallFailed::class)
|
||||
}
|
||||
|
||||
|
||||
@@ -88,12 +88,12 @@ class DuplexTest {
|
||||
@StartStop
|
||||
private val server = MockWebServer()
|
||||
|
||||
private var listener = RecordingEventListener()
|
||||
private var eventRecorder = EventRecorder()
|
||||
private val handshakeCertificates = platform.localhostHandshakeCertificates()
|
||||
private var client =
|
||||
clientTestRule
|
||||
.newClientBuilder()
|
||||
.eventListenerFactory(clientTestRule.wrap(listener))
|
||||
.eventListenerFactory(clientTestRule.wrap(eventRecorder))
|
||||
.build()
|
||||
private val executorService = Executors.newScheduledThreadPool(1)
|
||||
|
||||
@@ -323,7 +323,7 @@ class DuplexTest {
|
||||
requestBody.close()
|
||||
}
|
||||
body.awaitSuccess()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -395,26 +395,21 @@ class DuplexTest {
|
||||
fun duplexWithRedirect() {
|
||||
enableProtocol(Protocol.HTTP_2)
|
||||
val duplexResponseSent = CountDownLatch(1)
|
||||
listener =
|
||||
object : RecordingEventListener() {
|
||||
override fun responseHeadersEnd(
|
||||
val requestHeadersEndListener =
|
||||
object : EventListener() {
|
||||
override fun requestHeadersEnd(
|
||||
call: Call,
|
||||
response: Response,
|
||||
request: Request,
|
||||
) {
|
||||
try {
|
||||
// Wait for the server to send the duplex response before acting on the 301 response
|
||||
// and resetting the stream.
|
||||
duplexResponseSent.await()
|
||||
} catch (e: InterruptedException) {
|
||||
throw AssertionError()
|
||||
}
|
||||
super.responseHeadersEnd(call, response)
|
||||
// Wait for the server to send the duplex response before acting on the 301 response
|
||||
// and resetting the stream.
|
||||
duplexResponseSent.await()
|
||||
}
|
||||
}
|
||||
client =
|
||||
client
|
||||
.newBuilder()
|
||||
.eventListener(listener)
|
||||
.eventListener(eventRecorder.eventListener + requestHeadersEndListener)
|
||||
.build()
|
||||
val body =
|
||||
MockSocketHandler()
|
||||
@@ -458,7 +453,7 @@ class DuplexTest {
|
||||
.isEqualTo("stream was reset: CANCEL")
|
||||
}
|
||||
body.awaitSuccess()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -488,7 +483,7 @@ class DuplexTest {
|
||||
CallEnd::class,
|
||||
RequestFailed::class,
|
||||
)
|
||||
assertThat(listener.findEvent<FollowUpDecision>()).all {
|
||||
assertThat(eventRecorder.findEvent<FollowUpDecision>()).all {
|
||||
prop(FollowUpDecision::nextRequest).isNotNull()
|
||||
}
|
||||
}
|
||||
|
||||
@@ -54,6 +54,7 @@ import okhttp3.CallEvent.CallFailed
|
||||
import okhttp3.CallEvent.CallStart
|
||||
import okhttp3.CallEvent.Canceled
|
||||
import okhttp3.CallEvent.ConnectEnd
|
||||
import okhttp3.CallEvent.ConnectFailed
|
||||
import okhttp3.CallEvent.ConnectStart
|
||||
import okhttp3.CallEvent.ConnectionAcquired
|
||||
import okhttp3.CallEvent.ConnectionReleased
|
||||
@@ -114,12 +115,12 @@ class EventListenerTest {
|
||||
@StartStop
|
||||
private val server = MockWebServer()
|
||||
|
||||
private val listener: RecordingEventListener = RecordingEventListener()
|
||||
private val eventRecorder = EventRecorder()
|
||||
private val handshakeCertificates = platform.localhostHandshakeCertificates()
|
||||
private var client =
|
||||
clientTestRule
|
||||
.newClientBuilder()
|
||||
.eventListenerFactory(clientTestRule.wrap(listener))
|
||||
.eventListenerFactory(clientTestRule.wrap(eventRecorder))
|
||||
.build()
|
||||
private var socksProxy: SocksProxy? = null
|
||||
private var cache: Cache? = null
|
||||
@@ -127,8 +128,8 @@ class EventListenerTest {
|
||||
@BeforeEach
|
||||
fun setUp() {
|
||||
platform.assumeNotOpenJSSE()
|
||||
listener.forbidLock(get(client.connectionPool))
|
||||
listener.forbidLock(client.dispatcher)
|
||||
eventRecorder.forbidLock(get(client.connectionPool))
|
||||
eventRecorder.forbidLock(client.dispatcher)
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
@@ -160,7 +161,7 @@ class EventListenerTest {
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
assertThat(response.body.string()).isEqualTo("abc")
|
||||
response.body.close()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -206,7 +207,7 @@ class EventListenerTest {
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
assertThat(response.body.string()).isEqualTo("abc")
|
||||
response.body.close()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -260,7 +261,7 @@ class EventListenerTest {
|
||||
}
|
||||
call.enqueue(callback)
|
||||
completionLatch.await()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -306,7 +307,7 @@ class EventListenerTest {
|
||||
}.also { expected ->
|
||||
assertThat(expected.message).isIn("timeout", "Read timed out")
|
||||
}
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -322,7 +323,7 @@ class EventListenerTest {
|
||||
ConnectionReleased::class,
|
||||
CallFailed::class,
|
||||
)
|
||||
assertThat(listener.findEvent<RetryDecision>()).all {
|
||||
assertThat(eventRecorder.findEvent<RetryDecision>()).all {
|
||||
prop(RetryDecision::retry).isFalse()
|
||||
}
|
||||
}
|
||||
@@ -356,7 +357,7 @@ class EventListenerTest {
|
||||
}.also { expected ->
|
||||
assertThat(expected.message).isEqualTo("unexpected end of stream")
|
||||
}
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -375,7 +376,7 @@ class EventListenerTest {
|
||||
ConnectionReleased::class,
|
||||
CallFailed::class,
|
||||
)
|
||||
val responseFailed = listener.removeUpToEvent<ResponseFailed>()
|
||||
val responseFailed = eventRecorder.removeUpToEvent<ResponseFailed>()
|
||||
assertThat(responseFailed.ioe.message).isEqualTo("unexpected end of stream")
|
||||
}
|
||||
|
||||
@@ -394,7 +395,7 @@ class EventListenerTest {
|
||||
}.also { expected ->
|
||||
assertThat(expected.message).isEqualTo("Canceled")
|
||||
}
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
Canceled::class,
|
||||
CallStart::class,
|
||||
CallFailed::class,
|
||||
@@ -433,7 +434,7 @@ class EventListenerTest {
|
||||
},
|
||||
)
|
||||
call.cancel()
|
||||
assertThat(listener.recordedEventTypes()).contains(Canceled::class)
|
||||
assertThat(eventRecorder.recordedEventTypes()).contains(Canceled::class)
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -453,7 +454,7 @@ class EventListenerTest {
|
||||
)
|
||||
call.cancel()
|
||||
call.cancel()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(Canceled::class)
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(Canceled::class)
|
||||
}
|
||||
|
||||
private fun assertSuccessfulEventOrder(
|
||||
@@ -509,7 +510,7 @@ class EventListenerTest {
|
||||
ConnectionReleased::class,
|
||||
CallEnd::class,
|
||||
)
|
||||
assertThat(listener.recordedEventTypes()).isEqualTo(expectedEventTypes)
|
||||
assertThat(eventRecorder.recordedEventTypes()).isEqualTo(expectedEventTypes)
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -526,7 +527,7 @@ class EventListenerTest {
|
||||
.build(),
|
||||
).execute()
|
||||
.close()
|
||||
listener.removeUpToEvent<CallEnd>()
|
||||
eventRecorder.removeUpToEvent<CallEnd>()
|
||||
val call =
|
||||
client.newCall(
|
||||
Request
|
||||
@@ -536,7 +537,7 @@ class EventListenerTest {
|
||||
)
|
||||
val response = call.execute()
|
||||
response.close()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ConnectionAcquired::class,
|
||||
RequestHeadersStart::class,
|
||||
@@ -552,7 +553,7 @@ class EventListenerTest {
|
||||
}
|
||||
|
||||
private fun assertBytesReadWritten(
|
||||
listener: RecordingEventListener,
|
||||
listener: EventRecorder,
|
||||
requestHeaderLength: Matcher<Long?>?,
|
||||
requestBodyBytes: Matcher<Long?>?,
|
||||
responseHeaderLength: Matcher<Long?>?,
|
||||
@@ -628,7 +629,7 @@ class EventListenerTest {
|
||||
server.enqueue(MockResponse())
|
||||
assertSuccessfulEventOrder(matchesProtocol(Protocol.HTTP_2), emptyBody = true)
|
||||
assertBytesReadWritten(
|
||||
listener,
|
||||
eventRecorder,
|
||||
CoreMatchers.any(Long::class.java),
|
||||
null,
|
||||
greaterThan(0L),
|
||||
@@ -648,7 +649,7 @@ class EventListenerTest {
|
||||
)
|
||||
assertSuccessfulEventOrder(anyResponse)
|
||||
assertBytesReadWritten(
|
||||
listener,
|
||||
eventRecorder,
|
||||
CoreMatchers.any(Long::class.java),
|
||||
null,
|
||||
greaterThan(0L),
|
||||
@@ -669,7 +670,7 @@ class EventListenerTest {
|
||||
)
|
||||
assertSuccessfulEventOrder(anyResponse)
|
||||
assertBytesReadWritten(
|
||||
listener,
|
||||
eventRecorder,
|
||||
CoreMatchers.any(Long::class.java),
|
||||
null,
|
||||
greaterThan(0L),
|
||||
@@ -690,7 +691,7 @@ class EventListenerTest {
|
||||
)
|
||||
assertSuccessfulEventOrder(matchesProtocol(Protocol.HTTP_2))
|
||||
assertBytesReadWritten(
|
||||
listener,
|
||||
eventRecorder,
|
||||
CoreMatchers.any(Long::class.java),
|
||||
null,
|
||||
CoreMatchers.equalTo(0L),
|
||||
@@ -711,10 +712,10 @@ class EventListenerTest {
|
||||
val response = call.execute()
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
response.body.close()
|
||||
val dnsStart: DnsStart = listener.removeUpToEvent<DnsStart>()
|
||||
val dnsStart: DnsStart = eventRecorder.removeUpToEvent<DnsStart>()
|
||||
assertThat(dnsStart.call).isSameAs(call)
|
||||
assertThat(dnsStart.domainName).isEqualTo(server.hostName)
|
||||
val dnsEnd: DnsEnd = listener.removeUpToEvent<DnsEnd>()
|
||||
val dnsEnd: DnsEnd = eventRecorder.removeUpToEvent<DnsEnd>()
|
||||
assertThat(dnsEnd.call).isSameAs(call)
|
||||
assertThat(dnsEnd.domainName).isEqualTo(server.hostName)
|
||||
assertThat(dnsEnd.inetAddressList.size).isEqualTo(1)
|
||||
@@ -736,7 +737,7 @@ class EventListenerTest {
|
||||
val response1 = call1.execute()
|
||||
assertThat(response1.code).isEqualTo(200)
|
||||
response1.body.close()
|
||||
listener.clearAllEvents()
|
||||
eventRecorder.clearAllEvents()
|
||||
val call2 =
|
||||
client.newCall(
|
||||
Request
|
||||
@@ -747,7 +748,7 @@ class EventListenerTest {
|
||||
val response2 = call2.execute()
|
||||
assertThat(response2.code).isEqualTo(200)
|
||||
response2.body.close()
|
||||
val recordedEvents = listener.recordedEventTypes()
|
||||
val recordedEvents = eventRecorder.recordedEventTypes()
|
||||
assertThat(recordedEvents).doesNotContain(DnsStart::class)
|
||||
assertThat(recordedEvents).doesNotContain(DnsEnd::class)
|
||||
}
|
||||
@@ -780,10 +781,10 @@ class EventListenerTest {
|
||||
val response = call.execute()
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
response.body.close()
|
||||
listener.removeUpToEvent<DnsStart>()
|
||||
listener.removeUpToEvent<DnsEnd>()
|
||||
listener.removeUpToEvent<DnsStart>()
|
||||
listener.removeUpToEvent<DnsEnd>()
|
||||
eventRecorder.removeUpToEvent<DnsStart>()
|
||||
eventRecorder.removeUpToEvent<DnsEnd>()
|
||||
eventRecorder.removeUpToEvent<DnsStart>()
|
||||
eventRecorder.removeUpToEvent<DnsEnd>()
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -803,8 +804,8 @@ class EventListenerTest {
|
||||
assertFailsWith<IOException> {
|
||||
call.execute()
|
||||
}
|
||||
listener.removeUpToEvent<DnsStart>()
|
||||
val callFailed: CallFailed = listener.removeUpToEvent<CallFailed>()
|
||||
eventRecorder.removeUpToEvent<DnsStart>()
|
||||
val callFailed: CallFailed = eventRecorder.removeUpToEvent<CallFailed>()
|
||||
assertThat(callFailed.call).isSameAs(call)
|
||||
assertThat(callFailed.ioe).isInstanceOf<UnknownHostException>()
|
||||
}
|
||||
@@ -827,8 +828,8 @@ class EventListenerTest {
|
||||
assertFailsWith<IOException> {
|
||||
call.execute()
|
||||
}
|
||||
listener.removeUpToEvent<DnsStart>()
|
||||
val callFailed: CallFailed = listener.removeUpToEvent<CallFailed>()
|
||||
eventRecorder.removeUpToEvent<DnsStart>()
|
||||
val callFailed: CallFailed = eventRecorder.removeUpToEvent<CallFailed>()
|
||||
assertThat(callFailed.call).isSameAs(call)
|
||||
assertThat(callFailed.ioe).isInstanceOf(
|
||||
UnknownHostException::class.java,
|
||||
@@ -850,11 +851,11 @@ class EventListenerTest {
|
||||
response.body.close()
|
||||
val address = client.dns.lookup(server.hostName)[0]
|
||||
val expectedAddress = InetSocketAddress(address, server.port)
|
||||
val connectStart = listener.removeUpToEvent<ConnectStart>()
|
||||
val connectStart = eventRecorder.removeUpToEvent<ConnectStart>()
|
||||
assertThat(connectStart.call).isSameAs(call)
|
||||
assertThat(connectStart.inetSocketAddress).isEqualTo(expectedAddress)
|
||||
assertThat(connectStart.proxy).isEqualTo(Proxy.NO_PROXY)
|
||||
val connectEnd = listener.removeUpToEvent<CallEvent.ConnectEnd>()
|
||||
val connectEnd = eventRecorder.removeUpToEvent<ConnectEnd>()
|
||||
assertThat(connectEnd.call).isSameAs(call)
|
||||
assertThat(connectEnd.inetSocketAddress).isEqualTo(expectedAddress)
|
||||
assertThat(connectEnd.protocol).isEqualTo(Protocol.HTTP_1_1)
|
||||
@@ -881,11 +882,11 @@ class EventListenerTest {
|
||||
}
|
||||
val address = client.dns.lookup(server.hostName)[0]
|
||||
val expectedAddress = InetSocketAddress(address, server.port)
|
||||
val connectStart = listener.removeUpToEvent<ConnectStart>()
|
||||
val connectStart = eventRecorder.removeUpToEvent<ConnectStart>()
|
||||
assertThat(connectStart.call).isSameAs(call)
|
||||
assertThat(connectStart.inetSocketAddress).isEqualTo(expectedAddress)
|
||||
assertThat(connectStart.proxy).isEqualTo(Proxy.NO_PROXY)
|
||||
val connectFailed = listener.removeUpToEvent<CallEvent.ConnectFailed>()
|
||||
val connectFailed = eventRecorder.removeUpToEvent<ConnectFailed>()
|
||||
assertThat(connectFailed.call).isSameAs(call)
|
||||
assertThat(connectFailed.inetSocketAddress).isEqualTo(expectedAddress)
|
||||
assertThat(connectFailed.protocol).isNull()
|
||||
@@ -917,10 +918,10 @@ class EventListenerTest {
|
||||
val response = call.execute()
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
response.body.close()
|
||||
listener.removeUpToEvent<ConnectStart>()
|
||||
listener.removeUpToEvent<CallEvent.ConnectFailed>()
|
||||
listener.removeUpToEvent<ConnectStart>()
|
||||
listener.removeUpToEvent<CallEvent.ConnectEnd>()
|
||||
eventRecorder.removeUpToEvent<ConnectStart>()
|
||||
eventRecorder.removeUpToEvent<ConnectFailed>()
|
||||
eventRecorder.removeUpToEvent<ConnectStart>()
|
||||
eventRecorder.removeUpToEvent<ConnectEnd>()
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -944,13 +945,13 @@ class EventListenerTest {
|
||||
val address = client.dns.lookup(server.hostName)[0]
|
||||
val expectedAddress = InetSocketAddress(address, server.port)
|
||||
val connectStart: ConnectStart =
|
||||
listener.removeUpToEvent<ConnectStart>()
|
||||
eventRecorder.removeUpToEvent<ConnectStart>()
|
||||
assertThat(connectStart.call).isSameAs(call)
|
||||
assertThat(connectStart.inetSocketAddress).isEqualTo(expectedAddress)
|
||||
assertThat(connectStart.proxy).isEqualTo(
|
||||
server.proxyAddress,
|
||||
)
|
||||
val connectEnd = listener.removeUpToEvent<CallEvent.ConnectEnd>()
|
||||
val connectEnd = eventRecorder.removeUpToEvent<ConnectEnd>()
|
||||
assertThat(connectEnd.call).isSameAs(call)
|
||||
assertThat(connectEnd.inetSocketAddress).isEqualTo(expectedAddress)
|
||||
assertThat(connectEnd.protocol).isEqualTo(Protocol.HTTP_1_1)
|
||||
@@ -982,11 +983,11 @@ class EventListenerTest {
|
||||
SocksProxy.HOSTNAME_THAT_ONLY_THE_PROXY_KNOWS,
|
||||
server.port,
|
||||
)
|
||||
val connectStart = listener.removeUpToEvent<ConnectStart>()
|
||||
val connectStart = eventRecorder.removeUpToEvent<ConnectStart>()
|
||||
assertThat(connectStart.call).isSameAs(call)
|
||||
assertThat(connectStart.inetSocketAddress).isEqualTo(expectedAddress)
|
||||
assertThat(connectStart.proxy).isEqualTo(proxy)
|
||||
val connectEnd = listener.removeUpToEvent<CallEvent.ConnectEnd>()
|
||||
val connectEnd = eventRecorder.removeUpToEvent<ConnectEnd>()
|
||||
assertThat(connectEnd.call).isSameAs(call)
|
||||
assertThat(connectEnd.inetSocketAddress).isEqualTo(expectedAddress)
|
||||
assertThat(connectEnd.protocol).isEqualTo(Protocol.HTTP_1_1)
|
||||
@@ -1027,11 +1028,11 @@ class EventListenerTest {
|
||||
val response = call.execute()
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
response.body.close()
|
||||
listener.removeUpToEvent<ConnectStart>()
|
||||
val connectEnd = listener.removeUpToEvent<CallEvent.ConnectEnd>()
|
||||
eventRecorder.removeUpToEvent<ConnectStart>()
|
||||
val connectEnd = eventRecorder.removeUpToEvent<ConnectEnd>()
|
||||
assertThat(connectEnd.protocol).isNull()
|
||||
listener.removeUpToEvent<ConnectStart>()
|
||||
listener.removeUpToEvent<CallEvent.ConnectEnd>()
|
||||
eventRecorder.removeUpToEvent<ConnectStart>()
|
||||
eventRecorder.removeUpToEvent<ConnectEnd>()
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -1048,9 +1049,9 @@ class EventListenerTest {
|
||||
val response = call.execute()
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
response.body.close()
|
||||
val secureStart = listener.removeUpToEvent<SecureConnectStart>()
|
||||
val secureStart = eventRecorder.removeUpToEvent<SecureConnectStart>()
|
||||
assertThat(secureStart.call).isSameAs(call)
|
||||
val secureEnd = listener.removeUpToEvent<SecureConnectEnd>()
|
||||
val secureEnd = eventRecorder.removeUpToEvent<SecureConnectEnd>()
|
||||
assertThat(secureEnd.call).isSameAs(call)
|
||||
assertThat(secureEnd.handshake).isNotNull()
|
||||
}
|
||||
@@ -1074,9 +1075,9 @@ class EventListenerTest {
|
||||
assertFailsWith<IOException> {
|
||||
call.execute()
|
||||
}
|
||||
val secureStart = listener.removeUpToEvent<SecureConnectStart>()
|
||||
val secureStart = eventRecorder.removeUpToEvent<SecureConnectStart>()
|
||||
assertThat(secureStart.call).isSameAs(call)
|
||||
val callFailed = listener.removeUpToEvent<CallFailed>()
|
||||
val callFailed = eventRecorder.removeUpToEvent<CallFailed>()
|
||||
assertThat(callFailed.call).isSameAs(call)
|
||||
assertThat(callFailed.ioe).isNotNull()
|
||||
}
|
||||
@@ -1106,9 +1107,9 @@ class EventListenerTest {
|
||||
val response = call.execute()
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
response.body.close()
|
||||
val secureStart = listener.removeUpToEvent<SecureConnectStart>()
|
||||
val secureStart = eventRecorder.removeUpToEvent<SecureConnectStart>()
|
||||
assertThat(secureStart.call).isSameAs(call)
|
||||
val secureEnd = listener.removeUpToEvent<SecureConnectEnd>()
|
||||
val secureEnd = eventRecorder.removeUpToEvent<SecureConnectEnd>()
|
||||
assertThat(secureEnd.call).isSameAs(call)
|
||||
assertThat(secureEnd.handshake).isNotNull()
|
||||
}
|
||||
@@ -1138,10 +1139,10 @@ class EventListenerTest {
|
||||
val response = call.execute()
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
response.body.close()
|
||||
listener.removeUpToEvent<SecureConnectStart>()
|
||||
listener.removeUpToEvent<CallEvent.ConnectFailed>()
|
||||
listener.removeUpToEvent<SecureConnectStart>()
|
||||
listener.removeUpToEvent<SecureConnectEnd>()
|
||||
eventRecorder.removeUpToEvent<SecureConnectStart>()
|
||||
eventRecorder.removeUpToEvent<ConnectFailed>()
|
||||
eventRecorder.removeUpToEvent<SecureConnectStart>()
|
||||
eventRecorder.removeUpToEvent<SecureConnectEnd>()
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -1166,7 +1167,7 @@ class EventListenerTest {
|
||||
val response1 = call1.execute()
|
||||
assertThat(response1.code).isEqualTo(200)
|
||||
response1.body.close()
|
||||
listener.clearAllEvents()
|
||||
eventRecorder.clearAllEvents()
|
||||
val call2 =
|
||||
client.newCall(
|
||||
Request
|
||||
@@ -1177,7 +1178,7 @@ class EventListenerTest {
|
||||
val response2 = call2.execute()
|
||||
assertThat(response2.code).isEqualTo(200)
|
||||
response2.body.close()
|
||||
val recordedEvents = listener.recordedEventTypes()
|
||||
val recordedEvents = eventRecorder.recordedEventTypes()
|
||||
assertThat(recordedEvents).doesNotContain(SecureConnectStart::class)
|
||||
assertThat(recordedEvents).doesNotContain(SecureConnectEnd::class)
|
||||
}
|
||||
@@ -1195,7 +1196,7 @@ class EventListenerTest {
|
||||
val response = call.execute()
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
response.body.close()
|
||||
val connectionAcquired = listener.removeUpToEvent<ConnectionAcquired>()
|
||||
val connectionAcquired = eventRecorder.removeUpToEvent<ConnectionAcquired>()
|
||||
assertThat(connectionAcquired.call).isSameAs(call)
|
||||
assertThat(connectionAcquired.connection).isNotNull()
|
||||
}
|
||||
@@ -1224,8 +1225,8 @@ class EventListenerTest {
|
||||
)
|
||||
val response = call.execute()
|
||||
assertThat(response.body.string()).isEqualTo("ABC")
|
||||
listener.removeUpToEvent<ConnectionAcquired>()
|
||||
val remainingEvents = listener.recordedEventTypes()
|
||||
eventRecorder.removeUpToEvent<ConnectionAcquired>()
|
||||
val remainingEvents = eventRecorder.recordedEventTypes()
|
||||
assertThat(remainingEvents).doesNotContain(ConnectionAcquired::class)
|
||||
}
|
||||
|
||||
@@ -1245,8 +1246,8 @@ class EventListenerTest {
|
||||
val response1 = call1.execute()
|
||||
assertThat(response1.code).isEqualTo(200)
|
||||
response1.body.close()
|
||||
val connectionAcquired1 = listener.removeUpToEvent<ConnectionAcquired>()
|
||||
listener.clearAllEvents()
|
||||
val connectionAcquired1 = eventRecorder.removeUpToEvent<ConnectionAcquired>()
|
||||
eventRecorder.clearAllEvents()
|
||||
val call2 =
|
||||
client.newCall(
|
||||
Request
|
||||
@@ -1257,7 +1258,7 @@ class EventListenerTest {
|
||||
val response2 = call2.execute()
|
||||
assertThat(response2.code).isEqualTo(200)
|
||||
response2.body.close()
|
||||
val connectionAcquired2 = listener.removeUpToEvent<ConnectionAcquired>()
|
||||
val connectionAcquired2 = eventRecorder.removeUpToEvent<ConnectionAcquired>()
|
||||
assertThat(connectionAcquired2.connection).isSameAs(
|
||||
connectionAcquired1.connection,
|
||||
)
|
||||
@@ -1288,8 +1289,8 @@ class EventListenerTest {
|
||||
)
|
||||
val response = call.execute()
|
||||
assertThat(response.body.string()).isEqualTo("ABC")
|
||||
listener.removeUpToEvent<ConnectionAcquired>()
|
||||
listener.removeUpToEvent<ConnectionAcquired>()
|
||||
eventRecorder.removeUpToEvent<ConnectionAcquired>()
|
||||
eventRecorder.removeUpToEvent<ConnectionAcquired>()
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -1338,7 +1339,7 @@ class EventListenerTest {
|
||||
assertFailsWith<IOException> {
|
||||
response.body.string()
|
||||
}
|
||||
val callFailed = listener.removeUpToEvent<CallFailed>()
|
||||
val callFailed = eventRecorder.removeUpToEvent<CallFailed>()
|
||||
assertThat(callFailed.ioe).isNotNull()
|
||||
}
|
||||
|
||||
@@ -1361,7 +1362,7 @@ class EventListenerTest {
|
||||
)
|
||||
val response = call.execute()
|
||||
response.body.close()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -1400,7 +1401,7 @@ class EventListenerTest {
|
||||
)
|
||||
val response = call.execute()
|
||||
response.body.close()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -1440,7 +1441,7 @@ class EventListenerTest {
|
||||
)
|
||||
val response = call.execute()
|
||||
response.body.close()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -1501,11 +1502,11 @@ class EventListenerTest {
|
||||
call.execute()
|
||||
}
|
||||
if (expectedProtocol != null) {
|
||||
val connectionAcquired = listener.removeUpToEvent<ConnectionAcquired>()
|
||||
val connectionAcquired = eventRecorder.removeUpToEvent<ConnectionAcquired>()
|
||||
assertThat(connectionAcquired.connection.protocol())
|
||||
.isEqualTo(expectedProtocol)
|
||||
}
|
||||
val callFailed = listener.removeUpToEvent<CallFailed>()
|
||||
val callFailed = eventRecorder.removeUpToEvent<CallFailed>()
|
||||
assertThat(callFailed.ioe).isNotNull()
|
||||
assertThat(request.ioe).isNotNull()
|
||||
}
|
||||
@@ -1573,7 +1574,7 @@ class EventListenerTest {
|
||||
assertFailsWith<IOException> {
|
||||
call.execute()
|
||||
}
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -1674,7 +1675,7 @@ class EventListenerTest {
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
assertThat(response.body.string()).isEqualTo("abc")
|
||||
response.body.close()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -1718,7 +1719,7 @@ class EventListenerTest {
|
||||
val response = call.execute()
|
||||
assertThat(response.body.string()).isEqualTo("World!")
|
||||
assertBytesReadWritten(
|
||||
listener,
|
||||
eventRecorder,
|
||||
CoreMatchers.any(Long::class.java),
|
||||
requestBodyBytes,
|
||||
responseHeaderLength,
|
||||
@@ -1767,7 +1768,7 @@ class EventListenerTest {
|
||||
.build(),
|
||||
)
|
||||
warmUpCall.execute().use { warmUpResponse -> warmUpResponse.body.string() }
|
||||
listener.clearAllEvents()
|
||||
eventRecorder.clearAllEvents()
|
||||
|
||||
// Create a client with artificial delays.
|
||||
client =
|
||||
@@ -1830,19 +1831,19 @@ class EventListenerTest {
|
||||
}
|
||||
|
||||
// Confirm the events occur when expected.
|
||||
listener.takeEvent(CallStart::class.java, 0L)
|
||||
listener.takeEvent(ConnectionAcquired::class.java, applicationInterceptorDelay)
|
||||
listener.takeEvent(RequestHeadersStart::class.java, networkInterceptorDelay)
|
||||
listener.takeEvent(RequestHeadersEnd::class.java, 0L)
|
||||
listener.takeEvent(RequestBodyStart::class.java, 0L)
|
||||
listener.takeEvent(RequestBodyEnd::class.java, requestBodyDelay)
|
||||
listener.takeEvent(ResponseHeadersStart::class.java, responseHeadersStartDelay)
|
||||
listener.takeEvent(ResponseHeadersEnd::class.java, 0L)
|
||||
listener.takeEvent(FollowUpDecision::class.java, 0L)
|
||||
listener.takeEvent(ResponseBodyStart::class.java, responseBodyStartDelay)
|
||||
listener.takeEvent(ResponseBodyEnd::class.java, responseBodyEndDelay)
|
||||
listener.takeEvent(CallEvent.ConnectionReleased::class.java, 0L)
|
||||
listener.takeEvent(CallEnd::class.java, 0L)
|
||||
eventRecorder.takeEvent(CallStart::class.java, 0L)
|
||||
eventRecorder.takeEvent(ConnectionAcquired::class.java, applicationInterceptorDelay)
|
||||
eventRecorder.takeEvent(RequestHeadersStart::class.java, networkInterceptorDelay)
|
||||
eventRecorder.takeEvent(RequestHeadersEnd::class.java, 0L)
|
||||
eventRecorder.takeEvent(RequestBodyStart::class.java, 0L)
|
||||
eventRecorder.takeEvent(RequestBodyEnd::class.java, requestBodyDelay)
|
||||
eventRecorder.takeEvent(ResponseHeadersStart::class.java, responseHeadersStartDelay)
|
||||
eventRecorder.takeEvent(ResponseHeadersEnd::class.java, 0L)
|
||||
eventRecorder.takeEvent(FollowUpDecision::class.java, 0L)
|
||||
eventRecorder.takeEvent(ResponseBodyStart::class.java, responseBodyStartDelay)
|
||||
eventRecorder.takeEvent(ResponseBodyEnd::class.java, responseBodyEndDelay)
|
||||
eventRecorder.takeEvent(ConnectionReleased::class.java, 0L)
|
||||
eventRecorder.takeEvent(CallEnd::class.java, 0L)
|
||||
}
|
||||
|
||||
private fun enableTlsWithTunnel() {
|
||||
@@ -1869,7 +1870,7 @@ class EventListenerTest {
|
||||
server.enqueue(MockResponse())
|
||||
val call = client.newCall(Request.Builder().url(server.url("/")).build())
|
||||
call.execute()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -1895,7 +1896,7 @@ class EventListenerTest {
|
||||
ConnectionReleased::class,
|
||||
CallEnd::class,
|
||||
)
|
||||
assertThat(listener.findEvent<FollowUpDecision>()).all {
|
||||
assertThat(eventRecorder.findEvent<FollowUpDecision>()).all {
|
||||
prop(FollowUpDecision::nextRequest).isNotNull()
|
||||
}
|
||||
}
|
||||
@@ -1914,7 +1915,7 @@ class EventListenerTest {
|
||||
otherServer.enqueue(MockResponse())
|
||||
val call = client.newCall(Request.Builder().url(server.url("/")).build())
|
||||
call.execute()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -1948,7 +1949,7 @@ class EventListenerTest {
|
||||
ConnectionReleased::class,
|
||||
CallEnd::class,
|
||||
)
|
||||
assertThat(listener.findEvent<FollowUpDecision>()).all {
|
||||
assertThat(eventRecorder.findEvent<FollowUpDecision>()).all {
|
||||
prop(FollowUpDecision::nextRequest).isNotNull()
|
||||
}
|
||||
otherServer.close()
|
||||
@@ -1972,7 +1973,7 @@ class EventListenerTest {
|
||||
val call = client.newCall(Request.Builder().url(server.url("/")).build())
|
||||
val response = call.execute()
|
||||
assertThat(response.body.string()).isEqualTo("b")
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -2022,7 +2023,7 @@ class EventListenerTest {
|
||||
val call = client.newCall(Request.Builder().url(server.url("/")).build())
|
||||
val response = call.execute()
|
||||
assertThat(response.body.string()).isEqualTo("a")
|
||||
assertThat(listener.recordedEventTypes())
|
||||
assertThat(eventRecorder.recordedEventTypes())
|
||||
.containsExactly(CallStart::class, CallEnd::class)
|
||||
}
|
||||
|
||||
@@ -2044,7 +2045,7 @@ class EventListenerTest {
|
||||
.build()
|
||||
val call = client.newCall(request)
|
||||
call.execute()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -2088,10 +2089,10 @@ class EventListenerTest {
|
||||
call
|
||||
.execute()
|
||||
.use { response -> assertThat(response.body.string()).isEqualTo("") }
|
||||
listener.removeUpToEvent<ResponseHeadersStart>()
|
||||
listener.takeEvent(RequestBodyStart::class.java, 0L)
|
||||
listener.takeEvent(RequestBodyEnd::class.java, 0L)
|
||||
listener.takeEvent(ResponseHeadersEnd::class.java, responseHeadersStartDelay)
|
||||
eventRecorder.removeUpToEvent<ResponseHeadersStart>()
|
||||
eventRecorder.takeEvent(RequestBodyStart::class.java, 0L)
|
||||
eventRecorder.takeEvent(RequestBodyEnd::class.java, 0L)
|
||||
eventRecorder.takeEvent(ResponseHeadersEnd::class.java, responseHeadersStartDelay)
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -2114,7 +2115,7 @@ class EventListenerTest {
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
assertThat(response.body.string()).isEqualTo("abc")
|
||||
response.close()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
CacheMiss::class,
|
||||
ProxySelectStart::class,
|
||||
@@ -2162,13 +2163,13 @@ class EventListenerTest {
|
||||
var response = call.execute()
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
response.close()
|
||||
listener.clearAllEvents()
|
||||
eventRecorder.clearAllEvents()
|
||||
call = call.clone()
|
||||
response = call.execute()
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
assertThat(response.body.string()).isEqualTo("abc")
|
||||
response.close()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
CacheConditionalHit::class,
|
||||
ConnectionAcquired::class,
|
||||
@@ -2213,13 +2214,13 @@ class EventListenerTest {
|
||||
var response = call.execute()
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
response.close()
|
||||
listener.clearAllEvents()
|
||||
eventRecorder.clearAllEvents()
|
||||
call = call.clone()
|
||||
response = call.execute()
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
assertThat(response.body.string()).isEqualTo("abd")
|
||||
response.close()
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
CacheConditionalHit::class,
|
||||
ConnectionAcquired::class,
|
||||
@@ -2250,14 +2251,14 @@ class EventListenerTest {
|
||||
val response = call.execute()
|
||||
assertThat(response.code).isEqualTo(504)
|
||||
response.close()
|
||||
assertThat(listener.recordedEventTypes())
|
||||
assertThat(eventRecorder.recordedEventTypes())
|
||||
.containsExactly(
|
||||
CallStart::class,
|
||||
SatisfactionFailure::class,
|
||||
FollowUpDecision::class,
|
||||
CallEnd::class,
|
||||
)
|
||||
assertThat(listener.findEvent<FollowUpDecision>()).all {
|
||||
assertThat(eventRecorder.findEvent<FollowUpDecision>()).all {
|
||||
prop(FollowUpDecision::nextRequest).isNull()
|
||||
}
|
||||
}
|
||||
@@ -2283,13 +2284,13 @@ class EventListenerTest {
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
assertThat(response.body.string()).isEqualTo("abc")
|
||||
response.close()
|
||||
listener.clearAllEvents()
|
||||
eventRecorder.clearAllEvents()
|
||||
call = call.clone()
|
||||
response = call.execute()
|
||||
assertThat(response.code).isEqualTo(200)
|
||||
assertThat(response.body.string()).isEqualTo("abc")
|
||||
response.close()
|
||||
assertThat(listener.recordedEventTypes())
|
||||
assertThat(eventRecorder.recordedEventTypes())
|
||||
.containsExactly(
|
||||
CallStart::class,
|
||||
CacheHit::class,
|
||||
@@ -2301,8 +2302,8 @@ class EventListenerTest {
|
||||
/** Make sure we didn't mess up our special case for [EventListener.NONE]. */
|
||||
@Test
|
||||
fun eventListenerPlusNoneAggregation() {
|
||||
val a = RecordingEventListener(enforceOrder = false)
|
||||
val aPlusNone = a + EventListener.NONE
|
||||
val a = EventRecorder(enforceOrder = false)
|
||||
val aPlusNone = a.eventListener + EventListener.NONE
|
||||
|
||||
aPlusNone.callStart(FailingCall())
|
||||
assertThat(a.takeEvent()).isInstanceOf<CallStart>()
|
||||
@@ -2312,8 +2313,8 @@ class EventListenerTest {
|
||||
/** Make sure we didn't mess up our special case for [EventListener.NONE]. */
|
||||
@Test
|
||||
fun nonePlusEventListenerAggregation() {
|
||||
val a = RecordingEventListener(enforceOrder = false)
|
||||
val nonePlusA = EventListener.NONE + a
|
||||
val a = EventRecorder(enforceOrder = false)
|
||||
val nonePlusA = EventListener.NONE + a.eventListener
|
||||
|
||||
nonePlusA.callStart(FailingCall())
|
||||
assertThat(a.takeEvent()).isInstanceOf<CallStart>()
|
||||
@@ -2323,12 +2324,12 @@ class EventListenerTest {
|
||||
/** Make sure we didn't mess up our special case for combining aggregates. */
|
||||
@Test
|
||||
fun moreThanTwoAggregation() {
|
||||
val a = RecordingEventListener(enforceOrder = false)
|
||||
val b = RecordingEventListener(enforceOrder = false)
|
||||
val c = RecordingEventListener(enforceOrder = false)
|
||||
val d = RecordingEventListener(enforceOrder = false)
|
||||
val a = EventRecorder(enforceOrder = false)
|
||||
val b = EventRecorder(enforceOrder = false)
|
||||
val c = EventRecorder(enforceOrder = false)
|
||||
val d = EventRecorder(enforceOrder = false)
|
||||
|
||||
val abcd = (a + b) + (c + d)
|
||||
val abcd = (a.eventListener + b.eventListener) + (c.eventListener + d.eventListener)
|
||||
abcd.callStart(FailingCall())
|
||||
|
||||
assertThat(a.takeEvent()).isInstanceOf<CallStart>()
|
||||
@@ -2346,10 +2347,10 @@ class EventListenerTest {
|
||||
fun aggregateEventListenerIsComplete() {
|
||||
val sampleValues = sampleValuesMap()
|
||||
|
||||
val solo = RecordingEventListener(enforceOrder = false)
|
||||
val left = RecordingEventListener(enforceOrder = false)
|
||||
val right = RecordingEventListener(enforceOrder = false)
|
||||
val composite = left + right
|
||||
val solo = EventRecorder(enforceOrder = false)
|
||||
val left = EventRecorder(enforceOrder = false)
|
||||
val right = EventRecorder(enforceOrder = false)
|
||||
val composite = left.eventListener + right.eventListener
|
||||
|
||||
for (method in EventListener::class.java.declaredMethods) {
|
||||
if (method.name == "plus") continue
|
||||
@@ -2359,7 +2360,7 @@ class EventListenerTest {
|
||||
.map { sampleValues[it.type] ?: error("no sample value for ${it.type}") }
|
||||
.toTypedArray()
|
||||
|
||||
method.invoke(solo, *args)
|
||||
method.invoke(solo.eventListener, *args)
|
||||
method.invoke(composite, *args)
|
||||
|
||||
val expectedEvent = solo.takeEvent()
|
||||
|
||||
@@ -65,7 +65,7 @@ class FastFallbackTest {
|
||||
private lateinit var serverIpv4: MockWebServer
|
||||
private lateinit var serverIpv6: MockWebServer
|
||||
|
||||
private val listener = RecordingEventListener()
|
||||
private val eventRecorder = EventRecorder()
|
||||
private lateinit var client: OkHttpClient
|
||||
private lateinit var url: HttpUrl
|
||||
|
||||
@@ -98,7 +98,7 @@ class FastFallbackTest {
|
||||
client =
|
||||
clientTestRule
|
||||
.newClientBuilder()
|
||||
.eventListenerFactory(clientTestRule.wrap(listener))
|
||||
.eventListenerFactory(clientTestRule.wrap(eventRecorder))
|
||||
.connectTimeout(60, TimeUnit.SECONDS) // Deliberately exacerbate slow fallbacks.
|
||||
.dns { dnsResults }
|
||||
.fastFallback(true)
|
||||
@@ -136,8 +136,8 @@ class FastFallbackTest {
|
||||
assertThat(response.body.string()).isEqualTo("hello from IPv6")
|
||||
|
||||
// In the process we made one successful connection attempt.
|
||||
assertThat(listener.recordedEventTypes().filter { it == ConnectStart::class }).hasSize(1)
|
||||
assertThat(listener.recordedEventTypes().filter { it == ConnectFailed::class }).hasSize(0)
|
||||
assertThat(eventRecorder.recordedEventTypes().filter { it == ConnectStart::class }).hasSize(1)
|
||||
assertThat(eventRecorder.recordedEventTypes().filter { it == ConnectFailed::class }).hasSize(0)
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -160,8 +160,8 @@ class FastFallbackTest {
|
||||
assertThat(response.body.string()).isEqualTo("hello from IPv6")
|
||||
|
||||
// In the process we made one successful connection attempt.
|
||||
assertThat(listener.recordedEventTypes().filter { it == ConnectStart::class }).hasSize(1)
|
||||
assertThat(listener.recordedEventTypes().filter { it == ConnectFailed::class }).hasSize(0)
|
||||
assertThat(eventRecorder.recordedEventTypes().filter { it == ConnectStart::class }).hasSize(1)
|
||||
assertThat(eventRecorder.recordedEventTypes().filter { it == ConnectFailed::class }).hasSize(0)
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -176,9 +176,9 @@ class FastFallbackTest {
|
||||
assertThat(response.body.string()).isEqualTo("hello from IPv4")
|
||||
|
||||
// In the process we made one successful connection attempt.
|
||||
assertThat(listener.recordedEventTypes().filter { it == ConnectStart::class }).hasSize(2)
|
||||
assertThat(listener.recordedEventTypes().filter { it == ConnectFailed::class }).hasSize(1)
|
||||
assertThat(listener.recordedEventTypes().filter { it == ConnectEnd::class }).hasSize(1)
|
||||
assertThat(eventRecorder.recordedEventTypes().filter { it == ConnectStart::class }).hasSize(2)
|
||||
assertThat(eventRecorder.recordedEventTypes().filter { it == ConnectFailed::class }).hasSize(1)
|
||||
assertThat(eventRecorder.recordedEventTypes().filter { it == ConnectEnd::class }).hasSize(1)
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -193,9 +193,9 @@ class FastFallbackTest {
|
||||
assertThat(response.body.string()).isEqualTo("hello from IPv6")
|
||||
|
||||
// In the process we made two connection attempts including one failure.
|
||||
assertThat(listener.recordedEventTypes().filter { it == ConnectStart::class }).hasSize(1)
|
||||
assertThat(listener.recordedEventTypes().filter { it == ConnectEnd::class }).hasSize(1)
|
||||
assertThat(listener.recordedEventTypes().filter { it == ConnectFailed::class }).hasSize(0)
|
||||
assertThat(eventRecorder.recordedEventTypes().filter { it == ConnectStart::class }).hasSize(1)
|
||||
assertThat(eventRecorder.recordedEventTypes().filter { it == ConnectEnd::class }).hasSize(1)
|
||||
assertThat(eventRecorder.recordedEventTypes().filter { it == ConnectFailed::class }).hasSize(0)
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -209,8 +209,8 @@ class FastFallbackTest {
|
||||
}
|
||||
|
||||
// In the process we made two unsuccessful connection attempts.
|
||||
assertThat(listener.recordedEventTypes().filter { it == ConnectStart::class }).hasSize(2)
|
||||
assertThat(listener.recordedEventTypes().filter { it == ConnectFailed::class }).hasSize(2)
|
||||
assertThat(eventRecorder.recordedEventTypes().filter { it == ConnectStart::class }).hasSize(2)
|
||||
assertThat(eventRecorder.recordedEventTypes().filter { it == ConnectFailed::class }).hasSize(2)
|
||||
}
|
||||
|
||||
@RetryingTest(5)
|
||||
@@ -231,8 +231,8 @@ class FastFallbackTest {
|
||||
assertThat(response.body.string()).isEqualTo("hello from IPv4")
|
||||
|
||||
// In the process we made two connection attempts including one failure.
|
||||
assertThat(listener.recordedEventTypes().filter { it == ConnectStart::class }).hasSize(2)
|
||||
assertThat(listener.recordedEventTypes().filter { it == ConnectFailed::class }).hasSize(1)
|
||||
assertThat(eventRecorder.recordedEventTypes().filter { it == ConnectStart::class }).hasSize(2)
|
||||
assertThat(eventRecorder.recordedEventTypes().filter { it == ConnectFailed::class }).hasSize(1)
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
@@ -53,7 +53,7 @@ class RouteFailureTest {
|
||||
@StartStop
|
||||
val server2 = MockWebServer()
|
||||
|
||||
private var listener = RecordingEventListener()
|
||||
private var eventRecorder = EventRecorder()
|
||||
|
||||
private val handshakeCertificates = platform.localhostHandshakeCertificates()
|
||||
|
||||
@@ -78,7 +78,7 @@ class RouteFailureTest {
|
||||
.newClientBuilder()
|
||||
.dns(dns)
|
||||
.socketFactory(socketFactory)
|
||||
.eventListenerFactory(clientTestRule.wrap(listener))
|
||||
.eventListenerFactory(clientTestRule.wrap(eventRecorder))
|
||||
.build()
|
||||
}
|
||||
|
||||
|
||||
@@ -71,13 +71,13 @@ class ServerTruncatesRequestTest {
|
||||
@JvmField
|
||||
var clientTestRule = OkHttpClientTestRule()
|
||||
|
||||
private val listener = RecordingEventListener()
|
||||
private val eventRecorder = EventRecorder()
|
||||
private val handshakeCertificates = platform.localhostHandshakeCertificates()
|
||||
|
||||
private var client =
|
||||
clientTestRule
|
||||
.newClientBuilder()
|
||||
.eventListenerFactory(clientTestRule.wrap(listener))
|
||||
.eventListenerFactory(clientTestRule.wrap(eventRecorder))
|
||||
.build()
|
||||
|
||||
@StartStop
|
||||
@@ -147,7 +147,7 @@ class ServerTruncatesRequestTest {
|
||||
expectedEvents += ResponseBodyEnd::class
|
||||
expectedEvents += ConnectionReleased::class
|
||||
expectedEvents += CallEnd::class
|
||||
assertThat(listener.recordedEventTypes()).isEqualTo(expectedEvents)
|
||||
assertThat(eventRecorder.recordedEventTypes()).isEqualTo(expectedEvents)
|
||||
|
||||
// Confirm that the connection pool was not corrupted by making another call.
|
||||
makeSimpleCall()
|
||||
|
||||
@@ -45,11 +45,11 @@ import okhttp3.CallEvent.RequestFailed
|
||||
import okhttp3.CallEvent.ResponseFailed
|
||||
import okhttp3.DelegatingServerSocketFactory
|
||||
import okhttp3.DelegatingSocketFactory
|
||||
import okhttp3.EventRecorder
|
||||
import okhttp3.MediaType
|
||||
import okhttp3.OkHttpClient
|
||||
import okhttp3.OkHttpClientTestRule
|
||||
import okhttp3.Protocol.HTTP_1_1
|
||||
import okhttp3.RecordingEventListener
|
||||
import okhttp3.Request
|
||||
import okhttp3.RequestBody
|
||||
import okhttp3.SimpleProvider
|
||||
@@ -99,7 +99,7 @@ class CancelTest(
|
||||
private lateinit var server: MockWebServer
|
||||
private lateinit var client: OkHttpClient
|
||||
|
||||
val listener = RecordingEventListener()
|
||||
val eventRecorder = EventRecorder()
|
||||
|
||||
@BeforeEach
|
||||
fun setUp() {
|
||||
@@ -138,7 +138,7 @@ class CancelTest(
|
||||
).sslSocketFactory(
|
||||
handshakeCertificates.sslSocketFactory(),
|
||||
handshakeCertificates.trustManager,
|
||||
).eventListener(listener)
|
||||
).eventListener(eventRecorder.eventListener)
|
||||
.apply {
|
||||
if (connectionType == HTTPS) {
|
||||
protocols(listOf(HTTP_1_1))
|
||||
@@ -237,8 +237,8 @@ class CancelTest(
|
||||
|
||||
cancelLatch.await()
|
||||
|
||||
val events = listener.eventSequence.filter { isConnectionEvent(it) }.map { it.name }
|
||||
listener.clearAllEvents()
|
||||
val events = eventRecorder.eventSequence.filter { isConnectionEvent(it) }.map { it.name }
|
||||
eventRecorder.clearAllEvents()
|
||||
|
||||
assertThat(events).startsWith("CallStart", "ConnectStart", "ConnectEnd", "ConnectionAcquired")
|
||||
if (cancelMode == CANCEL) {
|
||||
@@ -254,7 +254,7 @@ class CancelTest(
|
||||
assertEquals(".", it.body.string())
|
||||
}
|
||||
|
||||
val events2 = listener.eventSequence.filter { isConnectionEvent(it) }.map { it.name }
|
||||
val events2 = eventRecorder.eventSequence.filter { isConnectionEvent(it) }.map { it.name }
|
||||
val expectedEvents2 =
|
||||
mutableListOf<String>().apply {
|
||||
add("CallStart")
|
||||
|
||||
@@ -18,7 +18,6 @@ package okhttp3.internal.http
|
||||
import assertk.assertThat
|
||||
import assertk.assertions.containsExactly
|
||||
import assertk.assertions.isEqualTo
|
||||
import assertk.assertions.isInstanceOf
|
||||
import assertk.assertions.isNull
|
||||
import assertk.assertions.isTrue
|
||||
import kotlin.test.assertFailsWith
|
||||
@@ -44,12 +43,10 @@ import okhttp3.CallEvent.ResponseBodyEnd
|
||||
import okhttp3.CallEvent.ResponseBodyStart
|
||||
import okhttp3.CallEvent.ResponseHeadersEnd
|
||||
import okhttp3.CallEvent.ResponseHeadersStart
|
||||
import okhttp3.CallEvent.SecureConnectEnd
|
||||
import okhttp3.CallEvent.SecureConnectStart
|
||||
import okhttp3.EventRecorder
|
||||
import okhttp3.Headers.Companion.headersOf
|
||||
import okhttp3.OkHttpClientTestRule
|
||||
import okhttp3.Protocol
|
||||
import okhttp3.RecordingEventListener
|
||||
import okhttp3.RecordingHostnameVerifier
|
||||
import okhttp3.Request
|
||||
import okhttp3.RequestBody
|
||||
@@ -72,12 +69,12 @@ class HttpUpgradesTest {
|
||||
@StartStop
|
||||
private val server = MockWebServer()
|
||||
|
||||
private var listener = RecordingEventListener()
|
||||
private var eventRecorder = EventRecorder()
|
||||
private val handshakeCertificates = platform.localhostHandshakeCertificates()
|
||||
private var client =
|
||||
clientTestRule
|
||||
.newClientBuilder()
|
||||
.eventListenerFactory(clientTestRule.wrap(listener))
|
||||
.eventListenerFactory(clientTestRule.wrap(eventRecorder))
|
||||
.build()
|
||||
|
||||
fun executeAndCheckUpgrade(request: Request) {
|
||||
@@ -159,7 +156,7 @@ class HttpUpgradesTest {
|
||||
assertThat(response.body.string()).isEqualTo("normal request")
|
||||
}
|
||||
// Confirm there's no RequestBodyStart/RequestBodyEnd on failed upgrades.
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -226,7 +223,7 @@ class HttpUpgradesTest {
|
||||
fun upgradeEventsWithoutRequestBody() {
|
||||
upgrade()
|
||||
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -253,7 +250,7 @@ class HttpUpgradesTest {
|
||||
fun upgradeEventsWithEmptyRequestBody() {
|
||||
upgradeWithEmptyRequestBody()
|
||||
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
@@ -282,7 +279,7 @@ class HttpUpgradesTest {
|
||||
fun upgradeEventsWithNonEmptyRequestBody() {
|
||||
upgradeWithNonEmptyRequestBody()
|
||||
|
||||
assertThat(listener.recordedEventTypes()).containsExactly(
|
||||
assertThat(eventRecorder.recordedEventTypes()).containsExactly(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
ProxySelectEnd::class,
|
||||
|
||||
@@ -46,9 +46,9 @@ import okhttp3.CallEvent.DnsStart
|
||||
import okhttp3.CallEvent.ProxySelectEnd
|
||||
import okhttp3.CallEvent.ProxySelectStart
|
||||
import okhttp3.CallEvent.SecureConnectStart
|
||||
import okhttp3.EventRecorder
|
||||
import okhttp3.OkHttpClient
|
||||
import okhttp3.OkHttpClientTestRule
|
||||
import okhttp3.RecordingEventListener
|
||||
import okhttp3.Request
|
||||
import okhttp3.internal.http2.ConnectionShutdownException
|
||||
import okhttp3.testing.Flaky
|
||||
@@ -329,11 +329,11 @@ class ClientAuthTest {
|
||||
.validityInterval(1, 2)
|
||||
.build()
|
||||
var client = buildClient(clientCert, clientIntermediateCa.certificate)
|
||||
val listener = RecordingEventListener()
|
||||
val eventRecorder = EventRecorder()
|
||||
client =
|
||||
client
|
||||
.newBuilder()
|
||||
.eventListener(listener)
|
||||
.eventListener(eventRecorder.eventListener)
|
||||
.build()
|
||||
val socketFactory = buildServerSslSocketFactory()
|
||||
server.useHttps(socketFactory)
|
||||
@@ -354,7 +354,7 @@ class ClientAuthTest {
|
||||
// Gradle - JDK 11
|
||||
// CallStart, ProxySelectStart, ProxySelectEnd, DnsStart, DnsEnd, ConnectStart, SecureConnectStart,
|
||||
// SecureConnectEnd, ConnectFailed, CallFailed
|
||||
val recordedEventTypes = listener.recordedEventTypes()
|
||||
val recordedEventTypes = eventRecorder.recordedEventTypes()
|
||||
assertThat(recordedEventTypes).startsWith(
|
||||
CallStart::class,
|
||||
ProxySelectStart::class,
|
||||
|
||||
@@ -46,11 +46,11 @@ import mockwebserver3.RecordedRequest
|
||||
import mockwebserver3.SocketEffect.CloseSocket
|
||||
import mockwebserver3.SocketEffect.Stall
|
||||
import mockwebserver3.junit5.StartStop
|
||||
import okhttp3.EventRecorder
|
||||
import okhttp3.Interceptor
|
||||
import okhttp3.OkHttpClient
|
||||
import okhttp3.OkHttpClientTestRule
|
||||
import okhttp3.Protocol
|
||||
import okhttp3.RecordingEventListener
|
||||
import okhttp3.RecordingHostnameVerifier
|
||||
import okhttp3.Request
|
||||
import okhttp3.Response
|
||||
@@ -938,11 +938,11 @@ class WebSocketHttpTest {
|
||||
|
||||
@Test
|
||||
fun webSocketsDontTriggerEventListener() {
|
||||
val listener = RecordingEventListener()
|
||||
val eventRecorder = EventRecorder()
|
||||
client =
|
||||
client
|
||||
.newBuilder()
|
||||
.eventListenerFactory(clientTestRule.wrap(listener))
|
||||
.eventListenerFactory(clientTestRule.wrap(eventRecorder))
|
||||
.build()
|
||||
webServer.enqueue(
|
||||
MockResponse
|
||||
@@ -961,7 +961,7 @@ class WebSocketHttpTest {
|
||||
clientListener.assertClosing(1000, "")
|
||||
clientListener.assertClosed(1000, "")
|
||||
serverListener.assertClosed(1000, "")
|
||||
assertThat(listener.recordedEventTypes()).isEmpty()
|
||||
assertThat(eventRecorder.recordedEventTypes()).isEmpty()
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
Reference in New Issue
Block a user