Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,6 @@

package org.springframework.graphql.execution;

import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.function.Consumer;

import graphql.GraphQL;
import graphql.execution.instrumentation.ChainedInstrumentation;
import graphql.execution.instrumentation.Instrumentation;
Expand All @@ -30,6 +24,9 @@
import graphql.schema.GraphQLTypeVisitor;
import graphql.schema.SchemaTraverser;

import java.util.*;
import java.util.function.Consumer;


/**
* Implementation of {@link GraphQlSource.Builder} that leaves it to subclasses
Expand All @@ -43,6 +40,8 @@ abstract class AbstractGraphQlSourceBuilder<B extends GraphQlSource.Builder<B>>

private final List<DataFetcherExceptionResolver> exceptionResolvers = new ArrayList<>();

private final List<SubscriptionExceptionResolver> subscriptionExceptionResolvers = new ArrayList<>();

private final List<GraphQLTypeVisitor> typeVisitors = new ArrayList<>();

private final List<Instrumentation> instrumentations = new ArrayList<>();
Expand All @@ -57,6 +56,12 @@ public B exceptionResolvers(List<DataFetcherExceptionResolver> resolvers) {
return self();
}

@Override
public B subscriptionExceptionResolvers(List<SubscriptionExceptionResolver> subscriptionExceptionResolvers) {
this.subscriptionExceptionResolvers.addAll(subscriptionExceptionResolvers);
return self();
}

@Override
public B typeVisitors(List<GraphQLTypeVisitor> typeVisitors) {
this.typeVisitors.addAll(typeVisitors);
Expand Down Expand Up @@ -105,8 +110,12 @@ public GraphQlSource build() {
protected abstract GraphQLSchema initGraphQlSchema();

private GraphQLSchema applyTypeVisitors(GraphQLSchema schema) {
SubscriptionExceptionResolver subscriptionExceptionResolver = new DelegatingSubscriptionExceptionResolver(
subscriptionExceptionResolvers);
GraphQLTypeVisitor visitor = ContextDataFetcherDecorator.createVisitor(subscriptionExceptionResolver);

List<GraphQLTypeVisitor> visitors = new ArrayList<>(this.typeVisitors);
visitors.add(ContextDataFetcherDecorator.TYPE_VISITOR);
visitors.add(visitor);

GraphQLCodeRegistry.Builder codeRegistry = GraphQLCodeRegistry.newCodeRegistry(schema.getCodeRegistry());
Map<Class<?>, Object> vars = Collections.singletonMap(GraphQLCodeRegistry.Builder.class, codeRegistry);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,22 +17,16 @@
package org.springframework.graphql.execution;

import graphql.ExecutionInput;
import graphql.schema.DataFetcher;
import graphql.schema.DataFetchingEnvironment;
import graphql.schema.GraphQLCodeRegistry;
import graphql.schema.GraphQLFieldDefinition;
import graphql.schema.GraphQLFieldsContainer;
import graphql.schema.GraphQLSchemaElement;
import graphql.schema.GraphQLTypeVisitor;
import graphql.schema.GraphQLTypeVisitorStub;
import graphql.schema.*;
import graphql.util.TraversalControl;
import graphql.util.TraverserContext;
import org.reactivestreams.Publisher;
import org.springframework.util.Assert;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import reactor.util.context.ContextView;

import org.springframework.util.Assert;
import java.util.function.Function;

/**
* Wrap a {@link DataFetcher} to enable the following:
Expand All @@ -51,10 +45,16 @@ final class ContextDataFetcherDecorator implements DataFetcher<Object> {

private final boolean subscription;

private ContextDataFetcherDecorator(DataFetcher<?> delegate, boolean subscription) {
private final SubscriptionExceptionResolver subscriptionExceptionResolver;

private ContextDataFetcherDecorator(
DataFetcher<?> delegate, boolean subscription,
SubscriptionExceptionResolver subscriptionExceptionResolver) {
Assert.notNull(delegate, "'delegate' DataFetcher is required");
Assert.notNull(subscriptionExceptionResolver, "'subscriptionExceptionResolver' is required");
this.delegate = delegate;
this.subscription = subscription;
this.subscriptionExceptionResolver = subscriptionExceptionResolver;
}

@Override
Expand All @@ -66,7 +66,8 @@ public Object get(DataFetchingEnvironment environment) throws Exception {
ContextView contextView = ReactorContextManager.getReactorContext(environment.getGraphQlContext());

if (this.subscription) {
return (!contextView.isEmpty() ? Flux.from((Publisher<?>) value).contextWrite(contextView) : value);
Publisher<?> publisher = interceptSubscriptionPublisherWithExceptionHandler((Publisher<?>) value);
return (!contextView.isEmpty() ? Flux.from(publisher).contextWrite(contextView) : publisher);
}

if (value instanceof Flux) {
Expand All @@ -84,29 +85,48 @@ public Object get(DataFetchingEnvironment environment) throws Exception {
return value;
}

@SuppressWarnings("unchecked")
private Publisher<?> interceptSubscriptionPublisherWithExceptionHandler(Publisher<?> publisher) {
Function<? super Throwable, Mono<?>> onErrorResumeFunction = e ->
subscriptionExceptionResolver.resolveException(e)
.flatMap(errors -> Mono.error(new SubscriptionStreamException(errors)));

if (publisher instanceof Flux) {
return ((Flux<Object>) publisher).onErrorResume(onErrorResumeFunction);
}

if (publisher instanceof Mono) {
return ((Mono<Object>) publisher).onErrorResume(onErrorResumeFunction);
}

throw new IllegalArgumentException("Unknown publisher type: '" + publisher.getClass().getName() +"'. " +
"Expected reactor.core.publisher.Mono or reactor.core.publisher.Flux");
}

/**
* {@link GraphQLTypeVisitor} that wraps non-GraphQL data fetchers and adapts them if
* they return {@link Flux} or {@link Mono}.
*/
static GraphQLTypeVisitor TYPE_VISITOR = new GraphQLTypeVisitorStub() {

@Override
public TraversalControl visitGraphQLFieldDefinition(GraphQLFieldDefinition fieldDefinition,
TraverserContext<GraphQLSchemaElement> context) {

GraphQLCodeRegistry.Builder codeRegistry = context.getVarFromParents(GraphQLCodeRegistry.Builder.class);
GraphQLFieldsContainer parent = (GraphQLFieldsContainer) context.getParentNode();
DataFetcher<?> dataFetcher = codeRegistry.getDataFetcher(parent, fieldDefinition);

if (dataFetcher.getClass().getPackage().getName().startsWith("graphql.")) {
static GraphQLTypeVisitor createVisitor(SubscriptionExceptionResolver subscriptionExceptionResolver) {
return new GraphQLTypeVisitorStub() {
@Override
public TraversalControl visitGraphQLFieldDefinition(GraphQLFieldDefinition fieldDefinition,
TraverserContext<GraphQLSchemaElement> context) {

GraphQLCodeRegistry.Builder codeRegistry = context.getVarFromParents(GraphQLCodeRegistry.Builder.class);
GraphQLFieldsContainer parent = (GraphQLFieldsContainer) context.getParentNode();
DataFetcher<?> dataFetcher = codeRegistry.getDataFetcher(parent, fieldDefinition);

if (dataFetcher.getClass().getPackage().getName().startsWith("graphql.")) {
return TraversalControl.CONTINUE;
}

boolean handlesSubscription = parent.getName().equals("Subscription");
dataFetcher = new ContextDataFetcherDecorator(dataFetcher, handlesSubscription, subscriptionExceptionResolver);
codeRegistry.dataFetcher(parent, fieldDefinition, dataFetcher);
return TraversalControl.CONTINUE;
}

boolean handlesSubscription = parent.getName().equals("Subscription");
dataFetcher = new ContextDataFetcherDecorator(dataFetcher, handlesSubscription);
codeRegistry.dataFetcher(parent, fieldDefinition, dataFetcher);
return TraversalControl.CONTINUE;
}
};
};
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
/*
* Copyright 2002-2022 the original author or authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.springframework.graphql.execution;

import graphql.ErrorType;
import graphql.GraphQLError;
import graphql.GraphqlErrorBuilder;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.springframework.util.Assert;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;

import java.util.Collections;
import java.util.List;

/**
* An implementation of {@link SubscriptionExceptionResolver} that is trying to map exception to GraphQL error
* using provided implementation of {@link SubscriptionExceptionResolver}.
* <br/>
* If none of provided implementations resolve exception to error or if any of implementation throw an exception,
* this {@link SubscriptionExceptionResolver} will return a default error.
*
* @author Mykyta Ivchenko
* @see SubscriptionExceptionResolver
*/
public class DelegatingSubscriptionExceptionResolver implements SubscriptionExceptionResolver {
private static final Log logger = LogFactory.getLog(DelegatingSubscriptionExceptionResolver.class);
private final List<SubscriptionExceptionResolver> resolvers;

public DelegatingSubscriptionExceptionResolver(List<SubscriptionExceptionResolver> resolvers) {
Assert.notNull(resolvers, "'resolvers' list must be not null.");
this.resolvers = resolvers;
}

@Override
public Mono<List<GraphQLError>> resolveException(Throwable exception) {
return Flux.fromIterable(resolvers)
.flatMap(resolver -> resolver.resolveException(exception))
.next()
.onErrorResume(error -> Mono.just(handleMappingException(error, exception)))
.defaultIfEmpty(createDefaultErrors());
}

private List<GraphQLError> handleMappingException(Throwable resolverException, Throwable originalException) {
if (logger.isWarnEnabled()) {
logger.warn("Failure while resolving " + originalException.getClass().getName(), resolverException);
}
return createDefaultErrors();
}

private List<GraphQLError> createDefaultErrors() {
GraphQLError error = GraphqlErrorBuilder.newError()
.message("Unknown error")
.errorType(ErrorType.DataFetchingException)
.build();

return Collections.singletonList(error);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,21 +16,20 @@

package org.springframework.graphql.execution;

import java.io.InputStream;
import java.util.List;
import java.util.function.BiFunction;
import java.util.function.Consumer;

import graphql.GraphQL;
import graphql.execution.instrumentation.Instrumentation;
import graphql.schema.GraphQLSchema;
import graphql.schema.GraphQLTypeVisitor;
import graphql.schema.TypeResolver;
import graphql.schema.idl.RuntimeWiring;
import graphql.schema.idl.TypeDefinitionRegistry;

import org.springframework.core.io.Resource;

import java.io.InputStream;
import java.util.List;
import java.util.function.BiFunction;
import java.util.function.Consumer;


/**
* Strategy to resolve a {@link GraphQL} and a {@link GraphQLSchema}.
Expand Down Expand Up @@ -91,6 +90,14 @@ interface Builder<B extends Builder<B>> {
*/
B exceptionResolvers(List<DataFetcherExceptionResolver> resolvers);

/**
* Add {@link SubscriptionExceptionResolver}s to map exceptions, thrown by
* GraphQL Subscription publisher.
* @param subscriptionExceptionResolver the subscription exception resolver
* @return the current builder
*/
B subscriptionExceptionResolvers(List<SubscriptionExceptionResolver> subscriptionExceptionResolvers);

/**
* Add {@link GraphQLTypeVisitor}s to visit all element of the created
* {@link graphql.schema.GraphQLSchema}.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
/*
* Copyright 2002-2022 the original author or authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.springframework.graphql.execution;

import graphql.GraphQLError;
import reactor.core.publisher.Mono;

import java.util.List;

/**
* Contract to resolve exceptions, that are thrown by subscription publisher.
* Implementations are typically declared as beans in Spring configuration and
* are invoked sequentially until one emits a List of {@link GraphQLError}s.
* <br/>
* Usually, it is enough to implement this interface by extending {@link SubscriptionExceptionResolverAdapter}
* and overriding one of its {@link SubscriptionExceptionResolverAdapter#resolveToSingleError(Throwable)}
* or {@link SubscriptionExceptionResolverAdapter#resolveToMultipleErrors(Throwable)}
*
* @author Mykyta Ivchenko
* @see SubscriptionExceptionResolverAdapter
* @see DelegatingSubscriptionExceptionResolver
* @see org.springframework.graphql.server.webflux.GraphQlWebSocketHandler
*/
@FunctionalInterface
public interface SubscriptionExceptionResolver {
/**
* Resolve given exception as list of {@link GraphQLError}s and send them as WebSocket message.
* @param exception the exception to resolve
* @return a {@code Mono} with errors to send in a WebSocket message;
* if the {@code Mono} completes with an empty List, the exception is resolved
* without any errors added to the response; if the {@code Mono} completes
* empty, without emitting a List, the exception remains unresolved and gives
* other resolvers a chance.
*/
Mono<List<GraphQLError>> resolveException(Throwable exception);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* Copyright 2002-2022 the original author or authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.springframework.graphql.execution;

import graphql.GraphQLError;
import reactor.core.publisher.Mono;

import java.util.Collections;
import java.util.List;

/**
* Abstract class for {@link SubscriptionExceptionResolver} implementations.
* This class provide an easy way to map an exception as GraphQL error synchronously.
* <br/>
* To use this class, you need to override either {@link SubscriptionExceptionResolverAdapter#resolveToSingleError(Throwable)}
* or {@link SubscriptionExceptionResolverAdapter#resolveToMultipleErrors(Throwable)}.
*
* @author Mykyta Ivchenko
* @see SubscriptionExceptionResolver
*/
public abstract class SubscriptionExceptionResolverAdapter implements SubscriptionExceptionResolver {
@Override
public Mono<List<GraphQLError>> resolveException(Throwable exception) {
return Mono.just(resolveToMultipleErrors(exception));
}

protected List<GraphQLError> resolveToMultipleErrors(Throwable exception) {
return Collections.singletonList(resolveToSingleError(exception));
}

protected GraphQLError resolveToSingleError(Throwable exception) {
return null;
}
}
Loading