vertices); - @Override Q adjacent(Vertex vertex); @@ -156,7 +155,8 @@ public interface JanusGraphMultiVertexQuery> drop(); + } diff --git a/janusgraph-core/src/main/java/org/janusgraph/core/TransactionBuilder.java b/janusgraph-core/src/main/java/org/janusgraph/core/TransactionBuilder.java index 6fb041070fa..8c6426d5d4c 100644 --- a/janusgraph-core/src/main/java/org/janusgraph/core/TransactionBuilder.java +++ b/janusgraph-core/src/main/java/org/janusgraph/core/TransactionBuilder.java @@ -15,6 +15,7 @@ package org.janusgraph.core; import org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration; +import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryDropStepStrategyMode; import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryHasStepStrategyMode; import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryLabelStepStrategyMode; import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryPropertiesStrategyMode; @@ -186,6 +187,15 @@ public interface TransactionBuilder { */ TransactionBuilder setLabelsStepStrategyMode(MultiQueryLabelStepStrategyMode labelStepStrategyMode); + /** + * Sets `drop` step strategy mode. + *+ * Doesn't have any effect if multi-query was disabled via config `query.batch.enabled = false`. + * + * @return Object with the set drop strategy mode settings + */ + TransactionBuilder setDropStepStrategyMode(MultiQueryDropStepStrategyMode dropStepStrategyMode); + /** * Sets the group name for this transaction which provides a way for gathering * reporting on multiple transactions into one group. diff --git a/janusgraph-core/src/main/java/org/janusgraph/graphdb/configuration/GraphDatabaseConfiguration.java b/janusgraph-core/src/main/java/org/janusgraph/graphdb/configuration/GraphDatabaseConfiguration.java index 10bf05bb3d4..dce5267a2b1 100644 --- a/janusgraph-core/src/main/java/org/janusgraph/graphdb/configuration/GraphDatabaseConfiguration.java +++ b/janusgraph-core/src/main/java/org/janusgraph/graphdb/configuration/GraphDatabaseConfiguration.java @@ -58,6 +58,7 @@ import org.janusgraph.graphdb.query.index.BruteForceIndexSelectionStrategy; import org.janusgraph.graphdb.query.index.IndexSelectionStrategy; import org.janusgraph.graphdb.query.index.ThresholdBasedIndexSelectionStrategy; +import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryDropStepStrategyMode; import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryLabelStepStrategyMode; import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryPropertiesStrategyMode; import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryStrategyRepeatStepMode; @@ -386,6 +387,15 @@ public class GraphDatabaseConfiguration { MultiQueryLabelStepStrategyMode.NONE.getConfigName()), ConfigOption.Type.MASKABLE, MultiQueryLabelStepStrategyMode.ALL.getConfigName()); + public static final ConfigOption
DROP_STEP_BATCH_MODE = new ConfigOption<>(QUERY_BATCH_NS,"drop-step-mode", + String.format("Batching mode for `drop()` step. Used only when `"+USE_MULTIQUERY.toStringWithoutRoot()+"` is `true`.
" + + "Supported modes:
" + + "- `%s` - Drops all vertices in a batch.
" + + "- `%s` - Skips drop batching optimization.
", + MultiQueryDropStepStrategyMode.ALL.getConfigName(), + MultiQueryDropStepStrategyMode.NONE.getConfigName()), + ConfigOption.Type.MASKABLE, MultiQueryDropStepStrategyMode.ALL.getConfigName()); + // ################ SCHEMA ####################### // ################################################ @@ -1371,6 +1381,7 @@ public boolean apply(@Nullable String s) { private MultiQueryHasStepStrategyMode hasStepStrategyMode; private MultiQueryPropertiesStrategyMode propertiesStrategyMode; private MultiQueryLabelStepStrategyMode labelStepStrategyMode; + private MultiQueryDropStepStrategyMode dropStepStrategyMode; private StoreFeatures storeFeatures = null; @@ -1497,6 +1508,10 @@ public MultiQueryLabelStepStrategyMode labelStepStrategyMode() { return labelStepStrategyMode; } + public MultiQueryDropStepStrategyMode dropStepStrategyMode() { + return dropStepStrategyMode; + } + public boolean adjustQueryLimit() { return adjustQueryLimit; } @@ -1627,6 +1642,7 @@ private void preLoadConfiguration() { hasStepStrategyMode = selectExactConfig(HAS_STEP_BATCH_MODE, MultiQueryHasStepStrategyMode.values()); propertiesStrategyMode = selectExactConfig(PROPERTIES_BATCH_MODE, MultiQueryPropertiesStrategyMode.values()); labelStepStrategyMode = selectExactConfig(LABEL_STEP_BATCH_MODE, MultiQueryLabelStepStrategyMode.values()); + dropStepStrategyMode = selectExactConfig(DROP_STEP_BATCH_MODE, MultiQueryDropStepStrategyMode.values()); indexSelectionStrategy = Backend.getImplementationClass(configuration, configuration.get(INDEX_SELECT_STRATEGY), REGISTERED_INDEX_SELECTION_STRATEGIES); diff --git a/janusgraph-core/src/main/java/org/janusgraph/graphdb/query/vertex/MultiVertexCentricQueryBuilder.java b/janusgraph-core/src/main/java/org/janusgraph/graphdb/query/vertex/MultiVertexCentricQueryBuilder.java index c461f418529..7e6420a52da 100644 --- a/janusgraph-core/src/main/java/org/janusgraph/graphdb/query/vertex/MultiVertexCentricQueryBuilder.java +++ b/janusgraph-core/src/main/java/org/janusgraph/graphdb/query/vertex/MultiVertexCentricQueryBuilder.java @@ -203,9 +203,9 @@ public MapvertexIds() { } @Override - public Integer drop() { + public Map > drop() { Map > allRelations = this.noPartitionRestriction().all().relations(); allRelations.forEach((vertex, relations) -> ((InternalVertex) vertex).remove(relations)); - return allRelations.size(); + return allRelations; } } diff --git a/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/step/JanusGraphDropStep.java b/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/step/JanusGraphDropStep.java new file mode 100644 index 00000000000..aad2f4c4d51 --- /dev/null +++ b/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/step/JanusGraphDropStep.java @@ -0,0 +1,145 @@ +// Copyright 2024 JanusGraph 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 +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.graphdb.tinkerpop.optimize.step; + +import org.apache.tinkerpop.gremlin.process.traversal.Traverser; +import org.apache.tinkerpop.gremlin.process.traversal.step.Profiling; +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.DropStep; +import org.apache.tinkerpop.gremlin.process.traversal.step.util.event.CallbackRegistry; +import org.apache.tinkerpop.gremlin.process.traversal.step.util.event.Event; +import org.apache.tinkerpop.gremlin.process.traversal.step.util.event.EventCallback; +import org.apache.tinkerpop.gremlin.process.traversal.strategy.decoration.EventStrategy; +import org.apache.tinkerpop.gremlin.process.traversal.util.MutableMetrics; +import org.apache.tinkerpop.gremlin.structure.Element; +import org.apache.tinkerpop.gremlin.structure.Vertex; +import org.janusgraph.core.BaseVertexQuery; +import org.janusgraph.graphdb.query.profile.QueryProfiler; +import org.janusgraph.graphdb.query.vertex.BasicVertexCentricQueryBuilder; +import org.janusgraph.graphdb.tinkerpop.optimize.step.fetcher.DropStepBatchFetcher; +import org.janusgraph.graphdb.tinkerpop.profile.TP3ProfileWrapper; +import org.janusgraph.graphdb.util.CopyStepUtil; +import org.janusgraph.graphdb.util.JanusGraphTraverserUtil; + +import java.util.List; + +/** + * This class extends the default TinkerPop's {@link DropStep} and adds vertices multi-query optimization to this step. + * + * Before this step is evaluated it usually receives multiple future vertices which might be processed next with this step. + * This step stores all these vertices which might be needed later for evaluation and whenever this step receives the + * vertex for evaluation which wasn't evaluated previously it sends multi-query for a batch of vertices to drop them. + *
+ * This step optimizes only drop of Vertices and skips optimization for any other Element. + */ +public class JanusGraphDropStep
extends DropStepimplements Profiling, MultiQueriable{ + + private boolean useMultiQuery = false; + private QueryProfiler queryProfiler = QueryProfiler.NO_OP; + private int batchSize = Integer.MAX_VALUE; + private DropStepBatchFetcher dropStepBatchFetcher; + + public JanusGraphDropStep(DropSteporiginalStep){ + super(originalStep.getTraversal()); + CopyStepUtil.copyAbstractStepModifiableFields(originalStep, this); + + CallbackRegistrycallbackRegistry = getMutatingCallbackRegistry(); + for(EventCallback callback : originalStep.getMutatingCallbackRegistry().getCallbacks()){ + callbackRegistry.addCallback(callback); + } + + if (originalStep instanceof JanusGraphDropStep) { + JanusGraphDropStep originalJanusGraphLabelStep = (JanusGraphDropStep) originalStep; + setBatchSize(originalJanusGraphLabelStep.batchSize); + setUseMultiQuery(originalJanusGraphLabelStep.useMultiQuery); + } + } + + @Override + protected boolean filter(final Traverser.Admin traverser) { + if (useMultiQuery && traverser.get() instanceof Vertex) { + dropStepBatchFetcher.fetchData(getTraversal(), (Vertex) traverser.get(), JanusGraphTraverserUtil.getLoops(traverser)); + return false; + } else { + return super.filter(traverser); + } + } + + @Override + public void setUseMultiQuery(boolean useMultiQuery) { + this.useMultiQuery = useMultiQuery; + if(this.useMultiQuery && dropStepBatchFetcher == null){ + dropStepBatchFetcher = new DropStepBatchFetcher(this::makeLabelsQuery, batchSize, (batchVertices, requiredVertex) -> { + List> callbacksForRemovalEvents = getMutatingCallbackRegistry().getCallbacks(); + if(!callbacksForRemovalEvents.isEmpty()){ + final EventStrategy eventStrategy = traversal.getStrategies().getStrategy(EventStrategy.class).get(); + produceRemovedEvent(eventStrategy, callbacksForRemovalEvents, requiredVertex); + for(Vertex vertexInBatch : batchVertices){ + if(vertexInBatch != requiredVertex){ + produceRemovedEvent(eventStrategy, callbacksForRemovalEvents, vertexInBatch); + } + } + } + }); + } + } + + private static void produceRemovedEvent(EventStrategy eventStrategy, + List > callbacksForRemovalEvents, + Vertex vertex){ + final Event removeEvent = new Event.VertexRemovedEvent(eventStrategy.detach(vertex)); + for(EventCallback callback : callbacksForRemovalEvents){ + callback.accept(removeEvent); + } + } + + private Q makeLabelsQuery(Q query) { + ((BasicVertexCentricQueryBuilder) query).profiler(queryProfiler); + return query; + } + + @Override + public void setBatchSize(int batchSize) { + this.batchSize = batchSize; + if(dropStepBatchFetcher != null){ + dropStepBatchFetcher.setBatchSize(batchSize); + } + } + + @Override + public void registerFirstNewLoopFutureVertexForPrefetching(Vertex futureVertex, int futureVertexTraverserLoop) { + if(useMultiQuery){ + dropStepBatchFetcher.registerFirstNewLoopFutureVertexForPrefetching(futureVertex); + } + } + + @Override + public void registerSameLoopFutureVertexForPrefetching(Vertex futureVertex, int futureVertexTraverserLoop) { + if(useMultiQuery){ + dropStepBatchFetcher.registerCurrentLoopFutureVertexForPrefetching(futureVertex, futureVertexTraverserLoop); + } + } + + @Override + public void registerNextLoopFutureVertexForPrefetching(Vertex futureVertex, int futureVertexTraverserLoop) { + if(useMultiQuery){ + dropStepBatchFetcher.registerNextLoopFutureVertexForPrefetching(futureVertex, futureVertexTraverserLoop); + } + } + + @Override + public void setMetrics(MutableMetrics metrics) { + queryProfiler = new TP3ProfileWrapper(metrics); + } +} diff --git a/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/step/fetcher/DropStepBatchFetcher.java b/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/step/fetcher/DropStepBatchFetcher.java new file mode 100644 index 00000000000..fb9c088a085 --- /dev/null +++ b/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/step/fetcher/DropStepBatchFetcher.java @@ -0,0 +1,38 @@ +// Copyright 2024 JanusGraph 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 +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.graphdb.tinkerpop.optimize.step.fetcher; + +import org.janusgraph.core.JanusGraphMultiVertexQuery; +import org.janusgraph.core.JanusGraphRelation; +import org.janusgraph.core.JanusGraphVertex; + +import java.util.Map; + +public class DropStepBatchFetcher extends MultiQueriableStepBatchFetcher>{ + + private final FetchQueryBuildFunction fetchQueryBuildFunction; + + public DropStepBatchFetcher(FetchQueryBuildFunction fetchQueryBuildFunction, int batchSize, + TriggeredBatchConsumer batchConsumerBeforeQuery) { + super(batchSize, batchConsumerBeforeQuery); + this.fetchQueryBuildFunction = fetchQueryBuildFunction; + } + + @Override + protected Map > makeQueryAndExecute(JanusGraphMultiVertexQuery multiQuery) { + multiQuery = fetchQueryBuildFunction.makeQuery(multiQuery); + return multiQuery.drop(); + } +} diff --git a/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/step/fetcher/MultiQueriableStepBatchFetcher.java b/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/step/fetcher/MultiQueriableStepBatchFetcher.java index 00fd7587d06..e4a25136c5e 100644 --- a/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/step/fetcher/MultiQueriableStepBatchFetcher.java +++ b/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/step/fetcher/MultiQueriableStepBatchFetcher.java @@ -31,17 +31,25 @@ */ public abstract class MultiQueriableStepBatchFetcher { + private static final TriggeredBatchConsumer EMPTY_BATCH_CONSUMER = (b, t) -> {}; + private Map multiQueryResults = null; private int batchSize; private int currentLoops = 0; private BatchProcessingQueue firstLoopBatchProcessingQueue; private BatchProcessingQueue currentLoopBatchProcessingQueue; private BatchProcessingQueue nextLoopBatchProcessingQueue; + private final TriggeredBatchConsumer batchConsumerBeforeQuery; public MultiQueriableStepBatchFetcher(int batchSize){ + this(batchSize, EMPTY_BATCH_CONSUMER); + } + + public MultiQueriableStepBatchFetcher(int batchSize, TriggeredBatchConsumer batchConsumerBeforeQuery){ this.batchSize = batchSize; this.currentLoopBatchProcessingQueue = generateNewBatchProcessingQueue(); this.nextLoopBatchProcessingQueue = generateNewBatchProcessingQueue(); + this.batchConsumerBeforeQuery = batchConsumerBeforeQuery; } public void registerCurrentLoopFutureVertexForPrefetching(Vertex forGeneralVertex, int traverserLoops) { @@ -105,9 +113,10 @@ private boolean hasNoFetchedData(Vertex forVertex){ } protected Map prefetchNextBatch(final Traversal.Admin, ?> traversal, JanusGraphVertex requiredFetchVertex){ - final JanusGraphMultiVertexQuery multiQuery = JanusGraphTraversalUtil.getTx(traversal) - .multiQuery(nextBatch()); + Collection batch = nextBatch(); + final JanusGraphMultiVertexQuery multiQuery = JanusGraphTraversalUtil.getTx(traversal).multiQuery(batch); multiQuery.addVertex(requiredFetchVertex); + batchConsumerBeforeQuery.consume(batch, requiredFetchVertex); try { return makeQueryAndExecute(multiQuery); } catch (JanusGraphException janusGraphException) { diff --git a/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/step/fetcher/TriggeredBatchConsumer.java b/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/step/fetcher/TriggeredBatchConsumer.java new file mode 100644 index 00000000000..7d6bfd2949f --- /dev/null +++ b/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/step/fetcher/TriggeredBatchConsumer.java @@ -0,0 +1,23 @@ +// Copyright 2024 JanusGraph 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 +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.graphdb.tinkerpop.optimize.step.fetcher; + +import java.util.Collection; + +public interface TriggeredBatchConsumer { + + void consume(Collection batch, E triggeringElement); + +} diff --git a/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/strategy/JanusGraphLocalQueryOptimizerStrategy.java b/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/strategy/JanusGraphLocalQueryOptimizerStrategy.java index 8e3a56dc03b..e1ba04c03a9 100644 --- a/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/strategy/JanusGraphLocalQueryOptimizerStrategy.java +++ b/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/strategy/JanusGraphLocalQueryOptimizerStrategy.java @@ -18,6 +18,7 @@ import org.apache.tinkerpop.gremlin.process.traversal.Traversal; import org.apache.tinkerpop.gremlin.process.traversal.TraversalStrategy; import org.apache.tinkerpop.gremlin.process.traversal.step.branch.LocalStep; +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.DropStep; import org.apache.tinkerpop.gremlin.process.traversal.step.filter.RangeGlobalStep; import org.apache.tinkerpop.gremlin.process.traversal.step.map.ElementMapStep; import org.apache.tinkerpop.gremlin.process.traversal.step.map.LabelStep; @@ -31,6 +32,7 @@ import org.janusgraph.graphdb.query.QueryUtil; import org.janusgraph.graphdb.tinkerpop.optimize.JanusGraphTraversalUtil; import org.janusgraph.graphdb.tinkerpop.optimize.step.HasStepFolder; +import org.janusgraph.graphdb.tinkerpop.optimize.step.JanusGraphDropStep; import org.janusgraph.graphdb.tinkerpop.optimize.step.JanusGraphElementMapStep; import org.janusgraph.graphdb.tinkerpop.optimize.step.JanusGraphLabelStep; import org.janusgraph.graphdb.tinkerpop.optimize.step.JanusGraphPropertiesStep; @@ -70,6 +72,7 @@ public void apply(final Traversal.Admin, ?> traversal) { final Optional tx = JanusGraphTraversalUtil.getJanusGraphTx(traversal); final MultiQueryPropertiesStrategyMode propertiesStrategyMode; final MultiQueryLabelStepStrategyMode labelStepStrategyMode; + final MultiQueryDropStepStrategyMode dropStepStrategyMode; final int txVertexCacheSize; if(tx.isPresent()){ @@ -77,16 +80,19 @@ public void apply(final Traversal.Admin, ?> traversal) { txVertexCacheSize = txConfig.getVertexCacheSize(); propertiesStrategyMode = txConfig.getPropertiesStrategyMode(); labelStepStrategyMode = txConfig.getLabelStepStrategyMode(); + dropStepStrategyMode = txConfig.getDropStepStrategyMode(); } else { GraphDatabaseConfiguration graphConfig = janusGraph.getConfiguration(); txVertexCacheSize = graphConfig.getTxVertexCacheSize(); propertiesStrategyMode = graphConfig.propertiesStrategyMode(); labelStepStrategyMode = graphConfig.labelStepStrategyMode(); + dropStepStrategyMode = graphConfig.dropStepStrategyMode(); } applyJanusGraphVertexSteps(traversal); applyJanusGraphPropertiesSteps(traversal, txVertexCacheSize, propertiesStrategyMode); applyJanusGraphLabelSteps(traversal, labelStepStrategyMode); + applyJanusGraphDropSteps(traversal, dropStepStrategyMode); inspectLocalTraversals(traversal, txVertexCacheSize, propertiesStrategyMode); } @@ -203,6 +209,19 @@ private void applyJanusGraphLabelSteps(Traversal.Admin, ?> traversal, MultiQue }); } + private void applyJanusGraphDropSteps(Traversal.Admin, ?> traversal, MultiQueryDropStepStrategyMode dropStepStrategyMode){ + if(MultiQueryDropStepStrategyMode.NONE.equals(dropStepStrategyMode)){ + return; + } + TraversalHelper.getStepsOfAssignableClass(DropStep.class, traversal).forEach(originalStep -> { + if(originalStep instanceof JanusGraphDropStep){ + return; + } + final JanusGraphDropStep janusGraphDropStep = new JanusGraphDropStep(originalStep); + TraversalHelper.replaceStep(originalStep, janusGraphDropStep, originalStep.getTraversal()); + }); + } + private static void unfoldLocalTraversal(final Traversal.Admin, ?> traversal, LocalStep, ?> localStep, Traversal.Admin localTraversal, MultiQueriable vertexStep) { diff --git a/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/strategy/JanusGraphMultiQueryStrategy.java b/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/strategy/JanusGraphMultiQueryStrategy.java index 85329a6ba46..d1673aa2386 100644 --- a/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/strategy/JanusGraphMultiQueryStrategy.java +++ b/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/strategy/JanusGraphMultiQueryStrategy.java @@ -20,7 +20,6 @@ import org.apache.tinkerpop.gremlin.process.traversal.TraversalStrategy; import org.apache.tinkerpop.gremlin.process.traversal.step.TraversalParent; import org.apache.tinkerpop.gremlin.process.traversal.step.branch.RepeatStep; -import org.apache.tinkerpop.gremlin.process.traversal.step.filter.DropStep; import org.apache.tinkerpop.gremlin.process.traversal.step.map.NoOpBarrierStep; import org.apache.tinkerpop.gremlin.process.traversal.strategy.AbstractTraversalStrategy; import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalHelper; @@ -55,11 +54,7 @@ private JanusGraphMultiQueryStrategy() { @Override public void apply(final Admin, ?> traversal) { - if (!traversal.getGraph().isPresent() - || TraversalHelper.onGraphComputer(traversal) - // The LazyBarrierStrategy is not allowed to run on traversals which use drop(). As a precaution, - // this strategy should not run on those traversals either, because it can also insert barrier(). - || !TraversalHelper.getStepsOfAssignableClassRecursively(DropStep.class, traversal).isEmpty()) { + if (!traversal.getGraph().isPresent() || TraversalHelper.onGraphComputer(traversal)) { return; } diff --git a/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/strategy/MultiQueryDropStepStrategyMode.java b/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/strategy/MultiQueryDropStepStrategyMode.java new file mode 100644 index 00000000000..3ee6a1a90bd --- /dev/null +++ b/janusgraph-core/src/main/java/org/janusgraph/graphdb/tinkerpop/optimize/strategy/MultiQueryDropStepStrategyMode.java @@ -0,0 +1,42 @@ +// Copyright 2024 JanusGraph 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 +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.graphdb.tinkerpop.optimize.strategy; + +import org.janusgraph.graphdb.configuration.ConfigName; + +public enum MultiQueryDropStepStrategyMode implements ConfigName { + + /** + * Prefetch labels for all vertices in a batch. + */ + ALL("all"), + + /** + * Skips `label` step pre-fetch optimization. + */ + NONE("none") + ; + + private final String configurationOptionName; + + MultiQueryDropStepStrategyMode(String configurationOptionName){ + this.configurationOptionName = configurationOptionName; + } + + @Override + public String getConfigName() { + return configurationOptionName; + } +} diff --git a/janusgraph-core/src/main/java/org/janusgraph/graphdb/transaction/StandardTransactionBuilder.java b/janusgraph-core/src/main/java/org/janusgraph/graphdb/transaction/StandardTransactionBuilder.java index 18285ee8d5a..8c79cd6784e 100644 --- a/janusgraph-core/src/main/java/org/janusgraph/graphdb/transaction/StandardTransactionBuilder.java +++ b/janusgraph-core/src/main/java/org/janusgraph/graphdb/transaction/StandardTransactionBuilder.java @@ -28,6 +28,7 @@ import org.janusgraph.diskstorage.util.time.TimestampProvider; import org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration; import org.janusgraph.graphdb.database.StandardJanusGraph; +import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryDropStepStrategyMode; import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryHasStepStrategyMode; import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryLabelStepStrategyMode; import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryPropertiesStrategyMode; @@ -91,9 +92,9 @@ public class StandardTransactionBuilder implements TransactionConfiguration, Tra private boolean isLazyLoadRelations; private MultiQueryHasStepStrategyMode hasStepStrategyMode; - private MultiQueryPropertiesStrategyMode propertiesStrategyMode; private MultiQueryLabelStepStrategyMode labelStepStrategyMode; + private MultiQueryDropStepStrategyMode dropStepStrategyMode; private final boolean forceIndexUsage; @@ -120,6 +121,7 @@ private StandardTransactionBuilder(GraphDatabaseConfiguration graphConfig, Stand this.hasStepStrategyMode = graphConfig.hasStepStrategyMode(); this.propertiesStrategyMode = graphConfig.propertiesStrategyMode(); this.labelStepStrategyMode = graphConfig.labelStepStrategyMode(); + this.dropStepStrategyMode = graphConfig.dropStepStrategyMode(); this.writableCustomOptions = writableCustomOptions; if(customOptions == null){ this.customOptions = new MergedConfiguration(writableCustomOptions, graphConfig.getConfiguration()); @@ -260,6 +262,12 @@ public TransactionBuilder setLabelsStepStrategyMode(MultiQueryLabelStepStrategyM return this; } + @Override + public TransactionBuilder setDropStepStrategyMode(MultiQueryDropStepStrategyMode dropStepStrategyMode) { + this.dropStepStrategyMode = dropStepStrategyMode; + return this; + } + @Override public void setCommitTime(Instant time) { throw new UnsupportedOperationException("Use setCommitTime(long,TimeUnit)"); @@ -307,7 +315,7 @@ propertyPrefetching, multiQuery, singleThreaded, threadBound, getTimestampProvid indexCacheWeight, getVertexCacheSize(), getDirtyVertexSize(), logIdentifier, restrictedPartitions, groupName, defaultSchemaMaker, hasDisabledSchemaConstraints, skipDBCacheRead, isLazyLoadRelations, hasStepStrategyMode, propertiesStrategyMode, - labelStepStrategyMode, customOptions); + labelStepStrategyMode, dropStepStrategyMode, customOptions); return graph.newTransaction(immutable); } @@ -444,6 +452,11 @@ public MultiQueryLabelStepStrategyMode getLabelStepStrategyMode() { return labelStepStrategyMode; } + @Override + public MultiQueryDropStepStrategyMode getDropStepStrategyMode() { + return dropStepStrategyMode; + } + @Override public String getGroupName() { return groupName; @@ -508,6 +521,7 @@ private static class ImmutableTxCfg implements TransactionConfiguration { private MultiQueryHasStepStrategyMode hasStepStrategyMode; private MultiQueryPropertiesStrategyMode propertiesStrategyMode; private MultiQueryLabelStepStrategyMode labelStepStrategyMode; + private MultiQueryDropStepStrategyMode dropStepStrategyMode; private final BaseTransactionConfig handleConfig; @@ -531,6 +545,7 @@ public ImmutableTxCfg(boolean isReadOnly, MultiQueryHasStepStrategyMode hasStepStrategyMode, MultiQueryPropertiesStrategyMode propertiesStrategyMode, MultiQueryLabelStepStrategyMode labelStepStrategyMode, + MultiQueryDropStepStrategyMode dropStepStrategyMode, Configuration customOptions) { this.isReadOnly = isReadOnly; this.hasEnabledBatchLoading = hasEnabledBatchLoading; @@ -557,6 +572,7 @@ public ImmutableTxCfg(boolean isReadOnly, this.hasStepStrategyMode = hasStepStrategyMode; this.propertiesStrategyMode = propertiesStrategyMode; this.labelStepStrategyMode = labelStepStrategyMode; + this.dropStepStrategyMode = dropStepStrategyMode; this.handleConfig = new StandardBaseTransactionConfig.Builder() .commitTime(commitTime) .timestampProvider(times) @@ -694,6 +710,11 @@ public MultiQueryLabelStepStrategyMode getLabelStepStrategyMode() { return labelStepStrategyMode; } + @Override + public MultiQueryDropStepStrategyMode getDropStepStrategyMode() { + return dropStepStrategyMode; + } + @Override public Instant getCommitTime() { return handleConfig.getCommitTime(); diff --git a/janusgraph-core/src/main/java/org/janusgraph/graphdb/transaction/TransactionConfiguration.java b/janusgraph-core/src/main/java/org/janusgraph/graphdb/transaction/TransactionConfiguration.java index 9f2aed31229..7373765fe72 100644 --- a/janusgraph-core/src/main/java/org/janusgraph/graphdb/transaction/TransactionConfiguration.java +++ b/janusgraph-core/src/main/java/org/janusgraph/graphdb/transaction/TransactionConfiguration.java @@ -16,6 +16,7 @@ import org.janusgraph.core.schema.DefaultSchemaMaker; import org.janusgraph.diskstorage.BaseTransactionConfig; +import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryDropStepStrategyMode; import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryHasStepStrategyMode; import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryLabelStepStrategyMode; import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryPropertiesStrategyMode; @@ -220,4 +221,9 @@ public interface TransactionConfiguration extends BaseTransactionConfig { */ MultiQueryLabelStepStrategyMode getLabelStepStrategyMode(); + /** + * @return Drop step strategy mode used for the transaction. Can be configured via config `query.batch.drop-step-mode`. + */ + MultiQueryDropStepStrategyMode getDropStepStrategyMode(); + } diff --git a/janusgraph-inmemory/src/test/java/org/janusgraph/graphdb/inmemory/InMemoryGraphTest.java b/janusgraph-inmemory/src/test/java/org/janusgraph/graphdb/inmemory/InMemoryGraphTest.java index a37c4cb9d91..e01c0eb03c2 100644 --- a/janusgraph-inmemory/src/test/java/org/janusgraph/graphdb/inmemory/InMemoryGraphTest.java +++ b/janusgraph-inmemory/src/test/java/org/janusgraph/graphdb/inmemory/InMemoryGraphTest.java @@ -134,4 +134,8 @@ public void testSupportDirectCommitOfSchemaChangesForConnection() {} @Override @Test @Disabled public void testSupportDirectCommitOfSchemaChangesForEdgeProperties() {} + + @Override @Test @Disabled + public void testMultiQueryDropsStrategyModes() {} + } diff --git a/janusgraph-inmemory/src/test/java/org/janusgraph/graphdb/inmemory/InMemoryLazyLoadGraphTest.java b/janusgraph-inmemory/src/test/java/org/janusgraph/graphdb/inmemory/InMemoryLazyLoadGraphTest.java index b91b2482baa..a8b5c9c5120 100644 --- a/janusgraph-inmemory/src/test/java/org/janusgraph/graphdb/inmemory/InMemoryLazyLoadGraphTest.java +++ b/janusgraph-inmemory/src/test/java/org/janusgraph/graphdb/inmemory/InMemoryLazyLoadGraphTest.java @@ -19,6 +19,7 @@ import org.janusgraph.diskstorage.configuration.WriteConfiguration; import org.janusgraph.graphdb.configuration.builder.GraphDatabaseConfigurationBuilder; import org.janusgraph.graphdb.database.LazyLoadGraphTest; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import static org.junit.Assert.assertThrows; @@ -50,4 +51,8 @@ public void testPropertyIdAccessInDifferentTransaction() { Exception exception = assertThrows(IllegalStateException.class, p::id); assertEquals(exception.getMessage(), "Any lazy load operation is not supported when transaction is already closed."); } + + @Override @Test @Disabled + public void testMultiQueryDropsStrategyModes() {} + } diff --git a/janusgraph-test/src/test/java/org/janusgraph/graphdb/tinkerpop/optimize/JanusGraphStepStrategyTest.java b/janusgraph-test/src/test/java/org/janusgraph/graphdb/tinkerpop/optimize/JanusGraphStepStrategyTest.java index 8316af3fddc..431055396d8 100644 --- a/janusgraph-test/src/test/java/org/janusgraph/graphdb/tinkerpop/optimize/JanusGraphStepStrategyTest.java +++ b/janusgraph-test/src/test/java/org/janusgraph/graphdb/tinkerpop/optimize/JanusGraphStepStrategyTest.java @@ -24,6 +24,7 @@ import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversal; import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversalSource; import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__; +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.DropStep; import org.apache.tinkerpop.gremlin.process.traversal.step.filter.HasStep; import org.apache.tinkerpop.gremlin.process.traversal.step.filter.IsStep; import org.apache.tinkerpop.gremlin.process.traversal.step.filter.OrStep; @@ -55,6 +56,7 @@ import org.janusgraph.graphdb.predicate.ConnectiveJanusPredicate; import org.janusgraph.graphdb.query.JanusGraphPredicateUtils; import org.janusgraph.graphdb.tinkerpop.optimize.step.HasStepFolder; +import org.janusgraph.graphdb.tinkerpop.optimize.step.JanusGraphDropStep; import org.janusgraph.graphdb.tinkerpop.optimize.step.JanusGraphElementMapStep; import org.janusgraph.graphdb.tinkerpop.optimize.step.JanusGraphHasStep; import org.janusgraph.graphdb.tinkerpop.optimize.step.JanusGraphLabelStep; @@ -68,6 +70,7 @@ import org.janusgraph.graphdb.tinkerpop.optimize.strategy.JanusGraphMultiQueryStrategy; import org.janusgraph.graphdb.tinkerpop.optimize.strategy.JanusGraphStepStrategy; import org.janusgraph.graphdb.tinkerpop.optimize.strategy.JanusGraphUnusedMultiQueryRemovalStrategy; +import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryDropStepStrategyMode; import org.janusgraph.graphdb.tinkerpop.optimize.strategy.MultiQueryStrategyRepeatStepMode; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -91,6 +94,7 @@ import static org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__.not; import static org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__.properties; import static org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__.values; +import static org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration.DROP_STEP_BATCH_MODE; import static org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration.REPEAT_STEP_BATCH_MODE; import static org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration.USE_MULTIQUERY; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -193,6 +197,10 @@ private void applyMultiQueryTraversalSteps(Traversal.Admin,?> traversal) { JanusGraphLabelStep janusGraphLabelStep = new JanusGraphLabelStep<>(labelStep); TraversalHelper.replaceStep(labelStep, janusGraphLabelStep, labelStep.getTraversal()); }); + TraversalHelper.getStepsOfAssignableClassRecursively(DropStep.class, traversal).forEach(dropStep -> { + JanusGraphDropStep janusGraphDropStep = new JanusGraphDropStep<>(dropStep); + TraversalHelper.replaceStep(dropStep, janusGraphDropStep, dropStep.getTraversal()); + }); } @SuppressWarnings({ "unchecked", "rawtypes" }) @@ -321,7 +329,9 @@ private static Stream generateTestParameters() { } private static Stream generateMultiQueryTestParameters() { - final StandardJanusGraph generalGraph = (StandardJanusGraph) StorageSetup.getInMemoryGraphWithMultiQuery(); + final StandardJanusGraph generalGraph = (StandardJanusGraph) JanusGraphFactory.open( + StorageSetup.getInMemoryConfiguration().set(USE_MULTIQUERY, true) + .set(DROP_STEP_BATCH_MODE, MultiQueryDropStepStrategyMode.ALL.getConfigName()));; final StandardJanusGraph graphWithRepeatClosestParent = (StandardJanusGraph) JanusGraphFactory.open( StorageSetup.getInMemoryConfiguration().set(USE_MULTIQUERY, true) .set(REPEAT_STEP_BATCH_MODE, MultiQueryStrategyRepeatStepMode.CLOSEST_REPEAT_PARENT.getConfigName())); @@ -474,6 +484,9 @@ private static Stream generateMultiQueryTestParameters() { // Need `JanusGraphLabelStep` to use instead of `LabelStep` arguments(g.V().label(), g_V().is(MQ_STEP).barrier(defaultBarrierSize).label(), otherStrategies), + // Need `JanusGraphDropStep` to use instead of `DropStep` + arguments(g.V().drop(), + g_V().is(MQ_STEP).barrier(defaultBarrierSize).drop(), otherStrategies), }); } }