http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/CacheFlag.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheFlag.java 
b/modules/core/src/main/java/org/apache/ignite/cache/CacheFlag.java
new file mode 100644
index 0000000..ed74b7d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheFlag.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.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.transactions.*;
+import org.jetbrains.annotations.*;
+
+import javax.cache.processor.*;
+
+/**
+ * Cache projection flags that specify projection behaviour. This flags can be 
explicitly passed into
+ * the following methods on {@link CacheProjection}:
+ * <ul>
+ * <li>{@link CacheProjection#flagsOn(CacheFlag...)}</li>
+ * <li>{@link CacheProjection#flagsOff(CacheFlag...)}</li>
+ * </ul>
+ * Also, some flags, like {@link #LOCAL}, or {@link #READ} may be implicitly 
set whenever
+ * creating new projections and passing entries to predicate filters.
+ */
+public enum CacheFlag {
+    /**
+     * Only operations that don't require any communication with
+     * other cache nodes are allowed. This flag is automatically set
+     * on underlying projection for all the entries that are given to
+     * predicate filters to make sure that no distribution happens
+     * from inside of predicate evaluation.
+     */
+    LOCAL,
+
+    /**
+     * Only operations that don't change cached data are allowed.
+     * This flag is automatically set on underlying projection for
+     * all the entries that are given to predicate filters to make
+     * sure that data cannot be updated during predicate evaluation.
+     */
+    READ,
+
+    /**
+     * Clone values prior to returning them to user.
+     * <p>
+     * Whenever values are returned from cache, they cannot be directly updated
+     * as cache holds the same references internally. If it is needed to
+     * update values that are returned from cache, this flag will provide
+     * automatic cloning of values prior to returning so they can be directly
+     * updated.
+     *
+     * @see CacheConfiguration#getCloner()
+     */
+    CLONE,
+
+    /** Skips store, i.e. no read-through and no write-through behavior. */
+    SKIP_STORE,
+
+    /** Skip swap space for reads and writes. */
+    SKIP_SWAP,
+
+    /** Synchronous commit. */
+    SYNC_COMMIT,
+
+    /**
+     * Switches a cache projection to work in {@code 'invalidation'} mode.
+     * Instead of updating remote entries with new values, small invalidation
+     * messages will be sent to set the values to {@code null}.
+     *
+     * @see IgniteTx#isInvalidate()
+     * @see CacheConfiguration#isInvalidate()
+     */
+    INVALIDATE,
+
+    /**
+     * Skips version check during {@link IgniteCache#invoke(Object, 
EntryProcessor, Object[])} writes in
+     * {@link CacheAtomicityMode#ATOMIC} mode. By default, in {@code ATOMIC} 
mode, whenever
+     * {@code transform(...)} is called, cache values (and not the {@code 
transform} closure) are sent from primary
+     * node to backup nodes to ensure proper update ordering.
+     * <p>
+     * By setting this flag, version check is skipped, and the {@code 
transform} closure is applied on both, primary
+     * and backup nodes. Use this flag for better performance if you are sure 
that there are no
+     * concurrent updates happening for the same key when {@code 
transform(...)} method is called.
+     */
+    FORCE_TRANSFORM_BACKUP;
+
+    /** */
+    private static final CacheFlag[] VALS = values();
+
+    /**
+     * Efficiently gets enumerated value from its ordinal.
+     *
+     * @param ord Ordinal value.
+     * @return Enumerated value or {@code null} if ordinal out of range.
+     */
+    @Nullable public static CacheFlag fromOrdinal(int ord) {
+        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/CacheFlagException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/CacheFlagException.java 
b/modules/core/src/main/java/org/apache/ignite/cache/CacheFlagException.java
new file mode 100644
index 0000000..014d4dd
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheFlagException.java
@@ -0,0 +1,67 @@
+/*
+ * 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.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Exception thrown when projection flags check fails.
+ */
+public class CacheFlagException extends IgniteException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Flags that caused this exception. */
+    private Collection<CacheFlag> flags;
+
+    /**
+     * @param flags Cause flags.
+     */
+    public CacheFlagException(@Nullable CacheFlag... flags) {
+        this(F.asList(flags));
+    }
+
+    /**
+     * @param flags Cause flags.
+     */
+    public CacheFlagException(@Nullable Collection<CacheFlag> flags) {
+        super(message(flags));
+
+        this.flags = flags;
+    }
+
+    /**
+     * @return Cause flags.
+     */
+    public Collection<CacheFlag> flags() {
+        return flags;
+    }
+
+    /**
+     * @param flags Flags.
+     * @return String information about cause flags.
+     */
+    private static String message(Collection<CacheFlag> flags) {
+        return "Cache projection flag violation (if flag is LOCAL, make sure 
to use peek(..) " +
+            "instead of get(..) methods)" + (F.isEmpty(flags) ? "." : " 
[flags=" + flags + ']');
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/CacheInterceptor.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/CacheInterceptor.java 
b/modules/core/src/main/java/org/apache/ignite/cache/CacheInterceptor.java
new file mode 100644
index 0000000..3af2cac
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheInterceptor.java
@@ -0,0 +1,120 @@
+/*
+ * 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.cache;
+
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * Cache interceptor. Cache interceptor can be used for getting callbacks 
before
+ * and after cache {@code get(...)}, {@code put(...)}, and {@code remove(...)}
+ * operations. The {@code onBefore} callbacks can also be used to change the 
values
+ * stored in cache or preventing entries from being removed from cache.
+ * <p>
+ * Cache interceptor is configured via {@link 
CacheConfiguration#getInterceptor()}
+ * configuration property.
+ * <p>
+ * Any grid resource from {@code org.apache.ignite.resources} package can be 
injected
+ * into implementation of this interface.
+ */
+public interface CacheInterceptor<K, V> {
+    /**
+     * This method is called within {@link CacheProjection#get(Object)}
+     * and similar operations to provide control over returned value.
+     * <p>
+     * If this method returns {@code null}, then {@code get()} operation
+     * results in {@code null} as well.
+     * <p>
+     * This method should not throw any exception.
+     *
+     * @param key Key.
+     * @param val Value mapped to {@code key} at the moment of {@code get()} 
operation.
+     * @return The new value to be returned as result of {@code get()} 
operation.
+     * @see CacheProjection#get(Object)
+     */
+    @Nullable public V onGet(K key, @Nullable V val);
+
+    /**
+     * This method is called within {@link CacheProjection#put(Object, Object, 
IgnitePredicate[])}
+     * and similar operations before new value is stored in cache.
+     * <p>
+     * Implementations should not execute any complex logic,
+     * including locking, networking or cache operations,
+     * as it may lead to deadlock, since this method is called
+     * from sensitive synchronization blocks.
+     * <p>
+     * This method should not throw any exception.
+     *
+     * @param key Key.
+     * @param oldVal Old value.
+     * @param newVal New value.
+     * @return Value to be put to cache. Returning {@code null} cancels the 
update.
+     * @see CacheProjection#put(Object, Object, IgnitePredicate[])
+     */
+    @Nullable public V onBeforePut(K key, @Nullable V oldVal, V newVal);
+
+    /**
+     * This method is called after new value has been stored.
+     * <p>
+     * Implementations should not execute any complex logic,
+     * including locking, networking or cache operations,
+     * as it may lead to deadlock, since this method is called
+     * from sensitive synchronization blocks.
+     * <p>
+     * This method should not throw any exception.
+     *
+     * @param key Key.
+     * @param val Current value.
+     */
+    public void onAfterPut(K key, V val);
+
+    /**
+     * This method is called within {@link CacheProjection#remove(Object, 
IgnitePredicate[])}
+     * and similar operations to provide control over returned value.
+     * <p>
+     * Implementations should not execute any complex logic,
+     * including locking, networking or cache operations,
+     * as it may lead to deadlock, since this method is called
+     * from sensitive synchronization blocks.
+     * <p>
+     * This method should not throw any exception.
+     *
+     * @param key Key.
+     * @param val Old value.
+     * @return Tuple. The first value is the flag whether remove should be 
cancelled or not.
+     *      The second is the value to be returned as result of {@code 
remove()} operation,
+     *      may be {@code null}.
+     * @see CacheProjection#remove(Object, IgnitePredicate[])
+     */
+    @Nullable public IgniteBiTuple<Boolean, V> onBeforeRemove(K key, @Nullable 
V val);
+
+    /**
+     * This method is called after value has been removed.
+     * <p>
+     * Implementations should not execute any complex logic,
+     * including locking, networking or cache operations,
+     * as it may lead to deadlock, since this method is called
+     * from sensitive synchronization blocks.
+     * <p>
+     * This method should not throw any exception.
+     *
+     * @param key Key.
+     * @param val Removed value.
+     */
+    public void onAfterRemove(K key, V val);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/CacheInterceptorAdapter.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/CacheInterceptorAdapter.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/CacheInterceptorAdapter.java
new file mode 100644
index 0000000..390e5b1
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/CacheInterceptorAdapter.java
@@ -0,0 +1,52 @@
+/*
+ * 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.cache;
+
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * Cache interceptor convenience adapter. It provides no-op implementations 
for all
+ * interceptor callbacks.
+ */
+public class CacheInterceptorAdapter<K, V> implements CacheInterceptor<K, V> {
+    /** {@inheritDoc} */
+    @Nullable @Override public V onGet(K key, V val) {
+        return val;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public V onBeforePut(K key, @Nullable V oldVal, V 
newVal) {
+        return newVal;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onAfterPut(K key, V val) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteBiTuple<Boolean, V> onBeforeRemove(K key, 
@Nullable V val) {
+        return new IgniteBiTuple<>(false, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onAfterRemove(K key, V val) {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/CacheMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheMBean.java 
b/modules/core/src/main/java/org/apache/ignite/cache/CacheMBean.java
new file mode 100644
index 0000000..643f9b5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheMBean.java
@@ -0,0 +1,295 @@
+/*
+ * 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.cache;
+
+import org.apache.ignite.mbean.*;
+
+/**
+ * This interface defines JMX view on {@link Cache}.
+ */
+@IgniteMBeanDescription("MBean that provides access to cache descriptor.")
+public interface CacheMBean {
+    /**
+     * Gets name of this cache.
+     *
+     * @return Cache name.
+     */
+    @IgniteMBeanDescription("Cache name.")
+    public String name();
+
+    /**
+     * Gets metrics (statistics) for this cache.
+     *
+     * @return Cache metrics.
+     */
+    @IgniteMBeanDescription("Formatted cache metrics.")
+    public String metricsFormatted();
+
+    /**
+     * Gets number of entries that was swapped to disk.
+     *
+     * @return Number of entries that was swapped to disk.
+     */
+    @IgniteMBeanDescription("Number of entries that was swapped to disk.")
+    public long getOverflowSize();
+
+    /**
+     * Gets number of entries stored in off-heap memory.
+     *
+     * @return Number of entries stored in off-heap memory.
+     */
+    @IgniteMBeanDescription("Number of entries stored in off-heap memory.")
+    public long getOffHeapEntriesCount();
+
+    /**
+     * Gets memory size allocated in off-heap.
+     *
+     * @return Memory size allocated in off-heap.
+     */
+    @IgniteMBeanDescription("Memory size allocated in off-heap.")
+    public long getOffHeapAllocatedSize();
+
+    /**
+     * Returns number of non-{@code null} values in the cache.
+     *
+     * @return Number of non-{@code null} values in the cache.
+     */
+    @IgniteMBeanDescription("Number of non-null values in the cache.")
+    public int getSize();
+
+    /**
+     * Gets number of keys in the cache, possibly with {@code null} values.
+     *
+     * @return Number of keys in the cache.
+     */
+    @IgniteMBeanDescription("Number of keys in the cache (possibly with null 
values).")
+    public int getKeySize();
+
+    /**
+     * Returns {@code true} if this cache is empty.
+     *
+     * @return {@code true} if this cache is empty.
+     */
+    @IgniteMBeanDescription("True if cache is empty.")
+    public boolean isEmpty();
+
+    /**
+     * Gets current size of evict queue used to batch up evictions.
+     *
+     * @return Current size of evict queue.
+     */
+    @IgniteMBeanDescription("Current size of evict queue.")
+    public int getDhtEvictQueueCurrentSize();
+
+    /**
+     * Gets transaction per-thread map size.
+     *
+     * @return Thread map size.
+     */
+    @IgniteMBeanDescription("Transaction per-thread map size.")
+    public int getTxThreadMapSize();
+
+    /**
+     * Gets transaction per-Xid map size.
+     *
+     * @return Transaction per-Xid map size.
+     */
+    @IgniteMBeanDescription("Transaction per-Xid map size.")
+    public int getTxXidMapSize();
+
+    /**
+     * Gets committed transaction queue size.
+     *
+     * @return Committed transaction queue size.
+     */
+    @IgniteMBeanDescription("Transaction committed queue size.")
+    public int getTxCommitQueueSize();
+
+    /**
+     * Gets prepared transaction queue size.
+     *
+     * @return Prepared transaction queue size.
+     */
+    @IgniteMBeanDescription("Transaction prepared queue size.")
+    public int getTxPrepareQueueSize();
+
+    /**
+     * Gets start version counts map size.
+     *
+     * @return Start version counts map size.
+     */
+    @IgniteMBeanDescription("Transaction start version counts map size.")
+    public int getTxStartVersionCountsSize();
+
+    /**
+     * Gets number of cached committed transaction IDs.
+     *
+     * @return Number of cached committed transaction IDs.
+     */
+    @IgniteMBeanDescription("Transaction committed ID map size.")
+    public int getTxCommittedVersionsSize();
+
+    /**
+     * Gets number of cached rolled back transaction IDs.
+     *
+     * @return Number of cached rolled back transaction IDs.
+     */
+    @IgniteMBeanDescription("Transaction rolled back ID map size.")
+    public int getTxRolledbackVersionsSize();
+
+    /**
+     * Gets transaction DHT per-thread map size.
+     *
+     * @return DHT thread map size.
+     */
+    @IgniteMBeanDescription("Transaction DHT per-thread map size.")
+    public int getTxDhtThreadMapSize();
+
+    /**
+     * Gets transaction DHT per-Xid map size.
+     *
+     * @return Transaction DHT per-Xid map size.
+     */
+    @IgniteMBeanDescription("Transaction DHT per-Xid map size.")
+    public int getTxDhtXidMapSize();
+
+    /**
+     * Gets committed DHT transaction queue size.
+     *
+     * @return Committed DHT transaction queue size.
+     */
+    @IgniteMBeanDescription("Transaction DHT committed queue size.")
+    public int getTxDhtCommitQueueSize();
+
+    /**
+     * Gets prepared DHT transaction queue size.
+     *
+     * @return Prepared DHT transaction queue size.
+     */
+    @IgniteMBeanDescription("Transaction DHT prepared queue size.")
+    public int getTxDhtPrepareQueueSize();
+
+    /**
+     * Gets DHT start version counts map size.
+     *
+     * @return DHT start version counts map size.
+     */
+    @IgniteMBeanDescription("Transaction DHT start version counts map size.")
+    public int getTxDhtStartVersionCountsSize();
+
+    /**
+     * Gets number of cached committed DHT transaction IDs.
+     *
+     * @return Number of cached committed DHT transaction IDs.
+     */
+    @IgniteMBeanDescription("Transaction DHT committed ID map size.")
+    public int getTxDhtCommittedVersionsSize();
+
+    /**
+     * Gets number of cached rolled back DHT transaction IDs.
+     *
+     * @return Number of cached rolled back DHT transaction IDs.
+     */
+    @IgniteMBeanDescription("Transaction DHT rolled back ID map size.")
+    public int getTxDhtRolledbackVersionsSize();
+
+    /**
+     * Returns {@code True} if write-behind is enabled.
+     *
+     * @return {@code True} if write-behind is enabled.
+     */
+    @IgniteMBeanDescription("True if write-behind is enabled for this cache.")
+    public boolean isWriteBehindEnabled();
+
+    /**
+     * Gets the maximum size of the write-behind buffer. When the count of 
unique keys
+     * in write buffer exceeds this value, the buffer is scheduled for write 
to the underlying store.
+     * <p/>
+     * If this value is {@code 0}, then flush is performed only on 
time-elapsing basis. However,
+     * when this value is {@code 0}, the cache critical size is set to
+     * {@link CacheConfiguration#DFLT_WRITE_BEHIND_CRITICAL_SIZE}
+     *
+     * @return Buffer size that triggers flush procedure.
+     */
+    @IgniteMBeanDescription("Size of internal buffer that triggers flush 
procedure.")
+    public int getWriteBehindFlushSize();
+
+    /**
+     * Gets the number of flush threads that will perform store update 
operations.
+     *
+     * @return Count of worker threads.
+     */
+    @IgniteMBeanDescription("Count of flush threads.")
+    public int getWriteBehindFlushThreadCount();
+
+    /**
+     * Gets the cache flush frequency. All pending operations on the 
underlying store will be performed
+     * within time interval not less then this value.
+     * <p/>
+     * If this value is {@code 0}, then flush is performed only when buffer 
size exceeds flush size.
+     *
+     * @return Flush frequency in milliseconds.
+     */
+    @IgniteMBeanDescription("Flush frequency interval in milliseconds.")
+    public long getWriteBehindFlushFrequency();
+
+    /**
+     * Gets the maximum count of similar (put or remove) operations that can 
be grouped to a single batch.
+     *
+     * @return Maximum size of batch.
+     */
+    @IgniteMBeanDescription("Maximum size of batch for similar operations.")
+    public int getWriteBehindStoreBatchSize();
+
+    /**
+     * Gets count of write buffer overflow events since initialization. Each 
overflow event causes
+     * the ongoing flush operation to be performed synchronously.
+     *
+     * @return Count of cache overflow events since start.
+     */
+    @IgniteMBeanDescription("Count of cache overflow events since write-behind 
cache has started.")
+    public int getWriteBehindTotalCriticalOverflowCount();
+
+    /**
+     * Gets count of write buffer overflow events in progress at the moment. 
Each overflow event causes
+     * the ongoing flush operation to be performed synchronously.
+     *
+     * @return Count of cache overflow events since start.
+     */
+    @IgniteMBeanDescription("Count of cache overflow events since write-behind 
cache has started.")
+    public int getWriteBehindCriticalOverflowCount();
+
+    /**
+     * Gets count of cache entries that are in a store-retry state. An entry 
is assigned a store-retry state
+     * when underlying store failed due some reason and cache has enough space 
to retain this entry till
+     * the next try.
+     *
+     * @return Count of entries in store-retry state.
+     */
+    @IgniteMBeanDescription("Count of cache cache entries that are currently 
in retry state.")
+    public int getWriteBehindErrorRetryCount();
+
+    /**
+     * Gets count of entries that were processed by the write-behind store and 
have not been
+     * flushed to the underlying store yet.
+     *
+     * @return Total count of entries in cache store internal buffer.
+     */
+    @IgniteMBeanDescription("Count of cache entries that are waiting to be 
flushed.")
+    public int getWriteBehindBufferSize();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/CacheMemoryMode.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/CacheMemoryMode.java 
b/modules/core/src/main/java/org/apache/ignite/cache/CacheMemoryMode.java
new file mode 100644
index 0000000..914a2e2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheMemoryMode.java
@@ -0,0 +1,59 @@
+/*
+ * 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.cache;
+
+/**
+ * Defines set of memory modes. Memory modes help control whether cache 
entries are
+ * stored on heap memory, offheap memory, or in swap space.
+ */
+public enum CacheMemoryMode {
+    /**
+     * Entries will be stored on-heap first. The onheap tiered storage works 
as follows:
+     * <nl>
+     * <li>Entries are cached on heap memory first.</li>
+     * <li>
+     *     If offheap memory is enabled and eviction policy evicts an entry 
from heap memory, entry will
+     *     be moved to offheap memory. If offheap memory is disabled, then 
entry is simply discarded.
+     * </li>
+     * <li>
+     *     If swap space is enabled and offheap memory fills up, then entry 
will be evicted into swap space.
+     *     If swap space is disabled, then entry will be discarded. If swap is 
enabled and offheap memory
+     *     is disabled, then entry will be evicted directly from heap memory 
into swap.
+     * </li>
+     * </nl>
+     * <p>
+     * <b>Note</b> that heap memory evictions are handled by configured {@link 
org.apache.ignite.cache.eviction.CacheEvictionPolicy}
+     * implementation. By default, no eviction policy is enabled, so entries 
never leave heap
+     * memory space unless explicitly removed.
+     */
+    ONHEAP_TIERED,
+
+    /**
+     * Works the same as {@link #ONHEAP_TIERED}, except that entries never end 
up in heap memory and get
+     * stored in offheap memory right away. Entries get cached in offheap 
memory first and then
+     * get evicted to swap, if one is configured.
+     */
+    OFFHEAP_TIERED,
+
+    /**
+     * Entry keys will be stored on heap memory, and values will be stored in 
offheap memory. Note
+     * that in this mode entries can be evicted only to swap. The evictions 
will happen according
+     * to configured {@link 
org.apache.ignite.cache.eviction.CacheEvictionPolicy}.
+     */
+    OFFHEAP_VALUES,
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java 
b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
new file mode 100644
index 0000000..1410e5b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
@@ -0,0 +1,103 @@
+/*
+ * 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.cache;
+
+import java.io.*;
+
+/**
+ * Cache metrics used to obtain statistics on cache itself.
+ * Use {@link Cache#metrics()} to obtain metrics for a cache.
+ */
+public interface CacheMetrics extends Serializable {
+    /**
+     * Gets create time of the owning entity (either cache or entry).
+     *
+     * @return Create time.
+     */
+    public long createTime();
+
+    /**
+     * Gets last write time of the owning entity (either cache or entry).
+     *
+     * @return Last write time.
+     */
+    public long writeTime();
+
+    /**
+     * Gets last read time of the owning entity (either cache or entry).
+     *
+     * @return Last read time.
+     */
+    public long readTime();
+
+    /**
+     * Gets last time transaction was committed.
+     *
+     * @return Last commit time.
+     */
+    public long commitTime();
+
+    /**
+     * Gets last time transaction was rollback.
+     *
+     * @return Last rollback time.
+     */
+    public long rollbackTime();
+
+    /**
+     * Gets total number of reads of the owning entity (either cache or entry).
+     *
+     * @return Total number of reads.
+     */
+    public int reads();
+
+    /**
+     * Gets total number of writes of the owning entity (either cache or 
entry).
+     *
+     * @return Total number of writes.
+     */
+    public int writes();
+
+    /**
+     * Gets total number of hits for the owning entity (either cache or entry).
+     *
+     * @return Number of hits.
+     */
+    public int hits();
+
+    /**
+     * Gets total number of misses for the owning entity (either cache or 
entry).
+     *
+     * @return Number of misses.
+     */
+    public int misses();
+
+    /**
+     * Gets total number of transaction commits.
+     *
+     * @return Number of transaction commits.
+     */
+    public int txCommits();
+
+    /**
+     * Gets total number of transaction rollbacks.
+     *
+     * @return Number of transaction rollbacks.
+     */
+    public int txRollbacks();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java 
b/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java
index 04c83a5..9d10d9f 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java
@@ -39,7 +39,7 @@ public enum CacheMode {
     /**
      * Specifies fully replicated cache behavior. In this mode all the keys 
are distributed
      * to all participating nodes. User still has affinity control
-     * over subset of nodes for any given key via {@link 
org.apache.ignite.cache.affinity.GridCacheAffinityFunction}
+     * over subset of nodes for any given key via {@link 
org.apache.ignite.cache.affinity.CacheAffinityFunction}
      * configuration.
      */
     REPLICATED,
@@ -48,7 +48,7 @@ public enum CacheMode {
      * Specifies partitioned cache behaviour. In this mode the overall
      * key set will be divided into partitions and all partitions will be split
      * equally between participating nodes. User has affinity
-     * control over key assignment via {@link 
org.apache.ignite.cache.affinity.GridCacheAffinityFunction}
+     * control over key assignment via {@link 
org.apache.ignite.cache.affinity.CacheAffinityFunction}
      * configuration.
      * <p>
      * Note that partitioned cache is always fronted by local

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/CacheName.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheName.java 
b/modules/core/src/main/java/org/apache/ignite/cache/CacheName.java
new file mode 100644
index 0000000..eb5e5f6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheName.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.cache;
+
+import java.lang.annotation.*;
+import java.util.concurrent.*;
+
+/**
+ * Allows to specify cache name from grid computations. It is used to provide 
cache name
+ * for affinity routing of grid computations, such as {@link 
org.apache.ignite.compute.ComputeJob}, {@link Runnable},
+ * {@link Callable}, or {@link org.apache.ignite.lang.IgniteClosure}. It 
should be used only in conjunction with
+ * {@link org.apache.ignite.cache.affinity.CacheAffinityKeyMapped 
@CacheAffinityKeyMapped} annotation, and should be attached to a method or field
+ * that provides cache name for the computation. Only one annotation per class
+ * is allowed. In the absence of this annotation, the default no-name cache
+ * will be used for providing key-to-node affinity.
+ * <p>
+ * Refer to {@link org.apache.ignite.cache.affinity.CacheAffinityKeyMapped 
@CacheAffinityKeyMapped} documentation for more information
+ * and examples about this annotation.
+ * @see org.apache.ignite.cache.affinity.CacheAffinityKeyMapped
+ */
+@Documented
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.FIELD, ElementType.METHOD})
+public @interface CacheName {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/CachePeekMode.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/CachePeekMode.java 
b/modules/core/src/main/java/org/apache/ignite/cache/CachePeekMode.java
index 9ea530a..0fffb38 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CachePeekMode.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CachePeekMode.java
@@ -21,9 +21,9 @@ import org.jetbrains.annotations.*;
 
 /**
  * Enumeration of all supported cache peek modes. Peek modes can be passed 
into various
- * {@code 'GridCacheProjection.peek(..)'} and {@code GridCacheEntry.peek(..)} 
methods,
- * such as {@link org.apache.ignite.cache.GridCacheProjection#peek(Object, 
java.util.Collection)},
- * {@link org.apache.ignite.cache.GridCacheEntry#peek()}, and others.
+ * {@code 'CacheProjection.peek(..)'} and {@code CacheEntry.peek(..)} methods,
+ * such as {@link CacheProjection#peek(Object, java.util.Collection)},
+ * {@link CacheEntry#peek()}, and others.
  * <p>
  * The following modes are supported:
  * <ul>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/CachePreloadMode.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/CachePreloadMode.java 
b/modules/core/src/main/java/org/apache/ignite/cache/CachePreloadMode.java
new file mode 100644
index 0000000..86b25e9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CachePreloadMode.java
@@ -0,0 +1,67 @@
+/*
+ * 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.cache;
+
+import org.jetbrains.annotations.*;
+
+/**
+ * Cache preload mode. When preloading is enabled (i.e. has value other than 
{@link #NONE}), distributed caches
+ * will attempt to preload all necessary values from other grid nodes. This 
enumeration is used to configure
+ * preloading via {@link CacheConfiguration#getPreloadMode()} configuration 
property. If not configured
+ * explicitly, then {@link CacheConfiguration#DFLT_PRELOAD_MODE} is used.
+ * <p>
+ * Replicated caches will try to load the full set of cache entries from other 
nodes (or as defined by
+ * pluggable {@link org.apache.ignite.cache.affinity.CacheAffinityFunction}), 
while partitioned caches will only load the entries for which
+ * current node is primary or back up.
+ * <p>
+ * Note that preload mode only makes sense for {@link CacheMode#REPLICATED} 
and {@link CacheMode#PARTITIONED}
+ * caches. Caches with {@link CacheMode#LOCAL} mode are local by definition 
and therefore cannot preload
+ * any values from neighboring nodes.
+ */
+public enum CachePreloadMode {
+    /**
+     * Synchronous preload mode. Distributed caches will not start until all 
necessary data
+     * is loaded from other available grid nodes.
+     */
+    SYNC,
+
+    /**
+     * Asynchronous preload mode. Distributed caches will start immediately 
and will load all necessary
+     * data from other available grid nodes in the background.
+     */
+    ASYNC,
+
+    /**
+     * In this mode no preloading will take place which means that caches will 
be either loaded on
+     * demand from persistent store whenever data is accessed, or will be 
populated explicitly.
+     */
+    NONE;
+
+    /** Enumerated values. */
+    private static final CachePreloadMode[] VALS = values();
+
+    /**
+     * Efficiently gets enumerated value from its ordinal.
+     *
+     * @param ord Ordinal value.
+     * @return Enumerated value or {@code null} if ordinal out of range.
+     */
+    @Nullable public static CachePreloadMode fromOrdinal(int ord) {
+        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
+    }
+}

Reply via email to