Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 73373184B8 for ; Mon, 16 Nov 2015 10:55:38 +0000 (UTC) Received: (qmail 4573 invoked by uid 500); 16 Nov 2015 10:55:38 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 4538 invoked by uid 500); 16 Nov 2015 10:55:38 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 4529 invoked by uid 99); 16 Nov 2015 10:55:38 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 16 Nov 2015 10:55:38 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 3B4F6E009E; Mon, 16 Nov 2015 10:55:38 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.apache.org Message-Id: <06995a9034c742008e1d1baf6ca53dac@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: ignite git commit: 'Single' operations optimizations for tx cache. Date: Mon, 16 Nov 2015 10:55:38 +0000 (UTC) Repository: ignite Updated Branches: refs/heads/ignite-single-op-tx 49b2dc4fe -> a03c16508 'Single' operations optimizations for tx cache. Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/a03c1650 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/a03c1650 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/a03c1650 Branch: refs/heads/ignite-single-op-tx Commit: a03c165080d63193e31ab8f0bcdbfd812337e97b Parents: 49b2dc4 Author: sboikov Authored: Mon Nov 16 13:55:26 2015 +0300 Committer: sboikov Committed: Mon Nov 16 13:55:26 2015 +0300 ---------------------------------------------------------------------- .../GridDistributedTxRemoteAdapter.java | 4 +- .../cache/distributed/dht/GridDhtTxRemote.java | 7 +- .../near/GridNearOptimisticTxPrepareFuture.java | 2 +- .../cache/distributed/near/GridNearTxLocal.java | 46 ++++--- .../cache/transactions/IgniteTxHandler.java | 7 +- .../IgniteTxImplicitSingleStateImpl.java | 2 +- .../transactions/IgniteTxLocalAdapter.java | 2 +- .../cache/transactions/IgniteTxLocalState.java | 44 ++++++ .../transactions/IgniteTxLocalStateAdapter.java | 41 ++++++ .../IgniteTxRemoteSingleStateImpl.java | 110 +++++++++++++++ .../cache/transactions/IgniteTxRemoteState.java | 39 ++++++ .../IgniteTxRemoteStateAdapter.java | 117 ++++++++++++++++ .../transactions/IgniteTxRemoteStateImpl.java | 133 +------------------ .../cache/transactions/IgniteTxState.java | 100 ++++++++++++-- .../transactions/IgniteTxStateAdapter.java | 41 ------ .../cache/transactions/IgniteTxStateImpl.java | 2 +- 16 files changed, 491 insertions(+), 206 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/a03c1650/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java index be59dd1..3ba88c5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java @@ -45,7 +45,7 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxRemoteEx; -import org.apache.ignite.internal.processors.cache.transactions.IgniteTxRemoteStateImpl; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxRemoteState; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxState; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext; @@ -105,7 +105,7 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter /** */ @GridToStringInclude - protected IgniteTxRemoteStateImpl txState; + protected IgniteTxRemoteState txState; /** * Empty constructor required for {@link Externalizable}. http://git-wip-us.apache.org/repos/asf/ignite/blob/a03c1650/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java index 569356c..15d1e0e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java @@ -36,6 +36,7 @@ import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxRemoteAdapter; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxRemoteSingleStateImpl; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxRemoteStateImpl; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.tostring.GridToStringBuilder; @@ -111,7 +112,8 @@ public class GridDhtTxRemote extends GridDistributedTxRemoteAdapter { GridCacheVersion nearXidVer, Map> txNodes, @Nullable UUID subjId, - int taskNameHash + int taskNameHash, + boolean single ) { super( ctx, @@ -138,7 +140,8 @@ public class GridDhtTxRemote extends GridDistributedTxRemoteAdapter { this.nearXidVer = nearXidVer; this.txNodes = txNodes; - txState = new IgniteTxRemoteStateImpl( + txState = single ? new IgniteTxRemoteSingleStateImpl() : + new IgniteTxRemoteStateImpl( Collections.emptyMap(), new ConcurrentLinkedHashMap(U.capacity(txSize), 0.75f, 1)); http://git-wip-us.apache.org/repos/asf/ignite/blob/a03c1650/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java index 36ad7d0..791d2f3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java @@ -311,7 +311,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa return; } - tx.addEntryMapping(mapping); + tx.addSingleEntryMapping(mapping, write); cctx.mvcc().recheckPendingLocks(); http://git-wip-us.apache.org/repos/asf/ignite/blob/a03c1650/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index 12c9958..22fff19 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -526,7 +526,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter { } /** - * @return Non {@code null} entry if tx has only one write entry. + * @return Non-null entry if tx has only one write entry. */ @Nullable IgniteTxEntry singleWrite() { return txState.singleWrite(); @@ -537,36 +537,48 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter { */ void addEntryMapping(@Nullable Collection maps) { if (!F.isEmpty(maps)) { - for (GridDistributedTxMapping map : maps) - addEntryMapping(map); + for (GridDistributedTxMapping map : maps) { + ClusterNode n = map.node(); - if (log.isDebugEnabled()) { - log.debug("Added mappings to transaction [locId=" + cctx.localNodeId() + - ", mappings=" + maps + - ", tx=" + this + ']'); + GridDistributedTxMapping m = mappings.get(n.id()); + + if (m == null) { + m = F.addIfAbsent(mappings, n.id(), new GridDistributedTxMapping(n)); + + m.near(map.near()); + + if (map.explicitLock()) + m.markExplicitLock(); + } + + assert m != null; + + for (IgniteTxEntry entry : map.entries()) + m.add(entry); } + + if (log.isDebugEnabled()) + log.debug("Added mappings to transaction [locId=" + cctx.localNodeId() + ", mappings=" + maps + + ", tx=" + this + ']'); } } /** * @param map Mapping. */ - void addEntryMapping(GridDistributedTxMapping map) { + void addSingleEntryMapping(GridDistributedTxMapping map, IgniteTxEntry entry) { ClusterNode n = map.node(); - GridDistributedTxMapping m = mappings.get(n.id()); + GridDistributedTxMapping m = new GridDistributedTxMapping(n); - if (m == null) { - m = F.addIfAbsent(mappings, n.id(), new GridDistributedTxMapping(n)); + mappings.put(n.id(), m); - m.near(map.near()); + m.near(map.near()); - if (map.explicitLock()) - m.markExplicitLock(); - } + if (map.explicitLock()) + m.markExplicitLock(); - for (IgniteTxEntry entry : map.entries()) - m.add(entry); + m.add(entry); } /** http://git-wip-us.apache.org/repos/asf/ignite/blob/a03c1650/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java index 014aa8a..173da35 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java @@ -1088,6 +1088,8 @@ public class IgniteTxHandler { GridDhtTxRemote tx = ctx.tm().tx(req.version()); if (tx == null) { + boolean single = req.last() && req.writes().size() == 1; + tx = new GridDhtTxRemote( ctx, req.nearNodeId(), @@ -1107,7 +1109,8 @@ public class IgniteTxHandler { req.nearXidVersion(), req.transactionNodes(), req.subjectId(), - req.taskNameHash()); + req.taskNameHash(), + single); tx.writeVersion(req.writeVersion()); @@ -1135,7 +1138,7 @@ public class IgniteTxHandler { tx.transactionNodes(req.transactionNodes()); } - if (!tx.isSystemInvalidate() && !F.isEmpty(req.writes())) { + if (!tx.isSystemInvalidate()) { int idx = 0; for (IgniteTxEntry entry : req.writes()) { http://git-wip-us.apache.org/repos/asf/ignite/blob/a03c1650/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java index 85a9561..1b99159 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java @@ -37,7 +37,7 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; /** * */ -public class IgniteTxImplicitSingleStateImpl extends IgniteTxStateAdapter { +public class IgniteTxImplicitSingleStateImpl extends IgniteTxLocalStateAdapter { /** */ private GridCacheContext cacheCtx; http://git-wip-us.apache.org/repos/asf/ignite/blob/a03c1650/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index 88e4fa2..d22d6f4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -153,7 +153,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter /** */ @GridToStringInclude - protected IgniteTxState txState; + protected IgniteTxLocalState txState; /** * Empty constructor required for {@link Externalizable}. http://git-wip-us.apache.org/repos/asf/ignite/blob/a03c1650/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalState.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalState.java new file mode 100644 index 0000000..6fd86de --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalState.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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.apache.ignite.internal.processors.cache.transactions; + +/** + * + */ +public interface IgniteTxLocalState extends IgniteTxState { + /** + * @param entry Entry. + */ + public void addEntry(IgniteTxEntry entry); + + /** + * @param txSize + * @return + */ + public boolean init(int txSize); + + /** + * @return + */ + public boolean initialized(); + + /** + * + */ + public void seal(); +} http://git-wip-us.apache.org/repos/asf/ignite/blob/a03c1650/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalStateAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalStateAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalStateAdapter.java new file mode 100644 index 0000000..cde5203 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalStateAdapter.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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.apache.ignite.internal.processors.cache.transactions; + +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * + */ +public abstract class IgniteTxLocalStateAdapter implements IgniteTxLocalState { + /** + * @param cacheCtx Cache context. + * @param tx Transaction. + * @param commit {@code False} if transaction rolled back. + */ + protected final void onTxEnd(GridCacheContext cacheCtx, IgniteInternalTx tx, boolean commit) { + if (cacheCtx.cache().configuration().isStatisticsEnabled()) { + // Convert start time from ms to ns. + if (commit) + cacheCtx.cache().metrics0().onTxCommit((U.currentTimeMillis() - tx.startTime()) * 1000); + else + cacheCtx.cache().metrics0().onTxRollback((U.currentTimeMillis() - tx.startTime()) * 1000); + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/a03c1650/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteSingleStateImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteSingleStateImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteSingleStateImpl.java new file mode 100644 index 0000000..528752f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteSingleStateImpl.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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.apache.ignite.internal.processors.cache.transactions; + +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.Set; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public class IgniteTxRemoteSingleStateImpl extends IgniteTxRemoteStateAdapter { + /** */ + private IgniteTxEntry entry; + + /** {@inheritDoc} */ + @Override public void setWriteValue(IgniteTxEntry e) { + if (entry == null) + entry = e; + else + copyEntry(e, entry); + } + + /** {@inheritDoc} */ + @Override public void addWriteEntry(IgniteTxKey key, IgniteTxEntry e) { + this.entry = e; + } + + /** {@inheritDoc} */ + @Override public void clearEntry(IgniteTxKey key) { + if (entry != null && entry.txKey().equals(key)) + entry = null; + } + + /** {@inheritDoc} */ + @Override public IgniteTxEntry entry(IgniteTxKey key) { + if (entry != null && entry.txKey().equals(key)) + return entry; + + return null; + } + + /** {@inheritDoc} */ + @Override public boolean hasWriteKey(IgniteTxKey key) { + return entry != null && entry.txKey().equals(key); + } + + /** {@inheritDoc} */ + @Override public Set readSet() { + return Collections.emptySet(); + } + + /** {@inheritDoc} */ + @Override public Set writeSet() { + return entry != null ? Collections.singleton(entry.txKey()) : Collections.emptySet(); + } + + /** {@inheritDoc} */ + @Override public Collection writeEntries() { + return entry != null ? Collections.singletonList(entry) : Collections.emptyList(); + } + + /** {@inheritDoc} */ + @Override public Collection readEntries() { + return Collections.emptyList(); + } + + /** {@inheritDoc} */ + @Override public Map writeMap() { + return entry != null ? Collections.singletonMap(entry.txKey(), entry) : + Collections.emptyMap(); + } + + /** {@inheritDoc} */ + @Override public Map readMap() { + return Collections.emptyMap(); + } + + /** {@inheritDoc} */ + @Override public boolean empty() { + return entry == null; + } + + /** {@inheritDoc} */ + @Override public Collection allEntries() { + return entry != null ? Collections.singletonList(entry) : Collections.emptyList(); + } + + /** {@inheritDoc} */ + @Nullable @Override public IgniteTxEntry singleWrite() { + return entry; + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/a03c1650/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteState.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteState.java new file mode 100644 index 0000000..f2ba3ba --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteState.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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.apache.ignite.internal.processors.cache.transactions; + +/** + * + */ +public interface IgniteTxRemoteState extends IgniteTxState { + /** + * @param e Entry. + */ + public void setWriteValue(IgniteTxEntry e); + + /** + * @param key Key. + * @param e Entry. + */ + public void addWriteEntry(IgniteTxKey key, IgniteTxEntry e); + + /** + * @param key Entry key. + */ + public void clearEntry(IgniteTxKey key); +} http://git-wip-us.apache.org/repos/asf/ignite/blob/a03c1650/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateAdapter.java new file mode 100644 index 0000000..9be37e1 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateAdapter.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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.apache.ignite.internal.processors.cache.transactions; + +import java.util.Collection; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture; +import org.apache.ignite.internal.processors.cache.store.CacheStoreManager; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public abstract class IgniteTxRemoteStateAdapter implements IgniteTxRemoteState { + /** {@inheritDoc} */ + @Override public boolean implicitSingle() { + return false; + } + + /** {@inheritDoc} */ + @Nullable @Override public Integer firstCacheId() { + assert false; + + return null; + } + + /** {@inheritDoc} */ + @Override public void awaitLastFut(GridCacheSharedContext cctx) { + assert false; + } + + /** {@inheritDoc} */ + @Override public IgniteCheckedException validateTopology(GridCacheSharedContext cctx, GridDhtTopologyFuture topFut) { + assert false; + + return null; + } + + /** {@inheritDoc} */ + @Override public boolean sync(GridCacheSharedContext cctx) { + assert false; + + return false; + } + + /** {@inheritDoc} */ + @Override public boolean hasNearCache(GridCacheSharedContext cctx) { + assert false; + + return false; + } + + /** {@inheritDoc} */ + @Override public void addActiveCache(GridCacheContext cacheCtx, IgniteTxLocalAdapter tx) + throws IgniteCheckedException { + assert false; + } + + /** {@inheritDoc} */ + @Override public GridDhtTopologyFuture topologyReadLock(GridCacheSharedContext cctx, GridFutureAdapter fut) { + assert false; + + return null; + } + + /** {@inheritDoc} */ + @Override public void topologyReadUnlock(GridCacheSharedContext cctx) { + assert false; + } + + /** {@inheritDoc} */ + @Override public boolean storeUsed(GridCacheSharedContext cctx) { + return false; + } + + /** {@inheritDoc} */ + @Override public Collection stores(GridCacheSharedContext cctx) { + assert false; + + return null; + } + + /** {@inheritDoc} */ + @Override public void onTxEnd(GridCacheSharedContext cctx, IgniteInternalTx tx, boolean commit) { + assert false; + } + + protected final void copyEntry(IgniteTxEntry e, IgniteTxEntry entry) { + entry.value(e.value(), e.hasWriteValue(), e.hasReadValue()); + entry.entryProcessors(e.entryProcessors()); + entry.op(e.op()); + entry.ttl(e.ttl()); + entry.explicitVersion(e.explicitVersion()); + + // Conflict resolution stuff. + entry.conflictVersion(e.conflictVersion()); + entry.conflictExpireTime(e.conflictExpireTime()); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/a03c1650/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateImpl.java index fb64655..4bbb9cf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateImpl.java @@ -20,20 +20,13 @@ package org.apache.ignite.internal.processors.cache.transactions; import java.util.Collection; import java.util.Map; import java.util.Set; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.processors.cache.GridCacheContext; -import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture; -import org.apache.ignite.internal.processors.cache.store.CacheStoreManager; -import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; -import org.jetbrains.annotations.Nullable; /** * */ -public class IgniteTxRemoteStateImpl implements IgniteTxState { +public class IgniteTxRemoteStateImpl extends IgniteTxRemoteStateAdapter { /** Read set. */ @GridToStringInclude protected Map readMap; @@ -53,79 +46,6 @@ public class IgniteTxRemoteStateImpl implements IgniteTxState { } /** {@inheritDoc} */ - @Override public boolean implicitSingle() { - return false; - } - - /** {@inheritDoc} */ - @Nullable @Override public Integer firstCacheId() { - assert false; - - return null; - } - - /** {@inheritDoc} */ - @Override public void awaitLastFut(GridCacheSharedContext cctx) { - assert false; - } - - /** {@inheritDoc} */ - @Override public IgniteCheckedException validateTopology(GridCacheSharedContext cctx, GridDhtTopologyFuture topFut) { - assert false; - - return null; - } - - /** {@inheritDoc} */ - @Override public boolean sync(GridCacheSharedContext cctx) { - assert false; - - return false; - } - - /** {@inheritDoc} */ - @Override public boolean hasNearCache(GridCacheSharedContext cctx) { - assert false; - - return false; - } - - /** {@inheritDoc} */ - @Override public void addActiveCache(GridCacheContext cacheCtx, IgniteTxLocalAdapter tx) - throws IgniteCheckedException { - assert false; - } - - /** {@inheritDoc} */ - @Override public GridDhtTopologyFuture topologyReadLock(GridCacheSharedContext cctx, GridFutureAdapter fut) { - assert false; - - return null; - } - - /** {@inheritDoc} */ - @Override public void topologyReadUnlock(GridCacheSharedContext cctx) { - assert false; - } - - /** {@inheritDoc} */ - @Override public boolean storeUsed(GridCacheSharedContext cctx) { - return false; - } - - /** {@inheritDoc} */ - @Override public Collection stores(GridCacheSharedContext cctx) { - assert false; - - return null; - } - - /** {@inheritDoc} */ - @Override public void onTxEnd(GridCacheSharedContext cctx, IgniteInternalTx tx, boolean commit) { - assert false; - } - - /** {@inheritDoc} */ @Override public IgniteTxEntry entry(IgniteTxKey key) { IgniteTxEntry e = writeMap == null ? null : writeMap.get(key); @@ -175,9 +95,7 @@ public class IgniteTxRemoteStateImpl implements IgniteTxState { return readMap.isEmpty() && writeMap.isEmpty(); } - /** - * @param e Entry. - */ + /** {@inheritDoc} */ public void setWriteValue(IgniteTxEntry e) { IgniteTxEntry entry = writeMap.get(e.txKey()); @@ -197,33 +115,16 @@ public class IgniteTxRemoteStateImpl implements IgniteTxState { writeMap.put(e.txKey(), e); } } - else { - // Copy values. - entry.value(e.value(), e.hasWriteValue(), e.hasReadValue()); - entry.entryProcessors(e.entryProcessors()); - entry.op(e.op()); - entry.ttl(e.ttl()); - entry.explicitVersion(e.explicitVersion()); - - // Conflict resolution stuff. - entry.conflictVersion(e.conflictVersion()); - entry.conflictExpireTime(e.conflictExpireTime()); - } + else + copyEntry(e, entry); } - /** - * @param key Key. - * @param e Entry. - */ + /** {@inheritDoc} */ public void addWriteEntry(IgniteTxKey key, IgniteTxEntry e) { writeMap.put(key, e); } - /** - * Clears entry from transaction as it never happened. - * - * @param key key to be removed. - */ + /** {@inheritDoc} */ public void clearEntry(IgniteTxKey key) { readMap.remove(key); writeMap.remove(key); @@ -235,28 +136,6 @@ public class IgniteTxRemoteStateImpl implements IgniteTxState { } /** {@inheritDoc} */ - @Override public boolean init(int txSize) { - return false; - } - - /** {@inheritDoc} */ - @Override public boolean initialized() { - assert false; - - return false; - } - - /** {@inheritDoc} */ - @Override public void addEntry(IgniteTxEntry entry) { - assert false; - } - - /** {@inheritDoc} */ - @Override public void seal() { - assert false; - } - - /** {@inheritDoc} */ @Override public IgniteTxEntry singleWrite() { return null; } http://git-wip-us.apache.org/repos/asf/ignite/blob/a03c1650/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxState.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxState.java index 1eece47..cb9d93d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxState.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxState.java @@ -32,57 +32,135 @@ import org.jetbrains.annotations.Nullable; * */ public interface IgniteTxState { + /** + * + * @return + */ public boolean implicitSingle(); + /** + * + * @return + */ @Nullable public Integer firstCacheId(); + /** + * + * @param cctx + */ public void awaitLastFut(GridCacheSharedContext cctx); + /** + * @param cctx Context. + * @param topFut Topology future. + * @return + */ public IgniteCheckedException validateTopology(GridCacheSharedContext cctx, GridDhtTopologyFuture topFut); + /** + * @param cctx Context. + * @return + */ public boolean sync(GridCacheSharedContext cctx); + /** + * @param cctx Context. + * @return + */ public boolean hasNearCache(GridCacheSharedContext cctx); + /** + * @param cacheCtx Ccntext. + * @param tx Transaction. + * @throws IgniteCheckedException If cache check failed. + */ public void addActiveCache(GridCacheContext cacheCtx, IgniteTxLocalAdapter tx) throws IgniteCheckedException; + /** + * @param cctx Context. + * @param fut + * @return + */ public GridDhtTopologyFuture topologyReadLock(GridCacheSharedContext cctx, GridFutureAdapter fut); + /** + * @param cctx Context. + */ public void topologyReadUnlock(GridCacheSharedContext cctx); + /** + * @param cctx Context. + * @return + */ public boolean storeUsed(GridCacheSharedContext cctx); + /** + * @param cctx Context. + * @return + */ public Collection stores(GridCacheSharedContext cctx); + /** + * @param cctx Context. + * @param tx Transaction. + * @param commit Commit flag. + */ public void onTxEnd(GridCacheSharedContext cctx, IgniteInternalTx tx, boolean commit); - public IgniteTxEntry entry(IgniteTxKey key); + /** + * @param key Key. + * @return Entry. + */ + @Nullable public IgniteTxEntry entry(IgniteTxKey key); + /** + * @param key Key. + * @return + */ public boolean hasWriteKey(IgniteTxKey key); + /** + * @return Read entries keys. + */ public Set readSet(); + /** + * @return Write entries keys. + */ public Set writeSet(); + /** + * @return Write entries. + */ public Collection writeEntries(); + /** + * @return Read entries. + */ public Collection readEntries(); + /** + * @return Write entries map. + */ public Map writeMap(); + /** + * @return Read entries map. + */ public Map readMap(); - public boolean empty(); - + /** + * @return All entries. + */ public Collection allEntries(); - public boolean init(int txSize); - - public boolean initialized(); - - public void addEntry(IgniteTxEntry entry); - - public void seal(); - + /** + * @return Non-null entry if tx has only one write entry. + */ @Nullable public IgniteTxEntry singleWrite(); + + /** + * @return {@code True} if transaction is empty. + */ + public boolean empty(); } http://git-wip-us.apache.org/repos/asf/ignite/blob/a03c1650/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateAdapter.java deleted file mode 100644 index 739bae4..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateAdapter.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You 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.apache.ignite.internal.processors.cache.transactions; - -import org.apache.ignite.internal.processors.cache.GridCacheContext; -import org.apache.ignite.internal.util.typedef.internal.U; - -/** - * - */ -public abstract class IgniteTxStateAdapter implements IgniteTxState { - /** - * @param cacheCtx Cache context. - * @param tx Transaction. - * @param commit {@code False} if transaction rolled back. - */ - protected final void onTxEnd(GridCacheContext cacheCtx, IgniteInternalTx tx, boolean commit) { - if (cacheCtx.cache().configuration().isStatisticsEnabled()) { - // Convert start time from ms to ns. - if (commit) - cacheCtx.cache().metrics0().onTxCommit((U.currentTimeMillis() - tx.startTime()) * 1000); - else - cacheCtx.cache().metrics0().onTxRollback((U.currentTimeMillis() - tx.startTime()) * 1000); - } - } -} http://git-wip-us.apache.org/repos/asf/ignite/blob/a03c1650/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java index 54cf953..1e12fe4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java @@ -42,7 +42,7 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; /** * */ -public class IgniteTxStateImpl extends IgniteTxStateAdapter { +public class IgniteTxStateImpl extends IgniteTxLocalStateAdapter { /** Active cache IDs. */ private Set activeCacheIds = new HashSet<>(); /** Per-transaction read map. */