Introduce Update annotation.

Switch update execution to an annotation based model that allows usage of both the classic update as well as the aggregation pipeline variant. Add the reactive variant of it.
Make sure to allow parameter binding for update expressions and verify method return types.
Update Javadoc and reference documentation.

See: #2107
Original Pull Request: #284
This commit is contained in:
Christoph Strobl
2021-03-09 07:55:19 +01:00
parent 6e3e8210d0
commit a6bd0fcea7
31 changed files with 1028 additions and 247 deletions

View File

@@ -0,0 +1,67 @@
/*
* Copyright 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.data.mongodb.repository;
import java.lang.annotation.Documented;
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
import org.springframework.core.annotation.AliasFor;
/**
* Annotation to declare update operators directly on repository methods. Both attributes allow using a placeholder
* notation of {@code ?0}, {@code ?1} and so on. The update will be applied to documents matching the either method name
* derived or annotated query, but not to any custom implementation methods.
*
* @author Christoph Strobl
*/
@Retention(RetentionPolicy.RUNTIME)
@Target({ ElementType.METHOD, ElementType.ANNOTATION_TYPE })
@Documented
public @interface Update {
/**
* Takes a MongoDB JSON string to define the actual update to be executed.
*
* @return the MongoDB JSON string representation of the update. Empty string by default.
* @see #update()
*/
@AliasFor("update")
String value() default "";
/**
* Takes a MongoDB JSON string to define the actual update to be executed.
*
* @return the MongoDB JSON string representation of the update. Empty string by default.
* @see <a href=
* "https://docs.mongodb.com/manual/tutorial/update-documents/">https://docs.mongodb.com/manual/tutorial/update-documents/</a>
*/
@AliasFor("value")
String update() default "";
/**
* Takes a MongoDB JSON string representation of an aggregation pipeline to define the update stages to be executed.
* <p>
* This allows to e.g. define update statement that can evaluate conditionals based on a field value, etc.
*
* @return the MongoDB JSON string representation of the update pipeline. Empty array by default.
* @see <a href=
* "https://docs.mongodb.com/manual/tutorial/update-documents-with-aggregation-pipeline/">https://docs.mongodb.com/manual/tutorial/update-documents-with-aggregation-pipeline</a>
*/
String[] pipeline() default {};
}

View File

@@ -15,30 +15,44 @@
*/
package org.springframework.data.mongodb.repository.query;
import java.util.ArrayList;
import java.util.List;
import org.bson.Document;
import org.bson.codecs.configuration.CodecRegistry;
import org.springframework.data.domain.Pageable;
import org.springframework.data.mapping.model.SpELExpressionEvaluator;
import org.springframework.data.mongodb.core.ExecutableFindOperation.ExecutableFind;
import org.springframework.data.mongodb.core.ExecutableFindOperation.FindWithQuery;
import org.springframework.data.mongodb.core.ExecutableFindOperation.TerminatingFind;
import org.springframework.data.mongodb.core.ExecutableUpdateOperation.ExecutableUpdate;
import org.springframework.data.mongodb.core.MongoOperations;
import org.springframework.data.mongodb.core.aggregation.AggregationOperation;
import org.springframework.data.mongodb.core.aggregation.AggregationUpdate;
import org.springframework.data.mongodb.core.query.BasicUpdate;
import org.springframework.data.mongodb.core.query.Query;
import org.springframework.data.mongodb.core.query.Update;
import org.springframework.data.mongodb.core.query.UpdateDefinition;
import org.springframework.data.mongodb.repository.Update;
import org.springframework.data.mongodb.repository.query.MongoQueryExecution.DeleteExecution;
import org.springframework.data.mongodb.repository.query.MongoQueryExecution.GeoNearExecution;
import org.springframework.data.mongodb.repository.query.MongoQueryExecution.PagedExecution;
import org.springframework.data.mongodb.repository.query.MongoQueryExecution.PagingGeoNearExecution;
import org.springframework.data.mongodb.repository.query.MongoQueryExecution.SlicedExecution;
import org.springframework.data.mongodb.repository.query.MongoQueryExecution.UpdateExecution;
import org.springframework.data.mongodb.util.json.ParameterBindingContext;
import org.springframework.data.mongodb.util.json.ParameterBindingDocumentCodec;
import org.springframework.data.repository.query.ParameterAccessor;
import org.springframework.data.repository.query.QueryMethodEvaluationContextProvider;
import org.springframework.data.repository.query.RepositoryQuery;
import org.springframework.data.repository.query.ResultProcessor;
import org.springframework.data.spel.ExpressionDependencies;
import org.springframework.data.util.Lazy;
import org.springframework.expression.EvaluationContext;
import org.springframework.expression.ExpressionParser;
import org.springframework.lang.NonNull;
import org.springframework.lang.Nullable;
import org.springframework.util.Assert;
import org.springframework.util.ObjectUtils;
import org.springframework.util.StringUtils;
import com.mongodb.client.MongoDatabase;
@@ -55,8 +69,11 @@ public abstract class AbstractMongoQuery implements RepositoryQuery {
private final MongoQueryMethod method;
private final MongoOperations operations;
private final ExecutableFind<?> executableFind;
private final ExecutableUpdate<?> executableUpdate;
private final ExpressionParser expressionParser;
private final QueryMethodEvaluationContextProvider evaluationContextProvider;
private final Lazy<ParameterBindingDocumentCodec> codec = Lazy
.of(() -> new ParameterBindingDocumentCodec(getCodecRegistry()));
/**
* Creates a new {@link AbstractMongoQuery} from the given {@link MongoQueryMethod} and {@link MongoOperations}.
@@ -81,6 +98,7 @@ public abstract class AbstractMongoQuery implements RepositoryQuery {
Class<?> type = metadata.getCollectionEntity().getType();
this.executableFind = operations.query(type);
this.executableUpdate = operations.update(type);
this.expressionParser = expressionParser;
this.evaluationContextProvider = evaluationContextProvider;
}
@@ -138,7 +156,17 @@ public abstract class AbstractMongoQuery implements RepositoryQuery {
if (isDeleteQuery()) {
return new DeleteExecution(operations, method);
} else if (method.isGeoNearQuery() && method.isPageQuery()) {
}
if (method.isModifyingQuery()) {
if (isLimiting()) {
throw new IllegalStateException(
String.format("Update method must not be limiting. Offending method: %s", method));
}
return new UpdateExecution(executableUpdate, method, () -> createUpdate(accessor), accessor);
}
if (method.isGeoNearQuery() && method.isPageQuery()) {
return new PagingGeoNearExecution(operation, method, accessor, this);
} else if (method.isGeoNearQuery()) {
return new GeoNearExecution(operation, method, accessor);
@@ -147,11 +175,6 @@ public abstract class AbstractMongoQuery implements RepositoryQuery {
} else if (method.isStreamQuery()) {
return q -> operation.matching(q).stream();
} else if (method.isCollectionQuery()) {
if (method.isModifyingQuery()) {
return q -> new UpdatingCollectionExecution(accessor.getPageable(), accessor.getUpdate()).execute(q);
}
return q -> operation.matching(q.with(accessor.getPageable()).with(accessor.getSort())).all();
} else if (method.isPageQuery()) {
return new PagedExecution(operation, accessor.getPageable());
@@ -161,11 +184,6 @@ public abstract class AbstractMongoQuery implements RepositoryQuery {
return q -> operation.matching(q).exists();
} else {
return q -> {
if (method.isModifyingQuery()) {
return new UpdatingSingleEntityExecution(accessor.getUpdate()).execute(q);
}
TerminatingFind<?> find = operation.matching(q);
return isLimiting() ? find.firstValue() : find.oneValue();
};
@@ -225,6 +243,94 @@ public abstract class AbstractMongoQuery implements RepositoryQuery {
return applyQueryMetaAttributesWhenPresent(createQuery(accessor));
}
/**
* Retrieves the {@link UpdateDefinition update} from the given
* {@link org.springframework.data.mongodb.repository.query.MongoParameterAccessor#getUpdate() accessor} or creates
* one via by parsing the annotated statement extracted from {@link Update}.
*
* @param accessor never {@literal null}.
* @return the computed {@link UpdateDefinition}.
* @throws IllegalStateException if no update could be found.
* @since 3.4
*/
protected UpdateDefinition createUpdate(ConvertingParameterAccessor accessor) {
if (accessor.getUpdate() != null) {
return accessor.getUpdate();
}
if (method.hasAnnotatedUpdate()) {
Update updateSource = method.getUpdateSource();
if (StringUtils.hasText(updateSource.update())) {
return new BasicUpdate(bindParameters(updateSource.update(), accessor));
}
if (!ObjectUtils.isEmpty(updateSource.pipeline())) {
return AggregationUpdate.from(parseAggregationPipeline(updateSource.pipeline(), accessor));
}
}
throw new IllegalStateException(String.format("No Update provided for method %s.", method));
}
/**
* Parse the given aggregation pipeline stages applying values to placeholders to compute the actual list of
* {@link AggregationOperation operations}.
*
* @param sourcePipeline must not be {@literal null}.
* @param accessor must not be {@literal null}.
* @return the parsed aggregation pipeline.
* @since 3.4
*/
protected List<AggregationOperation> parseAggregationPipeline(String[] sourcePipeline,
ConvertingParameterAccessor accessor) {
List<AggregationOperation> stages = new ArrayList<>(sourcePipeline.length);
for (String source : sourcePipeline) {
stages.add(computePipelineStage(source, accessor));
}
return stages;
}
private AggregationOperation computePipelineStage(String source, ConvertingParameterAccessor accessor) {
return ctx -> ctx.getMappedObject(bindParameters(source, accessor), getQueryMethod().getDomainClass());
}
protected Document decode(String source, ParameterBindingContext bindingContext) {
return getParameterBindingCodec().decode(source, bindingContext);
}
private Document bindParameters(String source, ConvertingParameterAccessor accessor) {
return decode(source, prepareBindingContext(source, accessor));
}
/**
* Create the {@link ParameterBindingContext binding context} used for SpEL evaluation.
*
* @param source the JSON source.
* @param accessor value provider for parameter binding.
* @return never {@literal null}.
* @since 3.4
*/
protected ParameterBindingContext prepareBindingContext(String source, ConvertingParameterAccessor accessor) {
ExpressionDependencies dependencies = getParameterBindingCodec().captureExpressionDependencies(source,
accessor::getBindableValue, expressionParser);
SpELExpressionEvaluator evaluator = getSpELExpressionEvaluatorFor(dependencies, accessor);
return new ParameterBindingContext(accessor::getBindableValue, evaluator);
}
/**
* Obtain the {@link ParameterBindingDocumentCodec} used for parsing JSON expressions.
*
* @return never {@literal null}.
* @since 3.4
*/
protected ParameterBindingDocumentCodec getParameterBindingCodec() {
return codec.get();
}
/**
* Obtain a the {@link EvaluationContext} suitable to evaluate expressions backed by the given dependencies.
*
@@ -286,53 +392,4 @@ public abstract class AbstractMongoQuery implements RepositoryQuery {
* @since 2.0.4
*/
protected abstract boolean isLimiting();
/**
* {@link MongoQueryExecution} for collection returning find and update queries.
*
* @author Thomas Darimont
*/
final class UpdatingCollectionExecution implements MongoQueryExecution {
private final Pageable pageable;
private final Update update;
UpdatingCollectionExecution(Pageable pageable, Update update) {
this.pageable = pageable;
this.update = update;
}
@Override
public Object execute(Query query) {
MongoEntityMetadata<?> metadata = method.getEntityInformation();
return operations.findAndModify(query.with(pageable), update, metadata.getJavaType(),
metadata.getCollectionName());
}
}
/**
* {@link MongoQueryExecution} to return a single entity with update.
*
* @author Thomas Darimont
*/
final class UpdatingSingleEntityExecution implements MongoQueryExecution {
private final Update update;
private UpdatingSingleEntityExecution(Update update) {
this.update = update;
}
/*
* (non-Javadoc)
* @see org.springframework.data.mongodb.repository.AbstractMongoQuery.Execution#execute(org.springframework.data.mongodb.core.core.query.Query)
*/
@Override
public Object execute(Query query) {
MongoEntityMetadata<?> metadata = method.getEntityInformation();
return operations.findAndModify(query.limit(1), update, metadata.getJavaType(), metadata.getCollectionName());
}
}
}

View File

@@ -15,12 +15,16 @@
*/
package org.springframework.data.mongodb.repository.query;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import reactor.util.function.Tuple2;
import java.util.ArrayList;
import java.util.List;
import org.bson.Document;
import org.bson.codecs.configuration.CodecRegistry;
import org.reactivestreams.Publisher;
import org.springframework.core.convert.converter.Converter;
import org.springframework.data.mapping.model.EntityInstantiators;
import org.springframework.data.mapping.model.SpELExpressionEvaluator;
@@ -29,11 +33,20 @@ import org.springframework.data.mongodb.core.ReactiveFindOperation.FindWithProje
import org.springframework.data.mongodb.core.ReactiveFindOperation.FindWithQuery;
import org.springframework.data.mongodb.core.ReactiveFindOperation.TerminatingFind;
import org.springframework.data.mongodb.core.ReactiveMongoOperations;
import org.springframework.data.mongodb.core.ReactiveUpdateOperation.ReactiveUpdate;
import org.springframework.data.mongodb.core.aggregation.AggregationOperation;
import org.springframework.data.mongodb.core.aggregation.AggregationUpdate;
import org.springframework.data.mongodb.core.query.BasicUpdate;
import org.springframework.data.mongodb.core.query.Query;
import org.springframework.data.mongodb.core.query.UpdateDefinition;
import org.springframework.data.mongodb.repository.Update;
import org.springframework.data.mongodb.repository.query.ReactiveMongoQueryExecution.DeleteExecution;
import org.springframework.data.mongodb.repository.query.ReactiveMongoQueryExecution.GeoNearExecution;
import org.springframework.data.mongodb.repository.query.ReactiveMongoQueryExecution.ResultProcessingConverter;
import org.springframework.data.mongodb.repository.query.ReactiveMongoQueryExecution.ResultProcessingExecution;
import org.springframework.data.mongodb.repository.query.ReactiveMongoQueryExecution.UpdateExecution;
import org.springframework.data.mongodb.util.json.ParameterBindingContext;
import org.springframework.data.mongodb.util.json.ParameterBindingDocumentCodec;
import org.springframework.data.repository.query.ParameterAccessor;
import org.springframework.data.repository.query.ReactiveQueryMethodEvaluationContextProvider;
import org.springframework.data.repository.query.RepositoryQuery;
@@ -43,6 +56,8 @@ import org.springframework.data.util.TypeInformation;
import org.springframework.expression.ExpressionParser;
import org.springframework.lang.Nullable;
import org.springframework.util.Assert;
import org.springframework.util.ObjectUtils;
import org.springframework.util.StringUtils;
import com.mongodb.MongoClientSettings;
@@ -59,6 +74,7 @@ public abstract class AbstractReactiveMongoQuery implements RepositoryQuery {
private final ReactiveMongoOperations operations;
private final EntityInstantiators instantiators;
private final FindWithProjection<?> findOperationWithProjection;
private final ReactiveUpdate<?> updateOps;
private final ExpressionParser expressionParser;
private final ReactiveQueryMethodEvaluationContextProvider evaluationContextProvider;
@@ -89,6 +105,7 @@ public abstract class AbstractReactiveMongoQuery implements RepositoryQuery {
Class<?> type = metadata.getCollectionEntity().getType();
this.findOperationWithProjection = operations.query(type);
this.updateOps = operations.update(type);
}
/*
@@ -180,6 +197,14 @@ public abstract class AbstractReactiveMongoQuery implements RepositoryQuery {
if (isDeleteQuery()) {
return new DeleteExecution(operations, method);
} else if (method.isModifyingQuery()) {
if (isLimiting()) {
throw new IllegalStateException(
String.format("Update method must not be limiting. Offending method: %s", method));
}
return new UpdateExecution(updateOps, method, accessor, createUpdate(accessor));
} else if (method.isGeoNearQuery()) {
return new GeoNearExecution(operations, accessor, method.getReturnType());
} else if (isTailable(method)) {
@@ -261,6 +286,97 @@ public abstract class AbstractReactiveMongoQuery implements RepositoryQuery {
return createQuery(accessor).map(this::applyQueryMetaAttributesWhenPresent);
}
/**
* Retrieves the {@link UpdateDefinition update} from the given
* {@link org.springframework.data.mongodb.repository.query.MongoParameterAccessor#getUpdate() accessor} or creates
* one via by parsing the annotated statement extracted from {@link Update}.
*
* @param accessor never {@literal null}.
* @return the computed {@link UpdateDefinition}.
* @throws IllegalStateException if no update could be found.
* @since 3.4
*/
protected Mono<UpdateDefinition> createUpdate(MongoParameterAccessor accessor) {
if (accessor.getUpdate() != null) {
return Mono.just(accessor.getUpdate());
}
if (method.hasAnnotatedUpdate()) {
Update updateSource = method.getUpdateSource();
if (StringUtils.hasText(updateSource.update())) {
String updateJson = updateSource.update();
return getParameterBindingCodec() //
.flatMap(codec -> expressionEvaluator(updateJson, accessor, codec)) //
.map(it -> decode(it.getT1(), updateJson, accessor, it.getT2())) //
.map(BasicUpdate::fromDocument);
}
if (!ObjectUtils.isEmpty(updateSource.pipeline())) {
return parseAggregationPipeline(updateSource.pipeline(), accessor).map(AggregationUpdate::from);
}
}
throw new IllegalStateException(String.format("No Update provided for method %s.", method));
}
/**
* Parse the given aggregation pipeline stages applying values to placeholders to compute the actual list of
* {@link AggregationOperation operations}.
*
* @param pipeline must not be {@literal null}.
* @param accessor must not be {@literal null}.
* @return the parsed aggregation pipeline.
* @since 3.4
*/
protected Mono<List<AggregationOperation>> parseAggregationPipeline(String[] pipeline,
MongoParameterAccessor accessor) {
return getCodecRegistry().map(ParameterBindingDocumentCodec::new).flatMap(codec -> {
List<Mono<AggregationOperation>> stages = new ArrayList<>(pipeline.length);
for (String source : pipeline) {
stages.add(computePipelineStage(source, accessor, codec));
}
return Flux.concat(stages).collectList();
});
}
private Mono<AggregationOperation> computePipelineStage(String source, MongoParameterAccessor accessor,
ParameterBindingDocumentCodec codec) {
return expressionEvaluator(source, accessor, codec).map(it -> {
return ctx -> ctx.getMappedObject(decode(it.getT1(), source, accessor, it.getT2()),
getQueryMethod().getDomainClass());
});
}
private Mono<Tuple2<SpELExpressionEvaluator, ParameterBindingDocumentCodec>> expressionEvaluator(String source,
MongoParameterAccessor accessor, ParameterBindingDocumentCodec codec) {
ExpressionDependencies dependencies = codec.captureExpressionDependencies(source, accessor::getBindableValue,
expressionParser);
return getSpelEvaluatorFor(dependencies, accessor).zipWith(Mono.just(codec));
}
private Document decode(SpELExpressionEvaluator expressionEvaluator, String source, MongoParameterAccessor accessor,
ParameterBindingDocumentCodec codec) {
ParameterBindingContext bindingContext = new ParameterBindingContext(accessor::getBindableValue,
expressionEvaluator);
return codec.decode(source, bindingContext);
}
/**
* Obtain the {@link ParameterBindingDocumentCodec} used for parsing JSON expressions.
*
* @return never {@literal null}.
* @since 3.4
*/
protected Mono<ParameterBindingDocumentCodec> getParameterBindingCodec() {
return getCodecRegistry().map(ParameterBindingDocumentCodec::new);
}
/**
* Obtain a {@link Mono publisher} emitting the {@link SpELExpressionEvaluator} suitable to evaluate expressions
* backed by the given dependencies.
@@ -269,10 +385,27 @@ public abstract class AbstractReactiveMongoQuery implements RepositoryQuery {
* @param accessor must not be {@literal null}.
* @return a {@link Mono} emitting the {@link SpELExpressionEvaluator} when ready.
* @since 2.4
* @deprecated in favor of {@link #getSpelEvaluatorFor(ExpressionDependencies, MongoParameterAccessor)}
*/
@Deprecated
protected Mono<SpELExpressionEvaluator> getSpelEvaluatorFor(ExpressionDependencies dependencies,
ConvertingParameterAccessor accessor) {
return getSpelEvaluatorFor(dependencies, (MongoParameterAccessor) accessor);
}
/**
* Obtain a {@link Mono publisher} emitting the {@link SpELExpressionEvaluator} suitable to evaluate expressions
* backed by the given dependencies.
*
* @param dependencies must not be {@literal null}.
* @param accessor must not be {@literal null}.
* @return a {@link Mono} emitting the {@link SpELExpressionEvaluator} when ready.
* @since 3.4
*/
protected Mono<SpELExpressionEvaluator> getSpelEvaluatorFor(ExpressionDependencies dependencies,
MongoParameterAccessor accessor) {
return evaluationContextProvider
.getEvaluationContextLater(getQueryMethod().getParameters(), accessor.getValues(), dependencies)
.map(evaluationContext -> (SpELExpressionEvaluator) new DefaultSpELExpressionEvaluator(expressionParser,

View File

@@ -31,7 +31,7 @@ import org.springframework.data.mongodb.core.convert.MongoWriter;
import org.springframework.data.mongodb.core.mapping.MongoPersistentProperty;
import org.springframework.data.mongodb.core.query.Collation;
import org.springframework.data.mongodb.core.query.TextCriteria;
import org.springframework.data.mongodb.core.query.Update;
import org.springframework.data.mongodb.core.query.UpdateDefinition;
import org.springframework.data.repository.query.ParameterAccessor;
import org.springframework.data.util.TypeInformation;
import org.springframework.lang.Nullable;
@@ -155,6 +155,11 @@ public class ConvertingParameterAccessor implements MongoParameterAccessor {
return delegate.getCollation();
}
@Override
public UpdateDefinition getUpdate() {
return delegate.getUpdate();
}
/**
* Converts the given value with the underlying {@link MongoWriter}.
*
@@ -297,12 +302,4 @@ public class ConvertingParameterAccessor implements MongoParameterAccessor {
*/
Object nextConverted(MongoPersistentProperty property);
}
/* (non-Javadoc)
* @see org.springframework.data.mongodb.repository.query.MongoParameterAccessor#getUpdate()
*/
@Override
public Update getUpdate() {
return delegate.getUpdate();
}
}

View File

@@ -21,6 +21,7 @@ import org.springframework.data.geo.Point;
import org.springframework.data.mongodb.core.query.Collation;
import org.springframework.data.mongodb.core.query.TextCriteria;
import org.springframework.data.mongodb.core.query.Update;
import org.springframework.data.mongodb.core.query.UpdateDefinition;
import org.springframework.data.repository.query.ParameterAccessor;
import org.springframework.lang.Nullable;
@@ -77,10 +78,11 @@ public interface MongoParameterAccessor extends ParameterAccessor {
Object[] getValues();
/**
* Returns the {@link Update} to be used for findAndUpdate query.
* Returns the {@link Update} to be used for an update execution.
*
* @return
* @since 1.7
* @return {@literal null} if not present.
* @since 3.4
*/
Update getUpdate();
@Nullable
UpdateDefinition getUpdate();
}

View File

@@ -25,7 +25,7 @@ import org.springframework.data.geo.Distance;
import org.springframework.data.geo.Point;
import org.springframework.data.mongodb.core.query.Collation;
import org.springframework.data.mongodb.core.query.TextCriteria;
import org.springframework.data.mongodb.core.query.Update;
import org.springframework.data.mongodb.core.query.UpdateDefinition;
import org.springframework.data.mongodb.repository.Near;
import org.springframework.data.mongodb.repository.query.MongoParameters.MongoParameter;
import org.springframework.data.repository.query.Parameter;
@@ -70,7 +70,7 @@ public class MongoParameters extends Parameters<MongoParameters, MongoParameter>
this.rangeIndex = getTypeIndex(parameterTypeInfo, Range.class, Distance.class);
this.maxDistanceIndex = this.rangeIndex == -1 ? getTypeIndex(parameterTypeInfo, Distance.class, null) : -1;
this.collationIndex = getTypeIndex(parameterTypeInfo, Collation.class, null);
this.updateIndex = parameterTypes.indexOf(Update.class);
this.updateIndex = QueryUtils.indexOfAssignableParameter(UpdateDefinition.class, parameterTypes);
int index = findNearIndexInParameters(method);
if (index == -1 && isGeoNearMethod) {
@@ -200,6 +200,15 @@ public class MongoParameters extends Parameters<MongoParameters, MongoParameter>
return collationIndex != null ? collationIndex.intValue() : -1;
}
/**
* Returns the index of the {@link UpdateDefinition} parameter or -1 if not present.
* @return -1 if not present.
* @since 3.4
*/
public int getUpdateIndex() {
return updateIndex;
}
/*
* (non-Javadoc)
* @see org.springframework.data.repository.query.Parameters#createFrom(java.util.List)
@@ -280,7 +289,4 @@ public class MongoParameters extends Parameters<MongoParameters, MongoParameter>
}
}
public int getUpdateIndex() {
return updateIndex;
}
}

View File

@@ -22,7 +22,7 @@ import org.springframework.data.geo.Point;
import org.springframework.data.mongodb.core.query.Collation;
import org.springframework.data.mongodb.core.query.Term;
import org.springframework.data.mongodb.core.query.TextCriteria;
import org.springframework.data.mongodb.core.query.Update;
import org.springframework.data.mongodb.core.query.UpdateDefinition;
import org.springframework.data.repository.query.ParametersParameterAccessor;
import org.springframework.lang.Nullable;
import org.springframework.util.Assert;
@@ -156,9 +156,9 @@ public class MongoParametersParameterAccessor extends ParametersParameterAccesso
}
@Override
public Update getUpdate() {
public UpdateDefinition getUpdate() {
int updateIndex = method.getParameters().getUpdateIndex();
return updateIndex == -1 ? null : (Update) getValue(updateIndex);
return updateIndex == -1 ? null : (UpdateDefinition) getValue(updateIndex);
}
}

View File

@@ -16,6 +16,7 @@
package org.springframework.data.mongodb.repository.query;
import java.util.List;
import java.util.function.Supplier;
import org.springframework.data.domain.Page;
import org.springframework.data.domain.Pageable;
@@ -30,9 +31,11 @@ import org.springframework.data.geo.Point;
import org.springframework.data.mongodb.core.ExecutableFindOperation;
import org.springframework.data.mongodb.core.ExecutableFindOperation.FindWithQuery;
import org.springframework.data.mongodb.core.ExecutableFindOperation.TerminatingFind;
import org.springframework.data.mongodb.core.ExecutableUpdateOperation.ExecutableUpdate;
import org.springframework.data.mongodb.core.MongoOperations;
import org.springframework.data.mongodb.core.query.NearQuery;
import org.springframework.data.mongodb.core.query.Query;
import org.springframework.data.mongodb.core.query.UpdateDefinition;
import org.springframework.data.repository.support.PageableExecutionUtils;
import org.springframework.data.util.TypeInformation;
import org.springframework.util.Assert;
@@ -298,4 +301,35 @@ interface MongoQueryExecution {
return writeResult.wasAcknowledged() ? writeResult.getDeletedCount() : 0L;
}
}
/**
* {@link MongoQueryExecution} updating documents matching the query.
*
* @author Christph Strobl
* @since 3.4
*/
final class UpdateExecution implements MongoQueryExecution {
private final ExecutableUpdate<?> updateOps;
private final MongoQueryMethod method;
private Supplier<UpdateDefinition> updateDefinitionSupplier;
private final MongoParameterAccessor accessor;
UpdateExecution(ExecutableUpdate<?> updateOps, MongoQueryMethod method, Supplier<UpdateDefinition> updateSupplier,
MongoParameterAccessor accessor) {
this.updateOps = updateOps;
this.method = method;
this.updateDefinitionSupplier = updateSupplier;
this.accessor = accessor;
}
@Override
public Object execute(Query query) {
return updateOps.matching(query.with(accessor.getSort())) //
.apply(updateDefinitionSupplier.get()) //
.all().getModifiedCount();
}
}
}

View File

@@ -30,15 +30,18 @@ import org.springframework.data.geo.GeoResults;
import org.springframework.data.mapping.context.MappingContext;
import org.springframework.data.mongodb.core.mapping.MongoPersistentEntity;
import org.springframework.data.mongodb.core.mapping.MongoPersistentProperty;
import org.springframework.data.mongodb.core.query.Update;
import org.springframework.data.mongodb.core.query.UpdateDefinition;
import org.springframework.data.mongodb.repository.Aggregation;
import org.springframework.data.mongodb.repository.Meta;
import org.springframework.data.mongodb.repository.Query;
import org.springframework.data.mongodb.repository.Tailable;
import org.springframework.data.mongodb.repository.Update;
import org.springframework.data.projection.ProjectionFactory;
import org.springframework.data.repository.core.RepositoryMetadata;
import org.springframework.data.repository.query.QueryMethod;
import org.springframework.data.repository.util.ReactiveWrappers;
import org.springframework.data.util.ClassTypeInformation;
import org.springframework.data.util.Lazy;
import org.springframework.data.util.TypeInformation;
import org.springframework.lang.Nullable;
import org.springframework.util.Assert;
@@ -64,6 +67,7 @@ public class MongoQueryMethod extends QueryMethod {
private final Map<Class<? extends Annotation>, Optional<Annotation>> annotationCache;
private @Nullable MongoEntityMetadata<?> metadata;
private Lazy<Boolean> isModifying = Lazy.of(this::resolveModifyingQueryIndicators);
/**
* Creates a new {@link MongoQueryMethod} from the given {@link Method}.
@@ -393,6 +397,10 @@ public class MongoQueryMethod extends QueryMethod {
return doFindAnnotation(Aggregation.class);
}
Optional<Update> lookupUpdateAnnotation() {
return doFindAnnotation(Update.class);
}
@SuppressWarnings("unchecked")
private <A extends Annotation> Optional<A> doFindAnnotation(Class<A> annotationType) {
@@ -402,8 +410,66 @@ public class MongoQueryMethod extends QueryMethod {
@Override
public boolean isModifyingQuery() {
return isModifying.get();
}
Class<?>[] parameterTypes = this.method.getParameterTypes();
return parameterTypes.length > 0 && parameterTypes[parameterTypes.length - 1] == Update.class;
private boolean resolveModifyingQueryIndicators() {
return hasAnnotatedUpdate()
|| QueryUtils.indexOfAssignableParameter(UpdateDefinition.class, method.getParameterTypes()) != -1;
}
/**
* @return {@literal true} if {@link Update} annotation is present.
* @since 3.4
*/
public boolean hasAnnotatedUpdate() {
return lookupUpdateAnnotation().isPresent();
}
/**
* @return the {@link Update} or {@literal null} if not present.
* @since 3.4
*/
public Update getUpdateSource() {
return lookupUpdateAnnotation().get();
}
/**
* Verify the actual {@link QueryMethod} is valid in terms of supported return and parameter types.
*
* @since 3.4
* @throws IllegalStateException
*/
public void verify() {
if (isModifyingQuery()) {
if (isCollectionQuery() || isSliceQuery() || isPageQuery() || isGeoNearQuery() || !isNumericOrVoidReturnValue()) { //
throw new IllegalStateException(
String.format("Update method may be void or return a numeric value (the number of updated documents)."
+ "Offending method: %s", method));
}
if (hasAnnotatedUpdate()) { // must define either an update or an update pipeline
if (!StringUtils.hasText(getUpdateSource().update()) && ObjectUtils.isEmpty(getUpdateSource().pipeline())) {
throw new IllegalStateException(
String.format("Update method must define either 'Update#update' or 'Update#pipeline' attribute. "
+ "Offending method: %s", method));
}
}
}
}
private boolean isNumericOrVoidReturnValue() {
Class<?> resultType = getReturnedObjectType();
if(ReactiveWrappers.usesReactiveType(resultType)) {
resultType = getReturnType().getComponentType().getType();
}
boolean isUpdateCountReturnType = ClassUtils.isAssignable(Number.class, resultType);
boolean isVoidReturnType = ClassUtils.isAssignable(Void.class, resultType);
return isUpdateCountReturnType || isVoidReturnType;
}
}

View File

@@ -15,6 +15,9 @@
*/
package org.springframework.data.mongodb.repository.query;
import java.util.Arrays;
import java.util.List;
import org.aopalliance.intercept.MethodInterceptor;
import org.bson.Document;
import org.springframework.aop.framework.ProxyFactory;
@@ -22,8 +25,8 @@ import org.springframework.data.mongodb.core.query.Collation;
import org.springframework.data.mongodb.core.query.Query;
import org.springframework.data.repository.query.QueryMethodEvaluationContextProvider;
import org.springframework.expression.ExpressionParser;
import org.springframework.expression.spel.standard.SpelExpressionParser;
import org.springframework.lang.Nullable;
import org.springframework.util.ClassUtils;
/**
* Internal utility class to help avoid duplicate code required in both the reactive and the sync {@link Query} support
@@ -84,4 +87,40 @@ class QueryUtils {
evaluationContextProvider);
return collation == null ? query : query.collation(collation);
}
/**
* Get the first index of the parameter that can be assigned to the given type.
*
* @param type the type to look for.
* @param parameters the actual parameters.
* @return -1 if not found.
* @since 3.4
*/
static int indexOfAssignableParameter(Class<?> type, Class<?>[] parameters) {
return indexOfAssignableParameter(type, Arrays.asList(parameters));
}
/**
* Get the first index of the parameter that can be assigned to the given type.
*
* @param type the type to look for.
* @param parameters the actual parameters.
* @return -1 if not found.
* @since 3.4
*/
static int indexOfAssignableParameter(Class<?> type, List<Class<?>> parameters) {
if(parameters.isEmpty()) {
return -1;
}
int i = 0;
for(Class<?> parameterType : parameters) {
if(ClassUtils.isAssignable(type, parameterType)) {
return i;
}
i++;
}
return -1;
}
}

View File

@@ -28,8 +28,10 @@ import org.springframework.data.geo.GeoResult;
import org.springframework.data.geo.Point;
import org.springframework.data.mapping.model.EntityInstantiators;
import org.springframework.data.mongodb.core.ReactiveMongoOperations;
import org.springframework.data.mongodb.core.ReactiveUpdateOperation.ReactiveUpdate;
import org.springframework.data.mongodb.core.query.NearQuery;
import org.springframework.data.mongodb.core.query.Query;
import org.springframework.data.mongodb.core.query.UpdateDefinition;
import org.springframework.data.repository.query.ResultProcessor;
import org.springframework.data.repository.query.ReturnedType;
import org.springframework.data.repository.util.ReactiveWrappers;
@@ -39,6 +41,8 @@ import org.springframework.lang.Nullable;
import org.springframework.util.Assert;
import org.springframework.util.ClassUtils;
import com.mongodb.client.result.UpdateResult;
/**
* Set of classes to contain query execution strategies. Depending (mostly) on the return type of a
* {@link org.springframework.data.repository.query.QueryMethod} a {@link AbstractReactiveMongoQuery} can be executed in
@@ -149,6 +153,39 @@ interface ReactiveMongoQueryExecution {
}
}
/**
* {@link MongoQueryExecution} updating documents matching the query.
*
* @author Christph Strobl
* @since 3.4
*/
final class UpdateExecution implements ReactiveMongoQueryExecution {
private final ReactiveUpdate<?> updateOps;
private final MongoQueryMethod method;
private final MongoParameterAccessor accessor;
private Mono<UpdateDefinition> update;
UpdateExecution(ReactiveUpdate<?> updateOps, ReactiveMongoQueryMethod method, MongoParameterAccessor accessor,
Mono<UpdateDefinition> update) {
this.updateOps = updateOps;
this.method = method;
this.accessor = accessor;
this.update = update;
}
@Override
public Publisher<? extends Object> execute(Query query, Class<?> type, String collection) {
return update.flatMap(it -> updateOps.inCollection(collection) //
.matching(query.with(accessor.getSort())) // actually we could do it unsorted
.apply(it) //
.all() //
.map(UpdateResult::getModifiedCount));
}
}
/**
* An {@link ReactiveMongoQueryExecution} that wraps the results of the given delegate with the given result
* processing.

View File

@@ -66,33 +66,6 @@ public class ReactiveMongoQueryMethod extends MongoQueryMethod {
super(method, metadata, projectionFactory, mappingContext);
if (hasParameterOfType(method, Pageable.class)) {
TypeInformation<?> returnType = ClassTypeInformation.fromReturnTypeOf(method);
boolean multiWrapper = ReactiveWrappers.isMultiValueType(returnType.getType());
boolean singleWrapperWithWrappedPageableResult = ReactiveWrappers.isSingleValueType(returnType.getType())
&& (PAGE_TYPE.isAssignableFrom(returnType.getRequiredComponentType())
|| SLICE_TYPE.isAssignableFrom(returnType.getRequiredComponentType()));
if (singleWrapperWithWrappedPageableResult) {
throw new InvalidDataAccessApiUsageException(
String.format("'%s.%s' must not use sliced or paged execution. Please use Flux.buffer(size, skip).",
ClassUtils.getShortName(method.getDeclaringClass()), method.getName()));
}
if (!multiWrapper) {
throw new IllegalStateException(String.format(
"Method has to use a either multi-item reactive wrapper return type or a wrapped Page/Slice type. Offending method: %s",
method.toString()));
}
if (hasParameterOfType(method, Sort.class)) {
throw new IllegalStateException(String.format("Method must not have Pageable *and* Sort parameter. "
+ "Use sorting capabilities on Pageable instead! Offending method: %s", method.toString()));
}
}
this.method = method;
this.isCollectionQuery = Lazy.of(() -> (!(isPageQuery() || isSliceQuery())
&& ReactiveWrappers.isMultiValueType(metadata.getReturnType(method).getType()) || super.isCollectionQuery()));
@@ -179,4 +152,36 @@ public class ReactiveMongoQueryMethod extends MongoQueryMethod {
return false;
}
@Override
public void verify() {
if (hasParameterOfType(method, Pageable.class)) {
TypeInformation<?> returnType = ClassTypeInformation.fromReturnTypeOf(method);
boolean multiWrapper = ReactiveWrappers.isMultiValueType(returnType.getType());
boolean singleWrapperWithWrappedPageableResult = ReactiveWrappers.isSingleValueType(returnType.getType())
&& (PAGE_TYPE.isAssignableFrom(returnType.getRequiredComponentType())
|| SLICE_TYPE.isAssignableFrom(returnType.getRequiredComponentType()));
if (singleWrapperWithWrappedPageableResult) {
throw new InvalidDataAccessApiUsageException(
String.format("'%s.%s' must not use sliced or paged execution. Please use Flux.buffer(size, skip).",
ClassUtils.getShortName(method.getDeclaringClass()), method.getName()));
}
if (!multiWrapper) {
throw new IllegalStateException(String.format(
"Method has to use a either multi-item reactive wrapper return type or a wrapped Page/Slice type. Offending method: %s",
method.toString()));
}
if (hasParameterOfType(method, Sort.class)) {
throw new IllegalStateException(String.format("Method must not have Pageable *and* Sort parameter. "
+ "Use sorting capabilities on Pageable instead! Offending method: %s", method.toString()));
}
}
super.verify();
}
}

View File

@@ -18,7 +18,6 @@ package org.springframework.data.mongodb.repository.query;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import java.util.ArrayList;
import java.util.List;
import org.bson.Document;
@@ -31,11 +30,8 @@ import org.springframework.data.mongodb.core.aggregation.TypedAggregation;
import org.springframework.data.mongodb.core.convert.MongoConverter;
import org.springframework.data.mongodb.core.mapping.MongoSimpleTypes;
import org.springframework.data.mongodb.core.query.Query;
import org.springframework.data.mongodb.util.json.ParameterBindingContext;
import org.springframework.data.mongodb.util.json.ParameterBindingDocumentCodec;
import org.springframework.data.repository.query.ReactiveQueryMethodEvaluationContextProvider;
import org.springframework.data.repository.query.ResultProcessor;
import org.springframework.data.spel.ExpressionDependencies;
import org.springframework.expression.ExpressionParser;
import org.springframework.util.ClassUtils;
@@ -122,30 +118,7 @@ public class ReactiveStringBasedAggregation extends AbstractReactiveMongoQuery {
}
private Mono<List<AggregationOperation>> computePipeline(ConvertingParameterAccessor accessor) {
return getCodecRegistry().map(ParameterBindingDocumentCodec::new).flatMap(codec -> {
String[] sourcePipeline = getQueryMethod().getAnnotatedAggregation();
List<Mono<AggregationOperation>> stages = new ArrayList<>(sourcePipeline.length);
for (String source : sourcePipeline) {
stages.add(computePipelineStage(source, accessor, codec));
}
return Flux.concat(stages).collectList();
});
}
private Mono<AggregationOperation> computePipelineStage(String source, ConvertingParameterAccessor accessor,
ParameterBindingDocumentCodec codec) {
ExpressionDependencies dependencies = codec.captureExpressionDependencies(source, accessor::getBindableValue,
expressionParser);
return getSpelEvaluatorFor(dependencies, accessor).map(it -> {
ParameterBindingContext bindingContext = new ParameterBindingContext(accessor::getBindableValue, it);
return ctx -> ctx.getMappedObject(codec.decode(source, bindingContext), getQueryMethod().getDomainClass());
});
return parseAggregationPipeline(getQueryMethod().getAnnotatedAggregation(), accessor);
}
private AggregationOptions computeOptions(MongoQueryMethod method, ConvertingParameterAccessor accessor) {

View File

@@ -24,7 +24,6 @@ import org.bson.Document;
import org.springframework.data.domain.Pageable;
import org.springframework.data.domain.SliceImpl;
import org.springframework.data.mapping.model.SpELExpressionEvaluator;
import org.springframework.data.mongodb.InvalidMongoDbApiUsageException;
import org.springframework.data.mongodb.core.MongoOperations;
import org.springframework.data.mongodb.core.aggregation.Aggregation;
@@ -35,11 +34,8 @@ import org.springframework.data.mongodb.core.aggregation.TypedAggregation;
import org.springframework.data.mongodb.core.convert.MongoConverter;
import org.springframework.data.mongodb.core.mapping.MongoSimpleTypes;
import org.springframework.data.mongodb.core.query.Query;
import org.springframework.data.mongodb.util.json.ParameterBindingContext;
import org.springframework.data.mongodb.util.json.ParameterBindingDocumentCodec;
import org.springframework.data.repository.query.QueryMethodEvaluationContextProvider;
import org.springframework.data.repository.query.ResultProcessor;
import org.springframework.data.spel.ExpressionDependencies;
import org.springframework.expression.ExpressionParser;
import org.springframework.util.ClassUtils;
@@ -172,26 +168,7 @@ public class StringBasedAggregation extends AbstractMongoQuery {
}
List<AggregationOperation> computePipeline(MongoQueryMethod method, ConvertingParameterAccessor accessor) {
ParameterBindingDocumentCodec codec = new ParameterBindingDocumentCodec(getCodecRegistry());
String[] sourcePipeline = method.getAnnotatedAggregation();
List<AggregationOperation> stages = new ArrayList<>(sourcePipeline.length);
for (String source : sourcePipeline) {
stages.add(computePipelineStage(source, accessor, codec));
}
return stages;
}
private AggregationOperation computePipelineStage(String source, ConvertingParameterAccessor accessor,
ParameterBindingDocumentCodec codec) {
ExpressionDependencies dependencies = codec.captureExpressionDependencies(source, accessor::getBindableValue,
expressionParser);
SpELExpressionEvaluator evaluator = getSpELExpressionEvaluatorFor(dependencies, accessor);
ParameterBindingContext bindingContext = new ParameterBindingContext(accessor::getBindableValue, evaluator);
return ctx -> ctx.getMappedObject(codec.decode(source, bindingContext), getQueryMethod().getDomainClass());
return parseAggregationPipeline(method.getAnnotatedAggregation(), accessor);
}
private AggregationOptions computeOptions(MongoQueryMethod method, ConvertingParameterAccessor accessor) {

View File

@@ -47,8 +47,6 @@ public class StringBasedMongoQuery extends AbstractMongoQuery {
private final String query;
private final String fieldSpec;
private final ExpressionParser expressionParser;
private final boolean isCountQuery;
private final boolean isExistsQuery;
private final boolean isDeleteQuery;
@@ -85,7 +83,6 @@ public class StringBasedMongoQuery extends AbstractMongoQuery {
Assert.notNull(expressionParser, "SpelExpressionParser must not be null!");
this.query = query;
this.expressionParser = expressionParser;
this.fieldSpec = method.getFieldSpecification();
if (method.hasAnnotatedQuery()) {
@@ -115,10 +112,8 @@ public class StringBasedMongoQuery extends AbstractMongoQuery {
@Override
protected Query createQuery(ConvertingParameterAccessor accessor) {
ParameterBindingDocumentCodec codec = getParameterBindingCodec();
Document queryObject = codec.decode(this.query, getBindingContext(this.query, accessor, codec));
Document fieldsObject = codec.decode(this.fieldSpec, getBindingContext(this.fieldSpec, accessor, codec));
Document queryObject = decode(this.query, prepareBindingContext(this.query, accessor));
Document fieldsObject = decode(this.fieldSpec, prepareBindingContext(this.fieldSpec, accessor));
Query query = new BasicQuery(queryObject, fieldsObject).with(accessor.getSort());
@@ -129,16 +124,6 @@ public class StringBasedMongoQuery extends AbstractMongoQuery {
return query;
}
private ParameterBindingContext getBindingContext(String json, ConvertingParameterAccessor accessor,
ParameterBindingDocumentCodec codec) {
ExpressionDependencies dependencies = codec.captureExpressionDependencies(json, accessor::getBindableValue,
expressionParser);
SpELExpressionEvaluator evaluator = getSpELExpressionEvaluatorFor(dependencies, accessor);
return new ParameterBindingContext(accessor::getBindableValue, evaluator);
}
/*
* (non-Javadoc)
* @see org.springframework.data.mongodb.repository.query.AbstractMongoQuery#isCountQuery()
@@ -179,8 +164,4 @@ public class StringBasedMongoQuery extends AbstractMongoQuery {
boolean isDeleteQuery) {
return BooleanUtil.countBooleanTrueValues(isCountQuery, isExistsQuery, isDeleteQuery) > 1;
}
private ParameterBindingDocumentCodec getParameterBindingCodec() {
return new ParameterBindingDocumentCodec(getCodecRegistry());
}
}

View File

@@ -199,6 +199,8 @@ public class MongoRepositoryFactory extends RepositoryFactorySupport {
NamedQueries namedQueries) {
MongoQueryMethod queryMethod = new MongoQueryMethod(method, metadata, factory, mappingContext);
queryMethod.verify();
String namedQueryName = queryMethod.getNamedQueryName();
if (namedQueries.hasQuery(namedQueryName)) {

View File

@@ -189,6 +189,8 @@ public class ReactiveMongoRepositoryFactory extends ReactiveRepositoryFactorySup
NamedQueries namedQueries) {
ReactiveMongoQueryMethod queryMethod = new ReactiveMongoQueryMethod(method, metadata, factory, mappingContext);
queryMethod.verify();
String namedQueryName = queryMethod.getNamedQueryName();
if (namedQueries.hasQuery(namedQueryName)) {

View File

@@ -17,6 +17,7 @@ package org.springframework.data.mongodb.repository;
import static java.util.Arrays.*;
import static org.assertj.core.api.Assertions.*;
import static org.assertj.core.api.Assumptions.*;
import static org.springframework.data.geo.Metrics.*;
import java.util.ArrayList;
@@ -40,6 +41,7 @@ import org.junit.jupiter.api.extension.ExtendWith;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.dao.DuplicateKeyException;
import org.springframework.dao.IncorrectResultSizeDataAccessException;
import org.springframework.dao.InvalidDataAccessApiUsageException;
import org.springframework.data.domain.Example;
import org.springframework.data.domain.Page;
import org.springframework.data.domain.PageRequest;
@@ -375,7 +377,7 @@ public abstract class AbstractPersonRepositoryIntegrationTests {
@Test
void rejectsDuplicateEmailAddressOnSave() {
assertThat(dave.getEmail()).isEqualTo("dave@dmband.com");
assumeThat(repository.findById(dave.getId()).map(Person::getEmail)).contains("dave@dmband.com");
Person daveSyer = new Person("Dave", "Syer");
assertThat(daveSyer.getEmail()).isEqualTo("dave@dmband.com");
@@ -1440,7 +1442,8 @@ public abstract class AbstractPersonRepositoryIntegrationTests {
@Test // GH-3633
void annotatedQueryWithNullEqualityCheckShouldWork() {
operations.updateFirst(Query.query(Criteria.where("id").is(dave.getId())), Update.update("age", null), Person.class);
operations.updateFirst(Query.query(Criteria.where("id").is(dave.getId())), Update.update("age", null),
Person.class);
Person byQueryWithNullEqualityCheck = repository.findByQueryWithNullEqualityCheck();
assertThat(byQueryWithNullEqualityCheck.getId()).isEqualTo(dave.getId());
@@ -1461,7 +1464,7 @@ public abstract class AbstractPersonRepositoryIntegrationTests {
assertThat(result).map(Person::getId).containsExactly(josh.getId());
}
@Test //GH-3656
@Test // GH-3656
void resultProjectionWithOptionalIsExcecutedCorrectly() {
carter.setAddress(new Address("batman", "robin", "gotham"));
@@ -1474,35 +1477,58 @@ public abstract class AbstractPersonRepositoryIntegrationTests {
assertThat(result.getFirstname()).contains("Carter");
}
/**
* @see DATAMONGO-1188
*/
@Test
public void shouldSupportFindAndModfiyForQueryDerivationWithCollectionResult() {
List<Person> result = repository.findAndModifyByFirstname("Dave", new Update().inc("visits", 42));
assertThat(result.size()).isOne();
assertThat(result.get(0)).isEqualTo(dave);
Person dave = repository.findById(result.get(0).getId()).get();
assertThat(dave.visits).isEqualTo(42);
@Test // GH-2107
void shouldAllowToUpdateAllElements() {
assertThat(repository.findAndUpdateViaMethodArgAllByLastname("Matthews", new Update().inc("visits", 1337))).isEqualTo(2);
}
/**
* @see DATAMONGO-1188
*/
@Test
public void shouldSupportFindAndModfiyForQueryDerivationWithSingleResult() {
@Test // GH-2107
void annotatedUpdateIsAppliedCorrectly() {
Person result = repository.findOneAndModifyByFirstname("Dave", new Update().inc("visits", 1337));
assertThat(repository.findAndIncrementVisitsByLastname("Matthews", 1337)).isEqualTo(2);
assertThat(result).isEqualTo(dave);
Person dave = repository.findById(result.getId()).get();
assertThat(dave.visits).isEqualTo(1337);
assertThat(repository.findByLastname("Matthews")).extracting(Person::getVisits).allMatch(it -> it.equals(1337));
}
@Test // GH-2107
void mixAnnotatedUpdateWithAnnotatedQuery() {
assertThat(repository.updateAllByLastname("Matthews", 1337)).isEqualTo(2);
assertThat(repository.findByLastname("Matthews")).extracting(Person::getVisits).allMatch(it -> it.equals(1337));
}
@Test // GH-2107
void annotatedUpdateWithSpELIsAppliedCorrectly() {
assertThat(repository.findAndIncrementVisitsUsingSpELByLastname("Matthews", 1337)).isEqualTo(2);
assertThat(repository.findByLastname("Matthews")).extracting(Person::getVisits).allMatch(it -> it.equals(1337));
}
@Test // GH-2107
@EnableIfMongoServerVersion(isGreaterThanEqual = "4.2")
void annotatedAggregationUpdateIsAppliedCorrectly() {
repository.findAndIncrementVisitsViaPipelineByLastname("Matthews", 1337);
assertThat(repository.findByLastname("Matthews")).extracting(Person::getVisits).allMatch(it -> it.equals(1337));
}
@Test // GH-2107
void shouldAllowToUpdateAllElementsWithVoidReturn() {
repository.findAndUpdateViaMethodArgAllByLastname("Matthews", new Update().inc("visits", 1337));
assertThat(repository.findByLastname("Matthews")).extracting(Person::getVisits).allMatch(visits -> visits == 1337);
}
@Test // GH-2107
void allowsToUseComplexTypesInUpdate() {
Address address = new Address("1007 Mountain Drive", "53540", "Gotham");
assertThat(repository.findAndPushShippingAddressByEmail(dave.getEmail(), address)).isEqualTo(1);
assertThat(repository.findById(dave.getId()).map(Person::getShippingAddresses)).contains(Collections.singleton(address));
}
}

View File

@@ -1,5 +1,5 @@
/*
* Copyright 2011-2021 the original author or authors.
* Copyright 2011-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.
@@ -15,6 +15,8 @@
*/
package org.springframework.data.mongodb.repository;
import org.springframework.util.ObjectUtils;
import com.querydsl.core.annotations.QueryEmbeddable;
/**
@@ -83,4 +85,28 @@ public class Address {
public void setCity(String city) {
this.city = city;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Address address = (Address) o;
if (!ObjectUtils.nullSafeEquals(street, address.street)) {
return false;
}
if (!ObjectUtils.nullSafeEquals(zipCode, address.zipCode)) {
return false;
}
return ObjectUtils.nullSafeEquals(city, address.city);
}
@Override
public int hashCode() {
int result = ObjectUtils.nullSafeHashCode(street);
result = 31 * result + ObjectUtils.nullSafeHashCode(zipCode);
result = 31 * result + ObjectUtils.nullSafeHashCode(city);
return result;
}
}

View File

@@ -48,7 +48,7 @@ public class Person extends Contact {
private String firstname;
private String lastname;
@Indexed(unique = true, dropDups = true) private String email;
@Indexed(unique = true) private String email;
private Integer age;
@SuppressWarnings("unused") private Sex sex;
Date createdAt;

View File

@@ -36,7 +36,7 @@ import org.springframework.data.geo.GeoResults;
import org.springframework.data.geo.Point;
import org.springframework.data.geo.Polygon;
import org.springframework.data.mongodb.core.aggregation.AggregationResults;
import org.springframework.data.mongodb.core.query.Update;
import org.springframework.data.mongodb.core.query.UpdateDefinition;
import org.springframework.data.mongodb.repository.Person.Sex;
import org.springframework.data.querydsl.QuerydslPredicateExecutor;
import org.springframework.data.repository.query.Param;
@@ -420,12 +420,27 @@ public interface PersonRepository extends MongoRepository<Person, String>, Query
List<Person> findByUnwrappedUser(User user);
List<Person> findAndModifyByFirstname(String firstname, Update update);
int findAndUpdateViaMethodArgAllByLastname(String lastname, UpdateDefinition update);
Person findOneAndModifyByFirstname(String firstname, Update update);
@Update("{ '$inc' : { 'visits' : ?1 } }")
int findAndIncrementVisitsByLastname(String lastname, int increment);
@Query("{ 'lastname' : ?0 }")
@Update("{ '$inc' : { 'visits' : ?1 } }")
int updateAllByLastname(String lastname, int increment);
@Update( pipeline = {"{ '$set' : { 'visits' : { '$add' : [ '$visits', ?1 ] } } }"})
void findAndIncrementVisitsViaPipelineByLastname(String lastname, int increment);
@Update("{ '$inc' : { 'visits' : ?#{[1]} } }")
int findAndIncrementVisitsUsingSpELByLastname(String lastname, int increment);
@Update("{ '$push' : { 'shippingAddresses' : ?1 } }")
int findAndPushShippingAddressByEmail(String email, Address address);
@Query("{ 'age' : null }")
Person findByQueryWithNullEqualityCheck();
List<Person> findBySpiritAnimal(User user);
}

View File

@@ -23,6 +23,7 @@ import static org.springframework.data.mongodb.test.util.Assertions.assertThat;
import lombok.Data;
import lombok.NoArgsConstructor;
import org.springframework.data.mongodb.test.util.EnableIfMongoServerVersion;
import reactor.core.Disposable;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
@@ -44,6 +45,7 @@ import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.context.annotation.Bean;
import org.springframework.context.annotation.Configuration;
import org.springframework.dao.IncorrectResultSizeDataAccessException;
import org.springframework.dao.InvalidDataAccessApiUsageException;
import org.springframework.data.domain.PageRequest;
import org.springframework.data.domain.Pageable;
import org.springframework.data.domain.Sort;
@@ -58,6 +60,8 @@ import org.springframework.data.mongodb.core.CollectionOptions;
import org.springframework.data.mongodb.core.ReactiveMongoOperations;
import org.springframework.data.mongodb.core.ReactiveMongoTemplate;
import org.springframework.data.mongodb.core.mapping.Document;
import org.springframework.data.mongodb.core.query.Update;
import org.springframework.data.mongodb.core.query.UpdateDefinition;
import org.springframework.data.mongodb.repository.Person.Sex;
import org.springframework.data.mongodb.repository.support.ReactiveMongoRepositoryFactory;
import org.springframework.data.mongodb.repository.support.SimpleReactiveMongoRepository;
@@ -625,6 +629,90 @@ class ReactiveMongoRepositoryTests {
.verifyComplete();
}
@Test // GH-2107
void shouldAllowToUpdateAllElements() {
repository.findAndUpdateViaMethodArgAllByLastname("Matthews", new Update().inc("visits", 1337))
.as(StepVerifier::create)
.expectNext(2L)
.verifyComplete();
}
@Test // GH-2107
void mixAnnotatedUpdateWithAnnotatedQuery() {
repository.updateAllByLastname("Matthews", 1337)
.as(StepVerifier::create)
.expectNext(2L)
.verifyComplete();
repository.findByLastname("Matthews")
.map(Person::getVisits)
.as(StepVerifier::create)
.expectNext(1337, 1337)
.verifyComplete();
}
@Test // GH-2107
void annotatedUpdateWithSpELIsAppliedCorrectly() {
repository.findAndIncrementVisitsUsingSpELByLastname("Matthews", 1337)
.as(StepVerifier::create)
.expectNext(2L)
.verifyComplete();
repository.findByLastname("Matthews")
.map(Person::getVisits)
.as(StepVerifier::create)
.expectNext(1337, 1337)
.verifyComplete();
}
@Test // GH-2107
@EnableIfMongoServerVersion(isGreaterThanEqual = "4.2")
void annotatedAggregationUpdateIsAppliedCorrectly() {
repository.findAndIncrementVisitsViaPipelineByLastname("Matthews", 1337)
.as(StepVerifier::create)
.verifyComplete();
repository.findByLastname("Matthews")
.map(Person::getVisits)
.as(StepVerifier::create)
.expectNext(1337, 1337)
.verifyComplete();
}
@Test // GH-2107
void shouldAllowToUpdateAllElementsWithVoidReturn() {
repository.findAndIncrementVisitsByLastname("Matthews", 1337)
.as(StepVerifier::create)
.expectNext(2L)
.verifyComplete();
repository.findByLastname("Matthews")
.map(Person::getVisits)
.as(StepVerifier::create)
.expectNext(1337, 1337)
.verifyComplete();
}
@Test // GH-2107
void allowsToUseComplexTypesInUpdate() {
Address address = new Address("1007 Mountain Drive", "53540", "Gotham");
repository.findAndPushShippingAddressByEmail(dave.getEmail(), address) //
.as(StepVerifier::create) //
.expectNext(1L) //
.verifyComplete();
repository.findById(dave.getId()).map(Person::getShippingAddresses)
.as(StepVerifier::create)
.consumeNextWith(it -> assertThat(it).containsExactly(address))
.verifyComplete();
}
interface ReactivePersonRepository
extends ReactiveMongoRepository<Person, String>, ReactiveQuerydslPredicateExecutor<Person> {
@@ -701,6 +789,24 @@ class ReactiveMongoRepositoryTests {
Mono<Long> deleteCountByLastname(String lastname);
Mono<Person> deleteSinglePersonByLastname(String lastname);
Mono<Long> findAndUpdateViaMethodArgAllByLastname(String lastname, UpdateDefinition update);
@org.springframework.data.mongodb.repository.Update("{ '$inc' : { 'visits' : ?1 } }")
Mono<Long> findAndIncrementVisitsByLastname(String lastname, int increment);
@Query("{ 'lastname' : ?0 }")
@org.springframework.data.mongodb.repository.Update("{ '$inc' : { 'visits' : ?1 } }")
Mono<Long> updateAllByLastname(String lastname, int increment);
@org.springframework.data.mongodb.repository.Update( pipeline = {"{ '$set' : { 'visits' : { '$add' : [ '$visits', ?1 ] } } }"})
Mono<Void> findAndIncrementVisitsViaPipelineByLastname(String lastname, int increment);
@org.springframework.data.mongodb.repository.Update("{ '$inc' : { 'visits' : ?#{[1]} } }")
Mono<Long> findAndIncrementVisitsUsingSpELByLastname(String lastname, int increment);
@org.springframework.data.mongodb.repository.Update("{ '$push' : { 'shippingAddresses' : ?1 } }")
Mono<Long> findAndPushShippingAddressByEmail(String email, Address address);
}
interface ReactiveContactRepository extends ReactiveMongoRepository<Contact, String> {}

View File

@@ -1,5 +1,5 @@
/*
* Copyright 2014-2021 the original author or authors.
* Copyright 2014-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.
@@ -25,6 +25,7 @@ import java.util.Locale;
import java.util.Optional;
import org.bson.Document;
import org.bson.codecs.configuration.CodecRegistry;
import org.bson.types.ObjectId;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
@@ -35,7 +36,6 @@ import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import org.springframework.data.domain.Page;
import org.springframework.data.domain.PageRequest;
import org.springframework.data.domain.Pageable;
@@ -45,6 +45,10 @@ import org.springframework.data.domain.Sort.Direction;
import org.springframework.data.mongodb.MongoDatabaseFactory;
import org.springframework.data.mongodb.core.ExecutableFindOperation.ExecutableFind;
import org.springframework.data.mongodb.core.ExecutableFindOperation.FindWithQuery;
import org.springframework.data.mongodb.core.ExecutableUpdateOperation.ExecutableUpdate;
import org.springframework.data.mongodb.core.ExecutableUpdateOperation.TerminatingUpdate;
import org.springframework.data.mongodb.core.ExecutableUpdateOperation.UpdateWithQuery;
import org.springframework.data.mongodb.core.ExecutableUpdateOperation.UpdateWithUpdate;
import org.springframework.data.mongodb.core.MongoExceptionTranslator;
import org.springframework.data.mongodb.core.MongoOperations;
import org.springframework.data.mongodb.core.Person;
@@ -56,8 +60,10 @@ import org.springframework.data.mongodb.core.mapping.MongoMappingContext;
import org.springframework.data.mongodb.core.query.BasicQuery;
import org.springframework.data.mongodb.core.query.Collation;
import org.springframework.data.mongodb.core.query.Query;
import org.springframework.data.mongodb.core.query.UpdateDefinition;
import org.springframework.data.mongodb.repository.Meta;
import org.springframework.data.mongodb.repository.MongoRepository;
import org.springframework.data.mongodb.repository.Update;
import org.springframework.data.projection.ProjectionFactory;
import org.springframework.data.projection.SpelAwareProxyProjectionFactory;
import org.springframework.data.repository.Repository;
@@ -65,7 +71,9 @@ import org.springframework.data.repository.core.support.DefaultRepositoryMetadat
import org.springframework.data.repository.query.QueryMethodEvaluationContextProvider;
import org.springframework.expression.spel.standard.SpelExpressionParser;
import com.mongodb.MongoClientSettings;
import com.mongodb.client.result.DeleteResult;
import com.mongodb.client.result.UpdateResult;
/**
* Unit tests for {@link AbstractMongoQuery}.
@@ -82,9 +90,14 @@ class AbstractMongoQueryUnitTests {
@Mock MongoOperations mongoOperationsMock;
@Mock ExecutableFind<?> executableFind;
@Mock FindWithQuery<?> withQueryMock;
@Mock ExecutableUpdate executableUpdate;
@Mock UpdateWithQuery updateWithQuery;
@Mock UpdateWithUpdate updateWithUpdate;
@Mock TerminatingUpdate terminatingUpdate;
@Mock BasicMongoPersistentEntity<?> persitentEntityMock;
@Mock MongoMappingContext mappingContextMock;
@Mock DeleteResult deleteResultMock;
@Mock UpdateResult updateResultMock;
@BeforeEach
void setUp() {
@@ -104,8 +117,12 @@ class AbstractMongoQueryUnitTests {
doReturn(executableFind).when(mongoOperationsMock).query(any());
doReturn(withQueryMock).when(executableFind).as(any());
doReturn(withQueryMock).when(withQueryMock).matching(any(Query.class));
doReturn(executableUpdate).when(mongoOperationsMock).update(any());
doReturn(updateWithQuery).when(executableUpdate).matching(any(Query.class));
doReturn(terminatingUpdate).when(updateWithQuery).apply(any(UpdateDefinition.class));
when(mongoOperationsMock.remove(any(), any(), anyString())).thenReturn(deleteResultMock);
when(mongoOperationsMock.updateMulti(any(), any(), any(), anyString())).thenReturn(updateResultMock);
}
@Test // DATAMONGO-566
@@ -437,6 +454,21 @@ class AbstractMongoQueryUnitTests {
.contains(Collation.of("en_US").toDocument());
}
@Test // GH-2107
void updateExecutionCallsUpdateAllCorrectly() {
when(terminatingUpdate.all()).thenReturn(updateResultMock);
createQueryForMethod("findAndIncreaseVisitsByLastname", String.class, int.class) //
.execute(new Object[] { "dalinar", 100 });
ArgumentCaptor<UpdateDefinition> update = ArgumentCaptor.forClass(UpdateDefinition.class);
verify(updateWithQuery).apply(update.capture());
verify(terminatingUpdate).all();
assertThat(update.getValue().getUpdateObject()).isEqualTo(Document.parse("{ '$inc' : { 'visits' : 100 } }"));
}
private MongoQueryFake createQueryForMethod(String methodName, Class<?>... paramTypes) {
return createQueryForMethod(Repo.class, methodName, paramTypes);
}
@@ -500,6 +532,11 @@ class AbstractMongoQueryUnitTests {
isLimitingQuery = limitingQuery;
return this;
}
@Override
protected CodecRegistry getCodecRegistry() {
return MongoClientSettings.getDefaultCodecRegistry();
}
}
private interface Repo extends MongoRepository<Person, Long> {
@@ -546,6 +583,9 @@ class AbstractMongoQueryUnitTests {
@org.springframework.data.mongodb.repository.Query(collation = "{ 'locale' : 'en_US' }")
List<Person> findWithWithCollationParameterAndAnnotationByFirstName(String firstname, Collation collation);
@Update("{ '$inc' : { 'visits' : ?1 } }")
void findAndIncreaseVisitsByLastname(String lastname, int value);
}
// DATAMONGO-1872

View File

@@ -30,6 +30,8 @@ import org.springframework.data.geo.Point;
import org.springframework.data.mongodb.core.mapping.MongoMappingContext;
import org.springframework.data.mongodb.core.query.Collation;
import org.springframework.data.mongodb.core.query.TextCriteria;
import org.springframework.data.mongodb.core.query.Update;
import org.springframework.data.mongodb.core.query.UpdateDefinition;
import org.springframework.data.mongodb.repository.Person;
import org.springframework.data.projection.ProjectionFactory;
import org.springframework.data.projection.SpelAwareProxyProjectionFactory;
@@ -126,6 +128,31 @@ public class MongoParametersParameterAccessorUnitTests {
assertThat(accessor.getCollation()).isEqualTo(collation);
}
@Test // GH-2107
public void shouldReturnUpdateIfPresent() throws NoSuchMethodException, SecurityException {
Method method = PersonRepository.class.getMethod("findAndModifyByFirstname", String.class, UpdateDefinition.class);
MongoQueryMethod queryMethod = new MongoQueryMethod(method, metadata, factory, context);
Update update = new Update();
MongoParameterAccessor accessor = new MongoParametersParameterAccessor(queryMethod,
new Object[] { "dalinar", update });
assertThat(accessor.getUpdate()).isSameAs(update);
}
@Test // GH-2107
public void shouldReturnNullIfNoUpdatePresent() throws NoSuchMethodException, SecurityException {
Method method = PersonRepository.class.getMethod("findByLocationNear", Point.class);
MongoQueryMethod queryMethod = new MongoQueryMethod(method, metadata, factory, context);
MongoParameterAccessor accessor = new MongoParametersParameterAccessor(queryMethod,
new Object[] { new Point(0,0) });
assertThat(accessor.getUpdate()).isNull();
}
interface PersonRepository extends Repository<Person, Long> {
List<Person> findByLocationNear(Point point);
@@ -138,5 +165,7 @@ public class MongoParametersParameterAccessorUnitTests {
List<Person> findByFirstname(String firstname, Collation collation);
List<Person> findAndModifyByFirstname(String firstname, UpdateDefinition update);
}
}

View File

@@ -25,12 +25,14 @@ import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
import org.springframework.data.domain.Pageable;
import org.springframework.data.domain.Range;
import org.springframework.data.geo.Distance;
import org.springframework.data.geo.GeoResults;
import org.springframework.data.geo.Point;
import org.springframework.data.mongodb.core.query.Collation;
import org.springframework.data.mongodb.core.query.TextCriteria;
import org.springframework.data.mongodb.core.query.UpdateDefinition;
import org.springframework.data.mongodb.repository.Near;
import org.springframework.data.mongodb.repository.Person;
import org.springframework.data.repository.query.Parameter;
@@ -163,6 +165,24 @@ class MongoParametersUnitTests {
assertThat(parameters.getCollationParameterIndex()).isOne();
}
@Test // GH-2107
void shouldReturnIndexUpdateIfExists() throws NoSuchMethodException, SecurityException {
Method method = PersonRepository.class.getMethod("findAndModifyByFirstname", String.class, UpdateDefinition.class, Pageable.class);
MongoParameters parameters = new MongoParameters(method, false);
assertThat(parameters.getUpdateIndex()).isOne();
}
@Test // GH-2107
void shouldReturnInvalidIndexIfUpdateDoesNotExist() throws NoSuchMethodException, SecurityException {
Method method = PersonRepository.class.getMethod("someOtherMethod", Point.class, Point.class);
MongoParameters parameters = new MongoParameters(method, false);
assertThat(parameters.getUpdateIndex()).isEqualTo(-1);
}
interface PersonRepository {
List<Person> findByLocationNear(Point point, Distance distance);
@@ -182,5 +202,7 @@ class MongoParametersUnitTests {
List<Person> findByLocationNear(Point point, Range<Distance> range);
List<Person> findByText(String text, Collation collation);
List<Person> findAndModifyByFirstname(String firstname, UpdateDefinition update, Pageable page);
}
}

View File

@@ -21,9 +21,8 @@ import java.lang.reflect.Method;
import java.util.Collection;
import java.util.List;
import org.assertj.core.api.Assertions;
import org.junit.Before;
import org.junit.Test;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.springframework.data.domain.Pageable;
import org.springframework.data.geo.Distance;
import org.springframework.data.geo.GeoPage;
@@ -31,7 +30,10 @@ import org.springframework.data.geo.GeoResult;
import org.springframework.data.geo.GeoResults;
import org.springframework.data.geo.Point;
import org.springframework.data.mongodb.core.User;
import org.springframework.data.mongodb.core.aggregation.AggregationUpdate;
import org.springframework.data.mongodb.core.mapping.MongoMappingContext;
import org.springframework.data.mongodb.core.query.Update;
import org.springframework.data.mongodb.core.query.UpdateDefinition;
import org.springframework.data.mongodb.repository.Address;
import org.springframework.data.mongodb.repository.Aggregation;
import org.springframework.data.mongodb.repository.Contact;
@@ -53,7 +55,7 @@ public class MongoQueryMethodUnitTests {
MongoMappingContext context;
@Before
@BeforeEach
public void setUp() {
context = new MongoMappingContext();
}
@@ -105,13 +107,13 @@ public class MongoQueryMethodUnitTests {
.isThrownBy(() -> queryMethod(PersonRepository.class, "findByLocationNear", Point.class, Distance.class));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void rejectsNullMappingContext() throws Exception {
Method method = PersonRepository.class.getMethod("findByFirstname", String.class, Point.class);
new MongoQueryMethod(method, new DefaultRepositoryMetadata(PersonRepository.class),
new SpelAwareProxyProjectionFactory(), null);
assertThatExceptionOfType(IllegalArgumentException.class).isThrownBy(() -> new MongoQueryMethod(method,
new DefaultRepositoryMetadata(PersonRepository.class), new SpelAwareProxyProjectionFactory(), null));
}
@Test
@@ -218,8 +220,8 @@ public class MongoQueryMethodUnitTests {
MongoQueryMethod method = queryMethod(PersonRepository.class, "findByAggregation");
Assertions.assertThat(method.hasAnnotatedAggregation()).isTrue();
Assertions.assertThat(method.getAnnotatedAggregation()).hasSize(1);
assertThat(method.hasAnnotatedAggregation()).isTrue();
assertThat(method.getAnnotatedAggregation()).hasSize(1);
}
@Test // DATAMONGO-2153
@@ -227,8 +229,53 @@ public class MongoQueryMethodUnitTests {
MongoQueryMethod method = queryMethod(PersonRepository.class, "findByAggregationWithCollation");
Assertions.assertThat(method.hasAnnotatedCollation()).isTrue();
Assertions.assertThat(method.getAnnotatedCollation()).isEqualTo("de_AT");
assertThat(method.hasAnnotatedCollation()).isTrue();
assertThat(method.getAnnotatedCollation()).isEqualTo("de_AT");
}
@Test // GH-2107
void detectsModifyingQueryByUpdateType() throws Exception {
MongoQueryMethod method = queryMethod(PersonRepository.class, "findAndUpdateBy", String.class, Update.class);
assertThat(method.isModifyingQuery()).isTrue();
}
@Test // GH-2107
void detectsModifyingQueryByUpdateDefinitionType() throws Exception {
MongoQueryMethod method = queryMethod(PersonRepository.class, "findAndUpdateBy", String.class,
UpdateDefinition.class);
assertThat(method.isModifyingQuery()).isTrue();
}
@Test // GH-2107
void detectsModifyingQueryByAggregationUpdateDefinitionType() throws Exception {
MongoQueryMethod method = queryMethod(PersonRepository.class, "findAndUpdateBy", String.class,
AggregationUpdate.class);
assertThat(method.isModifyingQuery()).isTrue();
}
@Test // GH-2107
void queryCreationFailsOnInvalidUpdate() throws Exception {
assertThatExceptionOfType(IllegalStateException.class) //
.isThrownBy(() -> queryMethod(InvalidUpdateMethodRepo.class, "findAndUpdateByLastname", String.class).verify()) //
.withMessageContaining("Update") //
.withMessageContaining("findAndUpdateByLastname");
}
@Test // GH-2107
void queryCreationForUpdateMethodFailsOnInvalidReturnType() throws Exception {
assertThatExceptionOfType(IllegalStateException.class) //
.isThrownBy(() -> queryMethod(InvalidUpdateMethodRepo.class, "findAndIncrementVisitsByFirstname", String.class).verify()) //
.withMessageContaining("Update") //
.withMessageContaining("numeric") //
.withMessageContaining("findAndIncrementVisitsByFirstname");
}
private MongoQueryMethod queryMethod(Class<?> repository, String name, Class<?>... parameters) throws Exception {
@@ -285,6 +332,12 @@ public class MongoQueryMethodUnitTests {
@Aggregation(pipeline = "{'$group': { _id: '$templateId', maxVersion : { $max : '$version'} } }",
collation = "de_AT")
List<User> findByAggregationWithCollation();
void findAndUpdateBy(String firstname, Update update);
void findAndUpdateBy(String firstname, UpdateDefinition update);
void findAndUpdateBy(String firstname, AggregationUpdate update);
}
interface SampleRepository extends Repository<Contact, Long> {
@@ -299,6 +352,15 @@ public class MongoQueryMethodUnitTests {
Customer methodReturningAnInterface();
}
interface InvalidUpdateMethodRepo extends Repository<Person, Long> {
@org.springframework.data.mongodb.repository.Update
void findAndUpdateByLastname(String lastname);
@org.springframework.data.mongodb.repository.Update("{ '$inc' : { 'visits' : 1 } }")
Person findAndIncrementVisitsByFirstname(String firstname);
}
interface Customer {
}

View File

@@ -1,5 +1,5 @@
/*
* Copyright 2016-2021 the original author or authors.
* Copyright 2016-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.
@@ -50,6 +50,7 @@ import org.springframework.data.repository.core.support.DefaultRepositoryMetadat
* Unit test for {@link ReactiveMongoQueryMethod}.
*
* @author Mark Paluch
* @author Christoph Strobl
*/
public class ReactiveMongoQueryMethodUnitTests {
@@ -113,7 +114,7 @@ public class ReactiveMongoQueryMethodUnitTests {
@Test // DATAMONGO-1444
public void rejectsMonoPageableResult() {
assertThatIllegalStateException()
.isThrownBy(() -> queryMethod(PersonRepository.class, "findMonoByLastname", String.class, Pageable.class));
.isThrownBy(() -> queryMethod(PersonRepository.class, "findMonoByLastname", String.class, Pageable.class).verify());
}
@Test // DATAMONGO-1444
@@ -142,13 +143,13 @@ public class ReactiveMongoQueryMethodUnitTests {
@Test // DATAMONGO-1444
public void throwsExceptionOnWrappedPage() {
assertThatExceptionOfType(InvalidDataAccessApiUsageException.class)
.isThrownBy(() -> queryMethod(PersonRepository.class, "findMonoPageByLastname", String.class, Pageable.class));
.isThrownBy(() -> queryMethod(PersonRepository.class, "findMonoPageByLastname", String.class, Pageable.class).verify());
}
@Test // DATAMONGO-1444
public void throwsExceptionOnWrappedSlice() {
assertThatExceptionOfType(InvalidDataAccessApiUsageException.class)
.isThrownBy(() -> queryMethod(PersonRepository.class, "findMonoSliceByLastname", String.class, Pageable.class));
.isThrownBy(() -> queryMethod(PersonRepository.class, "findMonoSliceByLastname", String.class, Pageable.class).verify());
}
@Test // DATAMONGO-1444
@@ -177,6 +178,25 @@ public class ReactiveMongoQueryMethodUnitTests {
Assertions.assertThat(method.getAnnotatedCollation()).isEqualTo("de_AT");
}
@Test // GH-2107
public void queryCreationFailsOnInvalidUpdate() throws Exception {
assertThatExceptionOfType(IllegalStateException.class) //
.isThrownBy(() -> queryMethod(InvalidUpdateMethodRepo.class, "findAndUpdateByLastname", String.class).verify()) //
.withMessageContaining("Update") //
.withMessageContaining("findAndUpdateByLastname");
}
@Test // GH-2107
public void queryCreationForUpdateMethodFailsOnInvalidReturnType() throws Exception {
assertThatExceptionOfType(IllegalStateException.class) //
.isThrownBy(() -> queryMethod(InvalidUpdateMethodRepo.class, "findAndIncrementVisitsByFirstname", String.class).verify()) //
.withMessageContaining("Update") //
.withMessageContaining("numeric") //
.withMessageContaining("findAndIncrementVisitsByFirstname");
}
private ReactiveMongoQueryMethod queryMethod(Class<?> repository, String name, Class<?>... parameters)
throws Exception {
@@ -232,5 +252,14 @@ public class ReactiveMongoQueryMethodUnitTests {
Customer methodReturningAnInterface();
}
interface InvalidUpdateMethodRepo extends Repository<Person, Long> {
@org.springframework.data.mongodb.repository.Update
Mono<Void> findAndUpdateByLastname(String lastname);
@org.springframework.data.mongodb.repository.Update("{ '$inc' : { 'visits' : 1 } }")
Mono<Person> findAndIncrementVisitsByFirstname(String firstname);
}
interface Customer {}
}

View File

@@ -262,13 +262,6 @@ public class StringBasedAggregationUnitTests {
assertThat(result).isInstanceOf(Stream.class);
}
@Test // DATAMONGO-2557
void aggregationRetrievesCodecFromDriverJustOnceForMultipleAggregationOperationsInPipeline() {
executeAggregation("multiOperationPipeline", "firstname");
verify(operations).execute(any());
}
@Test // DATAMONGO-2506
void aggregateRaisesErrorOnInvalidReturnType() {

View File

@@ -28,7 +28,7 @@ import org.springframework.data.geo.Point;
import org.springframework.data.mongodb.core.convert.MongoWriter;
import org.springframework.data.mongodb.core.query.Collation;
import org.springframework.data.mongodb.core.query.TextCriteria;
import org.springframework.data.mongodb.core.query.Update;
import org.springframework.data.mongodb.core.query.UpdateDefinition;
import org.springframework.data.repository.query.ParameterAccessor;
import org.springframework.lang.Nullable;
@@ -175,7 +175,7 @@ class StubParameterAccessor implements MongoParameterAccessor {
}
@Override
public Update getUpdate() {
public UpdateDefinition getUpdate() {
return null;
}
}

View File

@@ -1,6 +1,11 @@
[[new-features]]
= New & Noteworthy
[[new-features.3.4]]
== What's New in Spring Data MongoDB 3.4
* Find and update ``Document``s via <<mongodb.repositories.queries.update,Repository method>>.
[[new-features.3.3]]
== What's New in Spring Data MongoDB 3.3

View File

@@ -289,6 +289,56 @@ lower / upper bounds (`$gt` / `$gte` & `$lt` / `$lte`) according to `Range`
NOTE: If the property criterion compares a document, the order of the fields and exact equality in the document matters.
[[mongodb.repositories.queries.update]]
=== Repository Update Methods
The keywords in the preceding table can also be used to create queries that identify matching documents for running updates on them.
The actual update action is defined via the `@Update` annotation on the method itself as shown in the snippet below. +
Please note that the naming schema for derived queries starts with `find`.
Using _update_ (as in `updateAllByLastname(...)`) is only allowed in combination with `@Query`.
The update is applied to *all* matching documents and it is *not* possible to limit the scope by passing in a `Page` nor using any of the <<repositories.limit-query-result,limiting keywords>>. +
The return type can be either `void` or a _numeric_ type, such as `long` which holds the number of modified documents.
.Update Methods
====
[source,java]
----
public interface PersonRepository extends CrudRepository<Person, String> {
@Update("{ '$inc' : { 'visits' : 1 } }")
long findAndIncrementVisitsByLastname(String lastname); <1>
@Update("{ '$inc' : { 'visits' : ?1 } }")
void findAndIncrementVisitsByLastname(String lastname, int increment); <2>
@Update("{ '$inc' : { 'visits' : ?#{[1]} } }")
long findAndIncrementVisitsUsingSpELByLastname(String lastname, int increment); <3>
@Update(pipeline = {"{ '$set' : { 'visits' : { '$add' : [ '$visits', ?1 ] } } }"})
void findAndIncrementVisitsViaPipelineByLastname(String lastname, int increment); <4>
@Update("{ '$push' : { 'shippingAddresses' : ?1 } }")
long findAndPushShippingAddressByEmail(String email, Address address); <5>
@Query("{ 'lastname' : ?0 }")
@Update("{ '$inc' : { 'visits' : ?1 } }")
void updateAllByLastname(String lastname, int increment); <6>
}
----
<1> The filter query for the update is derived from the method name. The update is as is and does not bind any parameters.
<2> The actual increment value is defined by the _increment_ method argument that is bound to the `?1` placeholder.
<3> It is possible to use SpEL for parameter binding.
<4> Use the `pipeline` attribute to issue <<mongo-template.aggregation-update,aggregation pipeline updates>>.
<5> The update may contain complex objects.
<6> Combine a <<mongodb.repositories.queries.json-based,string based query>> with an update.
====
[WARNING]
====
Repository updates do not emit persistence nor mapping lifecycle events.
====
[[mongodb.repositories.queries.delete]]
=== Repository Delete Queries