X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-distributed-datastore%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fdatastore%2FTransactionProxy.java;h=da9789693cdca41e1be07f428775a2c74403bb29;hp=82258b46a44b237748342b9d43e8373cb04c0dde;hb=1d5ca4009be6c61d7b61989799037ad8f1ab7a75;hpb=daaef05cbf70e6cbec9af181258faead6d9620a6 diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionProxy.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionProxy.java index 82258b46a4..da9789693c 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionProxy.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionProxy.java @@ -7,15 +7,18 @@ */ package org.opendaylight.controller.cluster.datastore; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Verify.verifyNotNull; +import static java.util.Objects.requireNonNull; + import akka.actor.ActorSelection; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Optional; -import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; -import com.google.common.util.concurrent.CheckedFuture; +import com.google.common.util.concurrent.FluentFuture; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.SettableFuture; import java.util.ArrayList; import java.util.Collection; @@ -23,20 +26,30 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Optional; import java.util.Set; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; -import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier; -import org.opendaylight.controller.cluster.datastore.shardstrategy.ShardStrategyFactory; -import org.opendaylight.controller.cluster.datastore.utils.ActorContext; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; +import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier; +import org.opendaylight.controller.cluster.datastore.TransactionModificationOperation.DeleteOperation; +import org.opendaylight.controller.cluster.datastore.TransactionModificationOperation.MergeOperation; +import org.opendaylight.controller.cluster.datastore.TransactionModificationOperation.WriteOperation; +import org.opendaylight.controller.cluster.datastore.messages.AbstractRead; +import org.opendaylight.controller.cluster.datastore.messages.DataExists; +import org.opendaylight.controller.cluster.datastore.messages.ReadData; +import org.opendaylight.controller.cluster.datastore.utils.ActorUtils; import org.opendaylight.controller.cluster.datastore.utils.NormalizedNodeAggregator; -import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException; -import org.opendaylight.controller.sal.core.spi.data.AbstractDOMStoreTransaction; -import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction; -import org.opendaylight.yangtools.util.concurrent.MappingCheckedFuture; +import org.opendaylight.mdsal.dom.spi.store.AbstractDOMStoreTransaction; +import org.opendaylight.mdsal.dom.spi.store.DOMStoreReadWriteTransaction; import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier; +import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifier; +import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode; +import org.opendaylight.yangtools.yang.data.api.schema.DataContainerChild; import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode; -import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException; +import org.opendaylight.yangtools.yang.data.api.schema.builder.DataContainerNodeBuilder; +import org.opendaylight.yangtools.yang.data.impl.schema.Builders; +import org.opendaylight.yangtools.yang.data.tree.api.DataValidationFailedException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import scala.concurrent.Future; @@ -45,310 +58,321 @@ import scala.concurrent.Promise; /** * A transaction potentially spanning multiple backend shards. */ -public class TransactionProxy extends AbstractDOMStoreTransaction implements DOMStoreReadWriteTransaction { - private static enum TransactionState { +public class TransactionProxy extends AbstractDOMStoreTransaction + implements DOMStoreReadWriteTransaction { + private enum TransactionState { OPEN, READY, CLOSED, } + private static final Logger LOG = LoggerFactory.getLogger(TransactionProxy.class); + private static final DeleteOperation ROOT_DELETE_OPERATION = new DeleteOperation(YangInstanceIdentifier.empty()); - private final Map txContextAdapters = new HashMap<>(); + private final Map txContextWrappers = new TreeMap<>(); private final AbstractTransactionContextFactory txContextFactory; private final TransactionType type; private TransactionState state = TransactionState.OPEN; - private volatile OperationCompleter operationCompleter; - private volatile Semaphore operationLimiter; @VisibleForTesting public TransactionProxy(final AbstractTransactionContextFactory txContextFactory, final TransactionType type) { - super(txContextFactory.nextIdentifier(), false); + super(txContextFactory.nextIdentifier(), txContextFactory.getActorUtils().getDatastoreContext() + .isTransactionDebugContextEnabled()); this.txContextFactory = txContextFactory; - this.type = Preconditions.checkNotNull(type); + this.type = requireNonNull(type); LOG.debug("New {} Tx - {}", type, getIdentifier()); } @Override - public CheckedFuture exists(final YangInstanceIdentifier path) { - Preconditions.checkState(type != TransactionType.WRITE_ONLY, "Reads from write-only transactions are not allowed"); + public FluentFuture exists(final YangInstanceIdentifier path) { + return executeRead(shardNameFromIdentifier(path), new DataExists(path, DataStoreVersions.CURRENT_VERSION)); + } - LOG.debug("Tx {} exists {}", getIdentifier(), path); + private FluentFuture executeRead(final String shardName, final AbstractRead readCmd) { + checkState(type != TransactionType.WRITE_ONLY, "Reads from write-only transactions are not allowed"); - throttleOperation(); + LOG.trace("Tx {} {} {}", getIdentifier(), readCmd.getClass().getSimpleName(), readCmd.getPath()); - final SettableFuture proxyFuture = SettableFuture.create(); - TransactionContextWrapper contextAdapter = getContextAdapter(path); - contextAdapter.maybeExecuteTransactionOperation(new TransactionOperation() { + final SettableFuture proxyFuture = SettableFuture.create(); + AbstractTransactionContextWrapper contextWrapper = getContextWrapper(shardName); + contextWrapper.maybeExecuteTransactionOperation(new TransactionOperation() { @Override - public void invoke(TransactionContext transactionContext) { - transactionContext.dataExists(path, proxyFuture); + public void invoke(final TransactionContext transactionContext, final Boolean havePermit) { + transactionContext.executeRead(readCmd, proxyFuture, havePermit); } }); - return MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER); + return FluentFuture.from(proxyFuture); } @Override - public CheckedFuture>, ReadFailedException> read(final YangInstanceIdentifier path) { - Preconditions.checkState(type != TransactionType.WRITE_ONLY, "Reads from write-only transactions are not allowed"); + public FluentFuture> read(final YangInstanceIdentifier path) { + checkState(type != TransactionType.WRITE_ONLY, "Reads from write-only transactions are not allowed"); + requireNonNull(path, "path should not be null"); - LOG.debug("Tx {} read {}", getIdentifier(), path); - - if (YangInstanceIdentifier.EMPTY.equals(path)) { - return readAllData(); - } else { - throttleOperation(); - - return singleShardRead(shardNameFromIdentifier(path), path); - } + LOG.trace("Tx {} read {}", getIdentifier(), path); + return path.isEmpty() ? readAllData() : singleShardRead(shardNameFromIdentifier(path), path); } - private CheckedFuture>, ReadFailedException> singleShardRead( - final String shardName, final YangInstanceIdentifier path) { - final SettableFuture>> proxyFuture = SettableFuture.create(); - TransactionContextWrapper contextAdapter = getContextAdapter(shardName); - contextAdapter.maybeExecuteTransactionOperation(new TransactionOperation() { - @Override - public void invoke(TransactionContext transactionContext) { - transactionContext.readData(path, proxyFuture); - } - }); - - return MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER); + private FluentFuture> singleShardRead(final String shardName, + final YangInstanceIdentifier path) { + return executeRead(shardName, new ReadData(path, DataStoreVersions.CURRENT_VERSION)); } - private CheckedFuture>, ReadFailedException> readAllData() { - final Set allShardNames = txContextFactory.getActorContext().getConfiguration().getAllShardNames(); - final Collection>, ReadFailedException>> futures = new ArrayList<>(allShardNames.size()); + private FluentFuture> readAllData() { + final Set allShardNames = txContextFactory.getActorUtils().getConfiguration().getAllShardNames(); + final Collection>> futures = new ArrayList<>(allShardNames.size()); for (String shardName : allShardNames) { - futures.add(singleShardRead(shardName, YangInstanceIdentifier.EMPTY)); + futures.add(singleShardRead(shardName, YangInstanceIdentifier.empty())); } - final ListenableFuture>>> listFuture = Futures.allAsList(futures); - final ListenableFuture>> aggregateFuture; + final ListenableFuture>> listFuture = Futures.allAsList(futures); + final ListenableFuture> aggregateFuture; - aggregateFuture = Futures.transform(listFuture, new Function>>, Optional>>() { - @Override - public Optional> apply(final List>> input) { - try { - return NormalizedNodeAggregator.aggregate(YangInstanceIdentifier.EMPTY, input, txContextFactory.getActorContext().getSchemaContext()); - } catch (DataValidationFailedException e) { - throw new IllegalArgumentException("Failed to aggregate", e); - } + aggregateFuture = Futures.transform(listFuture, input -> { + try { + return NormalizedNodeAggregator.aggregate(YangInstanceIdentifier.empty(), input, + txContextFactory.getActorUtils().getSchemaContext(), + txContextFactory.getActorUtils().getDatastoreContext().getLogicalStoreType()); + } catch (DataValidationFailedException e) { + throw new IllegalArgumentException("Failed to aggregate", e); } - }); + }, MoreExecutors.directExecutor()); - return MappingCheckedFuture.create(aggregateFuture, ReadFailedException.MAPPER); + return FluentFuture.from(aggregateFuture); } @Override public void delete(final YangInstanceIdentifier path) { - checkModificationState(); - - LOG.debug("Tx {} delete {}", getIdentifier(), path); + checkModificationState("delete", path); - throttleOperation(); + if (path.isEmpty()) { + deleteAllData(); + } else { + executeModification(new DeleteOperation(path)); + } + } - TransactionContextWrapper contextAdapter = getContextAdapter(path); - contextAdapter.maybeExecuteTransactionOperation(new TransactionOperation() { - @Override - public void invoke(TransactionContext transactionContext) { - transactionContext.deleteData(path); - } - }); + private void deleteAllData() { + for (String shardName : getActorUtils().getConfiguration().getAllShardNames()) { + getContextWrapper(shardName).maybeExecuteTransactionOperation(ROOT_DELETE_OPERATION); + } } @Override - public void merge(final YangInstanceIdentifier path, final NormalizedNode data) { - checkModificationState(); + public void merge(final YangInstanceIdentifier path, final NormalizedNode data) { + checkModificationState("merge", path); - LOG.debug("Tx {} merge {}", getIdentifier(), path); + if (path.isEmpty()) { + mergeAllData(checkRootData(data)); + } else { + executeModification(new MergeOperation(path, data)); + } + } - throttleOperation(); + private void mergeAllData(final ContainerNode rootData) { + // Populate requests for individual shards that are being touched + final Map> rootBuilders = new HashMap<>(); + for (DataContainerChild child : rootData.body()) { + final String shardName = shardNameFromRootChild(child); + rootBuilders.computeIfAbsent(shardName, + unused -> Builders.containerBuilder().withNodeIdentifier(rootData.getIdentifier())) + .addChild(child); + } - TransactionContextWrapper contextAdapter = getContextAdapter(path); - contextAdapter.maybeExecuteTransactionOperation(new TransactionOperation() { - @Override - public void invoke(TransactionContext transactionContext) { - transactionContext.mergeData(path, data); - } - }); + // Now dispatch all merges + for (Entry> entry : rootBuilders.entrySet()) { + getContextWrapper(entry.getKey()).maybeExecuteTransactionOperation(new MergeOperation( + YangInstanceIdentifier.empty(), entry.getValue().build())); + } } @Override - public void write(final YangInstanceIdentifier path, final NormalizedNode data) { - checkModificationState(); + public void write(final YangInstanceIdentifier path, final NormalizedNode data) { + checkModificationState("write", path); - LOG.debug("Tx {} write {}", getIdentifier(), path); + if (path.isEmpty()) { + writeAllData(checkRootData(data)); + } else { + executeModification(new WriteOperation(path, data)); + } + } - throttleOperation(); + private void writeAllData(final ContainerNode rootData) { + // Open builders for all shards + final Map> rootBuilders = new HashMap<>(); + for (String shardName : getActorUtils().getConfiguration().getAllShardNames()) { + rootBuilders.put(shardName, Builders.containerBuilder().withNodeIdentifier(rootData.getIdentifier())); + } - TransactionContextWrapper contextAdapter = getContextAdapter(path); - contextAdapter.maybeExecuteTransactionOperation(new TransactionOperation() { - @Override - public void invoke(TransactionContext transactionContext) { - transactionContext.writeData(path, data); - } - }); + // Now distribute children as needed + for (DataContainerChild child : rootData.body()) { + final String shardName = shardNameFromRootChild(child); + verifyNotNull(rootBuilders.get(shardName), "Failed to find builder for %s", shardName).addChild(child); + } + + // Now dispatch all writes + for (Entry> entry : rootBuilders.entrySet()) { + getContextWrapper(entry.getKey()).maybeExecuteTransactionOperation(new WriteOperation( + YangInstanceIdentifier.empty(), entry.getValue().build())); + } + } + + private void executeModification(final TransactionModificationOperation operation) { + getContextWrapper(operation.path()).maybeExecuteTransactionOperation(operation); + } + + private static ContainerNode checkRootData(final NormalizedNode data) { + // Root has to be a container + checkArgument(data instanceof ContainerNode, "Invalid root data %s", data); + return (ContainerNode) data; } - private void checkModificationState() { - Preconditions.checkState(type != TransactionType.READ_ONLY, - "Modification operation on read-only transaction is not allowed"); - Preconditions.checkState(state == TransactionState.OPEN, - "Transaction is sealed - further modifications are not allowed"); + private void checkModificationState(final String opName, final YangInstanceIdentifier path) { + checkState(type != TransactionType.READ_ONLY, "Modification operation on read-only transaction is not allowed"); + checkState(state == TransactionState.OPEN, "Transaction is sealed - further modifications are not allowed"); + LOG.trace("Tx {} {} {}", getIdentifier(), opName, path); } private boolean seal(final TransactionState newState) { if (state == TransactionState.OPEN) { state = newState; return true; - } else { - return false; } + return false; } @Override public final void close() { if (!seal(TransactionState.CLOSED)) { - Preconditions.checkState(state == TransactionState.CLOSED, "Transaction %s is ready, it cannot be closed", + checkState(state == TransactionState.CLOSED, "Transaction %s is ready, it cannot be closed", getIdentifier()); // Idempotent no-op as per AutoCloseable recommendation return; } - for (TransactionContextWrapper contextAdapter : txContextAdapters.values()) { - contextAdapter.maybeExecuteTransactionOperation(new TransactionOperation() { + for (AbstractTransactionContextWrapper contextWrapper : txContextWrappers.values()) { + contextWrapper.maybeExecuteTransactionOperation(new TransactionOperation() { @Override - public void invoke(TransactionContext transactionContext) { + public void invoke(final TransactionContext transactionContext, final Boolean havePermit) { transactionContext.closeTransaction(); } }); } - txContextAdapters.clear(); + txContextWrappers.clear(); } @Override public final AbstractThreePhaseCommitCohort ready() { - Preconditions.checkState(type != TransactionType.READ_ONLY, "Read-only transactions cannot be readied"); + checkState(type != TransactionType.READ_ONLY, "Read-only transactions cannot be readied"); final boolean success = seal(TransactionState.READY); - Preconditions.checkState(success, "Transaction %s is %s, it cannot be readied", getIdentifier(), state); + checkState(success, "Transaction %s is %s, it cannot be readied", getIdentifier(), state); - LOG.debug("Tx {} Readying {} components for commit", getIdentifier(), txContextAdapters.size()); + LOG.debug("Tx {} Readying {} components for commit", getIdentifier(), txContextWrappers.size()); final AbstractThreePhaseCommitCohort ret; - switch (txContextAdapters.size()) { - case 0: - TransactionRateLimitingCallback.adjustRateLimitForUnusedTransaction(txContextFactory.getActorContext()); - ret = NoOpDOMStoreThreePhaseCommitCohort.INSTANCE; - break; - case 1: - final Entry e = Iterables.getOnlyElement(txContextAdapters.entrySet()); - ret = createSingleCommitCohort(e.getKey(), e.getValue()); - break; - default: - ret = createMultiCommitCohort(txContextAdapters.entrySet()); + switch (txContextWrappers.size()) { + case 0: + ret = NoOpDOMStoreThreePhaseCommitCohort.INSTANCE; + break; + case 1: + final Entry e = Iterables.getOnlyElement( + txContextWrappers.entrySet()); + ret = createSingleCommitCohort(e.getKey(), e.getValue()); + break; + default: + ret = createMultiCommitCohort(); } txContextFactory.onTransactionReady(getIdentifier(), ret.getCohortFutures()); - return ret; + + final Throwable debugContext = getDebugContext(); + return debugContext == null ? ret : new DebugThreePhaseCommitCohort(getIdentifier(), ret, debugContext); } + @SuppressWarnings({ "rawtypes", "unchecked" }) private AbstractThreePhaseCommitCohort createSingleCommitCohort(final String shardName, - final TransactionContextWrapper contextAdapter) { - throttleOperation(); + final AbstractTransactionContextWrapper contextWrapper) { LOG.debug("Tx {} Readying transaction for shard {}", getIdentifier(), shardName); final OperationCallback.Reference operationCallbackRef = new OperationCallback.Reference(OperationCallback.NO_OP_CALLBACK); - final TransactionContext transactionContext = contextAdapter.getTransactionContext(); + final TransactionContext transactionContext = contextWrapper.getTransactionContext(); final Future future; if (transactionContext == null) { final Promise promise = akka.dispatch.Futures.promise(); - contextAdapter.maybeExecuteTransactionOperation(new TransactionOperation() { + contextWrapper.maybeExecuteTransactionOperation(new TransactionOperation() { @Override - public void invoke(TransactionContext transactionContext) { - promise.completeWith(getReadyOrDirectCommitFuture(transactionContext, operationCallbackRef)); + public void invoke(final TransactionContext newTransactionContext, final Boolean havePermit) { + promise.completeWith(getDirectCommitFuture(newTransactionContext, operationCallbackRef, + havePermit)); } }); future = promise.future(); } else { // avoid the creation of a promise and a TransactionOperation - future = getReadyOrDirectCommitFuture(transactionContext, operationCallbackRef); + future = getDirectCommitFuture(transactionContext, operationCallbackRef, null); } - return new SingleCommitCohortProxy(txContextFactory.getActorContext(), future, getIdentifier().toString(), - operationCallbackRef); + return new SingleCommitCohortProxy(txContextFactory.getActorUtils(), future, getIdentifier(), + operationCallbackRef); } - private Future getReadyOrDirectCommitFuture(TransactionContext transactionContext, - OperationCallback.Reference operationCallbackRef) { - if (transactionContext.supportsDirectCommit()) { - TransactionRateLimitingCallback rateLimitingCallback = new TransactionRateLimitingCallback( - txContextFactory.getActorContext()); - operationCallbackRef.set(rateLimitingCallback); - rateLimitingCallback.run(); - return transactionContext.directCommit(); - } else { - return transactionContext.readyTransaction(); - } + private Future getDirectCommitFuture(final TransactionContext transactionContext, + final OperationCallback.Reference operationCallbackRef, final Boolean havePermit) { + TransactionRateLimitingCallback rateLimitingCallback = new TransactionRateLimitingCallback( + txContextFactory.getActorUtils()); + operationCallbackRef.set(rateLimitingCallback); + rateLimitingCallback.run(); + return transactionContext.directCommit(havePermit); } - private AbstractThreePhaseCommitCohort createMultiCommitCohort( - final Set> txContextAdapterEntries) { + private AbstractThreePhaseCommitCohort createMultiCommitCohort() { - throttleOperation(); - final List> cohortFutures = new ArrayList<>(txContextAdapterEntries.size()); - for (Entry e : txContextAdapterEntries) { + final List cohorts = new ArrayList<>(txContextWrappers.size()); + final Optional> shardNames = Optional.of(new TreeSet<>(txContextWrappers.keySet())); + for (Entry e : txContextWrappers.entrySet()) { LOG.debug("Tx {} Readying transaction for shard {}", getIdentifier(), e.getKey()); - TransactionContextWrapper contextAdapter = e.getValue(); - final TransactionContext transactionContext = contextAdapter.getTransactionContext(); - Future future; - if (transactionContext != null) { - // avoid the creation of a promise and a TransactionOperation - future = transactionContext.readyTransaction(); - } else { - final Promise promise = akka.dispatch.Futures.promise(); - contextAdapter.maybeExecuteTransactionOperation(new TransactionOperation() { - @Override - public void invoke(TransactionContext transactionContext) { - promise.completeWith(transactionContext.readyTransaction()); - } - }); - - future = promise.future(); - } + final AbstractTransactionContextWrapper wrapper = e.getValue(); - cohortFutures.add(future); + // The remote tx version is obtained the via TransactionContext which may not be available yet so + // we pass a Supplier to dynamically obtain it. Once the ready Future is resolved the + // TransactionContext is available. + cohorts.add(new ThreePhaseCommitCohortProxy.CohortInfo(wrapper.readyTransaction(shardNames), + () -> wrapper.getTransactionContext().getTransactionVersion())); } - return new ThreePhaseCommitCohortProxy(txContextFactory.getActorContext(), cohortFutures, getIdentifier().toString()); + return new ThreePhaseCommitCohortProxy(txContextFactory.getActorUtils(), cohorts, getIdentifier()); } - private static String shardNameFromIdentifier(final YangInstanceIdentifier path) { - return ShardStrategyFactory.getStrategy(path).findShard(path); + private String shardNameFromRootChild(final DataContainerChild child) { + return shardNameFromIdentifier(YangInstanceIdentifier.create(child.getIdentifier())); } - private TransactionContextWrapper getContextAdapter(final YangInstanceIdentifier path) { - return getContextAdapter(shardNameFromIdentifier(path)); + private String shardNameFromIdentifier(final YangInstanceIdentifier path) { + return getActorUtils().getShardStrategyFactory().getStrategy(path).findShard(path); } - private TransactionContextWrapper getContextAdapter(final String shardName) { - final TransactionContextWrapper existing = txContextAdapters.get(shardName); + private AbstractTransactionContextWrapper getContextWrapper(final YangInstanceIdentifier path) { + return getContextWrapper(shardNameFromIdentifier(path)); + } + + private AbstractTransactionContextWrapper getContextWrapper(final String shardName) { + final AbstractTransactionContextWrapper existing = txContextWrappers.get(shardName); if (existing != null) { return existing; } - final TransactionContextWrapper fresh = txContextFactory.newTransactionAdapter(this, shardName); - txContextAdapters.put(shardName, fresh); + final AbstractTransactionContextWrapper fresh = txContextFactory.newTransactionContextWrapper(this, shardName); + txContextWrappers.put(shardName, fresh); return fresh; } @@ -360,47 +384,7 @@ public class TransactionProxy extends AbstractDOMStoreTransaction