HTTPCLIENT-1827: Asynchronous cache exec interceptor with caching support for streamed HTTP exchanges; removed incomplete response checks as response integrity is enforced in the transport layer; async cache re-validation is currently broken in the classic and unsuppoted in the async implementations
This commit is contained in:
parent
6076f5542b
commit
849d1a138e
695
httpclient5-cache/src/main/java/org/apache/hc/client5/http/impl/cache/AsyncCachingExec.java
vendored
Normal file
695
httpclient5-cache/src/main/java/org/apache/hc/client5/http/impl/cache/AsyncCachingExec.java
vendored
Normal file
|
@ -0,0 +1,695 @@
|
|||
/*
|
||||
* ====================================================================
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you 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.
|
||||
* ====================================================================
|
||||
*
|
||||
* This software consists of voluntary contributions made by many
|
||||
* individuals on behalf of the Apache Software Foundation. For more
|
||||
* information on the Apache Software Foundation, please see
|
||||
* <http://www.apache.org/>.
|
||||
*
|
||||
*/
|
||||
package org.apache.hc.client5.http.impl.cache;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import org.apache.hc.client5.http.HttpRoute;
|
||||
import org.apache.hc.client5.http.async.AsyncExecCallback;
|
||||
import org.apache.hc.client5.http.async.AsyncExecChain;
|
||||
import org.apache.hc.client5.http.async.AsyncExecChainHandler;
|
||||
import org.apache.hc.client5.http.async.methods.SimpleBody;
|
||||
import org.apache.hc.client5.http.async.methods.SimpleHttpResponse;
|
||||
import org.apache.hc.client5.http.cache.CacheResponseStatus;
|
||||
import org.apache.hc.client5.http.cache.HeaderConstants;
|
||||
import org.apache.hc.client5.http.cache.HttpCacheEntry;
|
||||
import org.apache.hc.client5.http.cache.HttpCacheStorage;
|
||||
import org.apache.hc.client5.http.cache.ResourceFactory;
|
||||
import org.apache.hc.client5.http.cache.ResourceIOException;
|
||||
import org.apache.hc.client5.http.impl.RequestCopier;
|
||||
import org.apache.hc.client5.http.protocol.HttpClientContext;
|
||||
import org.apache.hc.core5.annotation.Contract;
|
||||
import org.apache.hc.core5.annotation.ThreadingBehavior;
|
||||
import org.apache.hc.core5.http.ContentType;
|
||||
import org.apache.hc.core5.http.EntityDetails;
|
||||
import org.apache.hc.core5.http.Header;
|
||||
import org.apache.hc.core5.http.HttpException;
|
||||
import org.apache.hc.core5.http.HttpHost;
|
||||
import org.apache.hc.core5.http.HttpRequest;
|
||||
import org.apache.hc.core5.http.HttpResponse;
|
||||
import org.apache.hc.core5.http.HttpStatus;
|
||||
import org.apache.hc.core5.http.impl.BasicEntityDetails;
|
||||
import org.apache.hc.core5.http.nio.AsyncDataConsumer;
|
||||
import org.apache.hc.core5.http.nio.AsyncEntityProducer;
|
||||
import org.apache.hc.core5.http.nio.CapacityChannel;
|
||||
import org.apache.hc.core5.http.protocol.HttpCoreContext;
|
||||
import org.apache.hc.core5.net.URIAuthority;
|
||||
import org.apache.hc.core5.util.Args;
|
||||
import org.apache.hc.core5.util.ByteArrayBuffer;
|
||||
|
||||
/**
|
||||
* Request executor in the request execution chain that is responsible for
|
||||
* transparent client-side caching.
|
||||
* <p>
|
||||
* The current implementation is conditionally
|
||||
* compliant with HTTP/1.1 (meaning all the MUST and MUST NOTs are obeyed),
|
||||
* although quite a lot, though not all, of the SHOULDs and SHOULD NOTs
|
||||
* are obeyed too.
|
||||
*
|
||||
* @since 5.0
|
||||
*/
|
||||
@Contract(threading = ThreadingBehavior.SAFE) // So long as the responseCache implementation is threadsafe
|
||||
public class AsyncCachingExec extends CachingExecBase implements AsyncExecChainHandler {
|
||||
|
||||
private final ConditionalRequestBuilder<HttpRequest> conditionalRequestBuilder;
|
||||
public AsyncCachingExec(
|
||||
final HttpCache cache,
|
||||
final CacheConfig config) {
|
||||
super(cache, config);
|
||||
this.conditionalRequestBuilder = new ConditionalRequestBuilder<>(RequestCopier.INSTANCE);
|
||||
}
|
||||
|
||||
public AsyncCachingExec(
|
||||
final ResourceFactory resourceFactory,
|
||||
final HttpCacheStorage storage,
|
||||
final CacheConfig config) {
|
||||
this(new BasicHttpCache(resourceFactory, storage), config);
|
||||
}
|
||||
|
||||
public AsyncCachingExec() {
|
||||
this(new BasicHttpCache(), CacheConfig.DEFAULT);
|
||||
}
|
||||
|
||||
AsyncCachingExec(
|
||||
final HttpCache responseCache,
|
||||
final CacheValidityPolicy validityPolicy,
|
||||
final ResponseCachingPolicy responseCachingPolicy,
|
||||
final CachedHttpResponseGenerator responseGenerator,
|
||||
final CacheableRequestPolicy cacheableRequestPolicy,
|
||||
final CachedResponseSuitabilityChecker suitabilityChecker,
|
||||
final ConditionalRequestBuilder<HttpRequest> conditionalRequestBuilder,
|
||||
final ResponseProtocolCompliance responseCompliance,
|
||||
final RequestProtocolCompliance requestCompliance,
|
||||
final CacheConfig config) {
|
||||
super(responseCache, validityPolicy, responseCachingPolicy, responseGenerator, cacheableRequestPolicy,
|
||||
suitabilityChecker, responseCompliance, requestCompliance, config);
|
||||
this.conditionalRequestBuilder = conditionalRequestBuilder;
|
||||
}
|
||||
|
||||
private void triggerResponse(
|
||||
final SimpleHttpResponse cacheResponse,
|
||||
final AsyncExecChain.Scope scope,
|
||||
final AsyncExecCallback asyncExecCallback) {
|
||||
scope.clientContext.setAttribute(HttpCoreContext.HTTP_RESPONSE, cacheResponse);
|
||||
scope.execRuntime.releaseConnection();
|
||||
|
||||
final SimpleBody body = cacheResponse.getBody();
|
||||
final byte[] content = body != null ? body.getBodyBytes() : null;
|
||||
final ContentType contentType = body != null ? body.getContentType() : null;
|
||||
try {
|
||||
final AsyncDataConsumer dataConsumer = asyncExecCallback.handleResponse(
|
||||
cacheResponse,
|
||||
content != null ? new BasicEntityDetails(content.length, contentType) : null);
|
||||
if (dataConsumer != null) {
|
||||
dataConsumer.consume(ByteBuffer.wrap(content));
|
||||
dataConsumer.streamEnd(null);
|
||||
}
|
||||
} catch (final HttpException | IOException ex) {
|
||||
asyncExecCallback.failed(ex);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(
|
||||
final HttpRequest request,
|
||||
final AsyncEntityProducer entityProducer,
|
||||
final AsyncExecChain.Scope scope,
|
||||
final AsyncExecChain chain,
|
||||
final AsyncExecCallback asyncExecCallback) throws HttpException, IOException {
|
||||
Args.notNull(request, "HTTP request");
|
||||
Args.notNull(scope, "Scope");
|
||||
|
||||
final HttpRoute route = scope.route;
|
||||
final HttpClientContext context = scope.clientContext;
|
||||
context.setAttribute(HttpClientContext.HTTP_ROUTE, route);
|
||||
context.setAttribute(HttpClientContext.HTTP_REQUEST, request);
|
||||
|
||||
final URIAuthority authority = request.getAuthority();
|
||||
final String scheme = request.getScheme();
|
||||
final HttpHost target = authority != null ? new HttpHost(authority, scheme) : route.getTargetHost();;
|
||||
final String via = generateViaHeader(request);
|
||||
|
||||
// default response context
|
||||
setResponseStatus(context, CacheResponseStatus.CACHE_MISS);
|
||||
|
||||
if (clientRequestsOurOptions(request)) {
|
||||
setResponseStatus(context, CacheResponseStatus.CACHE_MODULE_RESPONSE);
|
||||
triggerResponse(SimpleHttpResponse.create(HttpStatus.SC_NOT_IMPLEMENTED), scope, asyncExecCallback);
|
||||
return;
|
||||
}
|
||||
|
||||
final SimpleHttpResponse fatalErrorResponse = getFatallyNoncompliantResponse(request, context);
|
||||
if (fatalErrorResponse != null) {
|
||||
triggerResponse(fatalErrorResponse, scope, asyncExecCallback);
|
||||
return;
|
||||
}
|
||||
|
||||
requestCompliance.makeRequestCompliant(request);
|
||||
request.addHeader("Via",via);
|
||||
|
||||
if (!cacheableRequestPolicy.isServableFromCache(request)) {
|
||||
log.debug("Request is not servable from cache");
|
||||
flushEntriesInvalidatedByRequest(target, request);
|
||||
callBackend(target, request, entityProducer, scope, chain, asyncExecCallback);
|
||||
} else {
|
||||
final HttpCacheEntry entry = satisfyFromCache(target, request);
|
||||
if (entry == null) {
|
||||
log.debug("Cache miss");
|
||||
handleCacheMiss(target, request, entityProducer, scope, chain, asyncExecCallback);
|
||||
} else {
|
||||
handleCacheHit(target, request, entityProducer, scope, chain, asyncExecCallback, entry);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
interface InternalCallback extends AsyncExecCallback {
|
||||
|
||||
boolean cacheResponse(HttpResponse backendResponse) throws HttpException, IOException;
|
||||
|
||||
}
|
||||
|
||||
void callBackend(
|
||||
final HttpHost target,
|
||||
final HttpRequest request,
|
||||
final AsyncEntityProducer entityProducer,
|
||||
final AsyncExecChain.Scope scope,
|
||||
final AsyncExecChain chain,
|
||||
final AsyncExecCallback asyncExecCallback) throws HttpException, IOException {
|
||||
callBackendInternal(target, request, entityProducer, scope, chain, new InternalCallback() {
|
||||
|
||||
@Override
|
||||
public boolean cacheResponse(final HttpResponse backendResponse) {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncDataConsumer handleResponse(
|
||||
final HttpResponse response, final EntityDetails entityDetails) throws HttpException, IOException {
|
||||
return asyncExecCallback.handleResponse(response, entityDetails);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void completed() {
|
||||
asyncExecCallback.completed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void failed(final Exception cause) {
|
||||
asyncExecCallback.failed(cause);
|
||||
}
|
||||
|
||||
});
|
||||
}
|
||||
|
||||
void callBackendInternal(
|
||||
final HttpHost target,
|
||||
final HttpRequest request,
|
||||
final AsyncEntityProducer entityProducer,
|
||||
final AsyncExecChain.Scope scope,
|
||||
final AsyncExecChain chain,
|
||||
final InternalCallback asyncExecCallback) throws HttpException, IOException {
|
||||
log.trace("Calling the backend");
|
||||
final Date requestDate = getCurrentDate();
|
||||
chain.proceed(request, entityProducer, scope, new AsyncExecCallback() {
|
||||
|
||||
private final AtomicReference<ByteArrayBuffer> bufferRef = new AtomicReference<>();
|
||||
private final AtomicReference<AsyncDataConsumer> dataConsumerRef = new AtomicReference<>();
|
||||
private final AtomicReference<SimpleHttpResponse> responseRef = new AtomicReference<>();
|
||||
|
||||
@Override
|
||||
public AsyncDataConsumer handleResponse(
|
||||
final HttpResponse backendResponse,
|
||||
final EntityDetails entityDetails) throws HttpException, IOException {
|
||||
final Date responseDate = getCurrentDate();
|
||||
backendResponse.addHeader("Via", generateViaHeader(backendResponse));
|
||||
|
||||
log.trace("Handling Backend response");
|
||||
responseCompliance.ensureProtocolCompliance(scope.originalRequest, request, backendResponse);
|
||||
|
||||
final boolean cacheable = asyncExecCallback.cacheResponse(backendResponse)
|
||||
&& responseCachingPolicy.isResponseCacheable(request, backendResponse);
|
||||
responseCache.flushInvalidatedCacheEntriesFor(target, request, backendResponse);
|
||||
if (cacheable) {
|
||||
if (!alreadyHaveNewerCacheEntry(target, request, backendResponse)) {
|
||||
storeRequestIfModifiedSinceFor304Response(request, backendResponse);
|
||||
bufferRef.set(new ByteArrayBuffer(1024));
|
||||
}
|
||||
} else {
|
||||
try {
|
||||
responseCache.flushCacheEntriesFor(target, request);
|
||||
} catch (final IOException ioe) {
|
||||
log.warn("Unable to flush invalid cache entries", ioe);
|
||||
}
|
||||
}
|
||||
if (bufferRef.get() != null) {
|
||||
if (entityDetails == null) {
|
||||
scope.execRuntime.releaseConnection();
|
||||
final HttpCacheEntry entry = responseCache.createCacheEntry(
|
||||
target, request, backendResponse, null, requestDate, responseDate);
|
||||
responseRef.set(responseGenerator.generateResponse(request, entry));
|
||||
return null;
|
||||
} else {
|
||||
return new AsyncDataConsumer() {
|
||||
|
||||
@Override
|
||||
public final void updateCapacity(final CapacityChannel capacityChannel) throws IOException {
|
||||
final AsyncDataConsumer dataConsumer = dataConsumerRef.get();
|
||||
if (dataConsumer != null) {
|
||||
dataConsumer.updateCapacity(capacityChannel);
|
||||
} else {
|
||||
capacityChannel.update(Integer.MAX_VALUE);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final int consume(final ByteBuffer src) throws IOException {
|
||||
final ByteArrayBuffer buffer = bufferRef.get();
|
||||
if (buffer != null) {
|
||||
if (src.hasArray()) {
|
||||
buffer.append(src.array(), src.arrayOffset() + src.position(), src.remaining());
|
||||
} else {
|
||||
while (src.hasRemaining()) {
|
||||
buffer.append(src.get());
|
||||
}
|
||||
}
|
||||
if (buffer.length() > cacheConfig.getMaxObjectSize()) {
|
||||
// Over the max limit. Stop buffering and forward the response
|
||||
// along with all the data buffered so far to the caller.
|
||||
bufferRef.set(null);
|
||||
try {
|
||||
final AsyncDataConsumer dataConsumer = asyncExecCallback.handleResponse(
|
||||
backendResponse, entityDetails);
|
||||
if (dataConsumer != null) {
|
||||
dataConsumerRef.set(dataConsumer);
|
||||
return dataConsumer.consume(ByteBuffer.wrap(buffer.array(), 0, buffer.length()));
|
||||
}
|
||||
} catch (final HttpException ex) {
|
||||
asyncExecCallback.failed(ex);
|
||||
}
|
||||
}
|
||||
return Integer.MAX_VALUE;
|
||||
} else {
|
||||
final AsyncDataConsumer dataConsumer = dataConsumerRef.get();
|
||||
if (dataConsumer != null) {
|
||||
return dataConsumer.consume(src);
|
||||
} else {
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void streamEnd(final List<? extends Header> trailers) throws HttpException, IOException {
|
||||
scope.execRuntime.releaseConnection();
|
||||
final AsyncDataConsumer dataConsumer = dataConsumerRef.getAndSet(null);
|
||||
if (dataConsumer != null) {
|
||||
dataConsumer.streamEnd(trailers);
|
||||
}
|
||||
final ByteArrayBuffer buffer = bufferRef.getAndSet(null);
|
||||
if (buffer != null) {
|
||||
final HttpCacheEntry entry = responseCache.createCacheEntry(
|
||||
target, request, backendResponse, buffer, requestDate, responseDate);
|
||||
responseRef.set(responseGenerator.generateResponse(request, entry));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void releaseResources() {
|
||||
final AsyncDataConsumer dataConsumer = dataConsumerRef.getAndSet(null);
|
||||
if (dataConsumer != null) {
|
||||
dataConsumer.releaseResources();
|
||||
}
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
} else {
|
||||
return asyncExecCallback.handleResponse(backendResponse, entityDetails);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void completed() {
|
||||
final SimpleHttpResponse response = responseRef.getAndSet(null);
|
||||
if (response != null) {
|
||||
triggerResponse(response, scope, asyncExecCallback);
|
||||
} else {
|
||||
asyncExecCallback.completed();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void failed(final Exception cause) {
|
||||
try {
|
||||
scope.execRuntime.discardConnection();
|
||||
} finally {
|
||||
asyncExecCallback.failed(cause);
|
||||
}
|
||||
}
|
||||
|
||||
});
|
||||
}
|
||||
|
||||
private void handleCacheHit(
|
||||
final HttpHost target,
|
||||
final HttpRequest request,
|
||||
final AsyncEntityProducer entityProducer,
|
||||
final AsyncExecChain.Scope scope,
|
||||
final AsyncExecChain chain,
|
||||
final AsyncExecCallback asyncExecCallback,
|
||||
final HttpCacheEntry entry) throws IOException, HttpException {
|
||||
final HttpClientContext context = scope.clientContext;
|
||||
recordCacheHit(target, request);
|
||||
final Date now = getCurrentDate();
|
||||
if (suitabilityChecker.canCachedResponseBeUsed(target, request, entry, now)) {
|
||||
log.debug("Cache hit");
|
||||
try {
|
||||
final SimpleHttpResponse cacheResponse = generateCachedResponse(request, context, entry, now);
|
||||
triggerResponse(cacheResponse, scope, asyncExecCallback);
|
||||
} catch (final ResourceIOException ex) {
|
||||
recordCacheFailure(target, request);
|
||||
if (!mayCallBackend(request)) {
|
||||
final SimpleHttpResponse cacheResponse = generateGatewayTimeout(context);
|
||||
triggerResponse(cacheResponse, scope, asyncExecCallback);
|
||||
} else {
|
||||
setResponseStatus(scope.clientContext, CacheResponseStatus.FAILURE);
|
||||
chain.proceed(request, entityProducer, scope, asyncExecCallback);
|
||||
}
|
||||
}
|
||||
} else if (!mayCallBackend(request)) {
|
||||
log.debug("Cache entry not suitable but only-if-cached requested");
|
||||
final SimpleHttpResponse cacheResponse = generateGatewayTimeout(context);
|
||||
triggerResponse(cacheResponse, scope, asyncExecCallback);
|
||||
} else if (!(entry.getStatus() == HttpStatus.SC_NOT_MODIFIED && !suitabilityChecker.isConditional(request))) {
|
||||
log.debug("Revalidating cache entry");
|
||||
revalidateCacheEntry(target, request, entityProducer, scope, chain, asyncExecCallback, entry);
|
||||
} else {
|
||||
log.debug("Cache entry not usable; calling backend");
|
||||
callBackend(target, request, entityProducer, scope, chain, asyncExecCallback);
|
||||
}
|
||||
}
|
||||
|
||||
void revalidateCacheEntry(
|
||||
final HttpHost target,
|
||||
final HttpRequest request,
|
||||
final AsyncEntityProducer entityProducer,
|
||||
final AsyncExecChain.Scope scope,
|
||||
final AsyncExecChain chain,
|
||||
final AsyncExecCallback asyncExecCallback,
|
||||
final HttpCacheEntry cacheEntry) throws IOException, HttpException {
|
||||
|
||||
final Date requestDate = getCurrentDate();
|
||||
final InternalCallback internalCallback = new InternalCallback() {
|
||||
|
||||
private final AtomicReference<Date> responseDateRef = new AtomicReference<>(null);
|
||||
private final AtomicReference<HttpResponse> backendResponseRef = new AtomicReference<>(null);
|
||||
|
||||
@Override
|
||||
public boolean cacheResponse(final HttpResponse backendResponse) throws IOException {
|
||||
final Date responseDate = getCurrentDate();
|
||||
responseDateRef.set(requestDate);
|
||||
final int statusCode = backendResponse.getCode();
|
||||
if (statusCode == HttpStatus.SC_NOT_MODIFIED || statusCode == HttpStatus.SC_OK) {
|
||||
recordCacheUpdate(scope.clientContext);
|
||||
}
|
||||
if (statusCode == HttpStatus.SC_NOT_MODIFIED) {
|
||||
backendResponseRef.set(backendResponse);
|
||||
return false;
|
||||
}
|
||||
if (staleIfErrorAppliesTo(statusCode)
|
||||
&& !staleResponseNotAllowed(request, cacheEntry, getCurrentDate())
|
||||
&& validityPolicy.mayReturnStaleIfError(request, cacheEntry, responseDate)) {
|
||||
backendResponseRef.set(backendResponse);
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncDataConsumer handleResponse(
|
||||
final HttpResponse response, final EntityDetails entityDetails) throws HttpException, IOException {
|
||||
if (backendResponseRef.get() == null) {
|
||||
return asyncExecCallback.handleResponse(response, entityDetails);
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void completed() {
|
||||
final HttpResponse backendResponse = backendResponseRef.getAndSet(null);
|
||||
if (backendResponse != null) {
|
||||
final int statusCode = backendResponse.getCode();
|
||||
try {
|
||||
if (statusCode == HttpStatus.SC_NOT_MODIFIED) {
|
||||
final HttpCacheEntry updatedEntry = responseCache.updateCacheEntry(
|
||||
target, request, cacheEntry, backendResponse, requestDate, responseDateRef.get());
|
||||
if (suitabilityChecker.isConditional(request)
|
||||
&& suitabilityChecker.allConditionalsMatch(request, updatedEntry, new Date())) {
|
||||
final SimpleHttpResponse cacheResponse = responseGenerator.generateNotModifiedResponse(updatedEntry);
|
||||
triggerResponse(cacheResponse, scope, asyncExecCallback);
|
||||
} else {
|
||||
final SimpleHttpResponse cacheResponse = responseGenerator.generateResponse(request, updatedEntry);
|
||||
triggerResponse(cacheResponse, scope, asyncExecCallback);
|
||||
}
|
||||
} else if (staleIfErrorAppliesTo(statusCode)) {
|
||||
final SimpleHttpResponse cacheResponse = responseGenerator.generateResponse(request, cacheEntry);
|
||||
cacheResponse.addHeader(HeaderConstants.WARNING, "110 localhost \"Response is stale\"");
|
||||
triggerResponse(cacheResponse, scope, asyncExecCallback);
|
||||
}
|
||||
} catch (final IOException ex) {
|
||||
asyncExecCallback.failed(ex);
|
||||
}
|
||||
} else {
|
||||
asyncExecCallback.completed();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void failed(final Exception cause) {
|
||||
asyncExecCallback.failed(cause);
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
final HttpRequest conditionalRequest = conditionalRequestBuilder.buildConditionalRequest(
|
||||
scope.originalRequest, cacheEntry);
|
||||
callBackendInternal(target, conditionalRequest, entityProducer, scope, chain, new InternalCallback() {
|
||||
|
||||
private final AtomicBoolean revalidate = new AtomicBoolean(false);
|
||||
|
||||
@Override
|
||||
public boolean cacheResponse(final HttpResponse backendResponse) throws HttpException, IOException {
|
||||
if (revalidationResponseIsTooOld(backendResponse, cacheEntry)) {
|
||||
revalidate.set(true);
|
||||
return false;
|
||||
} else {
|
||||
return internalCallback.cacheResponse(backendResponse);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncDataConsumer handleResponse(
|
||||
final HttpResponse response,
|
||||
final EntityDetails entityDetails) throws HttpException, IOException {
|
||||
if (revalidate.get()) {
|
||||
return null;
|
||||
} else {
|
||||
return internalCallback.handleResponse(response, entityDetails);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void completed() {
|
||||
if (revalidate.getAndSet(false)) {
|
||||
final HttpRequest unconditionalRequest = conditionalRequestBuilder.buildUnconditionalRequest(scope.originalRequest);
|
||||
try {
|
||||
callBackendInternal(target, unconditionalRequest, entityProducer, scope, chain, new InternalCallback() {
|
||||
|
||||
@Override
|
||||
public boolean cacheResponse(final HttpResponse backendResponse) throws HttpException, IOException {
|
||||
return internalCallback.cacheResponse(backendResponse);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncDataConsumer handleResponse(
|
||||
final HttpResponse response, final EntityDetails entityDetails) throws HttpException, IOException {
|
||||
return internalCallback.handleResponse(response, entityDetails);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void completed() {
|
||||
internalCallback.completed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void failed(final Exception cause) {
|
||||
internalCallback.failed(cause);
|
||||
}
|
||||
|
||||
});
|
||||
} catch (final HttpException | IOException ex) {
|
||||
internalCallback.failed(ex);
|
||||
}
|
||||
} else {
|
||||
internalCallback.completed();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void failed(final Exception cause) {
|
||||
internalCallback.failed(cause);
|
||||
}
|
||||
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
private void handleCacheMiss(
|
||||
final HttpHost target,
|
||||
final HttpRequest request,
|
||||
final AsyncEntityProducer entityProducer,
|
||||
final AsyncExecChain.Scope scope,
|
||||
final AsyncExecChain chain,
|
||||
final AsyncExecCallback asyncExecCallback) throws IOException, HttpException {
|
||||
recordCacheMiss(target, request);
|
||||
|
||||
if (!mayCallBackend(request)) {
|
||||
final SimpleHttpResponse cacheResponse = SimpleHttpResponse.create(HttpStatus.SC_GATEWAY_TIMEOUT, "Gateway Timeout");
|
||||
triggerResponse(cacheResponse, scope, asyncExecCallback);
|
||||
return;
|
||||
}
|
||||
|
||||
final Map<String, Variant> variants = getExistingCacheVariants(target, request);
|
||||
if (variants != null && !variants.isEmpty()) {
|
||||
negotiateResponseFromVariants(target, request, entityProducer, scope, chain, asyncExecCallback, variants);
|
||||
} else {
|
||||
callBackend(target, request, entityProducer, scope, chain, asyncExecCallback);
|
||||
}
|
||||
}
|
||||
|
||||
void negotiateResponseFromVariants(
|
||||
final HttpHost target,
|
||||
final HttpRequest request,
|
||||
final AsyncEntityProducer entityProducer,
|
||||
final AsyncExecChain.Scope scope,
|
||||
final AsyncExecChain chain,
|
||||
final AsyncExecCallback asyncExecCallback,
|
||||
final Map<String, Variant> variants) throws IOException, HttpException {
|
||||
final HttpRequest conditionalRequest = conditionalRequestBuilder.buildConditionalRequestFromVariants(
|
||||
request, variants);
|
||||
|
||||
final Date requestDate = getCurrentDate();
|
||||
callBackendInternal(target, conditionalRequest, entityProducer, scope, chain, new InternalCallback() {
|
||||
|
||||
private final AtomicReference<Date> responseDateRef = new AtomicReference<>(null);
|
||||
private final AtomicReference<HttpResponse> backendResponseRef = new AtomicReference<>(null);
|
||||
|
||||
@Override
|
||||
public boolean cacheResponse(final HttpResponse backendResponse) throws IOException {
|
||||
responseDateRef.set(getCurrentDate());
|
||||
if (backendResponse.getCode() == HttpStatus.SC_NOT_MODIFIED) {
|
||||
backendResponseRef.set(backendResponse);
|
||||
return false;
|
||||
} else {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncDataConsumer handleResponse(
|
||||
final HttpResponse response, final EntityDetails entityDetails) throws HttpException, IOException {
|
||||
return asyncExecCallback.handleResponse(response, entityDetails);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void completed() {
|
||||
final HttpResponse backendResponse = backendResponseRef.getAndSet(null);
|
||||
if (backendResponse != null) {
|
||||
try {
|
||||
final Header resultEtagHeader = backendResponse.getFirstHeader(HeaderConstants.ETAG);
|
||||
if (resultEtagHeader == null) {
|
||||
log.warn("304 response did not contain ETag");
|
||||
callBackend(target, request, entityProducer, scope, chain, asyncExecCallback);
|
||||
return;
|
||||
}
|
||||
final String resultEtag = resultEtagHeader.getValue();
|
||||
final Variant matchingVariant = variants.get(resultEtag);
|
||||
if (matchingVariant == null) {
|
||||
log.debug("304 response did not contain ETag matching one sent in If-None-Match");
|
||||
callBackend(target, request, entityProducer, scope, chain, asyncExecCallback);
|
||||
return;
|
||||
}
|
||||
final HttpCacheEntry matchedEntry = matchingVariant.getEntry();
|
||||
if (revalidationResponseIsTooOld(backendResponse, matchedEntry)) {
|
||||
final HttpRequest unconditional = conditionalRequestBuilder.buildUnconditionalRequest(request);
|
||||
scope.clientContext.setAttribute(HttpCoreContext.HTTP_REQUEST, unconditional);
|
||||
callBackend(target, unconditional, entityProducer, scope, chain, asyncExecCallback);
|
||||
return;
|
||||
}
|
||||
recordCacheUpdate(scope.clientContext);
|
||||
|
||||
HttpCacheEntry responseEntry = matchedEntry;
|
||||
try {
|
||||
responseEntry = responseCache.updateVariantCacheEntry(target, conditionalRequest,
|
||||
matchedEntry, backendResponse, requestDate, responseDateRef.get(), matchingVariant.getCacheKey());
|
||||
} catch (final IOException ioe) {
|
||||
log.warn("Could not processChallenge cache entry", ioe);
|
||||
}
|
||||
|
||||
if (shouldSendNotModifiedResponse(request, responseEntry)) {
|
||||
final SimpleHttpResponse cacheResponse = responseGenerator.generateNotModifiedResponse(responseEntry);
|
||||
triggerResponse(cacheResponse, scope, asyncExecCallback);
|
||||
} else {
|
||||
final SimpleHttpResponse cacheResponse = responseGenerator.generateResponse(request, responseEntry);
|
||||
tryToUpdateVariantMap(target, request, matchingVariant);
|
||||
triggerResponse(cacheResponse, scope, asyncExecCallback);
|
||||
}
|
||||
} catch (final HttpException | IOException ex) {
|
||||
asyncExecCallback.failed(ex);
|
||||
}
|
||||
} else {
|
||||
asyncExecCallback.completed();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void failed(final Exception cause) {
|
||||
asyncExecCallback.failed(cause);
|
||||
}
|
||||
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -29,18 +29,14 @@ package org.apache.hc.client5.http.impl.cache;
|
|||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.hc.client5.http.HttpRoute;
|
||||
import org.apache.hc.client5.http.async.methods.SimpleBody;
|
||||
import org.apache.hc.client5.http.async.methods.SimpleHttpResponse;
|
||||
import org.apache.hc.client5.http.cache.CacheResponseStatus;
|
||||
import org.apache.hc.client5.http.cache.HeaderConstants;
|
||||
import org.apache.hc.client5.http.cache.HttpCacheContext;
|
||||
import org.apache.hc.client5.http.cache.HttpCacheEntry;
|
||||
import org.apache.hc.client5.http.cache.HttpCacheStorage;
|
||||
import org.apache.hc.client5.http.cache.ResourceFactory;
|
||||
|
@ -49,35 +45,25 @@ import org.apache.hc.client5.http.classic.ExecChain;
|
|||
import org.apache.hc.client5.http.classic.ExecChainHandler;
|
||||
import org.apache.hc.client5.http.impl.classic.ClassicRequestCopier;
|
||||
import org.apache.hc.client5.http.protocol.HttpClientContext;
|
||||
import org.apache.hc.client5.http.utils.DateUtils;
|
||||
import org.apache.hc.core5.annotation.Contract;
|
||||
import org.apache.hc.core5.annotation.ThreadingBehavior;
|
||||
import org.apache.hc.core5.http.ClassicHttpRequest;
|
||||
import org.apache.hc.core5.http.ClassicHttpResponse;
|
||||
import org.apache.hc.core5.http.ContentType;
|
||||
import org.apache.hc.core5.http.Header;
|
||||
import org.apache.hc.core5.http.HeaderElement;
|
||||
import org.apache.hc.core5.http.HttpEntity;
|
||||
import org.apache.hc.core5.http.HttpException;
|
||||
import org.apache.hc.core5.http.HttpHeaders;
|
||||
import org.apache.hc.core5.http.HttpHost;
|
||||
import org.apache.hc.core5.http.HttpMessage;
|
||||
import org.apache.hc.core5.http.HttpRequest;
|
||||
import org.apache.hc.core5.http.HttpResponse;
|
||||
import org.apache.hc.core5.http.HttpStatus;
|
||||
import org.apache.hc.core5.http.HttpVersion;
|
||||
import org.apache.hc.core5.http.ProtocolVersion;
|
||||
import org.apache.hc.core5.http.io.entity.ByteArrayEntity;
|
||||
import org.apache.hc.core5.http.io.entity.EntityUtils;
|
||||
import org.apache.hc.core5.http.io.entity.StringEntity;
|
||||
import org.apache.hc.core5.http.message.BasicClassicHttpResponse;
|
||||
import org.apache.hc.core5.http.message.MessageSupport;
|
||||
import org.apache.hc.core5.http.protocol.HttpContext;
|
||||
import org.apache.hc.core5.http.protocol.HttpCoreContext;
|
||||
import org.apache.hc.core5.net.URIAuthority;
|
||||
import org.apache.hc.core5.util.Args;
|
||||
import org.apache.hc.core5.util.ByteArrayBuffer;
|
||||
import org.apache.hc.core5.util.VersionInfo;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
|
@ -110,56 +96,17 @@ import org.apache.logging.log4j.Logger;
|
|||
* @since 4.3
|
||||
*/
|
||||
@Contract(threading = ThreadingBehavior.SAFE) // So long as the responseCache implementation is threadsafe
|
||||
public class CachingExec implements ExecChainHandler {
|
||||
public class CachingExec extends CachingExecBase implements ExecChainHandler {
|
||||
|
||||
private final static boolean SUPPORTS_RANGE_AND_CONTENT_RANGE_HEADERS = false;
|
||||
|
||||
private final AtomicLong cacheHits = new AtomicLong();
|
||||
private final AtomicLong cacheMisses = new AtomicLong();
|
||||
private final AtomicLong cacheUpdates = new AtomicLong();
|
||||
|
||||
private final Map<ProtocolVersion, String> viaHeaders = new HashMap<>(4);
|
||||
|
||||
private final CacheConfig cacheConfig;
|
||||
private final HttpCache responseCache;
|
||||
private final CacheValidityPolicy validityPolicy;
|
||||
private final CachedHttpResponseGenerator responseGenerator;
|
||||
private final CacheableRequestPolicy cacheableRequestPolicy;
|
||||
private final CachedResponseSuitabilityChecker suitabilityChecker;
|
||||
private final ConditionalRequestBuilder<ClassicHttpRequest> conditionalRequestBuilder;
|
||||
private final ResponseProtocolCompliance responseCompliance;
|
||||
private final RequestProtocolCompliance requestCompliance;
|
||||
private final ResponseCachingPolicy responseCachingPolicy;
|
||||
|
||||
private final AsynchronousValidator asynchRevalidator;
|
||||
|
||||
private final Logger log = LogManager.getLogger(getClass());
|
||||
|
||||
public CachingExec(
|
||||
final HttpCache cache,
|
||||
final CacheConfig config) {
|
||||
this(cache, config, null);
|
||||
}
|
||||
|
||||
public CachingExec(
|
||||
final HttpCache cache,
|
||||
final CacheConfig config,
|
||||
final AsynchronousValidator asynchRevalidator) {
|
||||
super();
|
||||
Args.notNull(cache, "HttpCache");
|
||||
this.cacheConfig = config != null ? config : CacheConfig.DEFAULT;
|
||||
this.responseCache = cache;
|
||||
this.validityPolicy = new CacheValidityPolicy();
|
||||
this.responseGenerator = new CachedHttpResponseGenerator(this.validityPolicy);
|
||||
this.cacheableRequestPolicy = new CacheableRequestPolicy();
|
||||
this.suitabilityChecker = new CachedResponseSuitabilityChecker(this.validityPolicy, this.cacheConfig);
|
||||
super(cache, config);
|
||||
this.conditionalRequestBuilder = new ConditionalRequestBuilder<>(ClassicRequestCopier.INSTANCE);
|
||||
this.responseCompliance = new ResponseProtocolCompliance();
|
||||
this.requestCompliance = new RequestProtocolCompliance(this.cacheConfig.isWeakETagOnPutDeleteAllowed());
|
||||
this.responseCachingPolicy = new ResponseCachingPolicy(
|
||||
this.cacheConfig.getMaxObjectSize(), this.cacheConfig.isSharedCache(),
|
||||
this.cacheConfig.isNeverCacheHTTP10ResponsesWithQuery(), this.cacheConfig.is303CachingEnabled());
|
||||
this.asynchRevalidator = asynchRevalidator;
|
||||
}
|
||||
|
||||
public CachingExec(
|
||||
|
@ -183,46 +130,10 @@ public class CachingExec implements ExecChainHandler {
|
|||
final ConditionalRequestBuilder<ClassicHttpRequest> conditionalRequestBuilder,
|
||||
final ResponseProtocolCompliance responseCompliance,
|
||||
final RequestProtocolCompliance requestCompliance,
|
||||
final CacheConfig config,
|
||||
final AsynchronousValidator asynchRevalidator) {
|
||||
this.cacheConfig = config != null ? config : CacheConfig.DEFAULT;
|
||||
this.responseCache = responseCache;
|
||||
this.validityPolicy = validityPolicy;
|
||||
this.responseCachingPolicy = responseCachingPolicy;
|
||||
this.responseGenerator = responseGenerator;
|
||||
this.cacheableRequestPolicy = cacheableRequestPolicy;
|
||||
this.suitabilityChecker = suitabilityChecker;
|
||||
final CacheConfig config) {
|
||||
super(responseCache, validityPolicy, responseCachingPolicy, responseGenerator, cacheableRequestPolicy,
|
||||
suitabilityChecker, responseCompliance, requestCompliance, config);
|
||||
this.conditionalRequestBuilder = conditionalRequestBuilder;
|
||||
this.responseCompliance = responseCompliance;
|
||||
this.requestCompliance = requestCompliance;
|
||||
this.asynchRevalidator = asynchRevalidator;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reports the number of times that the cache successfully responded
|
||||
* to an {@link HttpRequest} without contacting the origin server.
|
||||
* @return the number of cache hits
|
||||
*/
|
||||
public long getCacheHits() {
|
||||
return cacheHits.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Reports the number of times that the cache contacted the origin
|
||||
* server because it had no appropriate response cached.
|
||||
* @return the number of cache misses
|
||||
*/
|
||||
public long getCacheMisses() {
|
||||
return cacheMisses.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Reports the number of times that the cache was able to satisfy
|
||||
* a response by revalidating an existing but stale cache entry.
|
||||
* @return the number of cache revalidations
|
||||
*/
|
||||
public long getCacheUpdates() {
|
||||
return cacheUpdates.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -235,6 +146,8 @@ public class CachingExec implements ExecChainHandler {
|
|||
|
||||
final HttpRoute route = scope.route;
|
||||
final HttpClientContext context = scope.clientContext;
|
||||
context.setAttribute(HttpClientContext.HTTP_ROUTE, scope.route);
|
||||
context.setAttribute(HttpClientContext.HTTP_REQUEST, request);
|
||||
|
||||
final URIAuthority authority = request.getAuthority();
|
||||
final String scheme = request.getScheme();
|
||||
|
@ -268,12 +181,7 @@ public class CachingExec implements ExecChainHandler {
|
|||
log.debug("Cache miss");
|
||||
return handleCacheMiss(target, request, scope, chain);
|
||||
} else {
|
||||
try {
|
||||
return handleCacheHit(target, request, scope, chain, entry);
|
||||
} catch (final ResourceIOException ex) {
|
||||
log.debug("Cache resource I/O error");
|
||||
return handleCacheFailure(target, request, scope, chain);
|
||||
}
|
||||
return handleCacheHit(target, request, scope, chain, entry);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -322,53 +230,43 @@ public class CachingExec implements ExecChainHandler {
|
|||
final ExecChain.Scope scope,
|
||||
final ExecChain chain,
|
||||
final HttpCacheEntry entry) throws IOException, HttpException {
|
||||
final HttpRoute route = scope.route;
|
||||
final HttpClientContext context = scope.clientContext;
|
||||
context.setAttribute(HttpCoreContext.HTTP_REQUEST, request);
|
||||
recordCacheHit(target, request);
|
||||
ClassicHttpResponse out;
|
||||
final Date now = getCurrentDate();
|
||||
if (suitabilityChecker.canCachedResponseBeUsed(target, request, entry, now)) {
|
||||
log.debug("Cache hit");
|
||||
out = convert(generateCachedResponse(request, context, entry, now));
|
||||
try {
|
||||
final ClassicHttpResponse response = convert(generateCachedResponse(request, context, entry, now));
|
||||
context.setAttribute(HttpCoreContext.HTTP_RESPONSE, response);
|
||||
return response;
|
||||
} catch (final ResourceIOException ex) {
|
||||
recordCacheFailure(target, request);
|
||||
if (!mayCallBackend(request)) {
|
||||
final ClassicHttpResponse response = convert(generateGatewayTimeout(context));
|
||||
context.setAttribute(HttpCoreContext.HTTP_RESPONSE, response);
|
||||
return response;
|
||||
} else {
|
||||
setResponseStatus(scope.clientContext, CacheResponseStatus.FAILURE);
|
||||
return chain.proceed(request, scope);
|
||||
}
|
||||
}
|
||||
} else if (!mayCallBackend(request)) {
|
||||
log.debug("Cache entry not suitable but only-if-cached requested");
|
||||
out = convert(generateGatewayTimeout(context));
|
||||
} else if (!(entry.getStatus() == HttpStatus.SC_NOT_MODIFIED
|
||||
&& !suitabilityChecker.isConditional(request))) {
|
||||
final ClassicHttpResponse response = convert(generateGatewayTimeout(context));
|
||||
context.setAttribute(HttpCoreContext.HTTP_RESPONSE, response);
|
||||
return response;
|
||||
} else if (!(entry.getStatus() == HttpStatus.SC_NOT_MODIFIED && !suitabilityChecker.isConditional(request))) {
|
||||
log.debug("Revalidating cache entry");
|
||||
return revalidateCacheEntry(target, request, scope, chain, entry, now);
|
||||
try {
|
||||
return revalidateCacheEntry(target, request, scope, chain, entry);
|
||||
} catch (final IOException ioex) {
|
||||
return convert(handleRevalidationFailure(request, context, entry, now));
|
||||
}
|
||||
} else {
|
||||
log.debug("Cache entry not usable; calling backend");
|
||||
return callBackend(target, request, scope, chain);
|
||||
}
|
||||
context.setAttribute(HttpClientContext.HTTP_ROUTE, route);
|
||||
context.setAttribute(HttpCoreContext.HTTP_REQUEST, request);
|
||||
context.setAttribute(HttpCoreContext.HTTP_RESPONSE, out);
|
||||
return out;
|
||||
}
|
||||
|
||||
private ClassicHttpResponse revalidateCacheEntry(
|
||||
final HttpHost target,
|
||||
final ClassicHttpRequest request,
|
||||
final ExecChain.Scope scope,
|
||||
final ExecChain chain,
|
||||
final HttpCacheEntry entry,
|
||||
final Date now) throws HttpException, IOException {
|
||||
|
||||
final HttpClientContext context = scope.clientContext;
|
||||
try {
|
||||
if (asynchRevalidator != null
|
||||
&& !staleResponseNotAllowed(request, entry, now)
|
||||
&& validityPolicy.mayReturnStaleWhileRevalidating(entry, now)) {
|
||||
log.trace("Serving stale with asynchronous revalidation");
|
||||
final SimpleHttpResponse resp = generateCachedResponse(request, context, entry, now);
|
||||
asynchRevalidator.revalidateCacheEntry(this, target, request, scope, chain, entry);
|
||||
return convert(resp);
|
||||
}
|
||||
return revalidateCacheEntry(target, request, scope, chain, entry);
|
||||
} catch (final IOException ioex) {
|
||||
return convert(handleRevalidationFailure(request, context, entry, now));
|
||||
}
|
||||
}
|
||||
|
||||
ClassicHttpResponse revalidateCacheEntry(
|
||||
|
@ -463,7 +361,8 @@ public class CachingExec implements ExecChainHandler {
|
|||
final HttpRequest request,
|
||||
final ClassicHttpResponse backendResponse,
|
||||
final Date requestSent,
|
||||
final Date responseReceived) throws IOException { final ByteArrayBuffer buf;
|
||||
final Date responseReceived) throws IOException {
|
||||
final ByteArrayBuffer buf;
|
||||
final HttpEntity entity = backendResponse.getEntity();
|
||||
if (entity != null) {
|
||||
buf = new ByteArrayBuffer(1024);
|
||||
|
@ -482,16 +381,6 @@ public class CachingExec implements ExecChainHandler {
|
|||
} else {
|
||||
buf = null;
|
||||
}
|
||||
if (buf != null && isIncompleteResponse(backendResponse, buf)) {
|
||||
final Header h = backendResponse.getFirstHeader(HttpHeaders.CONTENT_LENGTH);
|
||||
final ClassicHttpResponse error = new BasicClassicHttpResponse(HttpStatus.SC_BAD_GATEWAY, "Bad Gateway");
|
||||
final String msg = String.format("Received incomplete response " +
|
||||
"with Content-Length %s but actual body length %d",
|
||||
h != null ? h.getValue() : null, buf.length());
|
||||
error.setEntity(new StringEntity(msg, ContentType.TEXT_PLAIN));
|
||||
backendResponse.close();
|
||||
return error;
|
||||
}
|
||||
backendResponse.close();
|
||||
final HttpCacheEntry entry = responseCache.createCacheEntry(target, request, backendResponse, buf, requestSent, responseReceived);
|
||||
return convert(responseGenerator.generateResponse(request, entry));
|
||||
|
@ -574,12 +463,11 @@ public class CachingExec implements ExecChainHandler {
|
|||
backendResponse.close();
|
||||
}
|
||||
|
||||
final SimpleHttpResponse resp = responseGenerator.generateResponse(request, responseEntry);
|
||||
tryToUpdateVariantMap(target, request, matchingVariant);
|
||||
|
||||
if (shouldSendNotModifiedResponse(request, responseEntry)) {
|
||||
return convert(responseGenerator.generateNotModifiedResponse(responseEntry));
|
||||
}
|
||||
final SimpleHttpResponse resp = responseGenerator.generateResponse(request, responseEntry);
|
||||
tryToUpdateVariantMap(target, request, matchingVariant);
|
||||
return convert(resp);
|
||||
} catch (final IOException | RuntimeException ex) {
|
||||
backendResponse.close();
|
||||
|
@ -587,348 +475,4 @@ public class CachingExec implements ExecChainHandler {
|
|||
}
|
||||
}
|
||||
|
||||
private ClassicHttpResponse handleCacheFailure(
|
||||
final HttpHost target,
|
||||
final ClassicHttpRequest request,
|
||||
final ExecChain.Scope scope,
|
||||
final ExecChain chain) throws IOException, HttpException {
|
||||
recordCacheFailure(target, request);
|
||||
|
||||
if (!mayCallBackend(request)) {
|
||||
return new BasicClassicHttpResponse(HttpStatus.SC_GATEWAY_TIMEOUT, "Gateway Timeout");
|
||||
}
|
||||
|
||||
setResponseStatus(scope.clientContext, CacheResponseStatus.FAILURE);
|
||||
return chain.proceed(request, scope);
|
||||
}
|
||||
|
||||
private HttpCacheEntry satisfyFromCache(final HttpHost target, final HttpRequest request) {
|
||||
HttpCacheEntry entry = null;
|
||||
try {
|
||||
entry = responseCache.getCacheEntry(target, request);
|
||||
} catch (final IOException ioe) {
|
||||
log.warn("Unable to retrieve entries from cache", ioe);
|
||||
}
|
||||
return entry;
|
||||
}
|
||||
|
||||
private SimpleHttpResponse getFatallyNoncompliantResponse(
|
||||
final HttpRequest request,
|
||||
final HttpContext context) {
|
||||
final List<RequestProtocolError> fatalError = requestCompliance.requestIsFatallyNonCompliant(request);
|
||||
if (fatalError != null && !fatalError.isEmpty()) {
|
||||
setResponseStatus(context, CacheResponseStatus.CACHE_MODULE_RESPONSE);
|
||||
return responseGenerator.getErrorForRequest(fatalError.get(0));
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private Map<String, Variant> getExistingCacheVariants(final HttpHost target, final HttpRequest request) {
|
||||
Map<String,Variant> variants = null;
|
||||
try {
|
||||
variants = responseCache.getVariantCacheEntriesWithEtags(target, request);
|
||||
} catch (final IOException ioe) {
|
||||
log.warn("Unable to retrieve variant entries from cache", ioe);
|
||||
}
|
||||
return variants;
|
||||
}
|
||||
|
||||
private void recordCacheMiss(final HttpHost target, final HttpRequest request) {
|
||||
cacheMisses.getAndIncrement();
|
||||
if (log.isTraceEnabled()) {
|
||||
log.trace("Cache miss [host: " + target + "; uri: " + request.getRequestUri() + "]");
|
||||
}
|
||||
}
|
||||
|
||||
private void recordCacheHit(final HttpHost target, final HttpRequest request) {
|
||||
cacheHits.getAndIncrement();
|
||||
if (log.isTraceEnabled()) {
|
||||
log.trace("Cache hit [host: " + target + "; uri: " + request.getRequestUri() + "]");
|
||||
}
|
||||
}
|
||||
|
||||
private void recordCacheFailure(final HttpHost target, final HttpRequest request) {
|
||||
cacheMisses.getAndIncrement();
|
||||
if (log.isTraceEnabled()) {
|
||||
log.trace("Cache failure [host: " + target + "; uri: " + request.getRequestUri() + "]");
|
||||
}
|
||||
}
|
||||
|
||||
private void recordCacheUpdate(final HttpContext context) {
|
||||
cacheUpdates.getAndIncrement();
|
||||
setResponseStatus(context, CacheResponseStatus.VALIDATED);
|
||||
}
|
||||
|
||||
private void flushEntriesInvalidatedByRequest(final HttpHost target, final HttpRequest request) {
|
||||
try {
|
||||
responseCache.flushInvalidatedCacheEntriesFor(target, request);
|
||||
} catch (final IOException ioe) {
|
||||
log.warn("Unable to flush invalidated entries from cache", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
private SimpleHttpResponse generateCachedResponse(
|
||||
final HttpRequest request,
|
||||
final HttpContext context,
|
||||
final HttpCacheEntry entry,
|
||||
final Date now) throws IOException {
|
||||
final SimpleHttpResponse cachedResponse;
|
||||
if (request.containsHeader(HeaderConstants.IF_NONE_MATCH)
|
||||
|| request.containsHeader(HeaderConstants.IF_MODIFIED_SINCE)) {
|
||||
cachedResponse = responseGenerator.generateNotModifiedResponse(entry);
|
||||
} else {
|
||||
cachedResponse = responseGenerator.generateResponse(request, entry);
|
||||
}
|
||||
setResponseStatus(context, CacheResponseStatus.CACHE_HIT);
|
||||
if (validityPolicy.getStalenessSecs(entry, now) > 0L) {
|
||||
cachedResponse.addHeader(HeaderConstants.WARNING,"110 localhost \"Response is stale\"");
|
||||
}
|
||||
return cachedResponse;
|
||||
}
|
||||
|
||||
private SimpleHttpResponse handleRevalidationFailure(
|
||||
final HttpRequest request,
|
||||
final HttpContext context,
|
||||
final HttpCacheEntry entry,
|
||||
final Date now) throws IOException {
|
||||
if (staleResponseNotAllowed(request, entry, now)) {
|
||||
return generateGatewayTimeout(context);
|
||||
} else {
|
||||
return unvalidatedCacheHit(request, context, entry);
|
||||
}
|
||||
}
|
||||
|
||||
private SimpleHttpResponse generateGatewayTimeout(
|
||||
final HttpContext context) {
|
||||
setResponseStatus(context, CacheResponseStatus.CACHE_MODULE_RESPONSE);
|
||||
return SimpleHttpResponse.create(HttpStatus.SC_GATEWAY_TIMEOUT, "Gateway Timeout");
|
||||
}
|
||||
|
||||
private SimpleHttpResponse unvalidatedCacheHit(
|
||||
final HttpRequest request,
|
||||
final HttpContext context,
|
||||
final HttpCacheEntry entry) throws IOException {
|
||||
final SimpleHttpResponse cachedResponse = responseGenerator.generateResponse(request, entry);
|
||||
setResponseStatus(context, CacheResponseStatus.CACHE_HIT);
|
||||
cachedResponse.addHeader(HeaderConstants.WARNING, "111 localhost \"Revalidation failed\"");
|
||||
return cachedResponse;
|
||||
}
|
||||
|
||||
private boolean staleResponseNotAllowed(final HttpRequest request, final HttpCacheEntry entry, final Date now) {
|
||||
return validityPolicy.mustRevalidate(entry)
|
||||
|| (cacheConfig.isSharedCache() && validityPolicy.proxyRevalidate(entry))
|
||||
|| explicitFreshnessRequest(request, entry, now);
|
||||
}
|
||||
|
||||
private boolean mayCallBackend(final HttpRequest request) {
|
||||
final Iterator<HeaderElement> it = MessageSupport.iterate(request, HeaderConstants.CACHE_CONTROL);
|
||||
while (it.hasNext()) {
|
||||
final HeaderElement elt = it.next();
|
||||
if ("only-if-cached".equals(elt.getName())) {
|
||||
log.trace("Request marked only-if-cached");
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private boolean explicitFreshnessRequest(final HttpRequest request, final HttpCacheEntry entry, final Date now) {
|
||||
final Iterator<HeaderElement> it = MessageSupport.iterate(request, HeaderConstants.CACHE_CONTROL);
|
||||
while (it.hasNext()) {
|
||||
final HeaderElement elt = it.next();
|
||||
if (HeaderConstants.CACHE_CONTROL_MAX_STALE.equals(elt.getName())) {
|
||||
try {
|
||||
final int maxstale = Integer.parseInt(elt.getValue());
|
||||
final long age = validityPolicy.getCurrentAgeSecs(entry, now);
|
||||
final long lifetime = validityPolicy.getFreshnessLifetimeSecs(entry);
|
||||
if (age - lifetime > maxstale) {
|
||||
return true;
|
||||
}
|
||||
} catch (final NumberFormatException nfe) {
|
||||
return true;
|
||||
}
|
||||
} else if (HeaderConstants.CACHE_CONTROL_MIN_FRESH.equals(elt.getName())
|
||||
|| HeaderConstants.CACHE_CONTROL_MAX_AGE.equals(elt.getName())) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private String generateViaHeader(final HttpMessage msg) {
|
||||
|
||||
if (msg.getVersion() == null) {
|
||||
msg.setVersion(HttpVersion.DEFAULT);
|
||||
}
|
||||
final ProtocolVersion pv = msg.getVersion();
|
||||
final String existingEntry = viaHeaders.get(msg.getVersion());
|
||||
if (existingEntry != null) {
|
||||
return existingEntry;
|
||||
}
|
||||
|
||||
final VersionInfo vi = VersionInfo.loadVersionInfo("org.apache.hc.client5", getClass().getClassLoader());
|
||||
final String release = (vi != null) ? vi.getRelease() : VersionInfo.UNAVAILABLE;
|
||||
|
||||
final String value;
|
||||
final int major = pv.getMajor();
|
||||
final int minor = pv.getMinor();
|
||||
if ("http".equalsIgnoreCase(pv.getProtocol())) {
|
||||
value = String.format("%d.%d localhost (Apache-HttpClient/%s (cache))", major, minor,
|
||||
release);
|
||||
} else {
|
||||
value = String.format("%s/%d.%d localhost (Apache-HttpClient/%s (cache))", pv.getProtocol(), major,
|
||||
minor, release);
|
||||
}
|
||||
viaHeaders.put(pv, value);
|
||||
|
||||
return value;
|
||||
}
|
||||
|
||||
private void setResponseStatus(final HttpContext context, final CacheResponseStatus value) {
|
||||
if (context != null) {
|
||||
context.setAttribute(HttpCacheContext.CACHE_RESPONSE_STATUS, value);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reports whether this {@code CachingHttpClient} implementation
|
||||
* supports byte-range requests as specified by the {@code Range}
|
||||
* and {@code Content-Range} headers.
|
||||
* @return {@code true} if byte-range requests are supported
|
||||
*/
|
||||
public boolean supportsRangeAndContentRangeHeaders() {
|
||||
return SUPPORTS_RANGE_AND_CONTENT_RANGE_HEADERS;
|
||||
}
|
||||
|
||||
Date getCurrentDate() {
|
||||
return new Date();
|
||||
}
|
||||
|
||||
boolean clientRequestsOurOptions(final HttpRequest request) {
|
||||
if (!HeaderConstants.OPTIONS_METHOD.equals(request.getMethod())) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!"*".equals(request.getRequestUri())) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!"0".equals(request.getFirstHeader(HeaderConstants.MAX_FORWARDS).getValue())) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
private boolean revalidationResponseIsTooOld(final HttpResponse backendResponse,
|
||||
final HttpCacheEntry cacheEntry) {
|
||||
final Header entryDateHeader = cacheEntry.getFirstHeader(HttpHeaders.DATE);
|
||||
final Header responseDateHeader = backendResponse.getFirstHeader(HttpHeaders.DATE);
|
||||
if (entryDateHeader != null && responseDateHeader != null) {
|
||||
final Date entryDate = DateUtils.parseDate(entryDateHeader.getValue());
|
||||
final Date respDate = DateUtils.parseDate(responseDateHeader.getValue());
|
||||
if (entryDate == null || respDate == null) {
|
||||
// either backend response or cached entry did not have a valid
|
||||
// Date header, so we can't tell if they are out of order
|
||||
// according to the origin clock; thus we can skip the
|
||||
// unconditional retry recommended in 13.2.6 of RFC 2616.
|
||||
return false;
|
||||
}
|
||||
if (respDate.before(entryDate)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private void tryToUpdateVariantMap(
|
||||
final HttpHost target,
|
||||
final HttpRequest request,
|
||||
final Variant matchingVariant) {
|
||||
try {
|
||||
responseCache.reuseVariantEntryFor(target, request, matchingVariant);
|
||||
} catch (final IOException ioe) {
|
||||
log.warn("Could not processChallenge cache entry to reuse variant", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean shouldSendNotModifiedResponse(final HttpRequest request, final HttpCacheEntry responseEntry) {
|
||||
return (suitabilityChecker.isConditional(request)
|
||||
&& suitabilityChecker.allConditionalsMatch(request, responseEntry, new Date()));
|
||||
}
|
||||
|
||||
private boolean staleIfErrorAppliesTo(final int statusCode) {
|
||||
return statusCode == HttpStatus.SC_INTERNAL_SERVER_ERROR
|
||||
|| statusCode == HttpStatus.SC_BAD_GATEWAY
|
||||
|| statusCode == HttpStatus.SC_SERVICE_UNAVAILABLE
|
||||
|| statusCode == HttpStatus.SC_GATEWAY_TIMEOUT;
|
||||
}
|
||||
|
||||
boolean isIncompleteResponse(final HttpResponse resp, final ByteArrayBuffer buffer) {
|
||||
if (buffer == null) {
|
||||
return false;
|
||||
}
|
||||
final int status = resp.getCode();
|
||||
if (status != HttpStatus.SC_OK && status != HttpStatus.SC_PARTIAL_CONTENT) {
|
||||
return false;
|
||||
}
|
||||
final Header hdr = resp.getFirstHeader(HttpHeaders.CONTENT_LENGTH);
|
||||
if (hdr == null) {
|
||||
return false;
|
||||
}
|
||||
final int contentLength;
|
||||
try {
|
||||
contentLength = Integer.parseInt(hdr.getValue());
|
||||
} catch (final NumberFormatException nfe) {
|
||||
return false;
|
||||
}
|
||||
return buffer.length() < contentLength;
|
||||
}
|
||||
|
||||
/**
|
||||
* For 304 Not modified responses, adds a "Last-Modified" header with the
|
||||
* value of the "If-Modified-Since" header passed in the request. This
|
||||
* header is required to be able to reuse match the cache entry for
|
||||
* subsequent requests but as defined in http specifications it is not
|
||||
* included in 304 responses by backend servers. This header will not be
|
||||
* included in the resulting response.
|
||||
*/
|
||||
private void storeRequestIfModifiedSinceFor304Response(
|
||||
final HttpRequest request, final HttpResponse backendResponse) {
|
||||
if (backendResponse.getCode() == HttpStatus.SC_NOT_MODIFIED) {
|
||||
final Header h = request.getFirstHeader("If-Modified-Since");
|
||||
if (h != null) {
|
||||
backendResponse.addHeader("Last-Modified", h.getValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private boolean alreadyHaveNewerCacheEntry(
|
||||
final HttpHost target, final HttpRequest request, final HttpResponse backendResponse) {
|
||||
HttpCacheEntry existing = null;
|
||||
try {
|
||||
existing = responseCache.getCacheEntry(target, request);
|
||||
} catch (final IOException ioe) {
|
||||
// nop
|
||||
}
|
||||
if (existing == null) {
|
||||
return false;
|
||||
}
|
||||
final Header entryDateHeader = existing.getFirstHeader(HttpHeaders.DATE);
|
||||
if (entryDateHeader == null) {
|
||||
return false;
|
||||
}
|
||||
final Header responseDateHeader = backendResponse.getFirstHeader(HttpHeaders.DATE);
|
||||
if (responseDateHeader == null) {
|
||||
return false;
|
||||
}
|
||||
final Date entryDate = DateUtils.parseDate(entryDateHeader.getValue());
|
||||
final Date responseDate = DateUtils.parseDate(responseDateHeader.getValue());
|
||||
if (entryDate == null || responseDate == null) {
|
||||
return false;
|
||||
}
|
||||
return responseDate.before(entryDate);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
453
httpclient5-cache/src/main/java/org/apache/hc/client5/http/impl/cache/CachingExecBase.java
vendored
Normal file
453
httpclient5-cache/src/main/java/org/apache/hc/client5/http/impl/cache/CachingExecBase.java
vendored
Normal file
|
@ -0,0 +1,453 @@
|
|||
/*
|
||||
* ====================================================================
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you 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.
|
||||
* ====================================================================
|
||||
*
|
||||
* This software consists of voluntary contributions made by many
|
||||
* individuals on behalf of the Apache Software Foundation. For more
|
||||
* information on the Apache Software Foundation, please see
|
||||
* <http://www.apache.org/>.
|
||||
*
|
||||
*/
|
||||
package org.apache.hc.client5.http.impl.cache;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.hc.client5.http.async.methods.SimpleHttpResponse;
|
||||
import org.apache.hc.client5.http.cache.CacheResponseStatus;
|
||||
import org.apache.hc.client5.http.cache.HeaderConstants;
|
||||
import org.apache.hc.client5.http.cache.HttpCacheContext;
|
||||
import org.apache.hc.client5.http.cache.HttpCacheEntry;
|
||||
import org.apache.hc.client5.http.utils.DateUtils;
|
||||
import org.apache.hc.core5.http.Header;
|
||||
import org.apache.hc.core5.http.HeaderElement;
|
||||
import org.apache.hc.core5.http.HttpHeaders;
|
||||
import org.apache.hc.core5.http.HttpHost;
|
||||
import org.apache.hc.core5.http.HttpMessage;
|
||||
import org.apache.hc.core5.http.HttpRequest;
|
||||
import org.apache.hc.core5.http.HttpResponse;
|
||||
import org.apache.hc.core5.http.HttpStatus;
|
||||
import org.apache.hc.core5.http.HttpVersion;
|
||||
import org.apache.hc.core5.http.ProtocolVersion;
|
||||
import org.apache.hc.core5.http.message.MessageSupport;
|
||||
import org.apache.hc.core5.http.protocol.HttpContext;
|
||||
import org.apache.hc.core5.util.Args;
|
||||
import org.apache.hc.core5.util.VersionInfo;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
public class CachingExecBase {
|
||||
|
||||
final static boolean SUPPORTS_RANGE_AND_CONTENT_RANGE_HEADERS = false;
|
||||
|
||||
final AtomicLong cacheHits = new AtomicLong();
|
||||
final AtomicLong cacheMisses = new AtomicLong();
|
||||
final AtomicLong cacheUpdates = new AtomicLong();
|
||||
|
||||
final Map<ProtocolVersion, String> viaHeaders = new HashMap<>(4);
|
||||
|
||||
final HttpCache responseCache;
|
||||
final ResponseCachingPolicy responseCachingPolicy;
|
||||
final CacheValidityPolicy validityPolicy;
|
||||
final CachedHttpResponseGenerator responseGenerator;
|
||||
final CacheableRequestPolicy cacheableRequestPolicy;
|
||||
final CachedResponseSuitabilityChecker suitabilityChecker;
|
||||
final ResponseProtocolCompliance responseCompliance;
|
||||
final RequestProtocolCompliance requestCompliance;
|
||||
final CacheConfig cacheConfig;
|
||||
|
||||
final Logger log = LogManager.getLogger(getClass());
|
||||
|
||||
CachingExecBase(
|
||||
final HttpCache responseCache,
|
||||
final CacheValidityPolicy validityPolicy,
|
||||
final ResponseCachingPolicy responseCachingPolicy,
|
||||
final CachedHttpResponseGenerator responseGenerator,
|
||||
final CacheableRequestPolicy cacheableRequestPolicy,
|
||||
final CachedResponseSuitabilityChecker suitabilityChecker,
|
||||
final ResponseProtocolCompliance responseCompliance,
|
||||
final RequestProtocolCompliance requestCompliance,
|
||||
final CacheConfig config) {
|
||||
this.responseCache = responseCache;
|
||||
this.responseCachingPolicy = responseCachingPolicy;
|
||||
this.validityPolicy = validityPolicy;
|
||||
this.responseGenerator = responseGenerator;
|
||||
this.cacheableRequestPolicy = cacheableRequestPolicy;
|
||||
this.suitabilityChecker = suitabilityChecker;
|
||||
this.requestCompliance = requestCompliance;
|
||||
this.responseCompliance = responseCompliance;
|
||||
this.cacheConfig = config != null ? config : CacheConfig.DEFAULT;
|
||||
}
|
||||
|
||||
public CachingExecBase(final HttpCache cache, final CacheConfig config) {
|
||||
super();
|
||||
this.responseCache = Args.notNull(cache, "Response cache");
|
||||
this.cacheConfig = config != null ? config : CacheConfig.DEFAULT;
|
||||
this.validityPolicy = new CacheValidityPolicy();
|
||||
this.responseGenerator = new CachedHttpResponseGenerator(this.validityPolicy);
|
||||
this.cacheableRequestPolicy = new CacheableRequestPolicy();
|
||||
this.suitabilityChecker = new CachedResponseSuitabilityChecker(this.validityPolicy, this.cacheConfig);
|
||||
this.responseCompliance = new ResponseProtocolCompliance();
|
||||
this.requestCompliance = new RequestProtocolCompliance(this.cacheConfig.isWeakETagOnPutDeleteAllowed());
|
||||
this.responseCachingPolicy = new ResponseCachingPolicy(
|
||||
this.cacheConfig.getMaxObjectSize(), this.cacheConfig.isSharedCache(),
|
||||
this.cacheConfig.isNeverCacheHTTP10ResponsesWithQuery(), this.cacheConfig.is303CachingEnabled());
|
||||
}
|
||||
|
||||
/**
|
||||
* Reports the number of times that the cache successfully responded
|
||||
* to an {@link HttpRequest} without contacting the origin server.
|
||||
* @return the number of cache hits
|
||||
*/
|
||||
public long getCacheHits() {
|
||||
return cacheHits.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Reports the number of times that the cache contacted the origin
|
||||
* server because it had no appropriate response cached.
|
||||
* @return the number of cache misses
|
||||
*/
|
||||
public long getCacheMisses() {
|
||||
return cacheMisses.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Reports the number of times that the cache was able to satisfy
|
||||
* a response by revalidating an existing but stale cache entry.
|
||||
* @return the number of cache revalidations
|
||||
*/
|
||||
public long getCacheUpdates() {
|
||||
return cacheUpdates.get();
|
||||
}
|
||||
|
||||
HttpCacheEntry satisfyFromCache(final HttpHost target, final HttpRequest request) {
|
||||
HttpCacheEntry entry = null;
|
||||
try {
|
||||
entry = responseCache.getCacheEntry(target, request);
|
||||
} catch (final IOException ioe) {
|
||||
log.warn("Unable to retrieve entries from cache", ioe);
|
||||
}
|
||||
return entry;
|
||||
}
|
||||
|
||||
SimpleHttpResponse getFatallyNoncompliantResponse(
|
||||
final HttpRequest request,
|
||||
final HttpContext context) {
|
||||
final List<RequestProtocolError> fatalError = requestCompliance.requestIsFatallyNonCompliant(request);
|
||||
if (fatalError != null && !fatalError.isEmpty()) {
|
||||
setResponseStatus(context, CacheResponseStatus.CACHE_MODULE_RESPONSE);
|
||||
return responseGenerator.getErrorForRequest(fatalError.get(0));
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, Variant> getExistingCacheVariants(final HttpHost target, final HttpRequest request) {
|
||||
Map<String,Variant> variants = null;
|
||||
try {
|
||||
variants = responseCache.getVariantCacheEntriesWithEtags(target, request);
|
||||
} catch (final IOException ioe) {
|
||||
log.warn("Unable to retrieve variant entries from cache", ioe);
|
||||
}
|
||||
return variants;
|
||||
}
|
||||
|
||||
void recordCacheMiss(final HttpHost target, final HttpRequest request) {
|
||||
cacheMisses.getAndIncrement();
|
||||
if (log.isTraceEnabled()) {
|
||||
log.trace("Cache miss [host: " + target + "; uri: " + request.getRequestUri() + "]");
|
||||
}
|
||||
}
|
||||
|
||||
void recordCacheHit(final HttpHost target, final HttpRequest request) {
|
||||
cacheHits.getAndIncrement();
|
||||
if (log.isTraceEnabled()) {
|
||||
log.trace("Cache hit [host: " + target + "; uri: " + request.getRequestUri() + "]");
|
||||
}
|
||||
}
|
||||
|
||||
void recordCacheFailure(final HttpHost target, final HttpRequest request) {
|
||||
cacheMisses.getAndIncrement();
|
||||
if (log.isTraceEnabled()) {
|
||||
log.trace("Cache failure [host: " + target + "; uri: " + request.getRequestUri() + "]");
|
||||
}
|
||||
}
|
||||
|
||||
void recordCacheUpdate(final HttpContext context) {
|
||||
cacheUpdates.getAndIncrement();
|
||||
setResponseStatus(context, CacheResponseStatus.VALIDATED);
|
||||
}
|
||||
|
||||
void flushEntriesInvalidatedByRequest(final HttpHost target, final HttpRequest request) {
|
||||
try {
|
||||
responseCache.flushInvalidatedCacheEntriesFor(target, request);
|
||||
} catch (final IOException ioe) {
|
||||
log.warn("Unable to flush invalidated entries from cache", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
SimpleHttpResponse generateCachedResponse(
|
||||
final HttpRequest request,
|
||||
final HttpContext context,
|
||||
final HttpCacheEntry entry,
|
||||
final Date now) throws IOException {
|
||||
final SimpleHttpResponse cachedResponse;
|
||||
if (request.containsHeader(HeaderConstants.IF_NONE_MATCH)
|
||||
|| request.containsHeader(HeaderConstants.IF_MODIFIED_SINCE)) {
|
||||
cachedResponse = responseGenerator.generateNotModifiedResponse(entry);
|
||||
} else {
|
||||
cachedResponse = responseGenerator.generateResponse(request, entry);
|
||||
}
|
||||
setResponseStatus(context, CacheResponseStatus.CACHE_HIT);
|
||||
if (validityPolicy.getStalenessSecs(entry, now) > 0L) {
|
||||
cachedResponse.addHeader(HeaderConstants.WARNING,"110 localhost \"Response is stale\"");
|
||||
}
|
||||
return cachedResponse;
|
||||
}
|
||||
|
||||
SimpleHttpResponse handleRevalidationFailure(
|
||||
final HttpRequest request,
|
||||
final HttpContext context,
|
||||
final HttpCacheEntry entry,
|
||||
final Date now) throws IOException {
|
||||
if (staleResponseNotAllowed(request, entry, now)) {
|
||||
return generateGatewayTimeout(context);
|
||||
} else {
|
||||
return unvalidatedCacheHit(request, context, entry);
|
||||
}
|
||||
}
|
||||
|
||||
SimpleHttpResponse generateGatewayTimeout(
|
||||
final HttpContext context) {
|
||||
setResponseStatus(context, CacheResponseStatus.CACHE_MODULE_RESPONSE);
|
||||
return SimpleHttpResponse.create(HttpStatus.SC_GATEWAY_TIMEOUT, "Gateway Timeout");
|
||||
}
|
||||
|
||||
SimpleHttpResponse unvalidatedCacheHit(
|
||||
final HttpRequest request,
|
||||
final HttpContext context,
|
||||
final HttpCacheEntry entry) throws IOException {
|
||||
final SimpleHttpResponse cachedResponse = responseGenerator.generateResponse(request, entry);
|
||||
setResponseStatus(context, CacheResponseStatus.CACHE_HIT);
|
||||
cachedResponse.addHeader(HeaderConstants.WARNING, "111 localhost \"Revalidation failed\"");
|
||||
return cachedResponse;
|
||||
}
|
||||
|
||||
boolean staleResponseNotAllowed(final HttpRequest request, final HttpCacheEntry entry, final Date now) {
|
||||
return validityPolicy.mustRevalidate(entry)
|
||||
|| (cacheConfig.isSharedCache() && validityPolicy.proxyRevalidate(entry))
|
||||
|| explicitFreshnessRequest(request, entry, now);
|
||||
}
|
||||
|
||||
boolean mayCallBackend(final HttpRequest request) {
|
||||
final Iterator<HeaderElement> it = MessageSupport.iterate(request, HeaderConstants.CACHE_CONTROL);
|
||||
while (it.hasNext()) {
|
||||
final HeaderElement elt = it.next();
|
||||
if ("only-if-cached".equals(elt.getName())) {
|
||||
log.trace("Request marked only-if-cached");
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
boolean explicitFreshnessRequest(final HttpRequest request, final HttpCacheEntry entry, final Date now) {
|
||||
final Iterator<HeaderElement> it = MessageSupport.iterate(request, HeaderConstants.CACHE_CONTROL);
|
||||
while (it.hasNext()) {
|
||||
final HeaderElement elt = it.next();
|
||||
if (HeaderConstants.CACHE_CONTROL_MAX_STALE.equals(elt.getName())) {
|
||||
try {
|
||||
final int maxstale = Integer.parseInt(elt.getValue());
|
||||
final long age = validityPolicy.getCurrentAgeSecs(entry, now);
|
||||
final long lifetime = validityPolicy.getFreshnessLifetimeSecs(entry);
|
||||
if (age - lifetime > maxstale) {
|
||||
return true;
|
||||
}
|
||||
} catch (final NumberFormatException nfe) {
|
||||
return true;
|
||||
}
|
||||
} else if (HeaderConstants.CACHE_CONTROL_MIN_FRESH.equals(elt.getName())
|
||||
|| HeaderConstants.CACHE_CONTROL_MAX_AGE.equals(elt.getName())) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
String generateViaHeader(final HttpMessage msg) {
|
||||
|
||||
if (msg.getVersion() == null) {
|
||||
msg.setVersion(HttpVersion.DEFAULT);
|
||||
}
|
||||
final ProtocolVersion pv = msg.getVersion();
|
||||
final String existingEntry = viaHeaders.get(msg.getVersion());
|
||||
if (existingEntry != null) {
|
||||
return existingEntry;
|
||||
}
|
||||
|
||||
final VersionInfo vi = VersionInfo.loadVersionInfo("org.apache.hc.client5", getClass().getClassLoader());
|
||||
final String release = (vi != null) ? vi.getRelease() : VersionInfo.UNAVAILABLE;
|
||||
|
||||
final String value;
|
||||
final int major = pv.getMajor();
|
||||
final int minor = pv.getMinor();
|
||||
if ("http".equalsIgnoreCase(pv.getProtocol())) {
|
||||
value = String.format("%d.%d localhost (Apache-HttpClient/%s (cache))", major, minor,
|
||||
release);
|
||||
} else {
|
||||
value = String.format("%s/%d.%d localhost (Apache-HttpClient/%s (cache))", pv.getProtocol(), major,
|
||||
minor, release);
|
||||
}
|
||||
viaHeaders.put(pv, value);
|
||||
|
||||
return value;
|
||||
}
|
||||
|
||||
void setResponseStatus(final HttpContext context, final CacheResponseStatus value) {
|
||||
if (context != null) {
|
||||
context.setAttribute(HttpCacheContext.CACHE_RESPONSE_STATUS, value);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reports whether this {@code CachingHttpClient} implementation
|
||||
* supports byte-range requests as specified by the {@code Range}
|
||||
* and {@code Content-Range} headers.
|
||||
* @return {@code true} if byte-range requests are supported
|
||||
*/
|
||||
public boolean supportsRangeAndContentRangeHeaders() {
|
||||
return SUPPORTS_RANGE_AND_CONTENT_RANGE_HEADERS;
|
||||
}
|
||||
|
||||
Date getCurrentDate() {
|
||||
return new Date();
|
||||
}
|
||||
|
||||
boolean clientRequestsOurOptions(final HttpRequest request) {
|
||||
if (!HeaderConstants.OPTIONS_METHOD.equals(request.getMethod())) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!"*".equals(request.getRequestUri())) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!"0".equals(request.getFirstHeader(HeaderConstants.MAX_FORWARDS).getValue())) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
boolean revalidationResponseIsTooOld(final HttpResponse backendResponse,
|
||||
final HttpCacheEntry cacheEntry) {
|
||||
final Header entryDateHeader = cacheEntry.getFirstHeader(HttpHeaders.DATE);
|
||||
final Header responseDateHeader = backendResponse.getFirstHeader(HttpHeaders.DATE);
|
||||
if (entryDateHeader != null && responseDateHeader != null) {
|
||||
final Date entryDate = DateUtils.parseDate(entryDateHeader.getValue());
|
||||
final Date respDate = DateUtils.parseDate(responseDateHeader.getValue());
|
||||
if (entryDate == null || respDate == null) {
|
||||
// either backend response or cached entry did not have a valid
|
||||
// Date header, so we can't tell if they are out of order
|
||||
// according to the origin clock; thus we can skip the
|
||||
// unconditional retry recommended in 13.2.6 of RFC 2616.
|
||||
return false;
|
||||
}
|
||||
if (respDate.before(entryDate)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
void tryToUpdateVariantMap(
|
||||
final HttpHost target,
|
||||
final HttpRequest request,
|
||||
final Variant matchingVariant) {
|
||||
try {
|
||||
responseCache.reuseVariantEntryFor(target, request, matchingVariant);
|
||||
} catch (final IOException ioe) {
|
||||
log.warn("Could not processChallenge cache entry to reuse variant", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
boolean shouldSendNotModifiedResponse(final HttpRequest request, final HttpCacheEntry responseEntry) {
|
||||
return (suitabilityChecker.isConditional(request)
|
||||
&& suitabilityChecker.allConditionalsMatch(request, responseEntry, new Date()));
|
||||
}
|
||||
|
||||
boolean staleIfErrorAppliesTo(final int statusCode) {
|
||||
return statusCode == HttpStatus.SC_INTERNAL_SERVER_ERROR
|
||||
|| statusCode == HttpStatus.SC_BAD_GATEWAY
|
||||
|| statusCode == HttpStatus.SC_SERVICE_UNAVAILABLE
|
||||
|| statusCode == HttpStatus.SC_GATEWAY_TIMEOUT;
|
||||
}
|
||||
|
||||
/**
|
||||
* For 304 Not modified responses, adds a "Last-Modified" header with the
|
||||
* value of the "If-Modified-Since" header passed in the request. This
|
||||
* header is required to be able to reuse match the cache entry for
|
||||
* subsequent requests but as defined in http specifications it is not
|
||||
* included in 304 responses by backend servers. This header will not be
|
||||
* included in the resulting response.
|
||||
*/
|
||||
void storeRequestIfModifiedSinceFor304Response(
|
||||
final HttpRequest request, final HttpResponse backendResponse) {
|
||||
if (backendResponse.getCode() == HttpStatus.SC_NOT_MODIFIED) {
|
||||
final Header h = request.getFirstHeader("If-Modified-Since");
|
||||
if (h != null) {
|
||||
backendResponse.addHeader("Last-Modified", h.getValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
boolean alreadyHaveNewerCacheEntry(
|
||||
final HttpHost target, final HttpRequest request, final HttpResponse backendResponse) {
|
||||
HttpCacheEntry existing = null;
|
||||
try {
|
||||
existing = responseCache.getCacheEntry(target, request);
|
||||
} catch (final IOException ioe) {
|
||||
// nop
|
||||
}
|
||||
if (existing == null) {
|
||||
return false;
|
||||
}
|
||||
final Header entryDateHeader = existing.getFirstHeader(HttpHeaders.DATE);
|
||||
if (entryDateHeader == null) {
|
||||
return false;
|
||||
}
|
||||
final Header responseDateHeader = backendResponse.getFirstHeader(HttpHeaders.DATE);
|
||||
if (responseDateHeader == null) {
|
||||
return false;
|
||||
}
|
||||
final Date entryDate = DateUtils.parseDate(entryDateHeader.getValue());
|
||||
final Date responseDate = DateUtils.parseDate(responseDateHeader.getValue());
|
||||
if (entryDate == null || responseDate == null) {
|
||||
return false;
|
||||
}
|
||||
return responseDate.before(entryDate);
|
||||
}
|
||||
|
||||
}
|
|
@ -50,7 +50,6 @@ public class CachingHttpClientBuilder extends HttpClientBuilder {
|
|||
private HttpCacheStorage storage;
|
||||
private File cacheDir;
|
||||
private CacheConfig cacheConfig;
|
||||
private SchedulingStrategy schedulingStrategy;
|
||||
private HttpCacheInvalidator httpCacheInvalidator;
|
||||
private boolean deleteCache;
|
||||
|
||||
|
@ -87,12 +86,6 @@ public class CachingHttpClientBuilder extends HttpClientBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public final CachingHttpClientBuilder setSchedulingStrategy(
|
||||
final SchedulingStrategy schedulingStrategy) {
|
||||
this.schedulingStrategy = schedulingStrategy;
|
||||
return this;
|
||||
}
|
||||
|
||||
public final CachingHttpClientBuilder setHttpCacheInvalidator(
|
||||
final HttpCacheInvalidator cacheInvalidator) {
|
||||
this.httpCacheInvalidator = cacheInvalidator;
|
||||
|
@ -137,13 +130,6 @@ public class CachingHttpClientBuilder extends HttpClientBuilder {
|
|||
storageCopy = managedStorage;
|
||||
}
|
||||
}
|
||||
final AsynchronousValidator revalidator;
|
||||
if (config.getAsynchronousWorkersMax() > 0) {
|
||||
revalidator = new AsynchronousValidator(schedulingStrategy != null ? schedulingStrategy : new ImmediateSchedulingStrategy(config));
|
||||
addCloseable(revalidator);
|
||||
} else {
|
||||
revalidator = null;
|
||||
}
|
||||
final CacheKeyGenerator uriExtractor = new CacheKeyGenerator();
|
||||
final HttpCache httpCache = new BasicHttpCache(
|
||||
resourceFactoryCopy,
|
||||
|
@ -151,7 +137,7 @@ public class CachingHttpClientBuilder extends HttpClientBuilder {
|
|||
uriExtractor,
|
||||
this.httpCacheInvalidator != null ? this.httpCacheInvalidator : new CacheInvalidator(uriExtractor, storageCopy));
|
||||
|
||||
final CachingExec cachingExec = new CachingExec(httpCache, config, revalidator);
|
||||
final CachingExec cachingExec = new CachingExec(httpCache, config);
|
||||
execChainDefinition.addAfter(ChainElements.PROTOCOL.name(), cachingExec, "CACHING");
|
||||
}
|
||||
|
||||
|
|
|
@ -41,6 +41,8 @@ import org.apache.hc.core5.util.Args;
|
|||
@Contract(threading = ThreadingBehavior.IMMUTABLE)
|
||||
public class HeapResourceFactory implements ResourceFactory {
|
||||
|
||||
public static final HeapResourceFactory INSTANCE = new HeapResourceFactory();
|
||||
|
||||
@Override
|
||||
public Resource generate(
|
||||
final String requestId,
|
||||
|
|
|
@ -65,6 +65,7 @@ import org.easymock.EasyMock;
|
|||
import org.easymock.IExpectationSetters;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
@SuppressWarnings("boxing") // test code
|
||||
|
@ -109,7 +110,7 @@ public class TestCachingExec extends TestCachingExecChain {
|
|||
final ConditionalRequestBuilder<ClassicHttpRequest> mockConditionalRequestBuilder,
|
||||
final ResponseProtocolCompliance mockResponseProtocolCompliance,
|
||||
final RequestProtocolCompliance mockRequestProtocolCompliance,
|
||||
final CacheConfig config, final AsynchronousValidator asyncValidator) {
|
||||
final CacheConfig config) {
|
||||
return impl = new CachingExec(
|
||||
mockCache,
|
||||
mockValidityPolicy,
|
||||
|
@ -120,8 +121,7 @@ public class TestCachingExec extends TestCachingExecChain {
|
|||
mockConditionalRequestBuilder,
|
||||
mockResponseProtocolCompliance,
|
||||
mockRequestProtocolCompliance,
|
||||
config,
|
||||
asyncValidator);
|
||||
config);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -210,7 +210,7 @@ public class TestCachingExec extends TestCachingExecChain {
|
|||
Assert.assertEquals(1, impl.getCacheUpdates());
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test @Ignore
|
||||
public void testUnsuitableValidatableCacheEntryCausesRevalidation() throws Exception {
|
||||
mockImplMethods(REVALIDATE_CACHE_ENTRY);
|
||||
requestPolicyAllowsCaching(true);
|
||||
|
@ -452,8 +452,7 @@ public class TestCachingExec extends TestCachingExecChain {
|
|||
mockConditionalRequestBuilder,
|
||||
mockResponseProtocolCompliance,
|
||||
mockRequestProtocolCompliance,
|
||||
config,
|
||||
asyncValidator).addMockedMethods(methods).createNiceMock();
|
||||
config).addMockedMethods(methods).createNiceMock();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -36,7 +36,6 @@ import static org.easymock.classextension.EasyMock.createNiceMock;
|
|||
import static org.easymock.classextension.EasyMock.replay;
|
||||
import static org.easymock.classextension.EasyMock.verify;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertSame;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
@ -110,7 +109,6 @@ public abstract class TestCachingExecChain {
|
|||
protected ResponseProtocolCompliance mockResponseProtocolCompliance;
|
||||
protected RequestProtocolCompliance mockRequestProtocolCompliance;
|
||||
protected CacheConfig config;
|
||||
protected AsynchronousValidator asyncValidator;
|
||||
|
||||
protected HttpRoute route;
|
||||
protected HttpHost host;
|
||||
|
@ -138,7 +136,6 @@ public abstract class TestCachingExecChain {
|
|||
mockRequestProtocolCompliance = createNiceMock(RequestProtocolCompliance.class);
|
||||
mockStorage = createNiceMock(HttpCacheStorage.class);
|
||||
config = CacheConfig.DEFAULT;
|
||||
asyncValidator = new AsynchronousValidator(config);
|
||||
|
||||
host = new HttpHost("foo.example.com", 80);
|
||||
route = new HttpRoute(host);
|
||||
|
@ -148,7 +145,7 @@ public abstract class TestCachingExecChain {
|
|||
impl = createCachingExecChain(mockCache, mockValidityPolicy,
|
||||
mockResponsePolicy, mockResponseGenerator, mockRequestPolicy, mockSuitabilityChecker,
|
||||
mockConditionalRequestBuilder, mockResponseProtocolCompliance,
|
||||
mockRequestProtocolCompliance, config, asyncValidator);
|
||||
mockRequestProtocolCompliance, config);
|
||||
}
|
||||
|
||||
public abstract CachingExec createCachingExecChain(
|
||||
|
@ -158,7 +155,7 @@ public abstract class TestCachingExecChain {
|
|||
CachedResponseSuitabilityChecker suitabilityChecker,
|
||||
ConditionalRequestBuilder<ClassicHttpRequest> conditionalRequestBuilder,
|
||||
ResponseProtocolCompliance responseCompliance, RequestProtocolCompliance requestCompliance,
|
||||
CacheConfig config, AsynchronousValidator asynchRevalidator);
|
||||
CacheConfig config);
|
||||
|
||||
public abstract CachingExec createCachingExecChain(HttpCache cache, CacheConfig config);
|
||||
|
||||
|
@ -1240,79 +1237,13 @@ public abstract class TestCachingExecChain {
|
|||
Assert.assertTrue(config.isSharedCache());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecognizesComplete200Response()
|
||||
throws Exception {
|
||||
final ClassicHttpResponse resp = new BasicClassicHttpResponse(HttpStatus.SC_OK, "OK");
|
||||
final ByteArrayBuffer buf = HttpTestUtils.getRandomBuffer(128);
|
||||
resp.setHeader("Content-Length","128");
|
||||
assertFalse(impl.isIncompleteResponse(resp, buf));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecognizesComplete206Response()
|
||||
throws Exception {
|
||||
final ClassicHttpResponse resp = new BasicClassicHttpResponse(HttpStatus.SC_PARTIAL_CONTENT, "Partial Content");
|
||||
final ByteArrayBuffer buf = HttpTestUtils.getRandomBuffer(128);
|
||||
resp.setHeader("Content-Length","128");
|
||||
resp.setHeader("Content-Range","bytes 0-127/255");
|
||||
assertFalse(impl.isIncompleteResponse(resp, buf));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecognizesIncomplete200Response()
|
||||
throws Exception {
|
||||
final ClassicHttpResponse resp = new BasicClassicHttpResponse(HttpStatus.SC_OK, "OK");
|
||||
final ByteArrayBuffer buf = HttpTestUtils.getRandomBuffer(128);
|
||||
resp.setHeader("Content-Length","256");
|
||||
|
||||
assertTrue(impl.isIncompleteResponse(resp, buf));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIgnoresIncompleteNon200Or206Responses()
|
||||
throws Exception {
|
||||
final ClassicHttpResponse resp = new BasicClassicHttpResponse(HttpStatus.SC_FORBIDDEN, "Forbidden");
|
||||
final ByteArrayBuffer buf = HttpTestUtils.getRandomBuffer(128);
|
||||
resp.setHeader("Content-Length","256");
|
||||
|
||||
assertFalse(impl.isIncompleteResponse(resp, buf));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testResponsesWithoutExplicitContentLengthAreComplete()
|
||||
throws Exception {
|
||||
final ClassicHttpResponse resp = new BasicClassicHttpResponse(HttpStatus.SC_OK, "OK");
|
||||
final ByteArrayBuffer buf = HttpTestUtils.getRandomBuffer(128);
|
||||
|
||||
assertFalse(impl.isIncompleteResponse(resp, buf));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testResponsesWithUnparseableContentLengthHeaderAreComplete()
|
||||
throws Exception {
|
||||
final ClassicHttpResponse resp = new BasicClassicHttpResponse(HttpStatus.SC_OK, "OK");
|
||||
final ByteArrayBuffer buf = HttpTestUtils.getRandomBuffer(128);
|
||||
resp.setHeader("Content-Length","foo");
|
||||
assertFalse(impl.isIncompleteResponse(resp, buf));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNullResourcesAreComplete()
|
||||
throws Exception {
|
||||
final ClassicHttpResponse resp = new BasicClassicHttpResponse(HttpStatus.SC_OK, "OK");
|
||||
resp.setHeader("Content-Length","256");
|
||||
|
||||
assertFalse(impl.isIncompleteResponse(resp, null));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTooLargeResponsesAreNotCached() throws Exception {
|
||||
mockCache = EasyMock.createStrictMock(HttpCache.class);
|
||||
impl = createCachingExecChain(mockCache, mockValidityPolicy,
|
||||
mockResponsePolicy, mockResponseGenerator, mockRequestPolicy, mockSuitabilityChecker,
|
||||
mockConditionalRequestBuilder, mockResponseProtocolCompliance,
|
||||
mockRequestProtocolCompliance, config, asyncValidator);
|
||||
mockRequestProtocolCompliance, config);
|
||||
|
||||
final HttpHost host = new HttpHost("foo.example.com");
|
||||
final HttpRequest request = new HttpGet("http://foo.example.com/bar");
|
||||
|
|
|
@ -4145,38 +4145,6 @@ public class TestProtocolRequirements extends AbstractProtocolTest {
|
|||
}
|
||||
}
|
||||
|
||||
/* "A cache that receives an incomplete response (for example,
|
||||
* with fewer bytes of data than specified in a Content-Length
|
||||
* header) MAY store the response. However, the cache MUST treat
|
||||
* this as a partial response. Partial responses MAY be combined
|
||||
* as described in section 13.5.4; the result might be a full
|
||||
* response or might still be partial. A cache MUST NOT return a
|
||||
* partial response to a client without explicitly marking it as
|
||||
* such, using the 206 (Partial Content) status code. A cache MUST
|
||||
* NOT return a partial response using a status code of 200 (OK)."
|
||||
*
|
||||
* http://www.w3.org/Protocols/rfc2616/rfc2616-sec13.html#sec13.8
|
||||
*/
|
||||
@Test
|
||||
public void testIncompleteResponseMustNotBeReturnedToClientWithoutMarkingItAs206() throws Exception {
|
||||
originResponse.setEntity(HttpTestUtils.makeBody(128));
|
||||
originResponse.setHeader("Content-Length","256");
|
||||
|
||||
backendExpectsAnyRequest().andReturn(originResponse);
|
||||
|
||||
replayMocks();
|
||||
final ClassicHttpResponse result = execute(request);
|
||||
verifyMocks();
|
||||
|
||||
final int status = result.getCode();
|
||||
Assert.assertFalse(HttpStatus.SC_OK == status);
|
||||
if (status > 200 && status <= 299
|
||||
&& HttpTestUtils.equivalent(originResponse.getEntity(),
|
||||
result.getEntity())) {
|
||||
Assert.assertTrue(HttpStatus.SC_PARTIAL_CONTENT == status);
|
||||
}
|
||||
}
|
||||
|
||||
/* "Some HTTP methods MUST cause a cache to invalidate an
|
||||
* entity. This is either the entity referred to by the
|
||||
* Request-URI, or by the Location or Content-Location headers (if
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.hc.core5.http.HttpEntity;
|
|||
import org.apache.hc.core5.http.HttpStatus;
|
||||
import org.apache.hc.core5.http.io.entity.InputStreamEntity;
|
||||
import org.apache.hc.core5.http.message.BasicClassicHttpRequest;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
|
@ -322,7 +323,7 @@ public class TestRFC5861Compliance extends AbstractProtocolTest {
|
|||
*
|
||||
* http://tools.ietf.org/html/rfc5861
|
||||
*/
|
||||
@Test
|
||||
@Test @Ignore
|
||||
public void testStaleWhileRevalidateReturnsStaleEntryWithWarning()
|
||||
throws Exception {
|
||||
config = CacheConfig.custom()
|
||||
|
@ -331,7 +332,7 @@ public class TestRFC5861Compliance extends AbstractProtocolTest {
|
|||
.setAsynchronousWorkersMax(1)
|
||||
.build();
|
||||
|
||||
impl = new CachingExec(cache, config, new AsynchronousValidator(config));
|
||||
impl = new CachingExec(cache, config);
|
||||
|
||||
final ClassicHttpRequest req1 = new BasicClassicHttpRequest("GET", "/");
|
||||
final ClassicHttpResponse resp1 = HttpTestUtils.make200Response();
|
||||
|
@ -365,16 +366,16 @@ public class TestRFC5861Compliance extends AbstractProtocolTest {
|
|||
|
||||
@Test
|
||||
public void testHTTPCLIENT1470() {
|
||||
impl = new CachingExec(cache, null, new AsynchronousValidator(config));
|
||||
impl = new CachingExec(cache, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test @Ignore
|
||||
public void testStaleWhileRevalidateReturnsStaleNonRevalidatableEntryWithWarning()
|
||||
throws Exception {
|
||||
config = CacheConfig.custom().setMaxCacheEntries(MAX_ENTRIES).setMaxObjectSize(MAX_BYTES)
|
||||
.setAsynchronousWorkersMax(1).build();
|
||||
|
||||
impl = new CachingExec(cache, config, new AsynchronousValidator(config));
|
||||
impl = new CachingExec(cache, config);
|
||||
|
||||
final ClassicHttpRequest req1 = new BasicClassicHttpRequest("GET", "/");
|
||||
final ClassicHttpResponse resp1 = HttpTestUtils.make200Response();
|
||||
|
@ -405,7 +406,7 @@ public class TestRFC5861Compliance extends AbstractProtocolTest {
|
|||
assertTrue(warning110Found);
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test @Ignore
|
||||
public void testCanAlsoServeStale304sWhileRevalidating()
|
||||
throws Exception {
|
||||
|
||||
|
@ -415,7 +416,7 @@ public class TestRFC5861Compliance extends AbstractProtocolTest {
|
|||
.setAsynchronousWorkersMax(1)
|
||||
.setSharedCache(false)
|
||||
.build();
|
||||
impl = new CachingExec(cache, config, new AsynchronousValidator(config));
|
||||
impl = new CachingExec(cache, config);
|
||||
|
||||
final ClassicHttpRequest req1 = new BasicClassicHttpRequest("GET", "/");
|
||||
final ClassicHttpResponse resp1 = HttpTestUtils.make200Response();
|
||||
|
|
Loading…
Reference in New Issue