# ignite-42
Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/1fa2a900
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/1fa2a900
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/1fa2a900
Branch: refs/heads/ignite-49
Commit: 1fa2a900e6bc62d276823bbd683a56740a34258e
Parents: f4da39a
Author: sboikov <sboikov@gridgain.com>
Authored: Wed Jan 21 15:03:25 2015 +0300
Committer: sboikov <sboikov@gridgain.com>
Committed: Wed Jan 21 17:27:35 2015 +0300
----------------------------------------------------------------------
.../processors/cache/GridCacheStoreManager.java | 12 +
.../dht/GridDhtTransactionalCacheAdapter.java | 2 +-
.../cache/distributed/near/GridNearTxLocal.java | 4 +-
.../transactions/IgniteTxLocalAdapter.java | 3 +-
...CacheAtomicLocalNoLoadPreviousValueTest.java | 49 +++
...gniteCacheAtomicLocalNoWriteThroughTest.java | 49 +++
...tomicNearEnabledNoLoadPreviousValueTest.java | 32 ++
...acheAtomicNearEnabledNoWriteThroughTest.java | 32 ++
...gniteCacheAtomicNoLoadPreviousValueTest.java | 55 +++
.../IgniteCacheAtomicNoWriteThroughTest.java | 55 +++
...iteCacheNoLoadPreviousValueAbstractTest.java | 249 +++++++++++++
.../IgniteCacheNoReadThroughAbstractTest.java | 34 +-
.../IgniteCacheNoWriteThroughAbstractTest.java | 345 +++++++++++++++++++
...niteCacheTxLocalNoLoadPreviousValueTest.java | 49 +++
.../IgniteCacheTxLocalNoWriteThroughTest.java | 49 +++
...cheTxNearEnabledNoLoadPreviousValueTest.java | 32 ++
...iteCacheTxNearEnabledNoWriteThroughTest.java | 32 ++
.../IgniteCacheTxNoLoadPreviousValueTest.java | 49 +++
.../IgniteCacheTxNoWriteThroughTest.java | 49 +++
.../near/GridCacheNearMultiNodeSelfTest.java | 1 +
.../bamboo/GridDataGridTestSuite.java | 15 +
.../GridSpringDynamicCacheManagerSelfTest.java | 3 +-
22 files changed, 1193 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheStoreManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheStoreManager.java
index 7d92f83..dddede9 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheStoreManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheStoreManager.java
@@ -62,6 +62,9 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
private final boolean locStore;
/** */
+ private final boolean writeThrough;
+
+ /** */
private boolean convertPortable;
/**
@@ -85,6 +88,8 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
ThreadLocal<SessionData> sesHolder0 = null;
+ writeThrough = cfg.isWriteThrough();
+
if (cfgStore != null) {
try {
if (!sesHolders.containsKey(cfgStore)) {
@@ -114,6 +119,13 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
}
/**
+ * @return {@code True} is write-through is enabled.
+ */
+ public boolean writeThrough() {
+ return writeThrough;
+ }
+
+ /**
* @return Unwrapped store provided in configuration.
*/
public CacheStore<?, ?> configuredStore() {
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index dfa8ad8..e911750 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@ -999,7 +999,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
if (ret)
val = e.innerGet(tx,
/*swap*/true,
- /*read-through*/true,
+ /*read-through*/ctx.loadPreviousValue(),
/*fail-fast.*/false,
/*unmarshal*/false,
/*update-metrics*/true,
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java
index 442293b..ba6f333 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -98,8 +98,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
* @param concurrency Concurrency.
* @param isolation Isolation.
* @param timeout Timeout.
- * @param invalidate
- * @param storeEnabled
+ * @param invalidate Invalidate flag.
+ * @param storeEnabled Store enabled flag.
* @param txSize Transaction size.
* @param grpLockKey Group lock key if this is a group lock transaction.
* @param partLock {@code True} if this is a group-lock transaction and the whole partition should be locked.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 06ee6b0..ea25015 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -465,7 +465,8 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
protected void batchStoreCommit(Iterable<IgniteTxEntry<K, V>> writeEntries) throws IgniteCheckedException {
GridCacheStoreManager<K, V> store = store();
- if (store != null && storeEnabled() && (!internal() || groupLock()) && (near() || store.writeToStoreFromDht())) {
+ if (store != null && store.writeThrough() && storeEnabled() &&
+ (!internal() || groupLock()) && (near() || store.writeToStoreFromDht())) {
try {
if (writeEntries != null) {
Map<K, IgniteBiTuple<V, GridCacheVersion>> putMap = null;
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicLocalNoLoadPreviousValueTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicLocalNoLoadPreviousValueTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicLocalNoLoadPreviousValueTest.java
new file mode 100644
index 0000000..194f474
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicLocalNoLoadPreviousValueTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.integration;
+
+import org.gridgain.grid.cache.*;
+
+import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
+import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheAtomicLocalNoLoadPreviousValueTest extends IgniteCacheNoLoadPreviousValueAbstractTest {
+ /** {@inheritDoc} */
+ @Override protected int gridCount() {
+ return 1;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheMode cacheMode() {
+ return LOCAL;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheAtomicityMode atomicityMode() {
+ return ATOMIC;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheDistributionMode distributionMode() {
+ return PARTITIONED_ONLY;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicLocalNoWriteThroughTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicLocalNoWriteThroughTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicLocalNoWriteThroughTest.java
new file mode 100644
index 0000000..b6f0312
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicLocalNoWriteThroughTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.integration;
+
+import org.gridgain.grid.cache.*;
+
+import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
+import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheAtomicLocalNoWriteThroughTest extends IgniteCacheNoWriteThroughAbstractTest {
+ /** {@inheritDoc} */
+ @Override protected int gridCount() {
+ return 1;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheMode cacheMode() {
+ return LOCAL;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheAtomicityMode atomicityMode() {
+ return ATOMIC;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheDistributionMode distributionMode() {
+ return PARTITIONED_ONLY;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicNearEnabledNoLoadPreviousValueTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicNearEnabledNoLoadPreviousValueTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicNearEnabledNoLoadPreviousValueTest.java
new file mode 100644
index 0000000..6d2879f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicNearEnabledNoLoadPreviousValueTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.integration;
+
+import org.gridgain.grid.cache.*;
+
+import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheAtomicNearEnabledNoLoadPreviousValueTest extends IgniteCacheAtomicNoLoadPreviousValueTest {
+ /** {@inheritDoc} */
+ @Override protected GridCacheDistributionMode distributionMode() {
+ return NEAR_PARTITIONED;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicNearEnabledNoWriteThroughTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicNearEnabledNoWriteThroughTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicNearEnabledNoWriteThroughTest.java
new file mode 100644
index 0000000..d4eb3dd
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicNearEnabledNoWriteThroughTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.integration;
+
+import org.gridgain.grid.cache.*;
+
+import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheAtomicNearEnabledNoWriteThroughTest extends IgniteCacheAtomicNoWriteThroughTest {
+ /** {@inheritDoc} */
+ @Override protected GridCacheDistributionMode distributionMode() {
+ return NEAR_PARTITIONED;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicNoLoadPreviousValueTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicNoLoadPreviousValueTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicNoLoadPreviousValueTest.java
new file mode 100644
index 0000000..bb11d01
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicNoLoadPreviousValueTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.integration;
+
+import org.gridgain.grid.cache.*;
+
+import static org.gridgain.grid.cache.GridCacheAtomicWriteOrderMode.*;
+import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
+import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheAtomicNoLoadPreviousValueTest extends IgniteCacheNoLoadPreviousValueAbstractTest {
+ /** {@inheritDoc} */
+ @Override protected int gridCount() {
+ return 3;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheMode cacheMode() {
+ return PARTITIONED;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheAtomicityMode atomicityMode() {
+ return ATOMIC;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheDistributionMode distributionMode() {
+ return PARTITIONED_ONLY;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheAtomicWriteOrderMode atomicWriteOrderMode() {
+ return PRIMARY;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicNoWriteThroughTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicNoWriteThroughTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicNoWriteThroughTest.java
new file mode 100644
index 0000000..4bfd623
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicNoWriteThroughTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.integration;
+
+import org.gridgain.grid.cache.*;
+
+import static org.gridgain.grid.cache.GridCacheAtomicWriteOrderMode.*;
+import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
+import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheAtomicNoWriteThroughTest extends IgniteCacheNoWriteThroughAbstractTest {
+ /** {@inheritDoc} */
+ @Override protected int gridCount() {
+ return 3;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheMode cacheMode() {
+ return PARTITIONED;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheAtomicityMode atomicityMode() {
+ return ATOMIC;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheDistributionMode distributionMode() {
+ return PARTITIONED_ONLY;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheAtomicWriteOrderMode atomicWriteOrderMode() {
+ return PRIMARY;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheNoLoadPreviousValueAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheNoLoadPreviousValueAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheNoLoadPreviousValueAbstractTest.java
new file mode 100644
index 0000000..a5b8414
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheNoLoadPreviousValueAbstractTest.java
@@ -0,0 +1,249 @@
+/*
+ * 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.integration;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.transactions.*;
+import org.gridgain.grid.cache.*;
+
+import javax.cache.processor.*;
+import java.util.*;
+
+import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
+
+/**
+ * Test for configuration property {@link CacheConfiguration#isLoadPreviousValue()}.
+ */
+public abstract class IgniteCacheNoLoadPreviousValueAbstractTest extends IgniteCacheAbstractTest {
+ /** */
+ private Integer lastKey = 0;
+
+ /** {@inheritDoc} */
+ @Override protected CacheStore<?, ?> cacheStore() {
+ return new TestStore();
+ }
+
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+ cfg.getTransactionsConfiguration().setTxSerializableEnabled(true);
+
+ return cfg;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+ CacheConfiguration ccfg = super.cacheConfiguration(gridName);
+
+ ccfg.setReadThrough(true);
+
+ ccfg.setWriteThrough(true);
+
+ ccfg.setLoadPreviousValue(false);
+
+ return ccfg;
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testNoLoadPreviousValue() throws Exception {
+ IgniteCache<Integer, Integer> cache = jcache(0);
+
+ for (Integer key : keys()) {
+ log.info("Test [key=" + key + ']');
+
+ storeMap.put(key, key);
+
+ assertEquals(key, cache.get(key));
+
+ assertEquals(key, storeMap.get(key));
+
+ cache.remove(key);
+
+ assertNull(storeMap.get(key));
+
+ storeMap.put(key, key);
+
+ assertNull(cache.getAndPut(key, -1));
+
+ assertEquals(-1, storeMap.get(key));
+
+ cache.remove(key);
+
+ assertNull(storeMap.get(key));
+
+ storeMap.put(key, key);
+
+ assertTrue(cache.putIfAbsent(key, -1));
+
+ assertEquals(-1, storeMap.get(key));
+
+ cache.remove(key);
+
+ assertNull(storeMap.get(key));
+
+ storeMap.put(key, key);
+
+ assertNull(cache.getAndRemove(key));
+
+ assertNull(storeMap.get(key));
+
+ storeMap.put(key, key);
+
+ assertNull(cache.getAndPutIfAbsent(key, -1));
+
+ assertEquals(-1, storeMap.get(key));
+
+ cache.remove(key);
+
+ assertNull(storeMap.get(key));
+
+ storeMap.put(key, key);
+
+ Object ret = cache.invoke(key, new EntryProcessor<Integer, Integer, Object>() {
+ @Override public Object process(MutableEntry<Integer, Integer> e, Object... args) {
+ Integer val = e.getValue();
+
+ assertTrue(e.exists());
+
+ e.setValue(-1);
+
+ return String.valueOf(val);
+ }
+ });
+
+ assertEquals(String.valueOf(key), ret);
+
+ assertEquals(-1, storeMap.get(key));
+
+ cache.remove(key);
+
+ assertNull(storeMap.get(key));
+
+ storeMap.put(key, key);
+
+ assertFalse(cache.replace(key, -1));
+
+ assertEquals(key, storeMap.get(key));
+
+ assertNull(cache.getAndReplace(key, -1));
+
+ assertEquals(key, storeMap.get(key));
+
+ assertFalse(cache.replace(key, key, -1));
+
+ assertEquals(key, storeMap.get(key));
+ }
+
+ Map<Integer, Integer> expData = new HashMap<>();
+
+ for (int i = 1000_0000; i < 1000_0000 + 1000; i++) {
+ storeMap.put(i, i);
+
+ expData.put(i, i);
+ }
+
+ assertEquals(expData, cache.getAll(expData.keySet()));
+
+ if (atomicityMode() == TRANSACTIONAL) {
+ for (IgniteTxConcurrency concurrency : IgniteTxConcurrency.values()) {
+ for (IgniteTxIsolation isolation : IgniteTxIsolation.values()) {
+ for (Integer key : keys()) {
+ log.info("Test tx [key=" + key +
+ ", concurrency=" + concurrency +
+ ", isolation=" + isolation + ']');
+
+ storeMap.put(key, key);
+
+ try (IgniteTx tx = ignite(0).transactions().txStart(concurrency, isolation)) {
+ assertEquals(key, cache.get(key));
+
+ tx.commit();
+ }
+
+ assertEquals(key, storeMap.get(key));
+
+ cache.remove(key);
+
+ assertNull(storeMap.get(key));
+
+ storeMap.put(key, key);
+
+ try (IgniteTx tx = ignite(0).transactions().txStart(concurrency, isolation)) {
+ assertNull(cache.getAndPut(key, -1));
+
+ tx.commit();
+ }
+
+ assertEquals(-1, storeMap.get(key));
+
+ cache.remove(key);
+
+ assertNull(storeMap.get(key));
+
+ storeMap.put(key, key);
+
+ try (IgniteTx tx = ignite(0).transactions().txStart(concurrency, isolation)) {
+ assertTrue(cache.putIfAbsent(key, -1));
+
+ tx.commit();
+ }
+
+ assertEquals(-1, storeMap.get(key));
+
+ try (IgniteTx tx = ignite(0).transactions().txStart(concurrency, isolation)) {
+ assertEquals(expData, cache.getAll(expData.keySet()));
+
+ tx.commit();
+ }
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * @return Test keys.
+ * @throws Exception If failed.
+ */
+ protected Collection<Integer> keys() throws Exception {
+ GridCache<Integer, Object> cache = cache(0);
+
+ ArrayList<Integer> keys = new ArrayList<>();
+
+ keys.add(primaryKeys(cache, 1, lastKey).get(0));
+
+ if (gridCount() > 1) {
+ keys.add(backupKeys(cache, 1, lastKey).get(0));
+
+ if (cache.configuration().getCacheMode() != REPLICATED)
+ keys.add(nearKeys(cache, 1, lastKey).get(0));
+ }
+
+ lastKey = Collections.max(keys) + 1;
+
+ return keys;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheNoReadThroughAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheNoReadThroughAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheNoReadThroughAbstractTest.java
index 7087934..fa0b148 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheNoReadThroughAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheNoReadThroughAbstractTest.java
@@ -35,7 +35,7 @@ import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
import static org.gridgain.grid.cache.GridCacheMode.*;
/**
- *
+ * Test for configuration property {@link CacheConfiguration#isReadThrough}.
*/
public abstract class IgniteCacheNoReadThroughAbstractTest extends IgniteCacheAbstractTest {
/** */
@@ -87,6 +87,8 @@ public abstract class IgniteCacheNoReadThroughAbstractTest extends IgniteCacheAb
ccfg.setWriteThrough(true);
+ ccfg.setLoadPreviousValue(true);
+
return ccfg;
}
@@ -228,6 +230,34 @@ public abstract class IgniteCacheNoReadThroughAbstractTest extends IgniteCacheAb
assertEquals(-1, storeMap.get(key));
+ cache.remove(key);
+
+ assertNull(storeMap.get(key));
+
+ storeMap.put(key, key);
+
+ try (IgniteTx tx = ignite(0).transactions().txStart(concurrency, isolation)) {
+ Object ret = cache.invoke(key, new EntryProcessor<Integer, Integer, Object>() {
+ @Override public Object process(MutableEntry<Integer, Integer> e, Object... args) {
+ Integer val = e.getValue();
+
+ assertFalse(e.exists());
+
+ assertNull(val);
+
+ e.setValue(-1);
+
+ return String.valueOf(val);
+ }
+ });
+
+ assertEquals("null", ret);
+
+ tx.commit();
+ }
+
+ assertEquals(-1, storeMap.get(key));
+
try (IgniteTx tx = ignite(0).transactions().txStart(concurrency, isolation)) {
assertTrue(cache.getAll(keys).isEmpty());
@@ -238,7 +268,7 @@ public abstract class IgniteCacheNoReadThroughAbstractTest extends IgniteCacheAb
}
}
- // Check
+ // Check can load cache when read-through is disabled.
allowLoad = true;
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheNoWriteThroughAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheNoWriteThroughAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheNoWriteThroughAbstractTest.java
new file mode 100644
index 0000000..4c0f151
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheNoWriteThroughAbstractTest.java
@@ -0,0 +1,345 @@
+/*
+ * 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.integration;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.CacheConfiguration;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.transactions.*;
+import org.gridgain.grid.cache.*;
+
+import javax.cache.processor.*;
+import java.util.*;
+
+import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
+
+/**
+ * Test for configuration property {@link CacheConfiguration#isWriteThrough}.
+ */
+public abstract class IgniteCacheNoWriteThroughAbstractTest extends IgniteCacheAbstractTest {
+ /** */
+ private Integer lastKey = 0;
+
+ /** {@inheritDoc} */
+ @Override protected CacheStore<?, ?> cacheStore() {
+ return new TestStore();
+ }
+
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+ cfg.getTransactionsConfiguration().setTxSerializableEnabled(true);
+
+ return cfg;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+ CacheConfiguration ccfg = super.cacheConfiguration(gridName);
+
+ ccfg.setReadThrough(true);
+
+ ccfg.setWriteThrough(false);
+
+ ccfg.setLoadPreviousValue(true);
+
+ return ccfg;
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ @SuppressWarnings("UnnecessaryLocalVariable")
+ public void testNoWriteThrough() throws Exception {
+ IgniteCache<Integer, Integer> cache = jcache(0);
+
+ for (Integer key : keys()) {
+ log.info("Test [key=" + key + ']');
+
+ final Integer storeVal = key;
+
+ storeMap.put(key, storeVal );
+
+ assertEquals(key, cache.get(key));
+
+ cache.remove(key);
+
+ assertEquals(storeVal, storeMap.get(key));
+
+ storeMap.remove(key);
+
+ assertNull(cache.get(key));
+
+ assertTrue(cache.putIfAbsent(key, key));
+
+ assertNull(storeMap.get(key));
+
+ assertEquals(key, cache.get(key));
+
+ cache.remove(key);
+
+ storeMap.put(key, storeVal);
+
+ Integer val = key + 1;
+
+ assertFalse(cache.putIfAbsent(key, val));
+
+ assertEquals(storeVal, storeMap.get(key));
+
+ cache.put(key, val);
+
+ assertEquals(val, cache.get(key));
+
+ assertEquals(storeVal, storeMap.get(key));
+
+ val = val + 1;
+
+ assertTrue(cache.replace(key, val));
+
+ assertEquals(val, cache.get(key));
+
+ assertEquals(storeVal, storeMap.get(key));
+
+ cache.remove(key);
+
+ assertEquals(storeVal, storeMap.get(key));
+
+ storeMap.remove(key);
+
+ assertNull(cache.get(key));
+
+ storeMap.put(key, storeVal);
+
+ val = val + 1;
+
+ assertEquals(storeVal, cache.getAndPut(key, val));
+
+ assertEquals(storeVal, storeMap.get(key));
+
+ assertEquals(val, cache.get(key));
+
+ cache.remove(key);
+
+ assertEquals(storeVal, storeMap.get(key));
+
+ assertEquals(storeVal, cache.getAndRemove(key));
+
+ cache.remove(key);
+
+ assertEquals(storeVal, storeMap.get(key));
+
+ Object ret = cache.invoke(key, new EntryProcessor<Integer, Integer, Object>() {
+ @Override public Object process(MutableEntry<Integer, Integer> entry, Object... args) {
+ Integer val = entry.getValue();
+
+ entry.setValue(val + 1);
+
+ return String.valueOf(val);
+ }
+ });
+
+ assertEquals(String.valueOf(storeVal), ret);
+
+ assertEquals(storeVal + 1, (int)cache.get(key));
+
+ assertEquals(storeVal, storeMap.get(key));
+
+ assertTrue(cache.replace(key, storeVal + 1, storeVal + 2));
+
+ assertEquals(storeVal, storeMap.get(key));
+
+ assertEquals(storeVal + 2, (int) cache.get(key));
+ }
+
+ Map<Integer, Integer> expData = new HashMap<>();
+
+ for (int i = 1000_0000; i < 1000_0000 + 1000; i++) {
+ storeMap.put(i, i);
+
+ expData.put(i, i);
+ }
+
+ assertEquals(expData, cache.getAll(expData.keySet()));
+
+ storeMap.clear();
+
+ cache.putAll(expData);
+
+ assertTrue(storeMap.isEmpty());
+
+ assertEquals(expData, cache.getAll(expData.keySet()));
+
+ Map<Integer, Integer> expData0 = new HashMap<>();
+
+ for (int i = 1000_0000; i < 1000_0000 + 1000; i++)
+ expData0.put(i, 1);
+
+ cache.invokeAll(expData.keySet(), new EntryProcessor<Integer, Integer, Object>() {
+ @Override public Object process(MutableEntry<Integer, Integer> entry, Object... args) {
+ entry.setValue(1);
+
+ return null;
+ }
+ });
+
+ assertEquals(expData0, cache.getAll(expData0.keySet()));
+
+ assertTrue(storeMap.isEmpty());
+
+ storeMap.putAll(expData);
+
+ cache.removeAll(expData.keySet());
+
+ assertEquals(1000, storeMap.size());
+
+ storeMap.clear();
+
+ assertTrue(cache.getAll(expData.keySet()).isEmpty());
+
+ if (atomicityMode() == TRANSACTIONAL) {
+ for (IgniteTxConcurrency concurrency : IgniteTxConcurrency.values()) {
+ for (IgniteTxIsolation isolation : IgniteTxIsolation.values()) {
+ for (Integer key : keys()) {
+ log.info("Test tx [key=" + key +
+ ", concurrency=" + concurrency +
+ ", isolation=" + isolation + ']');
+
+ storeMap.put(key, key);
+
+ try (IgniteTx tx = ignite(0).transactions().txStart(concurrency, isolation)) {
+ assertEquals(key, cache.getAndPut(key, -1));
+
+ tx.commit();
+ }
+
+ assertEquals(-1, (int)cache.get(key));
+
+ assertEquals(key, storeMap.get(key));
+
+ try (IgniteTx tx = ignite(0).transactions().txStart(concurrency, isolation)) {
+ cache.put(key, -2);
+
+ tx.commit();
+ }
+
+ assertEquals(-2, (int)cache.get(key));
+
+ assertEquals(key, storeMap.get(key));
+
+ try (IgniteTx tx = ignite(0).transactions().txStart(concurrency, isolation)) {
+ assertEquals(-2, (int)cache.getAndRemove(key));
+
+ tx.commit();
+ }
+
+ assertEquals(key, storeMap.get(key));
+
+ storeMap.remove(key);
+
+ assertNull(cache.get(key));
+
+ storeMap.put(key, key);
+
+ cache.put(key, key);
+
+ try (IgniteTx tx = ignite(0).transactions().txStart(concurrency, isolation)) {
+ assertTrue(cache.replace(key, -1));
+
+ tx.commit();
+ }
+
+ assertEquals(-1, (int)cache.get(key));
+
+ assertEquals(key, storeMap.get(key));
+
+ cache.remove(key);
+
+ storeMap.clear();
+
+ try (IgniteTx tx = ignite(0).transactions().txStart(concurrency, isolation)) {
+ cache.putAll(expData);
+
+ tx.commit();
+ }
+
+ assertTrue(storeMap.isEmpty());
+
+ assertEquals(expData, cache.getAll(expData.keySet()));
+
+ try (IgniteTx tx = ignite(0).transactions().txStart(concurrency, isolation)) {
+ cache.invokeAll(expData.keySet(), new EntryProcessor<Integer, Integer, Object>() {
+ @Override public Object process(MutableEntry<Integer, Integer> entry, Object... args) {
+ entry.setValue(1);
+
+ return null;
+ }
+ });
+
+ tx.commit();
+ }
+
+ assertEquals(expData0, cache.getAll(expData.keySet()));
+
+ assertTrue(storeMap.isEmpty());
+
+ storeMap.putAll(expData);
+
+ try (IgniteTx tx = ignite(0).transactions().txStart(concurrency, isolation)) {
+ cache.removeAll(expData.keySet());
+
+ tx.commit();
+ }
+
+ assertEquals(1000, storeMap.size());
+
+ storeMap.clear();
+
+ assertTrue(cache.getAll(expData.keySet()).isEmpty());
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * @return Test keys.
+ * @throws Exception If failed.
+ */
+ protected Collection<Integer> keys() throws Exception {
+ GridCache<Integer, Object> cache = cache(0);
+
+ ArrayList<Integer> keys = new ArrayList<>();
+
+ keys.add(primaryKeys(cache, 1, lastKey).get(0));
+
+ if (gridCount() > 1) {
+ keys.add(backupKeys(cache, 1, lastKey).get(0));
+
+ if (cache.configuration().getCacheMode() != REPLICATED)
+ keys.add(nearKeys(cache, 1, lastKey).get(0));
+ }
+
+ lastKey = Collections.max(keys) + 1;
+
+ return keys;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxLocalNoLoadPreviousValueTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxLocalNoLoadPreviousValueTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxLocalNoLoadPreviousValueTest.java
new file mode 100644
index 0000000..c3b45e3
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxLocalNoLoadPreviousValueTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.integration;
+
+import org.gridgain.grid.cache.*;
+
+import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
+import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheTxLocalNoLoadPreviousValueTest extends IgniteCacheNoLoadPreviousValueAbstractTest {
+ /** {@inheritDoc} */
+ @Override protected int gridCount() {
+ return 1;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheMode cacheMode() {
+ return LOCAL;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheAtomicityMode atomicityMode() {
+ return TRANSACTIONAL;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheDistributionMode distributionMode() {
+ return PARTITIONED_ONLY;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxLocalNoWriteThroughTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxLocalNoWriteThroughTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxLocalNoWriteThroughTest.java
new file mode 100644
index 0000000..b426b75
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxLocalNoWriteThroughTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.integration;
+
+import org.gridgain.grid.cache.*;
+
+import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
+import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheTxLocalNoWriteThroughTest extends IgniteCacheNoWriteThroughAbstractTest {
+ /** {@inheritDoc} */
+ @Override protected int gridCount() {
+ return 1;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheMode cacheMode() {
+ return LOCAL;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheAtomicityMode atomicityMode() {
+ return TRANSACTIONAL;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheDistributionMode distributionMode() {
+ return PARTITIONED_ONLY;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxNearEnabledNoLoadPreviousValueTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxNearEnabledNoLoadPreviousValueTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxNearEnabledNoLoadPreviousValueTest.java
new file mode 100644
index 0000000..b2019c5
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxNearEnabledNoLoadPreviousValueTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.integration;
+
+import org.gridgain.grid.cache.*;
+
+import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheTxNearEnabledNoLoadPreviousValueTest extends IgniteCacheTxNoLoadPreviousValueTest {
+ /** {@inheritDoc} */
+ @Override protected GridCacheDistributionMode distributionMode() {
+ return NEAR_PARTITIONED;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxNearEnabledNoWriteThroughTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxNearEnabledNoWriteThroughTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxNearEnabledNoWriteThroughTest.java
new file mode 100644
index 0000000..7ee2ef0
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxNearEnabledNoWriteThroughTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.integration;
+
+import org.gridgain.grid.cache.*;
+
+import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheTxNearEnabledNoWriteThroughTest extends IgniteCacheTxNoWriteThroughTest {
+ /** {@inheritDoc} */
+ @Override protected GridCacheDistributionMode distributionMode() {
+ return NEAR_PARTITIONED;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxNoLoadPreviousValueTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxNoLoadPreviousValueTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxNoLoadPreviousValueTest.java
new file mode 100644
index 0000000..7c6ee23
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxNoLoadPreviousValueTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.integration;
+
+import org.gridgain.grid.cache.*;
+
+import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
+import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheTxNoLoadPreviousValueTest extends IgniteCacheNoLoadPreviousValueAbstractTest {
+ /** {@inheritDoc} */
+ @Override protected int gridCount() {
+ return 3;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheMode cacheMode() {
+ return PARTITIONED;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheAtomicityMode atomicityMode() {
+ return TRANSACTIONAL;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheDistributionMode distributionMode() {
+ return PARTITIONED_ONLY;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxNoWriteThroughTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxNoWriteThroughTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxNoWriteThroughTest.java
new file mode 100644
index 0000000..7c781e3
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxNoWriteThroughTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.integration;
+
+import org.gridgain.grid.cache.*;
+
+import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
+import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheTxNoWriteThroughTest extends IgniteCacheNoWriteThroughAbstractTest {
+ /** {@inheritDoc} */
+ @Override protected int gridCount() {
+ return 3;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheMode cacheMode() {
+ return PARTITIONED;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheAtomicityMode atomicityMode() {
+ return TRANSACTIONAL;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected GridCacheDistributionMode distributionMode() {
+ return PARTITIONED_ONLY;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
index 3688772..a8e686b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
@@ -99,6 +99,7 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
cacheCfg.setCacheStoreFactory(new FactoryBuilder.SingletonFactory(store));
cacheCfg.setReadThrough(true);
cacheCfg.setWriteThrough(true);
+ cacheCfg.setLoadPreviousValue(true);
cacheCfg.setWriteSynchronizationMode(GridCacheWriteSynchronizationMode.FULL_SYNC);
cacheCfg.setAffinity(aff);
cacheCfg.setAtomicityMode(atomicityMode());
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java b/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
index 31d7449..5ccd297 100644
--- a/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
+++ b/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
@@ -353,6 +353,21 @@ public class GridDataGridTestSuite extends TestSuite {
suite.addTestSuite(IgniteCacheTxNearEnabledNoReadThroughTest.class);
suite.addTestSuite(IgniteCacheTxLocalNoReadThroughTest.class);
+ suite.addTestSuite(IgniteCacheAtomicNoLoadPreviousValueTest.class);
+ suite.addTestSuite(IgniteCacheAtomicNearEnabledNoLoadPreviousValueTest.class);
+ suite.addTestSuite(IgniteCacheAtomicLocalNoLoadPreviousValueTest.class);
+ // TODO IGNITE-42.
+ //suite.addTestSuite(IgniteCacheTxNoLoadPreviousValueTest.class);
+ //suite.addTestSuite(IgniteCacheTxNearEnabledNoLoadPreviousValueTest.class);
+ //suite.addTestSuite(IgniteCacheTxLocalNoLoadPreviousValueTest.class);
+
+ suite.addTestSuite(IgniteCacheAtomicNoWriteThroughTest.class);
+ suite.addTestSuite(IgniteCacheAtomicNearEnabledNoWriteThroughTest.class);
+ suite.addTestSuite(IgniteCacheAtomicLocalNoWriteThroughTest.class);
+ suite.addTestSuite(IgniteCacheTxNoWriteThroughTest.class);
+ suite.addTestSuite(IgniteCacheTxNearEnabledNoWriteThroughTest.class);
+ suite.addTestSuite(IgniteCacheTxLocalNoWriteThroughTest.class);
+
return suite;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fa2a900/modules/spring/src/test/java/org/gridgain/grid/cache/spring/GridSpringDynamicCacheManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/gridgain/grid/cache/spring/GridSpringDynamicCacheManagerSelfTest.java b/modules/spring/src/test/java/org/gridgain/grid/cache/spring/GridSpringDynamicCacheManagerSelfTest.java
index 29dfd45..37046a5 100644
--- a/modules/spring/src/test/java/org/gridgain/grid/cache/spring/GridSpringDynamicCacheManagerSelfTest.java
+++ b/modules/spring/src/test/java/org/gridgain/grid/cache/spring/GridSpringDynamicCacheManagerSelfTest.java
@@ -17,6 +17,7 @@
package org.gridgain.grid.cache.spring;
+import org.apache.ignite.cache.*;
import org.apache.ignite.configuration.*;
import org.apache.ignite.spi.discovery.tcp.*;
import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -50,7 +51,7 @@ public class GridSpringDynamicCacheManagerSelfTest extends GridCommonAbstractTes
@Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
IgniteConfiguration cfg = super.getConfiguration(gridName);
- GridCacheConfiguration cache = new GridCacheConfiguration();
+ CacheConfiguration cache = new CacheConfiguration();
cache.setName(DATA_CACHE_NAME);
|