Asynchronous ResourceResolver + ResourceTransformer

Issue: SPR-14521
This commit is contained in:
Rossen Stoyanchev 2016-09-09 11:05:01 -04:00
parent f592599349
commit 33d90747a1
30 changed files with 756 additions and 560 deletions

View File

@ -20,6 +20,7 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import reactor.core.publisher.Mono;
import org.springframework.core.io.Resource;
import org.springframework.web.server.ServerWebExchange;
@ -36,7 +37,7 @@ public abstract class AbstractResourceResolver implements ResourceResolver {
@Override
public Resource resolveResource(ServerWebExchange exchange, String requestPath,
public Mono<Resource> resolveResource(ServerWebExchange exchange, String requestPath,
List<? extends Resource> locations, ResourceResolverChain chain) {
if (logger.isTraceEnabled()) {
@ -46,7 +47,7 @@ public abstract class AbstractResourceResolver implements ResourceResolver {
}
@Override
public String resolveUrlPath(String resourceUrlPath, List<? extends Resource> locations,
public Mono<String> resolveUrlPath(String resourceUrlPath, List<? extends Resource> locations,
ResourceResolverChain chain) {
if (logger.isTraceEnabled()) {
@ -57,10 +58,10 @@ public abstract class AbstractResourceResolver implements ResourceResolver {
}
protected abstract Resource resolveResourceInternal(ServerWebExchange exchange, String requestPath,
List<? extends Resource> locations, ResourceResolverChain chain);
protected abstract Mono<Resource> resolveResourceInternal(ServerWebExchange exchange,
String requestPath, List<? extends Resource> locations, ResourceResolverChain chain);
protected abstract String resolveUrlPathInternal(String resourceUrlPath,
protected abstract Mono<String> resolveUrlPathInternal(String resourceUrlPath,
List<? extends Resource> locations, ResourceResolverChain chain);
}

View File

@ -21,13 +21,18 @@ import java.io.IOException;
import java.io.StringWriter;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Scanner;
import java.util.function.Consumer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import reactor.core.Exceptions;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import reactor.core.publisher.SynchronousSink;
import org.springframework.core.io.Resource;
import org.springframework.util.DigestUtils;
@ -62,15 +67,18 @@ import org.springframework.web.server.ServerWebExchange;
*/
public class AppCacheManifestTransformer extends ResourceTransformerSupport {
private static final Collection<String> MANIFEST_SECTION_HEADERS =
Arrays.asList("CACHE MANIFEST", "NETWORK:", "FALLBACK:", "CACHE:");
private static final String MANIFEST_HEADER = "CACHE MANIFEST";
private static final String CACHE_HEADER = "CACHE:";
private static final Charset DEFAULT_CHARSET = StandardCharsets.UTF_8;
private static final Log logger = LogFactory.getLog(AppCacheManifestTransformer.class);
private final Map<String, SectionTransformer> sectionTransformers = new HashMap<>();
private final String fileExtension;
@ -87,144 +95,198 @@ public class AppCacheManifestTransformer extends ResourceTransformerSupport {
*/
public AppCacheManifestTransformer(String fileExtension) {
this.fileExtension = fileExtension;
SectionTransformer noOpSection = new NoOpSection();
this.sectionTransformers.put(MANIFEST_HEADER, noOpSection);
this.sectionTransformers.put("NETWORK:", noOpSection);
this.sectionTransformers.put("FALLBACK:", noOpSection);
this.sectionTransformers.put("CACHE:", new CacheSection());
}
@Override
public Resource transform(ServerWebExchange exchange, Resource resource,
ResourceTransformerChain transformerChain) throws IOException {
public Mono<Resource> transform(ServerWebExchange exchange, Resource inputResource,
ResourceTransformerChain chain) {
resource = transformerChain.transform(exchange, resource);
if (!this.fileExtension.equals(StringUtils.getFilenameExtension(resource.getFilename()))) {
return resource;
}
byte[] bytes = FileCopyUtils.copyToByteArray(resource.getInputStream());
String content = new String(bytes, DEFAULT_CHARSET);
if (!content.startsWith(MANIFEST_HEADER)) {
if (logger.isTraceEnabled()) {
logger.trace("AppCache manifest does not start with 'CACHE MANIFEST', skipping: " + resource);
}
return resource;
}
if (logger.isTraceEnabled()) {
logger.trace("Transforming resource: " + resource);
}
StringWriter contentWriter = new StringWriter();
HashBuilder hashBuilder = new HashBuilder(content.length());
Scanner scanner = new Scanner(content);
SectionTransformer currentTransformer = this.sectionTransformers.get(MANIFEST_HEADER);
while (scanner.hasNextLine()) {
String line = scanner.nextLine();
if (this.sectionTransformers.containsKey(line.trim())) {
currentTransformer = this.sectionTransformers.get(line.trim());
contentWriter.write(line + "\n");
hashBuilder.appendString(line);
}
else {
contentWriter.write(
currentTransformer.transform(
line, hashBuilder, resource, transformerChain, exchange) + "\n");
}
}
String hash = hashBuilder.build();
contentWriter.write("\n" + "# Hash: " + hash);
if (logger.isTraceEnabled()) {
logger.trace("AppCache file: [" + resource.getFilename()+ "] hash: [" + hash + "]");
}
return new TransformedResource(resource, contentWriter.toString().getBytes(DEFAULT_CHARSET));
return chain.transform(exchange, inputResource)
.then(resource -> {
String name = resource.getFilename();
if (!this.fileExtension.equals(StringUtils.getFilenameExtension(name))) {
return Mono.just(resource);
}
String content = new String(getResourceBytes(resource), DEFAULT_CHARSET);
if (!content.startsWith(MANIFEST_HEADER)) {
if (logger.isTraceEnabled()) {
logger.trace("Manifest should start with 'CACHE MANIFEST', skip: " + resource);
}
return Mono.just(resource);
}
if (logger.isTraceEnabled()) {
logger.trace("Transforming resource: " + resource);
}
return Flux.generate(new LineGenerator(content))
.concatMap(info -> processLine(info, exchange, resource, chain))
.collect(() -> new LineAggregator(resource, content), LineAggregator::add)
.then(aggregator -> Mono.just(aggregator.createResource()));
});
}
@FunctionalInterface
private interface SectionTransformer {
/**
* Transforms a line in a section of the manifest.
* <p>The actual transformation depends on the chosen transformation strategy
* for the current manifest section (CACHE, NETWORK, FALLBACK, etc).
*/
String transform(String line, HashBuilder builder, Resource resource,
ResourceTransformerChain transformerChain, ServerWebExchange exchange) throws IOException;
}
private static class NoOpSection implements SectionTransformer {
public String transform(String line, HashBuilder builder, Resource resource,
ResourceTransformerChain transformerChain, ServerWebExchange exchange) throws IOException {
builder.appendString(line);
return line;
private static byte[] getResourceBytes(Resource resource) {
try {
return FileCopyUtils.copyToByteArray(resource.getInputStream());
}
catch (IOException ex) {
throw Exceptions.propagate(ex);
}
}
private Mono<LineOutput> processLine(LineInfo info, ServerWebExchange exchange,
Resource resource, ResourceTransformerChain chain) {
private class CacheSection implements SectionTransformer {
if (!info.isLink()) {
return Mono.just(new LineOutput(info.getLine(), null));
}
Mono<String> pathMono = resolveUrlPath(info.getLine(), exchange, resource, chain)
.doOnNext(path -> {
if (logger.isTraceEnabled()) {
logger.trace("Link modified: " + path + " (original: " + info.getLine() + ")");
}
});
Mono<Resource> resourceMono = chain.getResolverChain()
.resolveResource(null, info.getLine(), Collections.singletonList(resource));
return Flux.zip(pathMono, resourceMono, LineOutput::new).next();
}
private static class LineGenerator implements Consumer<SynchronousSink<LineInfo>> {
private final Scanner scanner;
private LineInfo previous;
public LineGenerator(String content) {
this.scanner = new Scanner(content);
}
private static final String COMMENT_DIRECTIVE = "#";
@Override
public String transform(String line, HashBuilder builder, Resource resource,
ResourceTransformerChain transformerChain, ServerWebExchange exchange) throws IOException {
if (isLink(line) && !hasScheme(line)) {
ResourceResolverChain resolverChain = transformerChain.getResolverChain();
Resource appCacheResource =
resolverChain.resolveResource(null, line, Collections.singletonList(resource));
String path = resolveUrlPath(line, exchange, resource, transformerChain);
builder.appendResource(appCacheResource);
if (logger.isTraceEnabled()) {
logger.trace("Link modified: " + path + " (original: " + line + ")");
}
return path;
public void accept(SynchronousSink<LineInfo> sink) {
if (this.scanner.hasNext()) {
String line = this.scanner.nextLine();
LineInfo current = new LineInfo(line, this.previous);
sink.next(current);
this.previous = current;
}
else {
sink.complete();
}
builder.appendString(line);
return line;
}
private boolean hasScheme(String link) {
int schemeIndex = link.indexOf(":");
return (link.startsWith("//") || (schemeIndex > 0 && !link.substring(0, schemeIndex).contains("/")));
}
private boolean isLink(String line) {
return (StringUtils.hasText(line) && !line.startsWith(COMMENT_DIRECTIVE));
}
}
private static class LineInfo {
private static class HashBuilder {
private final String line;
private final boolean cacheSection;
private final boolean link;
public LineInfo(String line, LineInfo previousLine) {
this.line = line;
this.cacheSection = initCacheSectionFlag(line, previousLine);
this.link = iniLinkFlag(line, this.cacheSection);
}
private static boolean initCacheSectionFlag(String line, LineInfo previousLine) {
if (MANIFEST_SECTION_HEADERS.contains(line.trim())) {
return line.trim().equals(CACHE_HEADER);
}
else if (previousLine != null) {
return previousLine.isCacheSection();
}
throw new IllegalStateException(
"Manifest does not start with " + MANIFEST_HEADER + ": " + line);
}
private static boolean iniLinkFlag(String line, boolean isCacheSection) {
return (isCacheSection && StringUtils.hasText(line) && !line.startsWith("#")
&& !line.startsWith("//") && !hasScheme(line));
}
private static boolean hasScheme(String line) {
int index = line.indexOf(":");
return (line.startsWith("//") || (index > 0 && !line.substring(0, index).contains("/")));
}
public String getLine() {
return this.line;
}
public boolean isCacheSection() {
return this.cacheSection;
}
public boolean isLink() {
return this.link;
}
}
private static class LineOutput {
private final String line;
private final Resource resource;
public LineOutput(String line, Resource resource) {
this.line = line;
this.resource = resource;
}
public String getLine() {
return this.line;
}
public Resource getResource() {
return this.resource;
}
}
private static class LineAggregator {
private final StringWriter writer = new StringWriter();
private final ByteArrayOutputStream baos;
public HashBuilder(int initialSize) {
this.baos = new ByteArrayOutputStream(initialSize);
private final Resource resource;
public LineAggregator(Resource resource, String content) {
this.resource = resource;
this.baos = new ByteArrayOutputStream(content.length());
}
public void appendResource(Resource resource) throws IOException {
byte[] content = FileCopyUtils.copyToByteArray(resource.getInputStream());
this.baos.write(DigestUtils.md5Digest(content));
public void add(LineOutput lineOutput) {
this.writer.write(lineOutput.getLine() + "\n");
try {
byte[] bytes = (lineOutput.getResource() != null ?
DigestUtils.md5Digest(getResourceBytes(lineOutput.getResource())) :
lineOutput.getLine().getBytes(DEFAULT_CHARSET));
this.baos.write(bytes);
}
catch (IOException ex) {
throw Exceptions.propagate(ex);
}
}
public void appendString(String content) throws IOException {
this.baos.write(content.getBytes(DEFAULT_CHARSET));
}
public String build() {
return DigestUtils.md5DigestAsHex(this.baos.toByteArray());
public TransformedResource createResource() {
String hash = DigestUtils.md5DigestAsHex(this.baos.toByteArray());
this.writer.write("\n" + "# Hash: " + hash);
if (logger.isTraceEnabled()) {
logger.trace("AppCache file: [" + resource.getFilename()+ "] hash: [" + hash + "]");
}
byte[] bytes = this.writer.toString().getBytes(DEFAULT_CHARSET);
return new TransformedResource(this.resource, bytes);
}
}

View File

@ -18,6 +18,8 @@ package org.springframework.web.reactive.resource;
import java.util.List;
import reactor.core.publisher.Mono;
import org.springframework.cache.Cache;
import org.springframework.cache.CacheManager;
import org.springframework.core.io.Resource;
@ -61,28 +63,26 @@ public class CachingResourceResolver extends AbstractResourceResolver {
@Override
protected Resource resolveResourceInternal(ServerWebExchange exchange, String requestPath,
protected Mono<Resource> resolveResourceInternal(ServerWebExchange exchange, String requestPath,
List<? extends Resource> locations, ResourceResolverChain chain) {
String key = computeKey(exchange, requestPath);
Resource resource = this.cache.get(key, Resource.class);
Resource cachedResource = this.cache.get(key, Resource.class);
if (resource != null) {
if (cachedResource != null) {
if (logger.isTraceEnabled()) {
logger.trace("Found match: " + resource);
logger.trace("Found match: " + cachedResource);
}
return resource;
return Mono.just(cachedResource);
}
resource = chain.resolveResource(exchange, requestPath, locations);
if (resource != null) {
if (logger.isTraceEnabled()) {
logger.trace("Putting resolved resource in cache: " + resource);
}
this.cache.put(key, resource);
}
return resource;
return chain.resolveResource(exchange, requestPath, locations)
.doOnNext(resource -> {
if (logger.isTraceEnabled()) {
logger.trace("Putting resolved resource in cache: " + resource);
}
this.cache.put(key, resource);
});
}
protected String computeKey(ServerWebExchange exchange, String requestPath) {
@ -98,28 +98,26 @@ public class CachingResourceResolver extends AbstractResourceResolver {
}
@Override
protected String resolveUrlPathInternal(String resourceUrlPath,
protected Mono<String> resolveUrlPathInternal(String resourceUrlPath,
List<? extends Resource> locations, ResourceResolverChain chain) {
String key = RESOLVED_URL_PATH_CACHE_KEY_PREFIX + resourceUrlPath;
String resolvedUrlPath = this.cache.get(key, String.class);
String cachedUrlPath = this.cache.get(key, String.class);
if (resolvedUrlPath != null) {
if (cachedUrlPath != null) {
if (logger.isTraceEnabled()) {
logger.trace("Found match: \"" + resolvedUrlPath + "\"");
logger.trace("Found match: \"" + cachedUrlPath + "\"");
}
return resolvedUrlPath;
return Mono.just(cachedUrlPath);
}
resolvedUrlPath = chain.resolveUrlPath(resourceUrlPath, locations);
if (resolvedUrlPath != null) {
if (logger.isTraceEnabled()) {
logger.trace("Putting resolved resource URL path in cache: \"" + resolvedUrlPath + "\"");
}
this.cache.put(key, resolvedUrlPath);
}
return resolvedUrlPath;
return chain.resolveUrlPath(resourceUrlPath, locations)
.doOnNext(resolvedPath -> {
if (logger.isTraceEnabled()) {
logger.trace("Putting resolved resource URL path in cache: \"" + resolvedPath + "\"");
}
this.cache.put(key, resolvedPath);
});
}
}

View File

@ -16,10 +16,9 @@
package org.springframework.web.reactive.resource;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import reactor.core.publisher.Mono;
import org.springframework.cache.Cache;
import org.springframework.cache.CacheManager;
@ -61,25 +60,24 @@ public class CachingResourceTransformer implements ResourceTransformer {
@Override
public Resource transform(ServerWebExchange exchange, Resource resource,
ResourceTransformerChain transformerChain) throws IOException {
public Mono<Resource> transform(ServerWebExchange exchange, Resource resource,
ResourceTransformerChain transformerChain) {
Resource transformed = this.cache.get(resource, Resource.class);
if (transformed != null) {
Resource cachedResource = this.cache.get(resource, Resource.class);
if (cachedResource != null) {
if (logger.isTraceEnabled()) {
logger.trace("Found match: " + transformed);
logger.trace("Found match: " + cachedResource);
}
return transformed;
return Mono.just(cachedResource);
}
transformed = transformerChain.transform(exchange, resource);
if (logger.isTraceEnabled()) {
logger.trace("Putting transformed resource in cache: " + transformed);
}
this.cache.put(resource, transformed);
return transformed;
return transformerChain.transform(exchange, resource)
.doOnNext(transformed -> {
if (logger.isTraceEnabled()) {
logger.trace("Putting transformed resource in cache: " + transformed);
}
this.cache.put(resource, transformed);
});
}
}

View File

@ -43,8 +43,7 @@ public class ContentVersionStrategy extends AbstractVersionStrategy {
@Override
public String getResourceVersion(Resource resource) {
try {
byte[] content = FileCopyUtils.copyToByteArray(resource.getInputStream());
return DigestUtils.md5DigestAsHex(content);
return DigestUtils.md5DigestAsHex(resource.getInputStream());
}
catch (IOException ex) {
throw new IllegalStateException("Failed to calculate hash for " + resource, ex);

View File

@ -28,6 +28,9 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import reactor.core.Exceptions;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import org.springframework.core.io.Resource;
import org.springframework.util.FileCopyUtils;
@ -54,71 +57,95 @@ public class CssLinkResourceTransformer extends ResourceTransformerSupport {
private static final Log logger = LogFactory.getLog(CssLinkResourceTransformer.class);
private final List<CssLinkParser> linkParsers = new ArrayList<>(2);
private final List<LinkParser> linkParsers = new ArrayList<>(2);
public CssLinkResourceTransformer() {
this.linkParsers.add(new ImportStatementCssLinkParser());
this.linkParsers.add(new UrlFunctionCssLinkParser());
this.linkParsers.add(new ImportLinkParser());
this.linkParsers.add(new UrlFunctionLinkParser());
}
@Override
public Resource transform(ServerWebExchange exchange, Resource resource,
ResourceTransformerChain transformerChain) throws IOException {
public Mono<Resource> transform(ServerWebExchange exchange, Resource resource,
ResourceTransformerChain transformerChain) {
resource = transformerChain.transform(exchange, resource);
return transformerChain.transform(exchange, resource)
.then(newResource -> {
String filename = newResource.getFilename();
if (!"css".equals(StringUtils.getFilenameExtension(filename))) {
return Mono.just(newResource);
}
String filename = resource.getFilename();
if (!"css".equals(StringUtils.getFilenameExtension(filename))) {
return resource;
if (logger.isTraceEnabled()) {
logger.trace("Transforming resource: " + newResource);
}
byte[] bytes = new byte[0];
try {
bytes = FileCopyUtils.copyToByteArray(newResource.getInputStream());
}
catch (IOException ex) {
return Mono.error(Exceptions.propagate(ex));
}
String fullContent = new String(bytes, DEFAULT_CHARSET);
List<Segment> segments = parseContent(fullContent);
if (segments.isEmpty()) {
if (logger.isTraceEnabled()) {
logger.trace("No links found.");
}
return Mono.just(newResource);
}
return Flux.fromIterable(segments)
.concatMap(segment -> {
String segmentContent = segment.getContent(fullContent);
if (segment.isLink() && !hasScheme(segmentContent)) {
return resolveUrlPath(segmentContent, exchange, newResource, transformerChain)
.defaultIfEmpty(segmentContent);
}
else {
return Mono.just(segmentContent);
}
})
.reduce(new StringWriter(), (writer, chunk) -> {
writer.write(chunk);
return writer;
})
.then(writer -> {
byte[] newContent = writer.toString().getBytes(DEFAULT_CHARSET);
return Mono.just(new TransformedResource(resource, newContent));
});
});
}
private List<Segment> parseContent(String fullContent) {
List<Segment> links = new ArrayList<>();
for (LinkParser parser : this.linkParsers) {
links.addAll(parser.parseLinks(fullContent));
}
if (logger.isTraceEnabled()) {
logger.trace("Transforming resource: " + resource);
if (links.isEmpty()) {
return Collections.emptyList();
}
byte[] bytes = FileCopyUtils.copyToByteArray(resource.getInputStream());
String content = new String(bytes, DEFAULT_CHARSET);
Set<CssLinkInfo> infos = new HashSet<>(8);
for (CssLinkParser parser : this.linkParsers) {
parser.parseLink(content, infos);
}
if (infos.isEmpty()) {
if (logger.isTraceEnabled()) {
logger.trace("No links found.");
}
return resource;
}
List<CssLinkInfo> sortedInfos = new ArrayList<>(infos);
Collections.sort(sortedInfos);
Collections.sort(links);
int index = 0;
StringWriter writer = new StringWriter();
for (CssLinkInfo info : sortedInfos) {
writer.write(content.substring(index, info.getStart()));
String link = content.substring(info.getStart(), info.getEnd());
String newLink = null;
if (!hasScheme(link)) {
newLink = resolveUrlPath(link, exchange, resource, transformerChain);
}
if (logger.isTraceEnabled()) {
if (newLink != null && !link.equals(newLink)) {
logger.trace("Link modified: " + newLink + " (original: " + link + ")");
}
else {
logger.trace("Link not modified: " + link);
}
}
writer.write(newLink != null ? newLink : link);
index = info.getEnd();
List<Segment> allSegments = new ArrayList<>(links);
for (Segment link : links) {
allSegments.add(new Segment(index, link.getStart(), false));
index = link.getEnd();
}
if (index < fullContent.length()) {
allSegments.add(new Segment(index, fullContent.length(), false));
}
writer.write(content.substring(index));
return new TransformedResource(resource, writer.toString().getBytes(DEFAULT_CHARSET));
Collections.sort(allSegments);
return allSegments;
}
private boolean hasScheme(String link) {
@ -128,40 +155,41 @@ public class CssLinkResourceTransformer extends ResourceTransformerSupport {
@FunctionalInterface
protected interface CssLinkParser {
protected interface LinkParser {
Set<Segment> parseLinks(String fullContent);
void parseLink(String content, Set<CssLinkInfo> linkInfos);
}
protected static abstract class AbstractCssLinkParser implements CssLinkParser {
protected static abstract class AbstractLinkParser implements LinkParser {
/**
* Return the keyword to use to search for links.
*/
/** Return the keyword to use to search for links. */
protected abstract String getKeyword();
@Override
public void parseLink(String content, Set<CssLinkInfo> linkInfos) {
public Set<Segment> parseLinks(String fullContent) {
Set<Segment> linksToAdd = new HashSet<>(8);
int index = 0;
do {
index = content.indexOf(getKeyword(), index);
index = fullContent.indexOf(getKeyword(), index);
if (index == -1) {
break;
}
index = skipWhitespace(content, index + getKeyword().length());
if (content.charAt(index) == '\'') {
index = addLink(index, "'", content, linkInfos);
index = skipWhitespace(fullContent, index + getKeyword().length());
if (fullContent.charAt(index) == '\'') {
index = addLink(index, "'", fullContent, linksToAdd);
}
else if (content.charAt(index) == '"') {
index = addLink(index, "\"", content, linkInfos);
else if (fullContent.charAt(index) == '"') {
index = addLink(index, "\"", fullContent, linksToAdd);
}
else {
index = extractLink(index, content, linkInfos);
index = extractLink(index, fullContent, linksToAdd);
}
}
while (true);
return linksToAdd;
}
private int skipWhitespace(String content, int index) {
@ -174,10 +202,10 @@ public class CssLinkResourceTransformer extends ResourceTransformerSupport {
}
}
protected int addLink(int index, String endKey, String content, Set<CssLinkInfo> linkInfos) {
protected int addLink(int index, String endKey, String content, Set<Segment> linksToAdd) {
int start = index + 1;
int end = content.indexOf(endKey, start);
linkInfos.add(new CssLinkInfo(start, end));
linksToAdd.add(new Segment(start, end, true));
return end + endKey.length();
}
@ -185,12 +213,12 @@ public class CssLinkResourceTransformer extends ResourceTransformerSupport {
* Invoked after a keyword match, after whitespaces removed, and when
* the next char is neither a single nor double quote.
*/
protected abstract int extractLink(int index, String content, Set<CssLinkInfo> linkInfos);
protected abstract int extractLink(int index, String content, Set<Segment> linksToAdd);
}
private static class ImportStatementCssLinkParser extends AbstractCssLinkParser {
private static class ImportLinkParser extends AbstractLinkParser {
@Override
protected String getKeyword() {
@ -198,7 +226,7 @@ public class CssLinkResourceTransformer extends ResourceTransformerSupport {
}
@Override
protected int extractLink(int index, String content, Set<CssLinkInfo> linkInfos) {
protected int extractLink(int index, String content, Set<Segment> linksToAdd) {
if (content.substring(index, index + 4).equals("url(")) {
// Ignore, UrlLinkParser will take care
}
@ -210,7 +238,7 @@ public class CssLinkResourceTransformer extends ResourceTransformerSupport {
}
private static class UrlFunctionCssLinkParser extends AbstractCssLinkParser {
private static class UrlFunctionLinkParser extends AbstractLinkParser {
@Override
protected String getKeyword() {
@ -218,24 +246,29 @@ public class CssLinkResourceTransformer extends ResourceTransformerSupport {
}
@Override
protected int extractLink(int index, String content, Set<CssLinkInfo> linkInfos) {
protected int extractLink(int index, String content, Set<Segment> linksToAdd) {
// A url() function without unquoted
return addLink(index - 1, ")", content, linkInfos);
return addLink(index - 1, ")", content, linksToAdd);
}
}
private static class CssLinkInfo implements Comparable<CssLinkInfo> {
private static class Segment implements Comparable<Segment> {
private final int start;
private final int end;
public CssLinkInfo(int start, int end) {
private final boolean link;
public Segment(int start, int end, boolean isLink) {
this.start = start;
this.end = end;
this.link = isLink;
}
public int getStart() {
return this.start;
}
@ -244,8 +277,16 @@ public class CssLinkResourceTransformer extends ResourceTransformerSupport {
return this.end;
}
public boolean isLink() {
return this.link;
}
public String getContent(String fullContent) {
return fullContent.substring(this.start, this.end);
}
@Override
public int compareTo(CssLinkInfo other) {
public int compareTo(Segment other) {
return (this.start < other.start ? -1 : (this.start == other.start ? 0 : 1));
}
@ -254,8 +295,8 @@ public class CssLinkResourceTransformer extends ResourceTransformerSupport {
if (this == obj) {
return true;
}
if (obj != null && obj instanceof CssLinkInfo) {
CssLinkInfo other = (CssLinkInfo) obj;
if (obj != null && obj instanceof Segment) {
Segment other = (Segment) obj;
return (this.start == other.start && this.end == other.end);
}
return false;

View File

@ -19,6 +19,8 @@ package org.springframework.web.reactive.resource;
import java.util.ArrayList;
import java.util.List;
import reactor.core.publisher.Mono;
import org.springframework.core.io.Resource;
import org.springframework.util.Assert;
import org.springframework.web.server.ServerWebExchange;
@ -45,7 +47,7 @@ class DefaultResourceResolverChain implements ResourceResolverChain {
@Override
public Resource resolveResource(ServerWebExchange exchange, String requestPath,
public Mono<Resource> resolveResource(ServerWebExchange exchange, String requestPath,
List<? extends Resource> locations) {
ResourceResolver resolver = getNext();
@ -62,7 +64,7 @@ class DefaultResourceResolverChain implements ResourceResolverChain {
}
@Override
public String resolveUrlPath(String resourcePath, List<? extends Resource> locations) {
public Mono<String> resolveUrlPath(String resourcePath, List<? extends Resource> locations) {
ResourceResolver resolver = getNext();
if (resolver == null) {
return null;

View File

@ -16,10 +16,11 @@
package org.springframework.web.reactive.resource;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import reactor.core.publisher.Mono;
import org.springframework.core.io.Resource;
import org.springframework.util.Assert;
import org.springframework.web.server.ServerWebExchange;
@ -57,10 +58,10 @@ class DefaultResourceTransformerChain implements ResourceTransformerChain {
@Override
public Resource transform(ServerWebExchange exchange, Resource resource) throws IOException {
public Mono<Resource> transform(ServerWebExchange exchange, Resource resource) {
ResourceTransformer transformer = getNext();
if (transformer == null) {
return resource;
return Mono.just(resource);
}
try {
return transformer.transform(exchange, resource, this);

View File

@ -23,6 +23,8 @@ import java.net.URI;
import java.net.URL;
import java.util.List;
import reactor.core.publisher.Mono;
import org.springframework.core.io.AbstractResource;
import org.springframework.core.io.Resource;
import org.springframework.http.HttpHeaders;
@ -41,25 +43,24 @@ import org.springframework.web.server.ServerWebExchange;
public class GzipResourceResolver extends AbstractResourceResolver {
@Override
protected Resource resolveResourceInternal(ServerWebExchange exchange, String requestPath,
protected Mono<Resource> resolveResourceInternal(ServerWebExchange exchange, String requestPath,
List<? extends Resource> locations, ResourceResolverChain chain) {
Resource resource = chain.resolveResource(exchange, requestPath, locations);
if ((resource == null) || (exchange != null && !isGzipAccepted(exchange))) {
return resource;
}
try {
Resource gzipped = new GzippedResource(resource);
if (gzipped.exists()) {
return gzipped;
}
}
catch (IOException ex) {
logger.trace("No gzipped resource for [" + resource.getFilename() + "]", ex);
}
return resource;
return chain.resolveResource(exchange, requestPath, locations)
.map(resource -> {
if (exchange == null || isGzipAccepted(exchange)) {
try {
Resource gzipped = new GzippedResource(resource);
if (gzipped.exists()) {
resource = gzipped;
}
}
catch (IOException ex) {
logger.trace("No gzipped resource for [" + resource.getFilename() + "]", ex);
}
}
return resource;
});
}
private boolean isGzipAccepted(ServerWebExchange exchange) {
@ -68,8 +69,8 @@ public class GzipResourceResolver extends AbstractResourceResolver {
}
@Override
protected String resolveUrlPathInternal(String resourceUrlPath, List<? extends Resource> locations,
ResourceResolverChain chain) {
protected Mono<String> resolveUrlPathInternal(String resourceUrlPath,
List<? extends Resource> locations, ResourceResolverChain chain) {
return chain.resolveUrlPath(resourceUrlPath, locations);
}

View File

@ -21,6 +21,9 @@ import java.net.URLDecoder;
import java.util.Arrays;
import java.util.List;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import org.springframework.core.io.ClassPathResource;
import org.springframework.core.io.Resource;
import org.springframework.core.io.UrlResource;
@ -67,41 +70,28 @@ public class PathResourceResolver extends AbstractResourceResolver {
@Override
protected Resource resolveResourceInternal(ServerWebExchange exchange, String requestPath,
protected Mono<Resource> resolveResourceInternal(ServerWebExchange exchange, String requestPath,
List<? extends Resource> locations, ResourceResolverChain chain) {
return getResource(requestPath, locations);
}
@Override
protected String resolveUrlPathInternal(String path, List<? extends Resource> locations,
protected Mono<String> resolveUrlPathInternal(String path, List<? extends Resource> locations,
ResourceResolverChain chain) {
return (StringUtils.hasText(path) && getResource(path, locations) != null ? path : null);
if (StringUtils.hasText(path)) {
return getResource(path, locations).map(resource -> path);
}
else {
return Mono.empty();
}
}
private Resource getResource(String resourcePath, List<? extends Resource> locations) {
for (Resource location : locations) {
try {
if (logger.isTraceEnabled()) {
logger.trace("Checking location: " + location);
}
Resource resource = getResource(resourcePath, location);
if (resource != null) {
if (logger.isTraceEnabled()) {
logger.trace("Found match: " + resource);
}
return resource;
}
else if (logger.isTraceEnabled()) {
logger.trace("No match for location: " + location);
}
}
catch (IOException ex) {
logger.trace("Failure checking for relative resource - trying next location", ex);
}
}
return null;
private Mono<Resource> getResource(String resourcePath, List<? extends Resource> locations) {
return Flux.fromIterable(locations)
.concatMap(location -> getResource(resourcePath, location))
.next();
}
/**
@ -112,20 +102,34 @@ public class PathResourceResolver extends AbstractResourceResolver {
* @param location the location to check
* @return the resource, or {@code null} if none found
*/
protected Resource getResource(String resourcePath, Resource location) throws IOException {
Resource resource = location.createRelative(resourcePath);
if (resource.exists() && resource.isReadable()) {
if (checkResource(resource, location)) {
return resource;
protected Mono<Resource> getResource(String resourcePath, Resource location) {
try {
Resource resource = location.createRelative(resourcePath);
if (resource.exists() && resource.isReadable()) {
if (checkResource(resource, location)) {
if (logger.isTraceEnabled()) {
logger.trace("Found match: " + resource);
}
return Mono.just(resource);
}
else if (logger.isTraceEnabled()) {
logger.trace("Resource path=\"" + resourcePath + "\" was successfully resolved " +
"but resource=\"" + resource.getURL() + "\" is neither under the " +
"current location=\"" + location.getURL() + "\" nor under any of the " +
"allowed locations=" + Arrays.asList(getAllowedLocations()));
}
}
else if (logger.isTraceEnabled()) {
logger.trace("Resource path=\"" + resourcePath + "\" was successfully resolved " +
"but resource=\"" + resource.getURL() + "\" is neither under the " +
"current location=\"" + location.getURL() + "\" nor under any of the " +
"allowed locations=" + Arrays.asList(getAllowedLocations()));
logger.trace("No match for location: " + location);
}
return Mono.empty();
}
catch (IOException ex) {
if (logger.isTraceEnabled()) {
logger.trace("Failure checking for relative resource under location + " + location, ex);
}
return Mono.error(ex);
}
return null;
}
/**

View File

@ -18,6 +18,8 @@ package org.springframework.web.reactive.resource;
import java.util.List;
import reactor.core.publisher.Mono;
import org.springframework.core.io.Resource;
import org.springframework.web.server.ServerWebExchange;
@ -40,10 +42,10 @@ public interface ResourceResolver {
* @param requestPath the portion of the request path to use
* @param locations the locations to search in when looking up resources
* @param chain the chain of remaining resolvers to delegate to
* @return the resolved resource or {@code null} if unresolved
* @return the resolved resource or an empty {@code Mono} if unresolved
*/
Resource resolveResource(ServerWebExchange exchange, String requestPath, List<? extends Resource> locations,
ResourceResolverChain chain);
Mono<Resource> resolveResource(ServerWebExchange exchange, String requestPath,
List<? extends Resource> locations, ResourceResolverChain chain);
/**
* Resolve the externally facing <em>public</em> URL path for clients to use
@ -53,8 +55,9 @@ public interface ResourceResolver {
* @param resourcePath the internal resource path
* @param locations the locations to search in when looking up resources
* @param chain the chain of resolvers to delegate to
* @return the resolved public URL path or {@code null} if unresolved
* @return the resolved public URL path or an empty {@code Mono} if unresolved
*/
String resolveUrlPath(String resourcePath, List<? extends Resource> locations, ResourceResolverChain chain);
Mono<String> resolveUrlPath(String resourcePath, List<? extends Resource> locations,
ResourceResolverChain chain);
}

View File

@ -18,6 +18,8 @@ package org.springframework.web.reactive.resource;
import java.util.List;
import reactor.core.publisher.Mono;
import org.springframework.core.io.Resource;
import org.springframework.web.server.ServerWebExchange;
@ -36,9 +38,10 @@ public interface ResourceResolverChain {
* @param exchange the current exchange
* @param requestPath the portion of the request path to use
* @param locations the locations to search in when looking up resources
* @return the resolved resource or {@code null} if unresolved
* @return the resolved resource or an empty {@code Mono} if unresolved
*/
Resource resolveResource(ServerWebExchange exchange, String requestPath, List<? extends Resource> locations);
Mono<Resource> resolveResource(ServerWebExchange exchange, String requestPath,
List<? extends Resource> locations);
/**
* Resolve the externally facing <em>public</em> URL path for clients to use
@ -47,8 +50,8 @@ public interface ResourceResolverChain {
* <p>This is useful when rendering URL links to clients.
* @param resourcePath the internal resource path
* @param locations the locations to search in when looking up resources
* @return the resolved public URL path or {@code null} if unresolved
* @return the resolved public URL path or an empty {@code Mono} if unresolved
*/
String resolveUrlPath(String resourcePath, List<? extends Resource> locations);
Mono<String> resolveUrlPath(String resourcePath, List<? extends Resource> locations);
}

View File

@ -16,7 +16,7 @@
package org.springframework.web.reactive.resource;
import java.io.IOException;
import reactor.core.publisher.Mono;
import org.springframework.core.io.Resource;
import org.springframework.web.server.ServerWebExchange;
@ -36,9 +36,8 @@ public interface ResourceTransformer {
* @param resource the resource to transform
* @param transformerChain the chain of remaining transformers to delegate to
* @return the transformed resource (never {@code null})
* @throws IOException if the transformation fails
*/
Resource transform(ServerWebExchange exchange, Resource resource,
ResourceTransformerChain transformerChain) throws IOException;
Mono<Resource> transform(ServerWebExchange exchange, Resource resource,
ResourceTransformerChain transformerChain);
}

View File

@ -18,6 +18,8 @@ package org.springframework.web.reactive.resource;
import java.io.IOException;
import reactor.core.publisher.Mono;
import org.springframework.core.io.Resource;
import org.springframework.web.server.ServerWebExchange;
@ -42,8 +44,7 @@ public interface ResourceTransformerChain {
* @param exchange the current exchange
* @param resource the candidate resource to transform
* @return the transformed or the same resource, never {@code null}
* @throws IOException if transformation fails
*/
Resource transform(ServerWebExchange exchange, Resource resource) throws IOException;
Mono<Resource> transform(ServerWebExchange exchange, Resource resource);
}

View File

@ -18,6 +18,8 @@ package org.springframework.web.reactive.resource;
import java.util.Collections;
import reactor.core.publisher.Mono;
import org.springframework.core.io.Resource;
import org.springframework.web.server.ServerWebExchange;
@ -64,7 +66,7 @@ public abstract class ResourceTransformerSupport implements ResourceTransformer
* @param transformerChain the transformer chain
* @return the resolved URL or null
*/
protected String resolveUrlPath(String resourcePath, ServerWebExchange exchange,
protected Mono<String> resolveUrlPath(String resourcePath, ServerWebExchange exchange,
Resource resource, ResourceTransformerChain transformerChain) {
if (resourcePath.startsWith("/")) {

View File

@ -25,6 +25,8 @@ import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import org.springframework.context.ApplicationContext;
import org.springframework.context.ApplicationListener;
@ -167,7 +169,7 @@ public class ResourceUrlProvider implements ApplicationListener<ContextRefreshed
* @param requestUrl the request URL path to resolve
* @return the resolved public URL path, or {@code null} if unresolved
*/
public final String getForRequestUrl(ServerWebExchange exchange, String requestUrl) {
public final Mono<String> getForRequestUrl(ServerWebExchange exchange, String requestUrl) {
if (logger.isTraceEnabled()) {
logger.trace("Getting resource URL for request URL \"" + requestUrl + "\"");
}
@ -176,8 +178,7 @@ public class ResourceUrlProvider implements ApplicationListener<ContextRefreshed
String prefix = requestUrl.substring(0, prefixIndex);
String suffix = requestUrl.substring(suffixIndex);
String lookupPath = requestUrl.substring(prefixIndex, suffixIndex);
String resolvedLookupPath = getForLookupPath(lookupPath);
return (resolvedLookupPath != null ? prefix + resolvedLookupPath + suffix : null);
return getForLookupPath(lookupPath).map(resolvedPath -> prefix + resolvedPath + suffix);
}
private int getLookupPathIndex(ServerWebExchange exchange) {
@ -204,7 +205,7 @@ public class ResourceUrlProvider implements ApplicationListener<ContextRefreshed
* @param lookupPath the lookup path to check
* @return the resolved public URL path, or {@code null} if unresolved
*/
public final String getForLookupPath(String lookupPath) {
public final Mono<String> getForLookupPath(String lookupPath) {
if (logger.isTraceEnabled()) {
logger.trace("Getting resource URL for lookup path \"" + lookupPath + "\"");
}
@ -216,32 +217,31 @@ public class ResourceUrlProvider implements ApplicationListener<ContextRefreshed
}
}
if (!matchingPatterns.isEmpty()) {
Comparator<String> patternComparator = getPathMatcher().getPatternComparator(lookupPath);
Collections.sort(matchingPatterns, patternComparator);
for (String pattern : matchingPatterns) {
String pathWithinMapping = getPathMatcher().extractPathWithinPattern(pattern, lookupPath);
String pathMapping = lookupPath.substring(0, lookupPath.indexOf(pathWithinMapping));
if (logger.isTraceEnabled()) {
logger.trace("Invoking ResourceResolverChain for URL pattern \"" + pattern + "\"");
}
ResourceWebHandler handler = this.handlerMap.get(pattern);
ResourceResolverChain chain = new DefaultResourceResolverChain(handler.getResourceResolvers());
String resolved = chain.resolveUrlPath(pathWithinMapping, handler.getLocations());
if (resolved == null) {
continue;
}
if (logger.isTraceEnabled()) {
logger.trace("Resolved public resource URL path \"" + resolved + "\"");
}
return pathMapping + resolved;
}
if (matchingPatterns.isEmpty()) {
return Mono.empty();
}
if (logger.isDebugEnabled()) {
logger.debug("No matching resource mapping for lookup path \"" + lookupPath + "\"");
}
return null;
Comparator<String> patternComparator = getPathMatcher().getPatternComparator(lookupPath);
Collections.sort(matchingPatterns, patternComparator);
return Flux.fromIterable(matchingPatterns)
.concatMap(pattern -> {
String pathWithinMapping = getPathMatcher().extractPathWithinPattern(pattern, lookupPath);
String pathMapping = lookupPath.substring(0, lookupPath.indexOf(pathWithinMapping));
if (logger.isTraceEnabled()) {
logger.trace("Invoking ResourceResolverChain for URL pattern \"" + pattern + "\"");
}
ResourceWebHandler handler = this.handlerMap.get(pattern);
ResourceResolverChain chain = new DefaultResourceResolverChain(handler.getResourceResolvers());
return chain.resolveUrlPath(pathWithinMapping, handler.getLocations())
.map(resolvedPath -> {
if (logger.isTraceEnabled()) {
logger.trace("Resolved public resource URL path \"" + resolvedPath + "\"");
}
return pathMapping + resolvedPath;
});
})
.next();
}
}

View File

@ -17,6 +17,7 @@
package org.springframework.web.reactive.resource;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.net.URLDecoder;
import java.time.Instant;
import java.util.ArrayList;
@ -31,6 +32,7 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import reactor.core.Exceptions;
import reactor.core.publisher.Mono;
import org.springframework.beans.factory.InitializingBean;
@ -276,84 +278,92 @@ public class ResourceWebHandler
*/
@Override
public Mono<Void> handle(ServerWebExchange exchange) {
try {
// For very general mappings (e.g. "/") we need to check 404 first
Resource resource = getResource(exchange);
if (resource == null) {
logger.trace("No matching resource found - returning 404");
exchange.getResponse().setStatusCode(HttpStatus.NOT_FOUND);
return Mono.empty();
}
if (HttpMethod.OPTIONS.equals(exchange.getRequest().getMethod())) {
exchange.getResponse().getHeaders().add("Allow", "GET,HEAD,OPTIONS");
return Mono.empty();
}
return getResource(exchange)
.otherwiseIfEmpty(Mono.defer(() -> {
logger.trace("No matching resource found - returning 404");
exchange.getResponse().setStatusCode(HttpStatus.NOT_FOUND);
return Mono.empty();
}))
.then(resource -> {
try {
if (HttpMethod.OPTIONS.equals(exchange.getRequest().getMethod())) {
exchange.getResponse().getHeaders().add("Allow", "GET,HEAD,OPTIONS");
return Mono.empty();
}
// Supported methods and required session
String httpMehtod = exchange.getRequest().getMethod().name();
if (!SUPPORTED_METHODS.contains(httpMehtod)) {
return Mono.error(new MethodNotAllowedException(httpMehtod, SUPPORTED_METHODS));
}
// Supported methods and required session
String httpMehtod = exchange.getRequest().getMethod().name();
if (!SUPPORTED_METHODS.contains(httpMehtod)) {
return Mono.error(new MethodNotAllowedException(httpMehtod, SUPPORTED_METHODS));
}
// Header phase
if (exchange.checkNotModified(Instant.ofEpochMilli(resource.lastModified()))) {
logger.trace("Resource not modified - returning 304");
return Mono.empty();
}
// Header phase
if (exchange.checkNotModified(Instant.ofEpochMilli(resource.lastModified()))) {
logger.trace("Resource not modified - returning 304");
return Mono.empty();
}
// Apply cache settings, if any
if (getCacheControl() != null) {
String value = getCacheControl().getHeaderValue();
if (value != null) {
exchange.getResponse().getHeaders().setCacheControl(value);
}
}
// Apply cache settings, if any
if (getCacheControl() != null) {
String value = getCacheControl().getHeaderValue();
if (value != null) {
exchange.getResponse().getHeaders().setCacheControl(value);
}
}
// Check the media type for the resource
MediaType mediaType = getMediaType(exchange, resource);
if (mediaType != null) {
if (logger.isTraceEnabled()) {
logger.trace("Determined media type '" + mediaType + "' for " + resource);
}
}
else {
if (logger.isTraceEnabled()) {
logger.trace("No media type found for " + resource + " - not sending a content-type header");
}
}
// Check the media type for the resource
MediaType mediaType = getMediaType(exchange, resource);
if (mediaType != null) {
if (logger.isTraceEnabled()) {
logger.trace("Determined media type '" + mediaType + "' for " + resource);
}
}
else {
if (logger.isTraceEnabled()) {
logger.trace("No media type found " +
"for " + resource + " - not sending a content-type header");
}
}
// Content phase
if (HttpMethod.HEAD.equals(exchange.getRequest().getMethod())) {
setHeaders(exchange, resource, mediaType);
logger.trace("HEAD request - skipping content");
return Mono.empty();
}
// Content phase
if (HttpMethod.HEAD.equals(exchange.getRequest().getMethod())) {
setHeaders(exchange, resource, mediaType);
logger.trace("HEAD request - skipping content");
return Mono.empty();
}
// TODO: range requests
// TODO: range requests
setHeaders(exchange, resource, mediaType);
return this.resourceHttpMessageWriter.write(Mono.just(resource),
ResolvableType.forClass(Resource.class), mediaType, exchange.getResponse(), Collections.emptyMap());
}
catch (IOException ex) {
return Mono.error(ex);
}
setHeaders(exchange, resource, mediaType);
return this.resourceHttpMessageWriter.write(
Mono.just(resource), ResolvableType.forClass(Resource.class),
mediaType, exchange.getResponse(), Collections.emptyMap());
}
catch (IOException ex) {
return Mono.error(ex);
}
});
}
protected Resource getResource(ServerWebExchange exchange) throws IOException {
String attrName = HandlerMapping.PATH_WITHIN_HANDLER_MAPPING_ATTRIBUTE;
Optional<String> optional = exchange.getAttribute(attrName);
protected Mono<Resource> getResource(ServerWebExchange exchange) {
String attributeName = HandlerMapping.PATH_WITHIN_HANDLER_MAPPING_ATTRIBUTE;
Optional<String> optional = exchange.getAttribute(attributeName);
if (!optional.isPresent()) {
throw new IllegalStateException("Required request attribute '" + attrName + "' is not set");
return Mono.error(new IllegalStateException(
"Required request attribute '" + attributeName + "' is not set"));
}
String path = processPath(optional.get());
if (!StringUtils.hasText(path) || isInvalidPath(path)) {
if (logger.isTraceEnabled()) {
logger.trace("Ignoring invalid resource path [" + path + "]");
}
return null;
return Mono.empty();
}
if (path.contains("%")) {
try {
// Use URLDecoder (vs UriUtils) to preserve potentially decoded UTF-8 chars
@ -361,23 +371,23 @@ public class ResourceWebHandler
if (logger.isTraceEnabled()) {
logger.trace("Ignoring invalid resource path with escape sequences [" + path + "].");
}
return null;
return Mono.empty();
}
}
catch (IllegalArgumentException ex) {
// ignore
}
}
ResourceResolverChain resolveChain = new DefaultResourceResolverChain(getResourceResolvers());
Resource resource = resolveChain.resolveResource(exchange, path, getLocations());
if (resource == null || getResourceTransformers().isEmpty()) {
return resource;
catch (UnsupportedEncodingException ex) {
return Mono.error(Exceptions.propagate(ex));
}
}
ResourceTransformerChain transformChain =
new DefaultResourceTransformerChain(resolveChain, getResourceTransformers());
resource = transformChain.transform(exchange, resource);
return resource;
ResourceResolverChain resolveChain = createResolverChain();
return resolveChain.resolveResource(exchange, path, getLocations())
.then(resource -> {
ResourceTransformerChain transformerChain = createTransformerChain(resolveChain);
return transformerChain.transform(exchange, resource);
});
}
/**
@ -452,6 +462,14 @@ public class ResourceWebHandler
return false;
}
private ResourceResolverChain createResolverChain() {
return new DefaultResourceResolverChain(getResourceResolvers());
}
private ResourceTransformerChain createTransformerChain(ResourceResolverChain resolverChain) {
return new DefaultResourceTransformerChain(resolverChain, getResourceTransformers());
}
/**
* Determine the media type for the given request and the resource matched
* to it. This implementation tries to determine the MediaType based on the
@ -470,7 +488,6 @@ public class ResourceWebHandler
* @param exchange current exchange
* @param resource the identified resource (never {@code null})
* @param mediaType the resource's media type (never {@code null})
* @throws IOException in case of errors while setting the headers
*/
protected void setHeaders(ServerWebExchange exchange, Resource resource, MediaType mediaType)
throws IOException {

View File

@ -55,7 +55,7 @@ public class TransformedResource extends ByteArrayResource {
}
@Override
public long lastModified() throws IOException {
public long lastModified() {
return this.lastModified;
}

View File

@ -29,6 +29,8 @@ import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import reactor.core.publisher.Mono;
import org.springframework.core.io.AbstractResource;
import org.springframework.core.io.Resource;
import org.springframework.http.HttpHeaders;
@ -150,17 +152,20 @@ public class VersionResourceResolver extends AbstractResourceResolver {
@Override
protected Resource resolveResourceInternal(ServerWebExchange exchange, String requestPath,
protected Mono<Resource> resolveResourceInternal(ServerWebExchange exchange, String requestPath,
List<? extends Resource> locations, ResourceResolverChain chain) {
Resource resolved = chain.resolveResource(exchange, requestPath, locations);
if (resolved != null) {
return resolved;
}
return chain.resolveResource(exchange, requestPath, locations)
.otherwiseIfEmpty(Mono.defer(() ->
resolveVersionedResource(exchange, requestPath, locations, chain)));
}
private Mono<Resource> resolveVersionedResource(ServerWebExchange exchange, String requestPath,
List<? extends Resource> locations, ResourceResolverChain chain) {
VersionStrategy versionStrategy = getStrategyForPath(requestPath);
if (versionStrategy == null) {
return null;
return Mono.empty();
}
String candidateVersion = versionStrategy.extractVersion(requestPath);
@ -168,7 +173,7 @@ public class VersionResourceResolver extends AbstractResourceResolver {
if (logger.isTraceEnabled()) {
logger.trace("No version found in path \"" + requestPath + "\"");
}
return null;
return Mono.empty();
}
String simplePath = versionStrategy.removeVersion(requestPath, candidateVersion);
@ -176,49 +181,51 @@ public class VersionResourceResolver extends AbstractResourceResolver {
logger.trace("Extracted version from path, re-resolving without version: \"" + simplePath + "\"");
}
Resource baseResource = chain.resolveResource(exchange, simplePath, locations);
if (baseResource == null) {
return null;
}
String actualVersion = versionStrategy.getResourceVersion(baseResource);
if (candidateVersion.equals(actualVersion)) {
if (logger.isTraceEnabled()) {
logger.trace("Resource matches extracted version [" + candidateVersion + "]");
}
return new FileNameVersionedResource(baseResource, candidateVersion);
}
else {
if (logger.isTraceEnabled()) {
logger.trace("Potential resource found for \"" + requestPath + "\", but version [" +
candidateVersion + "] does not match");
}
return null;
}
return chain.resolveResource(exchange, simplePath, locations)
.then(baseResource -> {
String actualVersion = versionStrategy.getResourceVersion(baseResource);
if (candidateVersion.equals(actualVersion)) {
if (logger.isTraceEnabled()) {
logger.trace("Resource matches extracted version [" + candidateVersion + "]");
}
return Mono.just(new FileNameVersionedResource(baseResource, candidateVersion));
}
else {
if (logger.isTraceEnabled()) {
logger.trace("Potential resource found for \"" + requestPath + "\", but version [" +
candidateVersion + "] does not match");
}
return Mono.empty();
}
});
}
@Override
protected String resolveUrlPathInternal(String resourceUrlPath, List<? extends Resource> locations,
ResourceResolverChain chain) {
protected Mono<String> resolveUrlPathInternal(String resourceUrlPath,
List<? extends Resource> locations, ResourceResolverChain chain) {
String baseUrl = chain.resolveUrlPath(resourceUrlPath, locations);
if (StringUtils.hasText(baseUrl)) {
VersionStrategy versionStrategy = getStrategyForPath(resourceUrlPath);
if (versionStrategy == null) {
return null;
}
if (logger.isTraceEnabled()) {
logger.trace("Getting the original resource to determine version " +
"for path \"" + resourceUrlPath + "\"");
}
Resource resource = chain.resolveResource(null, baseUrl, locations);
String version = versionStrategy.getResourceVersion(resource);
if (logger.isTraceEnabled()) {
logger.trace("Determined version [" + version + "] for " + resource);
}
return versionStrategy.addVersion(baseUrl, version);
}
return baseUrl;
return chain.resolveUrlPath(resourceUrlPath, locations)
.then(baseUrl -> {
if (StringUtils.hasText(baseUrl)) {
VersionStrategy versionStrategy = getStrategyForPath(resourceUrlPath);
if (versionStrategy == null) {
return Mono.empty();
}
if (logger.isTraceEnabled()) {
logger.trace("Getting the original resource to determine version " +
"for path \"" + resourceUrlPath + "\"");
}
return chain.resolveResource(null, baseUrl, locations)
.map(resource -> {
String version = versionStrategy.getResourceVersion(resource);
if (logger.isTraceEnabled()) {
logger.trace("Determined version [" + version + "] for " + resource);
}
return versionStrategy.addVersion(baseUrl, version);
});
}
return Mono.empty();
});
}
/**

View File

@ -20,6 +20,7 @@ import java.util.List;
import org.webjars.MultipleMatchesException;
import org.webjars.WebJarAssetLocator;
import reactor.core.publisher.Mono;
import org.springframework.core.io.Resource;
import org.springframework.web.server.ServerWebExchange;
@ -70,31 +71,35 @@ public class WebJarsResourceResolver extends AbstractResourceResolver {
@Override
protected Resource resolveResourceInternal(ServerWebExchange exchange, String requestPath,
protected Mono<Resource> resolveResourceInternal(ServerWebExchange exchange, String requestPath,
List<? extends Resource> locations, ResourceResolverChain chain) {
Resource resolved = chain.resolveResource(exchange, requestPath, locations);
if (resolved == null) {
String webJarResourcePath = findWebJarResourcePath(requestPath);
if (webJarResourcePath != null) {
return chain.resolveResource(exchange, webJarResourcePath, locations);
}
}
return resolved;
return chain.resolveResource(exchange, requestPath, locations)
.otherwiseIfEmpty(Mono.defer(() -> {
String webJarsResourcePath = findWebJarResourcePath(requestPath);
if (webJarsResourcePath != null) {
return chain.resolveResource(exchange, webJarsResourcePath, locations);
}
else {
return Mono.empty();
}
}));
}
@Override
protected String resolveUrlPathInternal(String resourceUrlPath,
protected Mono<String> resolveUrlPathInternal(String resourceUrlPath,
List<? extends Resource> locations, ResourceResolverChain chain) {
String path = chain.resolveUrlPath(resourceUrlPath, locations);
if (path == null) {
String webJarResourcePath = findWebJarResourcePath(resourceUrlPath);
if (webJarResourcePath != null) {
return chain.resolveUrlPath(webJarResourcePath, locations);
}
}
return path;
return chain.resolveUrlPath(resourceUrlPath, locations)
.otherwiseIfEmpty(Mono.defer(() -> {
String webJarResourcePath = findWebJarResourcePath(resourceUrlPath);
if (webJarResourcePath != null) {
return chain.resolveUrlPath(webJarResourcePath, locations);
}
else {
return Mono.empty();
}
}));
}
protected String findWebJarResourcePath(String path) {

View File

@ -24,6 +24,7 @@ import java.util.List;
import org.hamcrest.Matchers;
import org.junit.Before;
import org.junit.Test;
import reactor.core.publisher.Mono;
import org.springframework.core.io.ClassPathResource;
import org.springframework.core.io.Resource;
@ -74,18 +75,18 @@ public class AppCacheManifestTransformerTests {
public void noTransformIfExtensionNoMatch() throws Exception {
Resource resource = mock(Resource.class);
given(resource.getFilename()).willReturn("foobar.file");
given(this.chain.transform(this.exchange, resource)).willReturn(resource);
given(this.chain.transform(this.exchange, resource)).willReturn(Mono.just(resource));
Resource result = this.transformer.transform(this.exchange, resource, this.chain);
Resource result = this.transformer.transform(this.exchange, resource, this.chain).blockMillis(5000);
assertEquals(resource, result);
}
@Test
public void syntaxErrorInManifest() throws Exception {
Resource resource = new ClassPathResource("test/error.appcache", getClass());
given(this.chain.transform(this.exchange, resource)).willReturn(resource);
given(this.chain.transform(this.exchange, resource)).willReturn(Mono.just(resource));
Resource result = this.transformer.transform(this.exchange, resource, this.chain);
Resource result = this.transformer.transform(this.exchange, resource, this.chain).blockMillis(5000);
assertEquals(resource, result);
}
@ -106,7 +107,7 @@ public class AppCacheManifestTransformerTests {
this.chain = new DefaultResourceTransformerChain(resolverChain, transformers);
Resource resource = new ClassPathResource("test/test.appcache", getClass());
Resource result = this.transformer.transform(this.exchange, resource, this.chain);
Resource result = this.transformer.transform(this.exchange, resource, this.chain).blockMillis(5000);
byte[] bytes = FileCopyUtils.copyToByteArray(result.getInputStream());
String content = new String(bytes, "UTF-8");

View File

@ -81,7 +81,7 @@ public class CachingResourceResolverTests {
public void resolveResourceInternal() {
String file = "bar.css";
Resource expected = new ClassPathResource("test/" + file, getClass());
Resource actual = this.chain.resolveResource(this.exchange, file, this.locations);
Resource actual = this.chain.resolveResource(this.exchange, file, this.locations).blockMillis(5000);
assertEquals(expected, actual);
}
@ -93,20 +93,20 @@ public class CachingResourceResolverTests {
this.cache.put(CachingResourceResolver.RESOLVED_RESOURCE_CACHE_KEY_PREFIX + "bar.css", expected);
String file = "bar.css";
Resource actual = this.chain.resolveResource(this.exchange, file, this.locations);
Resource actual = this.chain.resolveResource(this.exchange, file, this.locations).blockMillis(5000);
assertSame(expected, actual);
}
@Test
public void resolveResourceInternalNoMatch() {
assertNull(this.chain.resolveResource(this.exchange, "invalid.css", this.locations));
assertNull(this.chain.resolveResource(this.exchange, "invalid.css", this.locations).blockMillis(5000));
}
@Test
public void resolverUrlPath() {
String expected = "/foo.css";
String actual = this.chain.resolveUrlPath(expected, this.locations);
String actual = this.chain.resolveUrlPath(expected, this.locations).blockMillis(5000);
assertEquals(expected, actual);
}
@ -115,14 +115,14 @@ public class CachingResourceResolverTests {
public void resolverUrlPathFromCache() {
String expected = "cached-imaginary.css";
this.cache.put(CachingResourceResolver.RESOLVED_URL_PATH_CACHE_KEY_PREFIX + "imaginary.css", expected);
String actual = this.chain.resolveUrlPath("imaginary.css", this.locations);
String actual = this.chain.resolveUrlPath("imaginary.css", this.locations).blockMillis(5000);
assertEquals(expected, actual);
}
@Test
public void resolverUrlPathNoMatch() {
assertNull(this.chain.resolveUrlPath("invalid.css", this.locations));
assertNull(this.chain.resolveUrlPath("invalid.css", this.locations).blockMillis(5000));
}
@Test
@ -130,7 +130,7 @@ public class CachingResourceResolverTests {
String file = "bar.css";
this.request.setUri(file).setHeader("Accept-Encoding", "gzip");
Resource expected = this.chain.resolveResource(this.exchange, file, this.locations);
Resource expected = this.chain.resolveResource(this.exchange, file, this.locations).blockMillis(5000);
String cacheKey = CachingResourceResolver.RESOLVED_RESOURCE_CACHE_KEY_PREFIX + file + "+encoding=gzip";
assertEquals(expected, this.cache.get(cacheKey).get());
@ -141,7 +141,7 @@ public class CachingResourceResolverTests {
String file = "bar.css";
this.request.setUri(file);
Resource expected = this.chain.resolveResource(this.exchange, file, this.locations);
Resource expected = this.chain.resolveResource(this.exchange, file, this.locations).blockMillis(5000);
String cacheKey = CachingResourceResolver.RESOLVED_RESOURCE_CACHE_KEY_PREFIX + file;
assertEquals(expected, this.cache.get(cacheKey).get());
@ -156,10 +156,10 @@ public class CachingResourceResolverTests {
String file = "bar.css";
this.request.setUri(file);
assertSame(resource, this.chain.resolveResource(this.exchange, file, this.locations));
assertSame(resource, this.chain.resolveResource(this.exchange, file, this.locations).blockMillis(5000));
request.addHeader("Accept-Encoding", "gzip");
assertSame(gzResource, this.chain.resolveResource(this.exchange, file, this.locations));
assertSame(gzResource, this.chain.resolveResource(this.exchange, file, this.locations).blockMillis(5000));
}
}

View File

@ -74,7 +74,9 @@ public class CssLinkResourceTransformerTests {
@Test
public void transform() throws Exception {
Resource css = new ClassPathResource("test/main.css", getClass());
TransformedResource actual = (TransformedResource) this.transformerChain.transform(this.exchange, css);
TransformedResource actual =
(TransformedResource) this.transformerChain.transform(this.exchange, css)
.blockMillis(5000);
String expected = "\n" +
"@import url(\"bar-11e16cf79faee7ac698c805cf28248d2.css\");\n" +
@ -92,7 +94,7 @@ public class CssLinkResourceTransformerTests {
@Test
public void transformNoLinks() throws Exception {
Resource expected = new ClassPathResource("test/foo.css", getClass());
Resource actual = this.transformerChain.transform(this.exchange, expected);
Resource actual = this.transformerChain.transform(this.exchange, expected).blockMillis(5000);
assertSame(expected, actual);
}
@ -103,7 +105,7 @@ public class CssLinkResourceTransformerTests {
Collections.singletonList(new CssLinkResourceTransformer()));
Resource externalCss = new ClassPathResource("test/external.css", getClass());
Resource resource = transformerChain.transform(this.exchange, externalCss);
Resource resource = transformerChain.transform(this.exchange, externalCss).blockMillis(5000);
TransformedResource transformedResource = (TransformedResource) resource;
String expected = "@import url(\"http://example.org/fonts/css\");\n" +
@ -124,7 +126,7 @@ public class CssLinkResourceTransformerTests {
@Test
public void transformWithNonCssResource() throws Exception {
Resource expected = new ClassPathResource("test/images/image.png", getClass());
Resource actual = this.transformerChain.transform(this.exchange, expected);
Resource actual = this.transformerChain.transform(this.exchange, expected).blockMillis(5000);
assertSame(expected, actual);
}

View File

@ -117,7 +117,7 @@ public class GzipResourceResolverTests {
public void resolveGzippedFile() throws IOException {
this.request.addHeader("Accept-Encoding", "gzip");
String file = "js/foo.js";
Resource resolved = this.resolver.resolveResource(this.exchange, file, this.locations);
Resource resolved = this.resolver.resolveResource(this.exchange, file, this.locations).blockMillis(5000);
String gzFile = file+".gz";
Resource resource = new ClassPathResource("test/" + gzFile, getClass());
@ -131,7 +131,7 @@ public class GzipResourceResolverTests {
public void resolveFingerprintedGzippedFile() throws IOException {
this.request.addHeader("Accept-Encoding", "gzip");
String file = "foo-e36d2e05253c6c7085a91522ce43a0b4.css";
Resource resolved = this.resolver.resolveResource(this.exchange, file, this.locations);
Resource resolved = this.resolver.resolveResource(this.exchange, file, this.locations).blockMillis(5000);
String gzFile = file + ".gz";
Resource resource = new ClassPathResource("test/" + gzFile, getClass());
@ -145,7 +145,7 @@ public class GzipResourceResolverTests {
public void resolveFromCacheWithEncodingVariants() throws IOException {
this.request.addHeader("Accept-Encoding", "gzip");
String file = "js/foo.js";
Resource resolved = this.resolver.resolveResource(this.exchange, file, this.locations);
Resource resolved = this.resolver.resolveResource(this.exchange, file, this.locations).blockMillis(5000);
String gzFile = file+".gz";
Resource gzResource = new ClassPathResource("test/"+gzFile, getClass());
@ -160,7 +160,7 @@ public class GzipResourceResolverTests {
MockServerHttpResponse response = new MockServerHttpResponse();
this.exchange = new DefaultServerWebExchange(this.request, response, new DefaultWebSessionManager());
resolved = this.resolver.resolveResource(this.exchange, file, this.locations);
resolved = this.resolver.resolveResource(this.exchange, file, this.locations).blockMillis(5000);
Resource resource = new ClassPathResource("test/"+file, getClass());
assertEquals(resource.getDescription(), resolved.getDescription());
@ -172,7 +172,7 @@ public class GzipResourceResolverTests {
@Test // SPR-13149
public void resolveWithNullRequest() throws IOException {
String file = "js/foo.js";
Resource resolved = this.resolver.resolveResource(null, file, this.locations);
Resource resolved = this.resolver.resolveResource(null, file, this.locations).blockMillis(5000);
String gzFile = file+".gz";
Resource gzResource = new ClassPathResource("test/" + gzFile, getClass());

View File

@ -16,6 +16,7 @@
package org.springframework.web.reactive.resource;
import java.io.IOException;
import java.util.List;
import org.junit.Test;
@ -43,7 +44,8 @@ public class PathResourceResolverTests {
public void resolveFromClasspath() throws IOException {
Resource location = new ClassPathResource("test/", PathResourceResolver.class);
String path = "bar.css";
Resource actual = this.resolver.resolveResource(null, path, singletonList(location), null);
List<Resource> locations = singletonList(location);
Resource actual = this.resolver.resolveResource(null, path, locations, null).blockMillis(5000);
assertEquals(location.createRelative(path), actual);
}
@ -51,7 +53,8 @@ public class PathResourceResolverTests {
public void resolveFromClasspathRoot() throws IOException {
Resource location = new ClassPathResource("/");
String path = "org/springframework/web/reactive/resource/test/bar.css";
Resource actual = this.resolver.resolveResource(null, path, singletonList(location), null);
List<Resource> locations = singletonList(location);
Resource actual = this.resolver.resolveResource(null, path, locations, null).blockMillis(5000);
assertNotNull(actual);
}
@ -75,7 +78,8 @@ public class PathResourceResolverTests {
}
private void testCheckResource(Resource location, String requestPath) throws IOException {
Resource actual = this.resolver.resolveResource(null, requestPath, singletonList(location), null);
List<Resource> locations = singletonList(location);
Resource actual = this.resolver.resolveResource(null, requestPath, locations, null).blockMillis(5000);
if (!location.createRelative(requestPath).exists() && !requestPath.contains(":")) {
fail(requestPath + " doesn't actually exist as a relative path");
}
@ -90,7 +94,9 @@ public class PathResourceResolverTests {
);
Resource location = new ClassPathResource("test/main.css", PathResourceResolver.class);
String actual = this.resolver.resolveUrlPath("../testalternatepath/bar.css", singletonList(location), null);
String actual = this.resolver.resolveUrlPath("../testalternatepath/bar.css",
singletonList(location), null).blockMillis(5000);
assertEquals("../testalternatepath/bar.css", actual);
}
@ -98,7 +104,8 @@ public class PathResourceResolverTests {
public void checkRelativeLocation() throws Exception {
String locationUrl= new UrlResource(getClass().getResource("./test/")).getURL().toExternalForm();
Resource location = new UrlResource(locationUrl.replace("/springframework","/../org/springframework"));
assertNotNull(this.resolver.resolveResource(null, "main.css", singletonList(location), null));
List<Resource> locations = singletonList(location);
assertNotNull(this.resolver.resolveResource(null, "main.css", locations, null).blockMillis(5000));
}
@Test // SPR-12747
@ -110,7 +117,9 @@ public class PathResourceResolverTests {
@Test // SPR-13241
public void resolvePathRootResource() throws Exception {
Resource webjarsLocation = new ClassPathResource("/META-INF/resources/webjars/", PathResourceResolver.class);
String path = this.resolver.resolveUrlPathInternal("", singletonList(webjarsLocation), null);
String path = this.resolver.resolveUrlPathInternal(
"", singletonList(webjarsLocation), null).blockMillis(5000);
assertNull(path);
}
}

View File

@ -21,6 +21,7 @@ import java.util.List;
import org.junit.Before;
import org.junit.Test;
import reactor.core.publisher.Mono;
import org.springframework.core.io.ClassPathResource;
import org.springframework.core.io.Resource;
@ -85,7 +86,9 @@ public class ResourceTransformerSupportTests {
this.request.setUri("/resources/main.css");
String resourcePath = "/resources/bar.css";
Resource css = new ClassPathResource("test/main.css", getClass());
String actual = this.transformer.resolveUrlPath(resourcePath, this.exchange, css, this.transformerChain);
String actual = this.transformer.resolveUrlPath(
resourcePath, this.exchange, css, this.transformerChain).blockMillis(5000);
assertEquals("/resources/bar-11e16cf79faee7ac698c805cf28248d2.css", actual);
assertEquals("/resources/bar-11e16cf79faee7ac698c805cf28248d2.css", actual);
}
@ -93,14 +96,18 @@ public class ResourceTransformerSupportTests {
@Test
public void resolveUrlPathWithRelativePath() throws Exception {
Resource css = new ClassPathResource("test/main.css", getClass());
String actual = this.transformer.resolveUrlPath("bar.css", this.exchange, css, this.transformerChain);
String actual = this.transformer.resolveUrlPath(
"bar.css", this.exchange, css, this.transformerChain).blockMillis(5000);
assertEquals("bar-11e16cf79faee7ac698c805cf28248d2.css", actual);
}
@Test
public void resolveUrlPathWithRelativePathInParentDirectory() throws Exception {
Resource imagePng = new ClassPathResource("test/images/image.png", getClass());
String actual = this.transformer.resolveUrlPath("../bar.css", this.exchange, imagePng, this.transformerChain);
String actual = this.transformer.resolveUrlPath(
"../bar.css", this.exchange, imagePng, this.transformerChain).blockMillis(5000);
assertEquals("../bar-11e16cf79faee7ac698c805cf28248d2.css", actual);
}
@ -108,8 +115,10 @@ public class ResourceTransformerSupportTests {
private static class TestResourceTransformerSupport extends ResourceTransformerSupport {
@Override
public Resource transform(ServerWebExchange exchange, Resource resource, ResourceTransformerChain chain) {
throw new IllegalStateException("Should never be called");
public Mono<Resource> transform(ServerWebExchange exchange, Resource resource,
ResourceTransformerChain chain) {
return Mono.error(new IllegalStateException("Should never be called"));
}
}

View File

@ -74,7 +74,7 @@ public class ResourceUrlProviderTests {
@Test
public void getStaticResourceUrl() {
String url = this.urlProvider.getForLookupPath("/resources/foo.css");
String url = this.urlProvider.getForLookupPath("/resources/foo.css").blockMillis(5000);
assertEquals("/resources/foo.css", url);
}
@ -86,7 +86,7 @@ public class ResourceUrlProviderTests {
ServerWebExchange exchange = new DefaultServerWebExchange(request, response, manager);
String url = "/resources/foo.css?foo=bar&url=http://example.org";
String resolvedUrl = this.urlProvider.getForRequestUrl(exchange, url);
String resolvedUrl = this.urlProvider.getForRequestUrl(exchange, url).blockMillis(5000);
assertEquals(url, resolvedUrl);
}
@ -102,7 +102,7 @@ public class ResourceUrlProviderTests {
resolvers.add(new PathResourceResolver());
this.handler.setResourceResolvers(resolvers);
String url = this.urlProvider.getForLookupPath("/resources/foo.css");
String url = this.urlProvider.getForLookupPath("/resources/foo.css").blockMillis(5000);
assertEquals("/resources/foo-e36d2e05253c6c7085a91522ce43a0b4.css", url);
}
@ -123,7 +123,7 @@ public class ResourceUrlProviderTests {
this.handlerMap.put("/resources/*.css", otherHandler);
this.urlProvider.setHandlerMap(this.handlerMap);
String url = this.urlProvider.getForLookupPath("/resources/foo.css");
String url = this.urlProvider.getForLookupPath("/resources/foo.css").blockMillis(5000);
assertEquals("/resources/foo-e36d2e05253c6c7085a91522ce43a0b4.css", url);
}

View File

@ -243,7 +243,7 @@ public class ResourceWebHandlerTests {
this.request.addHeader("Accept", "application/json,text/plain,*/*");
this.exchange.getAttributes().put(PATH_WITHIN_HANDLER_MAPPING_ATTRIBUTE, "foo.html");
handler.handle(this.exchange);
handler.handle(this.exchange).blockMillis(5000);
assertEquals(MediaType.TEXT_HTML, this.response.getHeaders().getContentType());
}

View File

@ -24,6 +24,7 @@ import java.util.Map;
import org.hamcrest.Matchers;
import org.junit.Before;
import org.junit.Test;
import reactor.core.publisher.Mono;
import org.springframework.core.io.ClassPathResource;
import org.springframework.core.io.Resource;
@ -76,10 +77,13 @@ public class VersionResourceResolverTests {
public void resolveResourceExisting() throws Exception {
String file = "bar.css";
Resource expected = new ClassPathResource("test/" + file, getClass());
given(this.chain.resolveResource(null, file, this.locations)).willReturn(expected);
given(this.chain.resolveResource(null, file, this.locations)).willReturn(Mono.just(expected));
this.resolver.setStrategyMap(Collections.singletonMap("/**", this.versionStrategy));
Resource actual = this.resolver.resolveResourceInternal(null, file, this.locations, this.chain);
Resource actual = this.resolver
.resolveResourceInternal(null, file, this.locations, this.chain)
.blockMillis(5000);
assertEquals(expected, actual);
verify(this.chain, times(1)).resolveResource(null, file, this.locations);
verify(this.versionStrategy, never()).extractVersion(file);
@ -88,10 +92,13 @@ public class VersionResourceResolverTests {
@Test
public void resolveResourceNoVersionStrategy() throws Exception {
String file = "missing.css";
given(this.chain.resolveResource(null, file, this.locations)).willReturn(null);
given(this.chain.resolveResource(null, file, this.locations)).willReturn(Mono.empty());
this.resolver.setStrategyMap(Collections.emptyMap());
Resource actual = this.resolver.resolveResourceInternal(null, file, this.locations, this.chain);
Resource actual = this.resolver
.resolveResourceInternal(null, file, this.locations, this.chain)
.blockMillis(5000);
assertNull(actual);
verify(this.chain, times(1)).resolveResource(null, file, this.locations);
}
@ -99,11 +106,14 @@ public class VersionResourceResolverTests {
@Test
public void resolveResourceNoVersionInPath() throws Exception {
String file = "bar.css";
given(this.chain.resolveResource(null, file, this.locations)).willReturn(null);
given(this.chain.resolveResource(null, file, this.locations)).willReturn(Mono.empty());
given(this.versionStrategy.extractVersion(file)).willReturn("");
this.resolver.setStrategyMap(Collections.singletonMap("/**", this.versionStrategy));
Resource actual = this.resolver.resolveResourceInternal(null, file, this.locations, this.chain);
Resource actual = this.resolver
.resolveResourceInternal(null, file, this.locations, this.chain)
.blockMillis(5000);
assertNull(actual);
verify(this.chain, times(1)).resolveResource(null, file, this.locations);
verify(this.versionStrategy, times(1)).extractVersion(file);
@ -114,13 +124,16 @@ public class VersionResourceResolverTests {
String versionFile = "bar-version.css";
String version = "version";
String file = "bar.css";
given(this.chain.resolveResource(null, versionFile, this.locations)).willReturn(null);
given(this.chain.resolveResource(null, file, this.locations)).willReturn(null);
given(this.chain.resolveResource(null, versionFile, this.locations)).willReturn(Mono.empty());
given(this.chain.resolveResource(null, file, this.locations)).willReturn(Mono.empty());
given(this.versionStrategy.extractVersion(versionFile)).willReturn(version);
given(this.versionStrategy.removeVersion(versionFile, version)).willReturn(file);
this.resolver.setStrategyMap(Collections.singletonMap("/**", this.versionStrategy));
Resource actual = this.resolver.resolveResourceInternal(null, versionFile, this.locations, this.chain);
Resource actual = this.resolver
.resolveResourceInternal(null, versionFile, this.locations, this.chain)
.blockMillis(5000);
assertNull(actual);
verify(this.versionStrategy, times(1)).removeVersion(versionFile, version);
}
@ -131,14 +144,17 @@ public class VersionResourceResolverTests {
String version = "version";
String file = "bar.css";
Resource expected = new ClassPathResource("test/" + file, getClass());
given(this.chain.resolveResource(null, versionFile, this.locations)).willReturn(null);
given(this.chain.resolveResource(null, file, this.locations)).willReturn(expected);
given(this.chain.resolveResource(null, versionFile, this.locations)).willReturn(Mono.empty());
given(this.chain.resolveResource(null, file, this.locations)).willReturn(Mono.just(expected));
given(this.versionStrategy.extractVersion(versionFile)).willReturn(version);
given(this.versionStrategy.removeVersion(versionFile, version)).willReturn(file);
given(this.versionStrategy.getResourceVersion(expected)).willReturn("newer-version");
this.resolver.setStrategyMap(Collections.singletonMap("/**", this.versionStrategy));
Resource actual = this.resolver.resolveResourceInternal(null, versionFile, this.locations, this.chain);
Resource actual = this.resolver
.resolveResourceInternal(null, versionFile, this.locations, this.chain)
.blockMillis(5000);
assertNull(actual);
verify(this.versionStrategy, times(1)).getResourceVersion(expected);
}
@ -153,14 +169,17 @@ public class VersionResourceResolverTests {
MockServerHttpResponse response = new MockServerHttpResponse();
DefaultWebSessionManager sessionManager = new DefaultWebSessionManager();
ServerWebExchange exchange = new DefaultServerWebExchange(request, response, sessionManager);
given(this.chain.resolveResource(exchange, versionFile, this.locations)).willReturn(null);
given(this.chain.resolveResource(exchange, file, this.locations)).willReturn(expected);
given(this.chain.resolveResource(exchange, versionFile, this.locations)).willReturn(Mono.empty());
given(this.chain.resolveResource(exchange, file, this.locations)).willReturn(Mono.just(expected));
given(this.versionStrategy.extractVersion(versionFile)).willReturn(version);
given(this.versionStrategy.removeVersion(versionFile, version)).willReturn(file);
given(this.versionStrategy.getResourceVersion(expected)).willReturn(version);
this.resolver.setStrategyMap(Collections.singletonMap("/**", this.versionStrategy));
Resource actual = this.resolver.resolveResourceInternal(exchange, versionFile, this.locations, this.chain);
Resource actual = this.resolver
.resolveResourceInternal(exchange, versionFile, this.locations, this.chain)
.blockMillis(5000);
assertEquals(expected.getFilename(), actual.getFilename());
verify(this.versionStrategy, times(1)).getResourceVersion(expected);
assertThat(actual, instanceOf(HttpResource.class));

View File

@ -16,11 +16,11 @@
package org.springframework.web.reactive.resource;
import java.util.Collections;
import java.util.List;
import org.junit.Before;
import org.junit.Test;
import reactor.core.publisher.Mono;
import org.springframework.core.io.ClassPathResource;
import org.springframework.core.io.Resource;
@ -33,6 +33,7 @@ import org.springframework.web.server.adapter.DefaultServerWebExchange;
import org.springframework.web.server.session.DefaultWebSessionManager;
import org.springframework.web.server.session.WebSessionManager;
import static java.util.Collections.singletonList;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.mockito.BDDMockito.given;
@ -61,7 +62,7 @@ public class WebJarsResourceResolverTests {
@Before
public void setup() {
// for this to work, an actual WebJar must be on the test classpath
this.locations = Collections.singletonList(new ClassPathResource("/META-INF/resources/webjars"));
this.locations = singletonList(new ClassPathResource("/META-INF/resources/webjars"));
this.resolver = new WebJarsResourceResolver();
this.chain = mock(ResourceResolverChain.class);
@ -74,11 +75,11 @@ public class WebJarsResourceResolverTests {
@Test
public void resolveUrlExisting() {
this.locations = Collections.singletonList(new ClassPathResource("/META-INF/resources/webjars/", getClass()));
this.locations = singletonList(new ClassPathResource("/META-INF/resources/webjars/", getClass()));
String file = "/foo/2.3/foo.txt";
given(this.chain.resolveUrlPath(file, this.locations)).willReturn(file);
given(this.chain.resolveUrlPath(file, this.locations)).willReturn(Mono.just(file));
String actual = this.resolver.resolveUrlPath(file, this.locations, this.chain);
String actual = this.resolver.resolveUrlPath(file, this.locations, this.chain).blockMillis(5000);
assertEquals(file, actual);
verify(this.chain, times(1)).resolveUrlPath(file, this.locations);
@ -86,11 +87,11 @@ public class WebJarsResourceResolverTests {
@Test
public void resolveUrlExistingNotInJarFile() {
this.locations = Collections.singletonList(new ClassPathResource("/META-INF/resources/webjars/", getClass()));
this.locations = singletonList(new ClassPathResource("/META-INF/resources/webjars/", getClass()));
String file = "foo/foo.txt";
given(this.chain.resolveUrlPath(file, this.locations)).willReturn(null);
given(this.chain.resolveUrlPath(file, this.locations)).willReturn(Mono.empty());
String actual = this.resolver.resolveUrlPath(file, this.locations, this.chain);
String actual = this.resolver.resolveUrlPath(file, this.locations, this.chain).blockMillis(5000);
assertNull(actual);
verify(this.chain, times(1)).resolveUrlPath(file, this.locations);
@ -101,10 +102,10 @@ public class WebJarsResourceResolverTests {
public void resolveUrlWebJarResource() {
String file = "underscorejs/underscore.js";
String expected = "underscorejs/1.8.3/underscore.js";
given(this.chain.resolveUrlPath(file, this.locations)).willReturn(null);
given(this.chain.resolveUrlPath(expected, this.locations)).willReturn(expected);
given(this.chain.resolveUrlPath(file, this.locations)).willReturn(Mono.empty());
given(this.chain.resolveUrlPath(expected, this.locations)).willReturn(Mono.just(expected));
String actual = this.resolver.resolveUrlPath(file, this.locations, this.chain);
String actual = this.resolver.resolveUrlPath(file, this.locations, this.chain).blockMillis(5000);
assertEquals(expected, actual);
verify(this.chain, times(1)).resolveUrlPath(file, this.locations);
@ -114,9 +115,9 @@ public class WebJarsResourceResolverTests {
@Test
public void resolveUrlWebJarResourceNotFound() {
String file = "something/something.js";
given(this.chain.resolveUrlPath(file, this.locations)).willReturn(null);
given(this.chain.resolveUrlPath(file, this.locations)).willReturn(Mono.empty());
String actual = this.resolver.resolveUrlPath(file, this.locations, this.chain);
String actual = this.resolver.resolveUrlPath(file, this.locations, this.chain).blockMillis(5000);
assertNull(actual);
verify(this.chain, times(1)).resolveUrlPath(file, this.locations);
@ -126,11 +127,13 @@ public class WebJarsResourceResolverTests {
@Test
public void resolveResourceExisting() {
Resource expected = mock(Resource.class);
this.locations = Collections.singletonList(new ClassPathResource("/META-INF/resources/webjars/", getClass()));
this.locations = singletonList(new ClassPathResource("/META-INF/resources/webjars/", getClass()));
String file = "foo/2.3/foo.txt";
given(this.chain.resolveResource(this.exchange, file, this.locations)).willReturn(expected);
given(this.chain.resolveResource(this.exchange, file, this.locations)).willReturn(Mono.just(expected));
Resource actual = this.resolver.resolveResource(this.exchange, file, this.locations, this.chain);
Resource actual = this.resolver
.resolveResource(this.exchange, file, this.locations, this.chain)
.blockMillis(5000);
assertEquals(expected, actual);
verify(this.chain, times(1)).resolveResource(this.exchange, file, this.locations);
@ -139,9 +142,11 @@ public class WebJarsResourceResolverTests {
@Test
public void resolveResourceNotFound() {
String file = "something/something.js";
given(this.chain.resolveUrlPath(file, this.locations)).willReturn(null);
given(this.chain.resolveResource(this.exchange, file, this.locations)).willReturn(Mono.empty());
Resource actual = this.resolver.resolveResource(this.exchange, file, this.locations, this.chain);
Resource actual = this.resolver
.resolveResource(this.exchange, file, this.locations, this.chain)
.blockMillis(5000);
assertNull(actual);
verify(this.chain, times(1)).resolveResource(this.exchange, file, this.locations);
@ -150,13 +155,20 @@ public class WebJarsResourceResolverTests {
@Test
public void resolveResourceWebJar() {
Resource expected = mock(Resource.class);
String file = "underscorejs/underscore.js";
String expectedPath = "underscorejs/1.8.3/underscore.js";
this.locations = Collections.singletonList(new ClassPathResource("/META-INF/resources/webjars/", getClass()));
given(this.chain.resolveResource(this.exchange, expectedPath, this.locations)).willReturn(expected);
this.locations = singletonList(new ClassPathResource("/META-INF/resources/webjars/", getClass()));
Resource actual = this.resolver.resolveResource(this.exchange, file, this.locations, this.chain);
String file = "underscorejs/underscore.js";
given(this.chain.resolveResource(this.exchange, file, this.locations)).willReturn(Mono.empty());
Resource expected = mock(Resource.class);
String expectedPath = "underscorejs/1.8.3/underscore.js";
given(this.chain.resolveResource(this.exchange, expectedPath, this.locations))
.willReturn(Mono.just(expected));
Resource actual = this.resolver
.resolveResource(this.exchange, file, this.locations, this.chain)
.blockMillis(5000);
assertEquals(expected, actual);
verify(this.chain, times(1)).resolveResource(this.exchange, file, this.locations);