http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNode.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNodeLocalMap.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobSibling.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/apache/ignite/compute/ComputeLoadBalancer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/apache/ignite/logger/java/IgniteJavaLogger.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTx.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/transactions/IgniteTx.java
index 0000000,80409f4..13a1d1b
mode 000000,100644..100644
--- a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTx.java
+++ b/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTx.java
@@@ -1,0 -1,247 +1,277 @@@
+ /*
+  * 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.transactions;
+ 
+ import org.apache.ignite.*;
+ import org.apache.ignite.lang.*;
+ import org.gridgain.grid.*;
+ import org.gridgain.grid.cache.*;
++import org.jetbrains.annotations.*;
+ 
+ import java.util.*;
+ 
+ /**
+  * Grid cache transaction. Cache transactions have a default 2PC 
(two-phase-commit) behavior and
+  * can be plugged into ongoing {@code JTA} transaction by properly 
implementing
+  * {@gglink org.gridgain.grid.cache.jta.GridCacheTmLookup}
+  * interface. Cache transactions can also be started explicitly directly from 
{@link GridCacheProjection} API
+  * via any of the {@code 'GridCacheProjection.txStart(..)'} methods.
+  * <p>
+  * Cache transactions support the following isolation levels:
+  * <ul>
+  * <li>
+  *  {@link IgniteTxIsolation#READ_COMMITTED} isolation level means that 
always a committed value
+  *  will be provided for read operations. With this isolation level values 
are always read
+  *  from cache global memory or persistent store every time a value is 
accessed. In other words,
+  *  if the same key is accessed more than once within the same transaction, 
it may have different
+  *  value every time since global cache memory may be updated concurrently by 
other threads.
+  * </li>
+  * <li>
+  *  {@link IgniteTxIsolation#REPEATABLE_READ} isolation level means that if a 
value was read once
+  *  within transaction, then all consecutive reads will provide the same 
in-transaction value. With
+  *  this isolation level accessed values are stored within in-transaction 
memory, so consecutive access
+  *  to the same key within the same transaction will always return the value 
that was previously read or
+  *  updated within this transaction. If concurrency is {@link 
IgniteTxConcurrency#PESSIMISTIC}, then a lock
+  *  on the key will be acquired prior to accessing the value.
+  * </li>
+  * <li>
+  *  {@link IgniteTxIsolation#SERIALIZABLE} isolation level means that all 
transactions occur in a completely
+  *  isolated fashion, as if all transactions in the system had executed 
serially, one after the other.
+  *  Read access with this level happens the same way as with {@link 
IgniteTxIsolation#REPEATABLE_READ} level.
+  *  However, in {@link IgniteTxConcurrency#OPTIMISTIC} mode, if some 
transactions cannot be serially isolated
+  *  from each other, then one winner will be picked and the other 
transactions in conflict will result in
+  * {@link IgniteTxOptimisticException} being thrown.
+  * </li>
+  * </ul>
+  * <p>
+  * Cache transactions support the following concurrency models:
+  * <ul>
+  * <li>
+  *  {@link IgniteTxConcurrency#OPTIMISTIC} - in this mode all cache 
operations are not distributed to other
+  *  nodes until {@link #commit()} is called. In this mode one {@code 
'PREPARE'}
+  *  message will be sent to participating cache nodes to start acquiring 
per-transaction locks, and once
+  *  all nodes reply {@code 'OK'} (i.e. {@code Phase 1} completes 
successfully), a one-way' {@code 'COMMIT'}
+  *  message is sent without waiting for reply. If it is necessary to know 
whenever remote nodes have committed
+  *  as well, synchronous commit or synchronous rollback should be enabled via
+  *  {@link GridCacheConfiguration#setWriteSynchronizationMode}
+  *  or by setting proper flags on cache projection, such as {@link 
GridCacheFlag#SYNC_COMMIT}.
+  *  <p>
+  *  Note that in this mode, optimistic failures are only possible in 
conjunction with
+  *  {@link IgniteTxIsolation#SERIALIZABLE} isolation level. In all other 
cases, optimistic
+  *  transactions will never fail optimistically and will always be 
identically ordered on all participating
+  *  grid nodes.
+  * </li>
+  * <li>
+  *  {@link IgniteTxConcurrency#PESSIMISTIC} - in this mode a lock is acquired 
on all cache operations
+  *  with exception of read operations in {@link 
IgniteTxIsolation#READ_COMMITTED} mode. All optional filters
+  *  passed into cache operations will be evaluated after successful lock 
acquisition. Whenever
+  *  {@link #commit()} is called, a single one-way {@code 'COMMIT'} message
+  *  is sent to participating cache nodes without waiting for reply. Note that 
there is no reason for
+  *  distributed 'PREPARE' step, as all locks have been already acquired. Just 
like with optimistic mode,
+  *  it is possible to configure synchronous commit or rollback and wait till 
transaction commits on
+  *  all participating remote nodes.
+  * </li>
+  * </ul>
+  * <p>
+  * <h1 class="header">Cache Atomicity Mode</h1>
+  * In addition to standard {@link GridCacheAtomicityMode#TRANSACTIONAL} 
behavior, GridGain also supports
+  * a lighter {@link GridCacheAtomicityMode#ATOMIC} mode as well. In this mode 
distributed transactions
+  * and distributed locking are not supported. Disabling transactions and 
locking allows to achieve much higher
+  * performance and throughput ratios. It is recommended that {@link 
GridCacheAtomicityMode#ATOMIC} mode
+  * is used whenever full {@code ACID}-compliant transactions are not needed.
+  * <p>
+  * <h1 class="header">Usage</h1>
+  * You can use cache transactions as follows:
+  * <pre name="code" class="java">
+  * GridCache&lt;String, Integer&gt; cache = GridGain.grid().cache();
+  *
+  * try (GridCacheTx tx = cache.txStart()) {
+  *     // Perform transactional operations.
+  *     Integer v1 = cache.get("k1");
+  *
+  *     // Check if v1 satisfies some condition before doing a put.
+  *     if (v1 != null && v1 > 0)
+  *         cache.putx("k1", 2);
+  *
+  *     cache.removex("k2");
+  *
+  *     // Commit the transaction.
+  *     tx.commit();
+  * }
+  * </pre>
+  */
 -public interface IgniteTx extends GridMetadataAware, AutoCloseable, 
IgniteAsyncSupport {
++public interface IgniteTx extends AutoCloseable, IgniteAsyncSupport {
+     /**
+      * Gets unique identifier for this transaction.
+      *
+      * @return Transaction UID.
+      */
+     public IgniteUuid xid();
+ 
+     /**
+      * ID of the node on which this transaction started.
+      *
+      * @return Originating node ID.
+      */
+     public UUID nodeId();
+ 
+     /**
+      * ID of the thread in which this transaction started.
+      *
+      * @return Thread ID.
+      */
+     public long threadId();
+ 
+     /**
+      * Start time of this transaction.
+      *
+      * @return Start time of this transaction on this node.
+      */
+     public long startTime();
+ 
+     /**
+      * Cache transaction isolation level.
+      *
+      * @return Isolation level.
+      */
+     public IgniteTxIsolation isolation();
+ 
+     /**
+      * Cache transaction concurrency mode.
+      *
+      * @return Concurrency mode.
+      */
+     public IgniteTxConcurrency concurrency();
+ 
+     /**
+      * Flag indicating whether transaction was started automatically by the
+      * system or not. System will start transactions implicitly whenever
+      * any cache {@code put(..)} or {@code remove(..)} operation is invoked
+      * outside of transaction.
+      *
+      * @return {@code True} if transaction was started implicitly.
+      */
+     public boolean implicit();
+ 
+     /**
+      * Get invalidation flag for this transaction. If set to {@code true}, 
then
+      * remote values will be {@code invalidated} (set to {@code null}) instead
+      * of updated.
+      * <p>
+      * Invalidation messages don't carry new values, so they are a lot lighter
+      * than update messages. However, when a value is accessed on a node after
+      * it's been invalidated, it must be loaded from persistent store.
+      *
+      * @return Invalidation flag.
+      */
+     public boolean isInvalidate();
+ 
+     /**
+      * Gets current transaction state value.
+      *
+      * @return Current transaction state.
+      */
+     public IgniteTxState state();
+ 
+     /**
+      * Gets timeout value in milliseconds for this transaction. If 
transaction times
+      * out prior to it's completion, {@link IgniteTxTimeoutException} will be 
thrown.
+      *
+      * @return Transaction timeout value.
+      */
+     public long timeout();
+ 
+     /**
+      * Sets transaction timeout value. This value can be set only before a 
first operation
+      * on transaction has been performed.
+      *
+      * @param timeout Transaction timeout value.
+      * @return Previous timeout.
+      */
+     public long timeout(long timeout);
+ 
+     /**
+      * Modify the transaction associated with the current thread such that the
+      * only possible outcome of the transaction is to roll back the
+      * transaction.
+      *
+      * @return {@code True} if rollback-only flag was set as a result of this 
operation,
+      *      {@code false} if it was already set prior to this call or could 
not be set
+      *      because transaction is already finishing up committing or rolling 
back.
+      */
+     public boolean setRollbackOnly();
+ 
+     /**
+      * If transaction was marked as rollback-only.
+      *
+      * @return {@code True} if transaction can only be rolled back.
+      */
+     public boolean isRollbackOnly();
+ 
+     /**
+      * Commits this transaction by initiating {@code two-phase-commit} 
process.
+      *
+      * @throws IgniteCheckedException If commit failed.
+      */
+     @IgniteAsyncSupported
+     public void commit() throws IgniteCheckedException;
+ 
+     /**
+      * Ends the transaction. Transaction will be rolled back if it has not 
been committed.
+      *
+      * @throws IgniteCheckedException If transaction could not be gracefully 
ended.
+      */
+     @Override public void close() throws IgniteCheckedException;
+ 
+     /**
+      * Rolls back this transaction.
+      *
+      * @throws IgniteCheckedException If rollback failed.
+      */
+     @IgniteAsyncSupported
+     public void rollback() throws IgniteCheckedException;
++
++    /**
++     * Removes metadata by name.
++     *
++     * @param name Name of the metadata to remove.
++     * @param <V> Type of the value.
++     * @return Value of removed metadata or {@code null}.
++     */
++    @Nullable public <V> V removeMeta(String name);
++
++    /**
++     * Gets metadata by name.
++     *
++     * @param name Metadata name.
++     * @param <V> Type of the value.
++     * @return Metadata value or {@code null}.
++     */
++    @Nullable public <V> V meta(String name);
++
++    /**
++     * Adds a new metadata.
++     *
++     * @param name Metadata name.
++     * @param val Metadata value.
++     * @param <V> Type of the value.
++     * @return Metadata previously associated with given name, or
++     *      {@code null} if there was none.
++     */
++    @Nullable public <V> V addMeta(String name, V val);
+ }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheEntry.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/gridgain/grid/cache/GridCacheEntry.java
index 6966cb5,e162178..11ef626
--- a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheEntry.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheEntry.java
@@@ -11,11 -19,13 +19,14 @@@ package org.gridgain.grid.cache
  
  import org.apache.ignite.*;
  import org.apache.ignite.lang.*;
+ import org.apache.ignite.transactions.*;
+ import org.gridgain.grid.*;
  import org.jetbrains.annotations.*;
  
+ import javax.cache.*;
  import java.util.*;
  import java.util.Map.*;
 +import java.util.concurrent.*;
  
  /**
   * This interface provides a rich API for working with individual cache 
entries. It
@@@ -78,7 -88,7 +89,7 @@@
   * @param <K> Key type.
   * @param <V> Value type.
   */
- public interface GridCacheEntry<K, V> extends Map.Entry<K, V> {
 -public interface GridCacheEntry<K, V> extends Map.Entry<K, V>, 
GridMetadataAware, Cache.Entry<K, V> {
++public interface GridCacheEntry<K, V> extends Map.Entry<K, V>, Cache.Entry<K, 
V> {
      /**
       * Cache projection to which this entry belongs. Note that entry and its
       * parent projections have same flags and filters.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStore.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStore.java
index 974c9a9,e2f4cc6..092660e
--- 
a/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStore.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStore.java
@@@ -41,9 -51,9 +51,9 @@@ import java.util.Date
   *     <li>{@link GridCacheJdbcBlobStore}</li>
   * </ul>
   * <p>
-  * All transactional operations of this API are provided with ongoing {@link 
GridCacheTx},
+  * All transactional operations of this API are provided with ongoing {@link 
IgniteTx},
 - * if any. As transaction is {@link GridMetadataAware}, you can attach any 
metadata to
 - * it, e.g. to recognize if several operations belong to the same transaction 
or not.
 + * if any. You can attach any metadata to it, e.g. to recognize if several 
operations belong
 + * to the same transaction or not.
   * Here is an example of how attach a JDBC connection as transaction metadata:
   * <pre name="code" class="java">
   * Connection conn = tx.meta("some.name");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/ClusterNodeLocalMapImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/GridJobContextImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/GridJobSiblingImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernalContext.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernalContextImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/GridLoggerProxy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/executor/GridExecutorService.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/managers/loadbalancer/GridLoadBalancerAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryEx.java
index f6eb3c2,4b5a936..580a896
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryEx.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryEx.java
@@@ -11,14 -19,17 +19,17 @@@ package org.gridgain.grid.kernal.proces
  
  import org.apache.ignite.*;
  import org.apache.ignite.lang.*;
 -import org.gridgain.grid.*;
  import org.gridgain.grid.cache.*;
  import org.gridgain.grid.kernal.processors.cache.distributed.*;
+ import org.gridgain.grid.kernal.processors.cache.transactions.*;
  import org.gridgain.grid.kernal.processors.dr.*;
  import org.gridgain.grid.util.lang.*;
  import org.jetbrains.annotations.*;
  
+ import javax.cache.expiry.*;
+ import javax.cache.processor.*;
  import java.util.*;
 +import java.util.concurrent.*;
  
  /**
   * Internal API for cache entry ({@code 'Ex'} stands for extended).

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryImpl.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryImpl.java
index 4184415,a23fc5c..b0df4b3
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryImpl.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryImpl.java
@@@ -11,8 -19,10 +19,9 @@@ package org.gridgain.grid.kernal.proces
  
  import org.apache.ignite.*;
  import org.apache.ignite.lang.*;
 -import org.gridgain.grid.*;
  import org.gridgain.grid.cache.*;
  import org.gridgain.grid.kernal.processors.cache.distributed.dht.*;
+ import org.gridgain.grid.kernal.processors.cache.transactions.*;
  import org.gridgain.grid.util.lang.*;
  import org.gridgain.grid.util.tostring.*;
  import org.gridgain.grid.util.typedef.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionEntry.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionEntry.java
index df673bd,967fb23..15aa1e5
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionEntry.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionEntry.java
@@@ -11,7 -19,9 +19,8 @@@ package org.gridgain.grid.kernal.proces
  
  import org.apache.ignite.*;
  import org.apache.ignite.lang.*;
 -import org.gridgain.grid.*;
  import org.gridgain.grid.cache.*;
+ import org.gridgain.grid.kernal.processors.cache.transactions.*;
  import org.gridgain.grid.util.typedef.*;
  import org.gridgain.grid.util.typedef.internal.*;
  import org.gridgain.grid.util.lang.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheFilterEvaluationEntry.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheLogger.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/GridPartitionedCacheEntryImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryEntry.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/transactions/IgniteTxProxyImpl.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/transactions/IgniteTxProxyImpl.java
index 0000000,0cb7f63..29c04b4
mode 000000,100644..100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/transactions/IgniteTxProxyImpl.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/transactions/IgniteTxProxyImpl.java
@@@ -1,0 -1,372 +1,314 @@@
+ /*
+  * 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.gridgain.grid.kernal.processors.cache.transactions;
+ 
+ import org.apache.ignite.*;
+ import org.apache.ignite.lang.*;
+ import org.apache.ignite.transactions.*;
+ import org.gridgain.grid.*;
+ import org.gridgain.grid.kernal.processors.cache.*;
+ import org.gridgain.grid.util.future.*;
+ import org.gridgain.grid.util.tostring.*;
+ import org.gridgain.grid.util.typedef.internal.*;
 -import org.jetbrains.annotations.*;
+ 
+ import java.io.*;
+ import java.util.*;
 -import java.util.concurrent.*;
+ 
+ /**
+  * Cache transaction proxy.
+  */
+ public class IgniteTxProxyImpl<K, V> implements IgniteTxProxy, Externalizable 
{
+     /** */
+     private static final long serialVersionUID = 0L;
+ 
+     /** Wrapped transaction. */
+     @GridToStringInclude
+     private IgniteTxEx<K, V> tx;
+ 
+     /** Gateway. */
+     @GridToStringExclude
+     private GridCacheSharedContext<K, V> cctx;
+ 
+     /** Async flag. */
+     private boolean async;
+ 
+     /** Async call result. */
+     private IgniteFuture asyncRes;
+ 
+     /**
+      * Empty constructor required for {@link Externalizable}.
+      */
+     public IgniteTxProxyImpl() {
+         // No-op.
+     }
+ 
+     /**
+      * @param tx Transaction.
+      * @param cctx Shared context.
+      * @param async Async flag.
+      */
+     public IgniteTxProxyImpl(IgniteTxEx<K, V> tx, GridCacheSharedContext<K, 
V> cctx, boolean async) {
+         assert tx != null;
+         assert cctx != null;
+ 
+         this.tx = tx;
+         this.cctx = cctx;
+         this.async = async;
+     }
+ 
+     /**
+      * Enters a call.
+      */
+     private void enter() {
+         if (cctx.deploymentEnabled())
+             cctx.deploy().onEnter();
+ 
+         try {
+             cctx.kernalContext().gateway().readLock();
+         }
+         catch (IllegalStateException e) {
+             throw e;
+         }
+         catch (RuntimeException | Error e) {
+             cctx.kernalContext().gateway().readUnlock();
+ 
+             throw e;
+         }
+     }
+ 
+     /**
+      * Leaves a call.
+      */
+     private void leave() {
+         try {
+             CU.unwindEvicts(cctx);
+         }
+         finally {
+             cctx.kernalContext().gateway().readUnlock();
+         }
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public IgniteUuid xid() {
+         return tx.xid();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public UUID nodeId() {
+         if (async)
+             save(tx.nodeId());
+ 
+         return tx.nodeId();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long threadId() {
+         if (async)
+             save(tx.threadId());
+ 
+         return tx.threadId();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long startTime() {
+         if (async)
+             save(tx.startTime());
+ 
+         return tx.startTime();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public IgniteTxIsolation isolation() {
+         if (async)
+             save(tx.isolation());
+ 
+         return tx.isolation();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public IgniteTxConcurrency concurrency() {
+         if (async)
+             save(tx.concurrency());
+ 
+         return tx.concurrency();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public boolean isInvalidate() {
+         if (async)
+             save(tx.isInvalidate());
+ 
+         return tx.isInvalidate();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public boolean implicit() {
+         if (async)
+             save(tx.implicit());
+ 
+         return tx.implicit();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long timeout() {
+         if (async)
+             save(tx.timeout());
+ 
+         return tx.timeout();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public IgniteTxState state() {
+         if (async)
+             save(tx.state());
+ 
+         return tx.state();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long timeout(long timeout) {
+         return tx.timeout(timeout);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public IgniteAsyncSupport enableAsync() {
+         return new IgniteTxProxyImpl<>(tx, cctx, true);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public boolean isAsync() {
+         return async;
+     }
+ 
+     /** {@inheritDoc} */
+     @SuppressWarnings("unchecked")
+     @Override public <R> IgniteFuture<R> future() {
+         return asyncRes;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public boolean setRollbackOnly() {
+         enter();
+ 
+         try {
+             return tx.setRollbackOnly();
+         }
+         finally {
+             leave();
+         }
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public boolean isRollbackOnly() {
+         enter();
+ 
+         try {
+             if (async)
+                 save(tx.isRollbackOnly());
+ 
+             return tx.isRollbackOnly();
+         }
+         finally {
+             leave();
+         }
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public void commit() throws IgniteCheckedException {
+         enter();
+ 
+         try {
+             IgniteFuture<IgniteTx> commitFut = cctx.commitTxAsync(tx);
+ 
+             if (async)
+                 asyncRes = commitFut;
+             else
+                 commitFut.get();
+         }
+         finally {
+             leave();
+         }
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public void close() throws IgniteCheckedException {
+         enter();
+ 
+         try {
+             cctx.endTx(tx);
+         }
+         finally {
+             leave();
+         }
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public void rollback() throws IgniteCheckedException {
+         enter();
+ 
+         try {
+             IgniteFuture rollbackFut = cctx.rollbackTxAsync(tx);
+ 
+             if (async)
+                 asyncRes = rollbackFut;
+             else
+                 rollbackFut.get();
+         }
+         finally {
+             leave();
+         }
+     }
+ 
+     /**
+      * @param res Result to convert to finished future.
+      */
+     private void save(Object res) {
+         asyncRes = new GridFinishedFutureEx<>(res);
+     }
 -
 -    /** {@inheritDoc} */
 -    @Override public void copyMeta(GridMetadataAware from) {
 -        tx.copyMeta(from);
 -    }
 -
 -    /** {@inheritDoc} */
 -    @Override public void copyMeta(Map<String, ?> data) {
 -        tx.copyMeta(data);
 -    }
 -
+     /** {@inheritDoc} */
+     @Override public <V1> V1 addMeta(String name, V1 val) {
+         return tx.addMeta(name, val);
+     }
+ 
+     /** {@inheritDoc} */
 -    @Override public <V1> V1 putMetaIfAbsent(String name, V1 val) {
 -        return tx.putMetaIfAbsent(name, val);
 -    }
 -
 -    /** {@inheritDoc} */
 -    @Override public <V1> V1 putMetaIfAbsent(String name, Callable<V1> c) {
 -        return tx.putMetaIfAbsent(name, c);
 -    }
 -
 -    /** {@inheritDoc} */
 -    @Nullable @Override public <V1> V1 addMetaIfAbsent(String name, V1 val) {
 -        return tx.addMeta(name, val);
 -    }
 -
 -    /** {@inheritDoc} */
 -    @Override public <V1> V1 addMetaIfAbsent(String name, @Nullable 
Callable<V1> c) {
 -        return tx.addMetaIfAbsent(name, c);
 -    }
 -
 -    /** {@inheritDoc} */
+     @SuppressWarnings({"RedundantTypeArguments"})
+     @Override public <V1> V1 meta(String name) {
+         return tx.<V1>meta(name);
+     }
+ 
+     /** {@inheritDoc} */
+     @SuppressWarnings({"RedundantTypeArguments"})
+     @Override public <V1> V1 removeMeta(String name) {
+         return tx.<V1>removeMeta(name);
+     }
+ 
+     /** {@inheritDoc} */
 -    @Override public <V1> boolean removeMeta(String name, V1 val) {
 -        return tx.removeMeta(name, val);
 -    }
 -
 -    /** {@inheritDoc} */
 -    @Override public <V1> Map<String, V1> allMeta() {
 -        return tx.allMeta();
 -    }
 -
 -    /** {@inheritDoc} */
 -    @Override public boolean hasMeta(String name) {
 -        return tx.hasMeta(name);
 -    }
 -
 -    /** {@inheritDoc} */
 -    @Override public <V1> boolean hasMeta(String name, V1 val) {
 -        return tx.hasMeta(name, val);
 -    }
 -
 -    /** {@inheritDoc} */
 -    @Override public <V1> boolean replaceMeta(String name, V1 curVal, V1 
newVal) {
 -        return tx.replaceMeta(name, curVal, newVal);
 -    }
 -
 -    /** {@inheritDoc} */
+     @Override public void writeExternal(ObjectOutput out) throws IOException {
+         out.writeObject(tx);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public void readExternal(ObjectInput in) throws IOException, 
ClassNotFoundException {
+         tx = (IgniteTxEx<K, V>)in.readObject();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public String toString() {
+         return S.toString(IgniteTxProxyImpl.class, this);
+     }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/util/GridUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/util/lang/GridComputeJobWrapper.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/util/lang/GridFunc.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/main/java/org/gridgain/grid/util/lang/GridMetadataAwareAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/test/java/org/gridgain/grid/GridTestJobContext.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/test/java/org/gridgain/grid/lang/GridTupleSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/core/src/test/java/org/gridgain/testframework/junits/logger/GridTestLog4jLogger.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/jcl/src/main/java/org/gridgain/grid/logger/jcl/GridJclLogger.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/IgniteLog4jLogger.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/schedule/src/main/java/org/gridgain/grid/kernal/processors/schedule/ScheduleFutureImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/slf4j/src/main/java/org/gridgain/grid/logger/slf4j/GridSlf4jLogger.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f08a9af7/modules/spring/src/main/java/org/gridgain/grid/GridSpringBean.java
----------------------------------------------------------------------

Reply via email to