Add support for DML with versioned entities

This commit is contained in:
Christian Beikov 2021-03-23 16:07:26 +01:00
parent af891c0997
commit 46a16c605a
15 changed files with 298 additions and 118 deletions

View File

@ -136,6 +136,7 @@ ARROW : '->';
ID : [iI][dD];
VERSION : [vV] [eE] [rR] [sS] [iI] [oO] [nN];
VERSIONED : [vV] [eE] [rR] [sS] [iI] [oO] [nN] [eE] [dD];
NATURALID : [nN] [aA] [tT] [uU] [rR] [aA] [lL] [iI] [dD];
ABS : [aA] [bB] [sS];

View File

@ -44,7 +44,7 @@ deleteStatement
;
updateStatement
: UPDATE dmlTarget setClause whereClause?
: UPDATE VERSIONED? dmlTarget setClause whereClause?
;
setClause
@ -1224,6 +1224,7 @@ identifier
| UPPER
| VALUE
| VERSION
| VERSIONED
| WEEK
| WHERE
| WITH

View File

@ -11,6 +11,10 @@ import java.sql.SQLException;
import org.hibernate.engine.spi.QueryParameters;
import org.hibernate.engine.spi.SharedSessionContractImplementor;
import org.hibernate.metamodel.mapping.JdbcMapping;
import org.hibernate.sql.exec.internal.JdbcParameterImpl;
import org.hibernate.sql.exec.spi.ExecutionContext;
import org.hibernate.sql.exec.spi.JdbcParameterBindings;
import org.hibernate.type.Type;
import org.hibernate.type.VersionType;
@ -19,7 +23,7 @@ import org.hibernate.type.VersionType;
*
* @author Steve Ebersole
*/
public class VersionTypeSeedParameterSpecification implements ParameterSpecification {
public class VersionTypeSeedParameterSpecification extends JdbcParameterImpl {
private final VersionType type;
/**
@ -28,31 +32,16 @@ public class VersionTypeSeedParameterSpecification implements ParameterSpecifica
* @param type The version type.
*/
public VersionTypeSeedParameterSpecification(VersionType type) {
super( (JdbcMapping) type );
this.type = type;
}
@Override
public int bind(
public void bindParameterValue(
PreparedStatement statement,
QueryParameters qp,
SharedSessionContractImplementor session,
int position) throws SQLException {
type.nullSafeSet( statement, type.seed( session ), position, session );
return 1;
}
@Override
public Type getExpectedType() {
return type;
}
@Override
public void setExpectedType(Type expectedType) {
// expected type is intrinsic here...
}
@Override
public String renderDisplayInfo() {
return "version-seed, type=" + type;
int startPosition,
JdbcParameterBindings jdbcParamBindings,
ExecutionContext executionContext) throws SQLException {
type.nullSafeSet( statement, type.seed( executionContext.getSession() ), startPosition, executionContext.getSession() );
}
}

View File

@ -12,4 +12,10 @@ import javax.persistence.criteria.CriteriaUpdate;
* @author Steve Ebersole
*/
public interface JpaCriteriaUpdate<T> extends JpaManipulationCriteria<T>, CriteriaUpdate<T> {
boolean isVersioned();
JpaCriteriaUpdate<T> versioned();
JpaCriteriaUpdate<T> versioned(boolean versioned);
}

View File

@ -93,6 +93,7 @@ import org.hibernate.query.sqm.tree.delete.SqmDeleteStatement;
import org.hibernate.query.sqm.tree.domain.AbstractSqmFrom;
import org.hibernate.query.sqm.tree.domain.SqmCorrelation;
import org.hibernate.query.sqm.tree.domain.SqmIndexedCollectionAccessPath;
import org.hibernate.query.sqm.tree.domain.SqmListJoin;
import org.hibernate.query.sqm.tree.domain.SqmMapEntryReference;
import org.hibernate.query.sqm.tree.domain.SqmMapJoin;
import org.hibernate.query.sqm.tree.domain.SqmMaxElementPath;
@ -427,6 +428,7 @@ public class SemanticQueryBuilder<R> extends HqlParserBaseVisitor<Object> implem
processingState.getPathRegistry().register( root );
try {
updateStatement.versioned( ctx.VERSIONED() != null );
for ( HqlParser.AssignmentContext assignmentContext : ctx.setClause().assignment() ) {
updateStatement.applyAssignment(
consumeDomainPath( assignmentContext.dotIdentifierSequence() ),
@ -3105,7 +3107,7 @@ public class SemanticQueryBuilder<R> extends HqlParserBaseVisitor<Object> implem
return getFunctionDescriptor("sqrt").generateSqmExpression(
arg,
(AllowableFunctionReturnType<?>) arg.getNodeType(),
null,
creationContext.getQueryEngine(),
creationContext.getJpaMetamodel().getTypeConfiguration()
);
@ -3567,7 +3569,7 @@ public class SemanticQueryBuilder<R> extends HqlParserBaseVisitor<Object> implem
return getFunctionDescriptor("sum").generateSqmExpression(
argument,
(AllowableFunctionReturnType<?>) arg.getNodeType(),
null,
creationContext.getQueryEngine(),
creationContext.getJpaMetamodel().getTypeConfiguration()
);
@ -3985,44 +3987,74 @@ public class SemanticQueryBuilder<R> extends HqlParserBaseVisitor<Object> implem
return visitMapKeyNavigablePath( ctx.mapKeyNavigablePath() );
}
else if ( ctx.dotIdentifierSequence() != null && ctx.indexedPathAccessFragment() != null ) {
final SqmAttributeJoin indexedJoinPath = (SqmAttributeJoin) ctx.dotIdentifierSequence().accept( this );
//noinspection unchecked
return new SqmIndexedCollectionAccessPath(
indexedJoinPath,
(SqmExpression) ctx.indexedPathAccessFragment().accept( this )
dotIdentifierConsumerStack.push(
new QualifiedJoinPathConsumer(
(SqmRoot<?>) dotIdentifierConsumerStack.getCurrent().getConsumedPart(),
SqmJoinType.INNER,
false,
null,
this
)
);
final SqmAttributeJoin<?, ?> indexedJoinPath;
try {
indexedJoinPath = (SqmAttributeJoin<?, ?>) ctx.dotIdentifierSequence().accept( this );
}
finally {
dotIdentifierConsumerStack.pop();
}
dotIdentifierConsumerStack.push(
new BasicDotIdentifierConsumer( indexedJoinPath, this ) {
@Override
protected void reset() {
}
}
);
try {
return (SemanticPathPart) ctx.indexedPathAccessFragment().accept( this );
}
finally {
dotIdentifierConsumerStack.pop();
}
}
throw new ParsingException( "Unsure how to process `syntacticDomainPath` over : " + ctx.getText() );
}
// @Override
// public SemanticPathPart visitDotIdentifierSequence(HqlParser.DotIdentifierSequenceContext ctx) {
// final int numberOfContinuations = ctx.dotIdentifierSequenceContinuation().size();
// final boolean hasContinuations = numberOfContinuations != 0;
//
// final SemanticPathPart currentPathPart = semanticPathPartStack.getCurrent();
//
// SemanticPathPart result = currentPathPart.resolvePathPart(
// ctx.identifier().getText(),
// !hasContinuations,
// this
// );
//
// if ( hasContinuations ) {
// int i = 1;
// for ( HqlParser.DotIdentifierSequenceContinuationContext continuation : ctx.dotIdentifierSequenceContinuation() ) {
// result = result.resolvePathPart(
// continuation.identifier().getText(),
// i++ >= numberOfContinuations,
// this
// );
// }
// }
//
// return result;
// }
@Override
public SemanticPathPart visitIndexedPathAccessFragment(HqlParser.IndexedPathAccessFragmentContext ctx) {
final DotIdentifierConsumer consumer = dotIdentifierConsumerStack.pop();
final SqmExpression<?> indexExpression = (SqmExpression<?>) ctx.expression().accept( this );
final SqmAttributeJoin<?, ?> attributeJoin = (SqmAttributeJoin<?, ?>) consumer.getConsumedPart();
final SqmExpression<?> index;
if ( attributeJoin instanceof SqmListJoin<?, ?> ) {
index = ( (SqmListJoin<?, ?>) attributeJoin ).index();
}
else if ( attributeJoin instanceof SqmMapJoin<?, ?, ?> ) {
index = ( (SqmMapJoin<?, ?, ?>) attributeJoin ).key();
}
else {
throw new SemanticException( "Index access is only supported on list or map attributes: " + attributeJoin.getNavigablePath() );
}
attributeJoin.setJoinPredicate( creationContext.getNodeBuilder().equal( index, indexExpression ) );
final SqmIndexedCollectionAccessPath<?> path = new SqmIndexedCollectionAccessPath<>(
attributeJoin,
indexExpression
);
dotIdentifierConsumerStack.push(
new BasicDotIdentifierConsumer( path, this ) {
@Override
protected void reset() {
}
}
);
HqlParser.GeneralPathFragmentContext generalPathFragmentContext = ctx.generalPathFragment();
if ( generalPathFragmentContext == null ) {
return path;
}
return (SemanticPathPart) generalPathFragmentContext.accept( this );
}
@Override
public SemanticPathPart visitDotIdentifierSequence(HqlParser.DotIdentifierSequenceContext ctx) {

View File

@ -352,7 +352,7 @@ public class SqmTreePrinter implements SemanticQueryWalker<Object> {
public Object visitUpdateStatement(SqmUpdateStatement<?> statement) {
if ( DEBUG_ENABLED ) {
processStanza(
"update",
statement.isVersioned() ? "update versioned" : "update",
() -> {
logWithIndentation( "[target = %s]", statement.getTarget().getNavigablePath().getFullPath() );

View File

@ -81,6 +81,7 @@ public class CteUpdateHandler extends AbstractCteMutationHandler implements Upda
assignments::add,
parameterResolutions::put
);
sqmConverter.addVersionedAssignment( assignments::add, updateStatement );
// ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
// cross-reference the TableReference by alias. The TableGroup already

View File

@ -156,6 +156,7 @@ public class TableBasedUpdateHandler
k -> new ArrayList<>( 1 )
).add( jdbcParameters )
);
converterDelegate.addVersionedAssignment( assignments::add, getSqmDeleteOrUpdateStatement() );
// ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
// visit the where-clause using our special converter, collecting information

View File

@ -7,6 +7,7 @@
package org.hibernate.query.sqm.sql;
import java.util.ArrayList;
import java.util.Calendar;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
@ -50,6 +51,7 @@ import org.hibernate.metamodel.mapping.EmbeddableValuedModelPart;
import org.hibernate.metamodel.mapping.EntityAssociationMapping;
import org.hibernate.metamodel.mapping.EntityIdentifierMapping;
import org.hibernate.metamodel.mapping.EntityMappingType;
import org.hibernate.metamodel.mapping.EntityVersionMapping;
import org.hibernate.metamodel.mapping.ForeignKeyDescriptor;
import org.hibernate.metamodel.mapping.MappingModelExpressable;
import org.hibernate.metamodel.mapping.ModelPart;
@ -64,6 +66,7 @@ import org.hibernate.metamodel.model.domain.AllowableParameterType;
import org.hibernate.metamodel.model.domain.EntityDomainType;
import org.hibernate.metamodel.model.domain.PluralPersistentAttribute;
import org.hibernate.metamodel.model.domain.internal.CompositeSqmPathSource;
import org.hibernate.param.VersionTypeSeedParameterSpecification;
import org.hibernate.persister.collection.CollectionPersister;
import org.hibernate.persister.entity.EntityPersister;
import org.hibernate.persister.entity.Joinable;
@ -173,12 +176,14 @@ import org.hibernate.query.sqm.tree.predicate.SqmWhereClause;
import org.hibernate.query.sqm.tree.select.SqmDynamicInstantiation;
import org.hibernate.query.sqm.tree.select.SqmDynamicInstantiationArgument;
import org.hibernate.query.sqm.tree.select.SqmDynamicInstantiationTarget;
import org.hibernate.query.sqm.tree.select.SqmJpaCompoundSelection;
import org.hibernate.query.sqm.tree.select.SqmOrderByClause;
import org.hibernate.query.sqm.tree.select.SqmQueryGroup;
import org.hibernate.query.sqm.tree.select.SqmQueryPart;
import org.hibernate.query.sqm.tree.select.SqmQuerySpec;
import org.hibernate.query.sqm.tree.select.SqmSelectClause;
import org.hibernate.query.sqm.tree.select.SqmSelectStatement;
import org.hibernate.query.sqm.tree.select.SqmSelectableNode;
import org.hibernate.query.sqm.tree.select.SqmSelection;
import org.hibernate.query.sqm.tree.select.SqmSortSpecification;
import org.hibernate.query.sqm.tree.select.SqmSubQuery;
@ -275,13 +280,16 @@ import org.hibernate.sql.results.internal.SqlSelectionImpl;
import org.hibernate.sql.results.internal.StandardEntityGraphTraversalStateImpl;
import org.hibernate.type.BasicType;
import org.hibernate.type.StandardBasicTypes;
import org.hibernate.type.VersionType;
import org.hibernate.type.descriptor.java.JavaTypeDescriptor;
import org.hibernate.type.descriptor.jdbc.JdbcTypeDescriptorIndicators;
import org.hibernate.type.spi.TypeConfiguration;
import org.hibernate.usertype.UserVersionType;
import org.jboss.logging.Logger;
import static org.hibernate.internal.util.NullnessHelper.coalesceSuppliedValues;
import static org.hibernate.query.BinaryArithmeticOperator.ADD;
import static org.hibernate.query.BinaryArithmeticOperator.MULTIPLY;
import static org.hibernate.query.BinaryArithmeticOperator.SUBTRACT;
import static org.hibernate.query.TemporalUnit.DAY;
@ -572,6 +580,7 @@ public abstract class BaseSqmToSqlAstConverter<T extends Statement> extends Base
);
final List<Assignment> assignments = visitSetClause( sqmStatement.getSetClause() );
addVersionedAssignment( assignments::add, sqmStatement );
final FilterPredicate filterPredicate = FilterHelper.createFilterPredicate(
getLoadQueryInfluencers(),
@ -625,6 +634,57 @@ public abstract class BaseSqmToSqlAstConverter<T extends Statement> extends Base
}
}
public void addVersionedAssignment(Consumer<Assignment> assignmentConsumer, SqmUpdateStatement<?> sqmStatement) {
if ( !sqmStatement.isVersioned() ) {
return;
}
final EntityPersister persister = creationContext.getDomainModel()
.findEntityDescriptor( sqmStatement.getTarget().getEntityName() );
if ( !persister.isVersioned() ) {
throw new SemanticException( "increment option specified for update of non-versioned entity" );
}
final VersionType<?> versionType = persister.getVersionType();
if ( versionType instanceof UserVersionType ) {
throw new SemanticException( "user-defined version types not supported for increment option" );
}
final EntityVersionMapping versionMapping = persister.getVersionMapping();
final List<ColumnReference> targetColumnReferences = BasicValuedPathInterpretation.from(
(SqmBasicValuedSimplePath<?>) sqmStatement
.getRoot()
.get( versionMapping.getPartName() ),
this,
this
).getColumnReferences();
assert targetColumnReferences.size() == 1;
final ColumnReference versionColumn = targetColumnReferences.get( 0 );
final Expression value;
if ( isTimestampBasedVersion( versionType ) ) {
value = new VersionTypeSeedParameterSpecification( versionType );
}
else {
final BasicValuedMapping basicValuedMapping = (BasicValuedMapping) versionType;
value = new BinaryArithmeticExpression(
versionColumn,
ADD,
new QueryLiteral<>( 1, basicValuedMapping ),
basicValuedMapping
);
}
assignmentConsumer.accept( new Assignment( versionColumn, value ) );
}
private boolean isTimestampBasedVersion(VersionType<?> versionType) {
if ( versionType instanceof BasicType<?> ) {
return ( (BasicType<?>) versionType ).getJdbcTypeDescriptor().isTemporal();
}
final Class<?> javaType = versionType.getReturnedClass();
return java.util.Date.class.isAssignableFrom( javaType )
|| Calendar.class.isAssignableFrom( javaType );
}
@Override
public List<Assignment> visitSetClause(SqmSetClause setClause) {
final List<Assignment> assignments = new ArrayList<>( setClause.getAssignments().size() );
@ -830,6 +890,7 @@ public abstract class BaseSqmToSqlAstConverter<T extends Statement> extends Base
);
currentClauseStack.push( Clause.INSERT );
final InsertStatement insertStatement;
boolean needsVersionInsert = entityDescriptor.isVersioned();
try {
final NavigablePath rootPath = sqmStatement.getTarget().getNavigablePath();
@ -857,10 +918,34 @@ public abstract class BaseSqmToSqlAstConverter<T extends Statement> extends Base
Collections.emptyList()
);
List<SqmPath> targetPaths = sqmStatement.getInsertionTargetPaths();
for ( SqmPath<?> target : targetPaths ) {
Assignable assignable = (Assignable) target.accept( this );
insertStatement.addTargetColumnReferences( assignable.getColumnReferences() );
final List<SqmPath> targetPaths = sqmStatement.getInsertionTargetPaths();
if ( needsVersionInsert ) {
final String versionAttributeName = entityDescriptor.getVersionMapping().getVersionAttribute().getAttributeName();
for ( int i = 0; i < targetPaths.size(); i++ ) {
final SqmPath<?> path = targetPaths.get( i );
if ( versionAttributeName.equals( path.getNavigablePath().getLocalName() ) ) {
needsVersionInsert = false;
}
final Assignable assignable = (Assignable) path.accept( this );
insertStatement.addTargetColumnReferences( assignable.getColumnReferences() );
}
if ( needsVersionInsert ) {
final List<ColumnReference> targetColumnReferences = BasicValuedPathInterpretation.from(
(SqmBasicValuedSimplePath<?>) sqmStatement.getTarget()
.get( versionAttributeName ),
this,
this
).getColumnReferences();
assert targetColumnReferences.size() == 1;
insertStatement.addTargetColumnReferences( targetColumnReferences );
}
}
else {
for ( int i = 0; i < targetPaths.size(); i++ ) {
final Assignable assignable = (Assignable) targetPaths.get( i ).accept( this );
insertStatement.addTargetColumnReferences( assignable.getColumnReferences() );
}
}
}
finally {
@ -872,6 +957,19 @@ public abstract class BaseSqmToSqlAstConverter<T extends Statement> extends Base
visitQueryPart( selectQueryPart )
);
if ( needsVersionInsert ) {
final VersionType versionType = entityDescriptor.getVersionType();
final Expression expression = new VersionTypeSeedParameterSpecification( versionType );
insertStatement.getSourceSelectStatement().forEachQuerySpec(
querySpec -> {
querySpec.getSelectClause().addSqlSelection(
// The position is irrelevant as this is only needed for insert
new SqlSelectionImpl( 1, 0, expression )
);
}
);
}
return insertStatement;
}
@ -1360,9 +1458,26 @@ public abstract class BaseSqmToSqlAstConverter<T extends Statement> extends Base
}
@Override
public Void visitSelection(SqmSelection sqmSelection) {
public Object visitSelection(SqmSelection<?> sqmSelection) {
currentSqlSelectionCollector().next();
final DomainResultProducer resultProducer = resolveDomainResultProducer( sqmSelection );
final Map<String, DomainResultProducer<?>> resultProducers;
if ( sqmSelection.getSelectableNode() instanceof SqmJpaCompoundSelection<?> ) {
SqmJpaCompoundSelection<?> selectableNode = (SqmJpaCompoundSelection<?>) sqmSelection.getSelectableNode();
resultProducers = new HashMap<>( selectableNode.getSelectionItems().size() );
for ( SqmSelectableNode<?> selectionItem : selectableNode.getSelectionItems() ) {
resultProducers.put(
selectionItem.getAlias(),
(DomainResultProducer<?>) selectionItem.accept( this )
);
}
}
else {
resultProducers = Collections.singletonMap(
sqmSelection.getAlias(),
(DomainResultProducer<?>) sqmSelection.getSelectableNode().accept( this )
);
}
if ( domainResults != null ) {
final Stack<SqlAstProcessingState> processingStateStack = getProcessingStateStack();
@ -1394,24 +1509,15 @@ public abstract class BaseSqmToSqlAstConverter<T extends Statement> extends Base
) == null;
}
if ( collectDomainResults ) {
final DomainResult domainResult = resultProducer.createDomainResult(
sqmSelection.getAlias(),
this
);
domainResults.add( domainResult );
resultProducers.forEach( (alias, r) -> domainResults.add( r.createDomainResult( alias, this ) ) );
}
else {
resultProducer.applySqlSelections( this );
resultProducers.forEach( (alias, r) -> r.applySqlSelections( this ) );
}
}
return null;
}
private DomainResultProducer resolveDomainResultProducer(SqmSelection sqmSelection) {
return (DomainResultProducer) sqmSelection.getSelectableNode().accept( this );
}
protected Expression resolveGroupOrOrderByExpression(SqmExpression<?> groupByClauseExpression) {
if ( groupByClauseExpression instanceof SqmLiteral<?> ) {
Object literal = ( (SqmLiteral<?>) groupByClauseExpression ).getLiteralValue();
@ -3539,7 +3645,7 @@ public abstract class BaseSqmToSqlAstConverter<T extends Statement> extends Base
}
@Override
public InSubQueryPredicate visitMemberOfPredicate(SqmMemberOfPredicate predicate) {
public Predicate visitMemberOfPredicate(SqmMemberOfPredicate predicate) {
final SqmPath<?> pluralPath = predicate.getPluralPath();
final PluralAttributeMapping mappingModelExpressable = (PluralAttributeMapping) determineValueMapping(
pluralPath );
@ -3564,63 +3670,58 @@ public abstract class BaseSqmToSqlAstConverter<T extends Statement> extends Base
inferrableTypeAccessStack.pop();
}
return new InSubQueryPredicate(
lhs,
createMemberOfSubQuery( pluralPath, mappingModelExpressable ),
predicate.isNegated()
);
}
private QueryPart createMemberOfSubQuery(SqmPath<?> pluralPath, PluralAttributeMapping mappingModelExpressable) {
final FromClauseAccess parentFromClauseAccess = getFromClauseAccess();
final QuerySpec querySpec = new QuerySpec( false );
final QuerySpec subQuerySpec = new QuerySpec( false );
pushProcessingState(
new SqlAstQueryPartProcessingStateImpl(
querySpec,
subQuerySpec,
getCurrentProcessingState(),
this,
currentClauseStack::getCurrent
)
);
try {
final TableGroup rootTableGroup = mappingModelExpressable.createRootTableGroup(
final TableGroup tableGroup = mappingModelExpressable.createRootTableGroup(
pluralPath.getNavigablePath(),
null,
true,
LockOptions.NONE.getLockMode(),
() -> querySpec::applyPredicate,
() -> subQuerySpec::applyPredicate,
this,
creationContext
);
getFromClauseAccess().registerTableGroup( pluralPath.getNavigablePath(), rootTableGroup );
querySpec.getFromClause().addRoot( rootTableGroup );
getFromClauseAccess().registerTableGroup( pluralPath.getNavigablePath(), tableGroup );
subQuerySpec.getFromClause().addRoot( tableGroup );
final CollectionPart elementDescriptor = mappingModelExpressable.getElementDescriptor();
elementDescriptor.createDomainResult(
pluralPath.getNavigablePath(),
rootTableGroup,
tableGroup,
null,
this
);
final Predicate predicate = mappingModelExpressable.getKeyDescriptor().generateJoinPredicate(
parentFromClauseAccess.findTableGroup( pluralPath.getNavigablePath().getParent() ),
rootTableGroup,
null,
getSqlExpressionResolver(),
creationContext
subQuerySpec.applyPredicate(
mappingModelExpressable.getKeyDescriptor().generateJoinPredicate(
parentFromClauseAccess.findTableGroup( pluralPath.getNavigablePath().getParent() ),
tableGroup,
SqlAstJoinType.INNER,
getSqlExpressionResolver(),
creationContext
)
);
querySpec.applyPredicate( predicate );
}
finally {
popProcessingStateStack();
}
return querySpec;
return new InSubQueryPredicate(
lhs,
subQuerySpec,
predicate.isNegated()
);
}
@Override
@ -3675,35 +3776,34 @@ public abstract class BaseSqmToSqlAstConverter<T extends Statement> extends Base
assert parentNavPath != null;
final TableGroup parentTableGroup = parentFromClauseAccess.getTableGroup( parentNavPath );
final SqlAliasBase sqlAliasBase = sqlAliasBaseManager.createSqlAliasBase( parentTableGroup.getGroupAlias() );
final TableGroup tableGroup = new CorrelatedTableGroup(
parentTableGroup,
sqlAliasBase,
subQuerySpec,
subQuerySpec::applyPredicate,
creationContext.getSessionFactory()
);
subQueryState.getSqlAstCreationState().getFromClauseAccess().registerTableGroup(
parentNavPath,
parentTableGroup
tableGroup
);
final SqmPathInterpretation<?> sqmPathInterpretation = visitPluralValuedPath( sqmPluralPath );
final PluralAttributeMapping pluralAttributeMapping = (PluralAttributeMapping) sqmPathInterpretation.getExpressionType();
// note : do not add to `parentTableGroup` as a join
final TableGroupJoin tableGroupJoin = pluralAttributeMapping.createTableGroupJoin(
// The creation of the table group join against the correlated table group
// has the side effect that the from and where clause of the sub-query are set
pluralAttributeMapping.createTableGroupJoin(
pluralPathNavPath,
parentTableGroup,
tableGroup,
sqmPluralPath.getExplicitAlias(),
SqlAstJoinType.LEFT,
SqlAstJoinType.INNER,
LockMode.NONE,
sqlAliasBaseManager,
subQueryState,
creationContext
);
final TableGroup collectionTableGroup = tableGroupJoin.getJoinedGroup();
subQuerySpec.getFromClause().addRoot( collectionTableGroup );
subQuerySpec.applyPredicate( tableGroupJoin.getPredicate() );
final ForeignKeyDescriptor collectionKeyDescriptor = pluralAttributeMapping.getKeyDescriptor();
final int jdbcTypeCount = collectionKeyDescriptor.getJdbcTypeCount();
assert jdbcTypeCount > 0;
@ -3713,7 +3813,11 @@ public abstract class BaseSqmToSqlAstConverter<T extends Statement> extends Base
new SqlSelectionImpl( 1, 0, jdbcLiteral )
);
return new ExistsPredicate( subQuerySpec );
final ExistsPredicate existsPredicate = new ExistsPredicate( subQuerySpec );
if ( predicate.isNegated() ) {
return existsPredicate;
}
return new NegatedPredicate( existsPredicate );
}
finally {
popProcessingStateStack();

View File

@ -35,6 +35,7 @@ import org.hibernate.query.sqm.tree.predicate.SqmWhereClause;
public class SqmUpdateStatement<T>
extends AbstractSqmDmlStatement<T>
implements SqmDeleteOrUpdateStatement<T>, JpaCriteriaUpdate<T> {
private boolean versioned;
private SqmSetClause setClause;
private SqmWhereClause whereClause;
@ -133,6 +134,23 @@ public class SqmUpdateStatement<T>
throw new NotYetImplementedFor6Exception();
}
@Override
public boolean isVersioned() {
return versioned;
}
@Override
public SqmUpdateStatement<T> versioned() {
this.versioned = true;
return this;
}
@Override
public SqmUpdateStatement<T> versioned(boolean versioned) {
this.versioned = versioned;
return this;
}
@Override
public SqmUpdateStatement<T> where(Expression<Boolean> restriction) {
getWhereClause().setPredicate( (SqmPredicate) restriction );

View File

@ -15,7 +15,7 @@ import org.hibernate.sql.ast.tree.select.QueryPart;
*/
public class ExistsPredicate implements Predicate {
private QueryPart expression;
private final QueryPart expression;
public ExistsPredicate(QueryPart expression) {
this.expression = expression;

View File

@ -7,6 +7,7 @@
package org.hibernate.sql.ast.tree.select;
import java.util.List;
import java.util.function.Consumer;
import org.hibernate.query.SetOperator;
import org.hibernate.metamodel.mapping.MappingModelExpressable;
@ -37,6 +38,13 @@ public class QueryGroup extends QueryPart {
return queryParts.get( queryParts.size() - 1 ).getLastQuerySpec();
}
@Override
public void forEachQuerySpec(Consumer<QuerySpec> querySpecConsumer) {
for ( int i = 0; i < queryParts.size(); i++ ) {
queryParts.get( i ).forEachQuerySpec( querySpecConsumer );
}
}
public SetOperator getSetOperator() {
return setOperator;
}

View File

@ -34,6 +34,8 @@ public abstract class QueryPart implements SqlAstNode, Expression, DomainResultP
public abstract QuerySpec getLastQuerySpec();
public abstract void forEachQuerySpec(Consumer<QuerySpec> querySpecConsumer);
/**
* Does this QueryPart map to the statement's root query (as
* opposed to one of its sub-queries)?

View File

@ -8,6 +8,7 @@ package org.hibernate.sql.ast.tree.select;
import java.util.Collections;
import java.util.List;
import java.util.function.Consumer;
import org.hibernate.metamodel.mapping.MappingModelExpressable;
import org.hibernate.query.sqm.sql.internal.DomainResultProducer;
@ -59,6 +60,11 @@ public class QuerySpec extends QueryPart implements SqlAstNode, PredicateContain
return this;
}
@Override
public void forEachQuerySpec(Consumer<QuerySpec> querySpecConsumer) {
querySpecConsumer.accept( this );
}
public FromClause getFromClause() {
return fromClause;
}

View File

@ -137,6 +137,17 @@ public interface JdbcTypeDescriptor extends Serializable {
return false;
}
default boolean isTemporal() {
switch ( getSqlType() ) {
case Types.DATE:
case Types.TIME:
case Types.TIMESTAMP:
case Types.TIMESTAMP_WITH_TIMEZONE:
return true;
}
return false;
}
default CastType getCastType() {
switch ( getJdbcType() ) {
case Types.INTEGER: