You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by nt...@apache.org on 2016/03/15 19:17:57 UTC

[01/35] ignite git commit: IGNITE-2702: .NET: Implemented compact footers optimization for binary serialization. This closes #523.

Repository: ignite
Updated Branches:
  refs/heads/ignite-2791 ef0425361 -> ecb2d58e0


http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
new file mode 100644
index 0000000..da86c07
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
@@ -0,0 +1,262 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Binary
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.IO;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Impl.Binary.IO;
+
+    /// <summary>
+    /// Schema reader/writer.
+    /// </summary>
+    internal static class BinaryObjectSchemaSerializer
+    {
+        /// <summary>
+        /// Converts schema fields to dictionary.
+        /// </summary>
+        /// <param name="fields">The fields.</param>
+        /// <returns>Fields as dictionary.</returns>
+        public static Dictionary<int, int> ToDictionary(this BinaryObjectSchemaField[] fields)
+        {
+            if (fields == null)
+                return null;
+
+            var res = new Dictionary<int, int>(fields.Length);
+
+            foreach (var field in fields)
+                res[field.Id] = field.Offset;
+
+            return res;
+        }
+
+        /// <summary>
+        /// Reads the schema according to this header data.
+        /// </summary>
+        /// <param name="stream">The stream.</param>
+        /// <param name="position">The position.</param>
+        /// <param name="hdr">The header.</param>
+        /// <param name="schema">The schema.</param>
+        /// <param name="marsh">The marshaller.</param>
+        /// <returns>
+        /// Schema.
+        /// </returns>
+        public static BinaryObjectSchemaField[] ReadSchema(IBinaryStream stream, int position, BinaryObjectHeader hdr, 
+            BinaryObjectSchema schema, Marshaller marsh)
+        {
+            Debug.Assert(stream != null);
+            Debug.Assert(schema != null);
+            Debug.Assert(marsh != null);
+
+            return ReadSchema(stream, position, hdr, () => GetFieldIds(hdr, schema, marsh));
+        }
+
+        /// <summary>
+        /// Reads the schema according to this header data.
+        /// </summary>
+        /// <param name="stream">The stream.</param>
+        /// <param name="position">The position.</param>
+        /// <param name="hdr">The header.</param>
+        /// <param name="fieldIdsFunc">The field ids function.</param>
+        /// <returns>
+        /// Schema.
+        /// </returns>
+        public static BinaryObjectSchemaField[] ReadSchema(IBinaryStream stream, int position, BinaryObjectHeader hdr, 
+            Func<int[]> fieldIdsFunc)
+        {
+            Debug.Assert(stream != null);
+            Debug.Assert(fieldIdsFunc != null);
+
+            var schemaSize = hdr.SchemaFieldCount;
+
+            if (schemaSize == 0)
+                return null;
+
+            stream.Seek(position + hdr.SchemaOffset, SeekOrigin.Begin);
+
+            var res = new BinaryObjectSchemaField[schemaSize];
+
+            var offsetSize = hdr.SchemaFieldOffsetSize;
+
+            if (hdr.IsCompactFooter)
+            {
+                var fieldIds = fieldIdsFunc();
+
+                Debug.Assert(fieldIds.Length == schemaSize);
+
+                if (offsetSize == 1)
+                {
+                    for (var i = 0; i < schemaSize; i++)
+                        res[i] = new BinaryObjectSchemaField(fieldIds[i], stream.ReadByte());
+
+                }
+                else if (offsetSize == 2)
+                {
+                    for (var i = 0; i < schemaSize; i++)
+                        res[i] = new BinaryObjectSchemaField(fieldIds[i], stream.ReadShort());
+                }
+                else
+                {
+                    for (var i = 0; i < schemaSize; i++)
+                        res[i] = new BinaryObjectSchemaField(fieldIds[i], stream.ReadInt());
+                }
+            }
+            else
+            {
+                if (offsetSize == 1)
+                {
+                    for (var i = 0; i < schemaSize; i++)
+                        res[i] = new BinaryObjectSchemaField(stream.ReadInt(), stream.ReadByte());
+                }
+                else if (offsetSize == 2)
+                {
+                    for (var i = 0; i < schemaSize; i++)
+                        res[i] = new BinaryObjectSchemaField(stream.ReadInt(), stream.ReadShort());
+                }
+                else
+                {
+                    for (var i = 0; i < schemaSize; i++)
+                        res[i] = new BinaryObjectSchemaField(stream.ReadInt(), stream.ReadInt());
+                }
+            }
+
+            return res;
+        }
+
+        /// <summary>
+        /// Writes an array of fields to a stream.
+        /// </summary>
+        /// <param name="fields">Fields.</param>
+        /// <param name="stream">Stream.</param>
+        /// <param name="offset">Offset in the array.</param>
+        /// <param name="count">Field count to write.</param>
+        /// <param name="compact">Compact mode without field ids.</param>
+        /// <returns>
+        /// Flags according to offset sizes: <see cref="BinaryObjectHeader.Flag.OffsetOneByte" />,
+        /// <see cref="BinaryObjectHeader.Flag.OffsetTwoBytes" />, or 0.
+        /// </returns>
+        public static unsafe BinaryObjectHeader.Flag WriteSchema(BinaryObjectSchemaField[] fields, IBinaryStream stream, int offset,
+            int count, bool compact)
+        {
+            Debug.Assert(fields != null);
+            Debug.Assert(stream != null);
+            Debug.Assert(count > 0);
+            Debug.Assert(offset >= 0);
+            Debug.Assert(offset < fields.Length);
+
+            unchecked
+            {
+                // Last field is the farthest in the stream
+                var maxFieldOffset = fields[offset + count - 1].Offset;
+
+                if (compact)
+                {
+                    if (maxFieldOffset <= byte.MaxValue)
+                    {
+                        for (int i = offset; i < count + offset; i++)
+                            stream.WriteByte((byte)fields[i].Offset);
+
+                        return BinaryObjectHeader.Flag.OffsetOneByte;
+                    }
+
+                    if (maxFieldOffset <= ushort.MaxValue)
+                    {
+                        for (int i = offset; i < count + offset; i++)
+                            stream.WriteShort((short)fields[i].Offset);
+
+                        return BinaryObjectHeader.Flag.OffsetTwoBytes;
+                    }
+
+                    for (int i = offset; i < count + offset; i++)
+                        stream.WriteInt(fields[i].Offset);
+                }
+                else
+                {
+                    if (maxFieldOffset <= byte.MaxValue)
+                    {
+                        for (int i = offset; i < count + offset; i++)
+                        {
+                            var field = fields[i];
+
+                            stream.WriteInt(field.Id);
+                            stream.WriteByte((byte)field.Offset);
+                        }
+
+                        return BinaryObjectHeader.Flag.OffsetOneByte;
+                    }
+
+                    if (maxFieldOffset <= ushort.MaxValue)
+                    {
+                        for (int i = offset; i < count + offset; i++)
+                        {
+                            var field = fields[i];
+
+                            stream.WriteInt(field.Id);
+
+                            stream.WriteShort((short)field.Offset);
+                        }
+
+                        return BinaryObjectHeader.Flag.OffsetTwoBytes;
+                    }
+
+                    if (BitConverter.IsLittleEndian)
+                    {
+                        fixed (BinaryObjectSchemaField* ptr = &fields[offset])
+                        {
+                            stream.Write((byte*)ptr, count / BinaryObjectSchemaField.Size);
+                        }
+                    }
+                    else
+                    {
+                        for (int i = offset; i < count + offset; i++)
+                        {
+                            var field = fields[i];
+
+                            stream.WriteInt(field.Id);
+                            stream.WriteInt(field.Offset);
+                        }
+                    }
+                }
+
+
+                return BinaryObjectHeader.Flag.None;
+            }
+        }
+
+        /// <summary>
+        /// Gets the field ids.
+        /// </summary>
+        private static int[] GetFieldIds(BinaryObjectHeader hdr, BinaryObjectSchema schema, Marshaller marsh)
+        {
+            var fieldIds = schema.Get(hdr.SchemaId);
+
+            if (fieldIds == null)
+            {
+                if (marsh.Ignite != null)
+                    fieldIds = marsh.Ignite.ClusterGroup.GetSchema(hdr.TypeId, hdr.SchemaId);
+
+                if (fieldIds == null)
+                    throw new BinaryObjectException("Cannot find schema for object with compact footer [" +
+                                                    "typeId=" + hdr.TypeId + ", schemaId=" + hdr.SchemaId + ']');
+            }
+            return fieldIds;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
index 16aae93..21c1642 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
@@ -718,16 +718,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                     // Set new frame.
                     _curHdr = hdr;
                     _curPos = pos;
-                    
-                    _curSchema = desc.Schema.Get(hdr.SchemaId);
-
-                    if (_curSchema == null)
-                    {
-                        _curSchema = ReadSchema();
-
-                        desc.Schema.Add(hdr.SchemaId, _curSchema);
-                    }
-
+                    SetCurSchema(desc);
                     _curStruct = new BinaryStructureTracker(desc, desc.ReaderTypeStructure);
                     _curRaw = false;
 
@@ -790,10 +781,40 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
+        /// Sets the current schema.
+        /// </summary>
+        private void SetCurSchema(IBinaryTypeDescriptor desc)
+        {
+            if (_curHdr.HasSchema)
+            {
+                _curSchema = desc.Schema.Get(_curHdr.SchemaId);
+
+                if (_curSchema == null)
+                {
+                    _curSchema = ReadSchema();
+
+                    desc.Schema.Add(_curHdr.SchemaId, _curSchema);
+                }
+            }
+        }
+
+        /// <summary>
         /// Reads the schema.
         /// </summary>
         private int[] ReadSchema()
         {
+            if (_curHdr.IsCompactFooter)
+            {
+                // Get schema from Java
+                var schema = Marshaller.Ignite.ClusterGroup.GetSchema(_curHdr.TypeId, _curHdr.SchemaId);
+
+                if (schema == null)
+                    throw new BinaryObjectException("Cannot find schema for object with compact footer [" +
+                        "typeId=" + _curHdr.TypeId + ", schemaId=" + _curHdr.SchemaId + ']');
+
+                return schema;
+            }
+
             Stream.Seek(_curPos + _curHdr.SchemaOffset, SeekOrigin.Begin);
 
             var count = _curHdr.SchemaFieldCount;
@@ -929,9 +950,10 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             if (_curSchema == null || actionId >= _curSchema.Length || fieldId != _curSchema[actionId])
             {
-                _curSchema = null; // read order is different, ignore schema for future reads
+                _curSchemaMap = _curSchemaMap ?? BinaryObjectSchemaSerializer.ReadSchema(Stream, _curPos, _curHdr,
+                                    () => _curSchema).ToDictionary();
 
-                _curSchemaMap = _curSchemaMap ?? _curHdr.ReadSchemaAsDictionary(Stream, _curPos);
+                _curSchema = null; // read order is different, ignore schema for future reads
 
                 int pos;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
index 5b1273e..47bc2b6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
@@ -1137,6 +1137,9 @@ namespace Apache.Ignite.Core.Impl.Binary
                         ? BinaryObjectHeader.Flag.UserType
                         : BinaryObjectHeader.Flag.None;
 
+                    if (Marshaller.CompactFooter && desc.UserType)
+                        flags |= BinaryObjectHeader.Flag.CompactFooter;
+
                     var hasSchema = _schema.WriteSchema(_stream, schemaIdx, out schemaId, ref flags);
 
                     if (hasSchema)
@@ -1146,6 +1149,10 @@ namespace Apache.Ignite.Core.Impl.Binary
                         // Calculate and write header.
                         if (_curRawPos > 0)
                             _stream.WriteInt(_curRawPos - pos); // raw offset is in the last 4 bytes
+
+                        // Update schema in type descriptor
+                        if (desc.Schema.Get(schemaId) == null)
+                            desc.Schema.Add(schemaId, _schema.GetSchema(schemaIdx));
                     }
                     else
                         schemaOffset = BinaryObjectHeader.Size;
@@ -1451,18 +1458,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                 BinaryType meta;
 
                 if (_metas.TryGetValue(desc.TypeId, out meta))
-                {
-                    if (fields != null)
-                    {
-                        IDictionary<string, int> existingFields = meta.GetFieldsMap();
-
-                        foreach (KeyValuePair<string, int> field in fields)
-                        {
-                            if (!existingFields.ContainsKey(field.Key))
-                                existingFields[field.Key] = field.Value;
-                        }
-                    }
-                }
+                    meta.UpdateFields(fields);
                 else
                     _metas[desc.TypeId] = new BinaryType(desc, fields);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
index 538fbcf..1a01f2c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
@@ -66,6 +66,8 @@ namespace Apache.Ignite.Core.Impl.Binary
             if (cfg == null)
                 cfg = new BinaryConfiguration();
 
+            CompactFooter = cfg.CompactFooter;
+
             if (cfg.TypeConfigurations == null)
                 cfg.TypeConfigurations = new List<BinaryTypeConfiguration>();
 
@@ -107,6 +109,11 @@ namespace Apache.Ignite.Core.Impl.Binary
         public Ignite Ignite { get; set; }
 
         /// <summary>
+        /// Gets the compact footer flag.
+        /// </summary>
+        public bool CompactFooter { get; set; }
+
+        /// <summary>
         /// Marshal object.
         /// </summary>
         /// <param name="val">Value.</param>
@@ -281,15 +288,14 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// Puts the binary type metadata to Ignite.
         /// </summary>
         /// <param name="desc">Descriptor.</param>
-        /// <param name="fields">Fields.</param>
-        public void PutBinaryType(IBinaryTypeDescriptor desc, IDictionary<string, int> fields = null)
+        public void PutBinaryType(IBinaryTypeDescriptor desc)
         {
             Debug.Assert(desc != null);
 
             GetBinaryTypeHandler(desc);  // ensure that handler exists
 
             if (Ignite != null)
-                Ignite.PutBinaryTypes(new[] {new BinaryType(desc, fields)});
+                Ignite.PutBinaryTypes(new[] {new BinaryType(desc)});
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Metadata/BinaryType.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Metadata/BinaryType.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Metadata/BinaryType.cs
index 28dfb1a..cb0d3cd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Metadata/BinaryType.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Metadata/BinaryType.cs
@@ -18,6 +18,7 @@
 namespace Apache.Ignite.Core.Impl.Binary.Metadata
 {
     using System.Collections.Generic;
+    using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
     using Apache.Ignite.Core.Binary;
 
@@ -54,6 +55,9 @@ namespace Apache.Ignite.Core.Impl.Binary.Metadata
         /** Aff key field name. */
         private readonly string _affinityKeyFieldName;
 
+        /** Type descriptor. */
+        private readonly IBinaryTypeDescriptor _descriptor;
+
         /// <summary>
         /// Initializes the <see cref="BinaryType"/> class.
         /// </summary>
@@ -129,7 +133,7 @@ namespace Apache.Ignite.Core.Impl.Binary.Metadata
         public BinaryType(IBinaryTypeDescriptor desc, IDictionary<string, int> fields = null) 
             : this (desc.TypeId, desc.TypeName, fields, desc.AffinityKeyFieldName, desc.IsEnum)
         {
-            // No-op.
+            _descriptor = desc;
         }
 
         /// <summary>
@@ -211,6 +215,14 @@ namespace Apache.Ignite.Core.Impl.Binary.Metadata
         }
 
         /// <summary>
+        /// Gets the descriptor.
+        /// </summary>
+        public IBinaryTypeDescriptor Descriptor
+        {
+            get { return _descriptor; }
+        }
+
+        /// <summary>
         /// Gets fields map.
         /// </summary>
         /// <returns>Fields map.</returns>
@@ -218,5 +230,19 @@ namespace Apache.Ignite.Core.Impl.Binary.Metadata
         {
             return _fields ?? EmptyDict;
         }
+
+        /// <summary>
+        /// Updates the fields.
+        /// </summary>
+        public void UpdateFields(IDictionary<string, int> fields)
+        {
+            if (fields == null || fields.Count == 0)
+                return;
+
+            Debug.Assert(_fields != null);
+
+            foreach (var field in fields)
+                _fields[field.Key] = field.Value;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
index fc673a6..e6c0005 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
@@ -96,6 +96,9 @@ namespace Apache.Ignite.Core.Impl.Cluster
         /** */
         private const int OpTopology = 14;
 
+        /** */
+        private const int OpSchema = 15;
+
         /** Initial Ignite instance. */
         private readonly Ignite _ignite;
         
@@ -570,5 +573,17 @@ namespace Apache.Ignite.Core.Impl.Cluster
                 return res;
             });
         }
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public int[] GetSchema(int typeId, int schemaId)
+        {
+            return DoOutInOp<int[]>(OpSchema, writer =>
+            {
+                writer.WriteInt(typeId);
+                writer.WriteInt(schemaId);
+            });
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
index 0271fa2..1735fb8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+#pragma warning disable 618   // SpringConfigUrl
 namespace Apache.Ignite.Core.Impl
 {
     using System;
@@ -135,6 +136,24 @@ namespace Apache.Ignite.Core.Impl
 
             // Set reconnected task to completed state for convenience.
             _clientReconnectTaskCompletionSource.SetResult(false);
+
+            SetCompactFooter();
+        }
+
+        /// <summary>
+        /// Sets the compact footer setting.
+        /// </summary>
+        private void SetCompactFooter()
+        {
+            if (!string.IsNullOrEmpty(_cfg.SpringConfigUrl))
+            {
+                // If there is a Spring config, use setting from Spring, 
+                // since we ignore .NET config in legacy mode.
+                var cfg0 = GetConfiguration().BinaryConfiguration;
+
+                if (cfg0 != null)
+                    _marsh.CompactFooter = cfg0.CompactFooter;
+            }
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
index c4258bd..26b6033 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
@@ -604,32 +604,53 @@ namespace Apache.Ignite.Core.Impl
         {
             DoOutOp(OpMeta, stream =>
             {
-                BinaryWriter metaWriter = _marsh.StartMarshal(stream);
+                BinaryWriter w = _marsh.StartMarshal(stream);
 
-                metaWriter.WriteInt(types.Count);
+                w.WriteInt(types.Count);
 
                 foreach (var meta in types)
                 {
-                    BinaryType meta0 = meta;
+                    w.WriteInt(meta.TypeId);
+                    w.WriteString(meta.TypeName);
+                    w.WriteString(meta.AffinityKeyFieldName);
 
-                    metaWriter.WriteInt(meta0.TypeId);
-                    metaWriter.WriteString(meta0.TypeName);
-                    metaWriter.WriteString(meta0.AffinityKeyFieldName);
+                    IDictionary<string, int> fields = meta.GetFieldsMap();
 
-                    IDictionary<string, int> fields = meta0.GetFieldsMap();
-
-                    metaWriter.WriteInt(fields.Count);
+                    w.WriteInt(fields.Count);
 
                     foreach (var field in fields)
                     {
-                        metaWriter.WriteString(field.Key);
-                        metaWriter.WriteInt(field.Value);
+                        w.WriteString(field.Key);
+                        w.WriteInt(field.Value);
+                    }
+
+                    w.WriteBoolean(meta.IsEnum);
+
+                    // Send schemas
+                    var desc = meta.Descriptor;
+                    Debug.Assert(desc != null);
+
+                    var count = 0;
+                    var countPos = stream.Position;
+                    w.WriteInt(0);  // Reserve for count
+
+                    foreach (var schema in desc.Schema.GetAll())
+                    {
+                        w.WriteInt(schema.Key);
+
+                        var ids = schema.Value;
+                        w.WriteInt(ids.Length);
+
+                        foreach (var id in ids)
+                            w.WriteInt(id);
+
+                        count++;
                     }
 
-                    metaWriter.WriteBoolean(meta.IsEnum);
+                    stream.WriteInt(countPos, count);
                 }
 
-                _marsh.FinishMarshal(metaWriter);
+                _marsh.FinishMarshal(w);
             });
 
             _marsh.OnBinaryTypesSent(types);


[34/35] ignite git commit: IGNITE-2838: IGFS: Opimized format of IgfsListingEntry. Now it contains only file ID and boolean flag endicating whether this a directory or file.

Posted by nt...@apache.org.
IGNITE-2838: IGFS: Opimized format of IgfsListingEntry. Now it contains only file ID and boolean flag endicating whether this a directory or file.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/7b7233ef
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/7b7233ef
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/7b7233ef

Branch: refs/heads/ignite-2791
Commit: 7b7233efeeacc0c65833dec925a73906d9880608
Parents: 18de768
Author: thatcoach <pp...@list.ru>
Authored: Tue Mar 15 20:46:13 2016 +0300
Committer: thatcoach <pp...@list.ru>
Committed: Tue Mar 15 20:46:13 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/igfs/IgfsFileImpl.java  |  29 ---
 .../internal/processors/igfs/IgfsFileInfo.java  |  38 +++-
 .../internal/processors/igfs/IgfsImpl.java      |   8 +-
 .../processors/igfs/IgfsListingEntry.java       | 129 ++---------
 .../processors/igfs/IgfsMetaManager.java        | 221 +++----------------
 .../processors/igfs/IgfsOutputStreamImpl.java   |   2 -
 6 files changed, 88 insertions(+), 339 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7b7233ef/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java
index 4a96e81..3576a06 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java
@@ -124,35 +124,6 @@ public final class IgfsFileImpl implements IgfsFile, Externalizable {
         modificationTime = info.modificationTime();
     }
 
-    /**
-     * Constructs file instance.
-     *
-     * @param path Path.
-     * @param entry Listing entry.
-     */
-    public IgfsFileImpl(IgfsPath path, IgfsListingEntry entry, long globalGrpSize) {
-        A.notNull(path, "path");
-        A.notNull(entry, "entry");
-
-        this.path = path;
-        fileId = entry.fileId();
-
-        blockSize = entry.blockSize();
-
-        // By contract file must have blockSize > 0, while directory's blockSize == 0:
-        assert entry.isFile() == (blockSize > 0);
-        assert entry.isDirectory() == (blockSize == 0);
-
-        grpBlockSize = entry.affinityKey() == null ? globalGrpSize :
-            entry.length() == 0 ? globalGrpSize : entry.length();
-
-        len = entry.length();
-        props = entry.properties();
-
-        accessTime = entry.accessTime();
-        modificationTime = entry.modificationTime();
-    }
-
     /** {@inheritDoc} */
     @Override public IgfsPath path() {
         return path;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b7233ef/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
index 0a85657..ba484bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
@@ -297,9 +297,6 @@ public final class IgfsFileInfo implements Externalizable {
         this.props = props == null || props.isEmpty() ? null :
             cpProps ? new GridLeanMap<>(props) : props;
 
-        if (listing == null && isDir)
-            this.listing = Collections.emptyMap();
-
         this.lockId = lockId;
         this.evictExclude = evictExclude;
     }
@@ -410,13 +407,38 @@ public final class IgfsFileInfo implements Externalizable {
      * @return Directory listing.
      */
     public Map<String, IgfsListingEntry> listing() {
-        // Always wrap into unmodifiable map to be able to avoid illegal modifications in order pieces of the code.
-        if (isFile())
-            return Collections.unmodifiableMap(Collections.<String, IgfsListingEntry>emptyMap());
+        return listing != null ? listing : Collections.<String, IgfsListingEntry>emptyMap();
+    }
+
+    /**
+     * @return {@code True} if at least one child exists.
+     */
+    public boolean hasChildren() {
+        return !F.isEmpty(listing);
+    }
+
+    /**
+     * @param name Child name.
+     * @return {@code True} if child with such name exists.
+     */
+    public boolean hasChild(String name) {
+        return listing != null && listing.containsKey(name);
+    }
 
-        assert listing != null;
+    /**
+     * @param name Child name.
+     * @param expId Expected child ID.
+     * @return {@code True} if child with such name exists.
+     */
+    public boolean hasChild(String name, IgniteUuid expId) {
+        if (listing != null) {
+            IgfsListingEntry entry = listing.get(name);
+
+            if (entry != null)
+                return F.eq(expId, entry.fileId());
+        }
 
-        return Collections.unmodifiableMap(listing);
+        return false;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b7233ef/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index 0e52927..f44eda8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -870,9 +870,13 @@ public final class IgfsImpl implements IgfsEx {
 
                         // Perform the listing.
                         for (Map.Entry<String, IgfsListingEntry> e : info.listing().entrySet()) {
-                            IgfsPath p = new IgfsPath(path, e.getKey());
+                            IgfsFileInfo childInfo = meta.info(e.getValue().fileId());
 
-                            files.add(new IgfsFileImpl(p, e.getValue(), data.groupBlockSize()));
+                            if (childInfo != null) {
+                                IgfsPath childPath = new IgfsPath(path, e.getKey());
+
+                                files.add(new IgfsFileImpl(childPath, childInfo, data.groupBlockSize()));
+                            }
                         }
                     }
                 } else if (mode == PRIMARY) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b7233ef/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java
index 1cdc8a9..61d9265 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java
@@ -21,7 +21,8 @@ import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
-import java.util.Map;
+
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
@@ -34,25 +35,10 @@ public class IgfsListingEntry implements Externalizable {
     private static final long serialVersionUID = 0L;
 
     /** File id. */
-    private IgniteUuid fileId;
-
-    /** File affinity key. */
-    private IgniteUuid affKey;
-
-    /** Positive block size if file, 0 if directory. */
-    private int blockSize;
-
-    /** File length. */
-    private long len;
-
-    /** Last access time. */
-    private long accessTime;
+    private IgniteUuid id;
 
-    /** Last modification time. */
-    private long modificationTime;
-
-    /** File properties. */
-    private Map<String, String> props;
+    /** Directory marker. */
+    private boolean dir;
 
     /**
      * Empty constructor required by {@link Externalizable}.
@@ -65,132 +51,51 @@ public class IgfsListingEntry implements Externalizable {
      * @param fileInfo File info to construct listing entry from.
      */
     public IgfsListingEntry(IgfsFileInfo fileInfo) {
-        fileId = fileInfo.id();
-        affKey = fileInfo.affinityKey();
-
-        if (fileInfo.isFile()) {
-            blockSize = fileInfo.blockSize();
-            len = fileInfo.length();
-        }
-
-        props = fileInfo.properties();
-        accessTime = fileInfo.accessTime();
-        modificationTime = fileInfo.modificationTime();
-    }
-
-    /**
-     * Creates listing entry with updated length.
-     *
-     * @param entry Entry.
-     * @param len New length.
-     */
-    public IgfsListingEntry(IgfsListingEntry entry, long len, long accessTime, long modificationTime) {
-        fileId = entry.fileId;
-        affKey = entry.affKey;
-        blockSize = entry.blockSize;
-        props = entry.props;
-        this.accessTime = accessTime;
-        this.modificationTime = modificationTime;
-
-        this.len = len;
+        id = fileInfo.id();
+        dir = fileInfo.isDirectory();
     }
 
     /**
      * @return Entry file ID.
      */
     public IgniteUuid fileId() {
-        return fileId;
-    }
-
-    /**
-     * @return File affinity key, if specified.
-     */
-    public IgniteUuid affinityKey() {
-        return affKey;
+        return id;
     }
 
     /**
      * @return {@code True} if entry represents file.
      */
     public boolean isFile() {
-        return blockSize > 0;
+        return !dir;
     }
 
     /**
      * @return {@code True} if entry represents directory.
      */
     public boolean isDirectory() {
-        return blockSize == 0;
-    }
-
-    /**
-     * @return Block size.
-     */
-    public int blockSize() {
-        return blockSize;
-    }
-
-    /**
-     * @return Length.
-     */
-    public long length() {
-        return len;
-    }
-
-    /**
-     * @return Last access time.
-     */
-    public long accessTime() {
-        return accessTime;
-    }
-
-    /**
-     * @return Last modification time.
-     */
-    public long modificationTime() {
-        return modificationTime;
-    }
-
-    /**
-     * @return Properties map.
-     */
-    public Map<String, String> properties() {
-        return props;
+        return dir;
     }
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
-        U.writeGridUuid(out, fileId);
-        out.writeInt(blockSize);
-        out.writeLong(len);
-        U.writeStringMap(out, props);
-        out.writeLong(accessTime);
-        out.writeLong(modificationTime);
+        U.writeGridUuid(out, id);
+        out.writeBoolean(dir);
     }
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        fileId = U.readGridUuid(in);
-        blockSize = in.readInt();
-        len = in.readLong();
-        props = U.readStringMap(in);
-        accessTime = in.readLong();
-        modificationTime = in.readLong();
+        id = U.readGridUuid(in);
+        dir = in.readBoolean();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (this == o) return true;
-        if (!(o instanceof IgfsListingEntry)) return false;
-
-        IgfsListingEntry that = (IgfsListingEntry)o;
-
-        return fileId.equals(that.fileId);
+    @Override public boolean equals(Object other) {
+        return this == other || other instanceof IgfsListingEntry && F.eq(id, ((IgfsListingEntry)other).id);
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return fileId.hashCode();
+        return id.hashCode();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b7233ef/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index d3aae58..463e7a8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -801,18 +801,12 @@ public class IgfsMetaManager extends IgfsManager {
         if (!parentInfo.isDirectory())
             throw fsException(new IgfsPathIsNotDirectoryException("Parent file is not a directory: " + parentInfo));
 
-        Map<String, IgfsListingEntry> parentListing = parentInfo.listing();
+        IgfsListingEntry childEntry = parentInfo.listing().get(fileName);
 
-        assert parentListing != null;
+        if (childEntry != null)
+            return childEntry.fileId();
 
-        IgfsListingEntry entry = parentListing.get(fileName);
-
-        if (entry != null)
-            return entry.fileId();
-
-        IgniteUuid fileId = newFileInfo.id();
-
-        if (!id2InfoPrj.putIfAbsent(fileId, newFileInfo))
+        if (!id2InfoPrj.putIfAbsent(newFileInfo.id(), newFileInfo))
             throw fsException("Failed to add file details into cache: " + newFileInfo);
 
         id2InfoPrj.invoke(parentId, new ListingAdd(fileName, new IgfsListingEntry(newFileInfo)));
@@ -915,7 +909,7 @@ public class IgfsMetaManager extends IgfsManager {
                     assert dstTargetInfo.isDirectory();
 
                     // 7. Last check: does destination target already have listing entry with the same name?
-                    if (dstTargetInfo.listing().containsKey(dstName)) {
+                    if (dstTargetInfo.hasChild(dstName)) {
                         throw new IgfsPathAlreadyExistsException("Failed to perform move because destination already " +
                             "contains entry with the same name existing file [src=" + srcPath +
                             ", dst=" + dstPath + ']');
@@ -973,14 +967,8 @@ public class IgfsMetaManager extends IgfsManager {
 
                 // If parent info is null, it doesn't exist.
                 if (parentInfo != null) {
-                    IgfsListingEntry childEntry = parentInfo.listing().get(pathParts.get(i));
-
-                    // If expected child exists.
-                    if (childEntry != null) {
-                        // If child ID matches expected ID.
-                        if (F.eq(childEntry.fileId(), expIds.get(i + 1)))
-                            continue;
-                    }
+                    if (parentInfo.hasChild(pathParts.get(i), expIds.get(i + 1)))
+                        continue;
                 }
             }
 
@@ -1046,7 +1034,6 @@ public class IgfsMetaManager extends IgfsManager {
                 fileId + ']'));
 
         IgfsListingEntry srcEntry = srcInfo.listing().get(srcFileName);
-        IgfsListingEntry destEntry = destInfo.listing().get(destFileName);
 
         // If source file does not exist or was re-created.
         if (srcEntry == null || !srcEntry.fileId().equals(fileId))
@@ -1055,10 +1042,10 @@ public class IgfsMetaManager extends IgfsManager {
                 ", srcParentId=" + srcParentId + ", srcEntry=" + srcEntry + ']'));
 
         // If stored file already exist.
-        if (destEntry != null)
+        if (destInfo.hasChild(destFileName))
             throw fsException(new IgfsPathAlreadyExistsException("Failed to add file name into the destination " +
                 " directory (file already exists) [fileId=" + fileId + ", destFileName=" + destFileName +
-                ", destParentId=" + destParentId + ", destEntry=" + destEntry + ']'));
+                ", destParentId=" + destParentId + ']'));
 
         // Remove listing entry from the source parent listing.
         id2InfoPrj.invoke(srcParentId, new ListingRemove(srcFileName, srcEntry.fileId()));
@@ -1174,7 +1161,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                     final IgfsFileInfo victimInfo = infoMap.get(victimId);
 
-                    if (!recursive && victimInfo.isDirectory() && !victimInfo.listing().isEmpty())
+                    if (!recursive && victimInfo.hasChildren())
                         // Throw exception if not empty and not recursive.
                         throw new IgfsDirectoryNotEmptyException("Failed to remove directory (directory is not " +
                             "empty and recursive flag is not set).");
@@ -1187,7 +1174,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                     final String destFileName = victimId.toString();
 
-                    assert destInfo.listing().get(destFileName) == null : "Failed to add file name into the " +
+                    assert !destInfo.hasChild(destFileName) : "Failed to add file name into the " +
                         "destination directory (file already exists) [destName=" + destFileName + ']';
 
                     IgfsFileInfo srcParentInfo = infoMap.get(pathIdList.get(pathIdList.size() - 2));
@@ -1338,8 +1325,8 @@ public class IgfsMetaManager extends IgfsManager {
 
                     int i = 1;
 
-                    for (IgfsListingEntry entry : listing.values())
-                        allIds[i++] = entry.fileId();
+                    for (IgfsListingEntry childEntry : listing.values())
+                        allIds[i++] = childEntry.fileId();
 
                     Map<IgniteUuid, IgfsFileInfo> locks = lockIds(allIds);
 
@@ -1347,35 +1334,37 @@ public class IgfsMetaManager extends IgfsManager {
 
                     // Ensure parent is still in place.
                     if (parentInfo != null) {
-                        Map<String, IgfsListingEntry> newListing =
-                            new HashMap<>(parentInfo.listing().size(), 1.0f);
+                        Map<String, IgfsListingEntry> parentListing = parentInfo.listing();
+
+                        Map<String, IgfsListingEntry> newListing = new HashMap<>(parentListing.size(), 1.0f);
 
-                        newListing.putAll(parentInfo.listing());
+                        newListing.putAll(parentListing);
 
                         // Remove child entries if possible.
                         for (Map.Entry<String, IgfsListingEntry> entry : listing.entrySet()) {
-                            IgniteUuid entryId = entry.getValue().fileId();
+                            String childName = entry.getKey();
+                            IgniteUuid childId = entry.getValue().fileId();
 
-                            IgfsFileInfo entryInfo = locks.get(entryId);
+                            IgfsFileInfo entryInfo = locks.get(childId);
 
                             if (entryInfo != null) {
                                 // File must be locked for deletion:
                                 assert entryInfo.isDirectory() || IgfsUtils.DELETE_LOCK_ID.equals(entryInfo.lockId());
 
                                 // Delete only files or empty folders.
-                                if (entryInfo.isFile() || entryInfo.isDirectory() && entryInfo.listing().isEmpty()) {
-                                    id2InfoPrj.getAndRemove(entryId);
+                                if (!entryInfo.hasChildren()) {
+                                    id2InfoPrj.getAndRemove(childId);
 
-                                    newListing.remove(entry.getKey());
+                                    newListing.remove(childName);
 
-                                    res.add(entryId);
+                                    res.add(childId);
                                 }
                             }
                             else {
                                 // Entry was deleted concurrently.
-                                newListing.remove(entry.getKey());
+                                newListing.remove(childName);
 
-                                res.add(entryId);
+                                res.add(childId);
                             }
                         }
 
@@ -1436,10 +1425,10 @@ public class IgfsMetaManager extends IgfsManager {
 
                         assert parentInfo != null;
 
-                        IgfsListingEntry listingEntry = parentInfo.listing().get(name);
+                        IgfsListingEntry childEntry = parentInfo.listing().get(name);
 
-                        if (listingEntry != null)
-                            id2InfoPrj.invoke(parentId, new ListingRemove(name, listingEntry.fileId()));
+                        if (childEntry != null)
+                            id2InfoPrj.invoke(parentId, new ListingRemove(name, childEntry.fileId()));
 
                         IgfsFileInfo deleted = id2InfoPrj.getAndRemove(id);
 
@@ -1481,13 +1470,9 @@ public class IgfsMetaManager extends IgfsManager {
 
                     IgfsFileInfo trashInfo = id2InfoPrj.get(trashId);
 
-                    if (trashInfo != null) {
-                        Map<String, IgfsListingEntry> listing = trashInfo.listing();
-
-                        if (listing != null && !listing.isEmpty()) {
-                            for (IgfsListingEntry entry : listing.values())
-                                ids.add(entry.fileId());
-                        }
+                    if (trashInfo != null && trashInfo.hasChildren()) {
+                        for (IgfsListingEntry entry : trashInfo.listing().values())
+                            ids.add(entry.fileId());
                     }
                 }
 
@@ -1543,11 +1528,7 @@ public class IgfsMetaManager extends IgfsManager {
                 return null; // File not found.
 
             if (parentInfo != null) {
-                Map<String, IgfsListingEntry> listing = parentInfo.listing();
-
-                IgfsListingEntry entry = listing.get(fileName);
-
-                if (entry == null || !entry.fileId().equals(fileId)) // File was removed or recreated.
+                if (!parentInfo.hasChild(fileName, fileId)) // File was removed or recreated.
                     return null;
             }
 
@@ -1622,35 +1603,6 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Asynchronously updates record in parent listing.
-     *
-     * @param parentId Parent ID.
-     * @param fileId File ID.
-     * @param fileName File name.
-     * @param lenDelta Length delta.
-     * @param modificationTime Last modification time.
-     */
-    public void updateParentListingAsync(IgniteUuid parentId, IgniteUuid fileId, String fileName, long lenDelta,
-        long modificationTime) {
-        if (busyLock.enterBusy()) {
-            try {
-                assert parentId != null;
-
-                assert validTxState(false);
-
-                id2InfoPrj.invokeAsync(parentId, new UpdateListingEntry(fileId, fileName, lenDelta, -1,
-                    modificationTime));
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to update parent listing because Grid is stopping [parentId=" +
-                parentId + ", fileId=" + fileId + ", fileName=" + fileName + ']');
-    }
-
-    /**
      * Reserve space for file.
      *
      * @param path File path.
@@ -2980,10 +2932,8 @@ public class IgfsMetaManager extends IgfsManager {
                         throw fsException(new IgfsPathNotFoundException("Failed to update times " +
                             "(parent was not found): " + fileName));
 
-                    IgfsListingEntry entry = parentInfo.listing().get(fileName);
-
                     // Validate listing.
-                    if (entry == null || !entry.fileId().equals(fileId))
+                    if (!parentInfo.hasChild(fileName, fileId))
                         throw fsException(new IgfsConcurrentModificationException("Failed to update times " +
                                 "(file concurrently modified): " + fileName));
 
@@ -2994,9 +2944,6 @@ public class IgfsMetaManager extends IgfsManager {
                         modificationTime == -1 ? fileInfo.modificationTime() : modificationTime)
                     );
 
-                    id2InfoPrj.invoke(parentId, new UpdateListingEntry(fileId, fileName, 0, accessTime,
-                        modificationTime));
-
                     tx.commit();
                 }
                 finally {
@@ -3143,100 +3090,6 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Updates file length information in parent listing.
-     */
-    private static final class UpdateListingEntry implements EntryProcessor<IgniteUuid, IgfsFileInfo, Void>,
-        Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** File name. */
-        private String fileName;
-
-        /** File id. */
-        private IgniteUuid fileId;
-
-        /** Length delta. */
-        private long lenDelta;
-
-        /** Last access time. */
-        private long accessTime;
-
-        /** Last modification time. */
-        private long modificationTime;
-
-        /**
-         * Empty constructor required by {@link Externalizable}.
-         */
-        public UpdateListingEntry() {
-            // No-op.
-        }
-
-        /**
-         * @param fileId Expected file id in parent directory listing.
-         * @param fileName File name.
-         * @param lenDelta Length delta.
-         * @param accessTime Last access time.
-         * @param modificationTime Last modification time.
-         */
-        private UpdateListingEntry(IgniteUuid fileId,
-            String fileName,
-            long lenDelta,
-            long accessTime,
-            long modificationTime) {
-            this.fileId = fileId;
-            this.fileName = fileName;
-            this.lenDelta = lenDelta;
-            this.accessTime = accessTime;
-            this.modificationTime = modificationTime;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<IgniteUuid, IgfsFileInfo> e, Object... args) {
-            IgfsFileInfo fileInfo = e.getValue();
-
-            Map<String, IgfsListingEntry> listing = fileInfo.listing();
-
-            IgfsListingEntry entry = listing.get(fileName);
-
-            if (entry == null || !entry.fileId().equals(fileId))
-                return null;
-
-            entry = new IgfsListingEntry(entry, entry.length() + lenDelta,
-                accessTime == -1 ? entry.accessTime() : accessTime,
-                modificationTime == -1 ? entry.modificationTime() : modificationTime);
-
-            // Create new map to replace info.
-            listing = new HashMap<>(listing);
-
-            // Modify listing map in-place since map is serialization-safe.
-            listing.put(fileName, entry);
-
-            e.setValue(new IgfsFileInfo(listing, fileInfo));
-
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            U.writeGridUuid(out, fileId);
-            out.writeUTF(fileName);
-            out.writeLong(lenDelta);
-            out.writeLong(accessTime);
-            out.writeLong(modificationTime);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException {
-            fileId = U.readGridUuid(in);
-            fileName = in.readUTF();
-            lenDelta = in.readLong();
-            accessTime = in.readLong();
-            modificationTime = in.readLong();
-        }
-    }
-
-    /**
      * Remove entry from directory listing.
      */
     @GridInternal
@@ -3608,13 +3461,9 @@ public class IgfsMetaManager extends IgfsManager {
                                 throw new IgfsParentNotDirectoryException("Failed to " + (append ? "open" : "create" )
                                     + " file (parent element is not a directory)");
 
-                            Map<String, IgfsListingEntry> parentListing = lowermostExistingInfo.listing();
-
                             final String uppermostFileToBeCreatedName = b.components.get(b.existingIdCnt - 1);
 
-                            final IgfsListingEntry entry = parentListing.get(uppermostFileToBeCreatedName);
-
-                            if (entry == null) {
+                            if (!lowermostExistingInfo.hasChild(uppermostFileToBeCreatedName)) {
                                 b.doBuild();
 
                                 assert b.leafInfo != null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b7233ef/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
index 8c11073..3bf1011 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
@@ -381,8 +381,6 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
                     throw new IOException("File to read file metadata: " + fileInfo.path(), e);
                 }
 
-                meta.updateParentListingAsync(parentId, fileInfo.id(), fileName, bytes, modificationTime);
-
                 if (err != null)
                     throw err;
             }


[27/35] ignite git commit: IGNITE-2810: IGFS: Minor correction to IgfsUtils.isRootOrTrashId() method.

Posted by nt...@apache.org.
IGNITE-2810: IGFS: Minor correction to IgfsUtils.isRootOrTrashId() method.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/d83fa116
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d83fa116
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d83fa116

Branch: refs/heads/ignite-2791
Commit: d83fa11663f121097753952429c10fd80451bfd4
Parents: 2d14842
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Mar 14 12:05:39 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Mar 14 12:05:39 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/igfs/IgfsUtils.java     | 22 ++++++++++----------
 1 file changed, 11 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d83fa116/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
index 6f8960a..edded2f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
@@ -98,12 +98,22 @@ public class IgfsUtils {
     }
 
     /**
+     * Check whether provided ID is either root ID or trash ID.
+     *
+     * @param id ID.
+     * @return {@code True} if this is root ID or trash ID.
+     */
+    public static boolean isRootOrTrashId(@Nullable IgniteUuid id) {
+        return id != null && (ROOT_ID.equals(id) || isTrashId(id));
+    }
+
+    /**
      * Check whether provided ID is trash ID.
      *
      * @param id ID.
      * @return {@code True} if this is trash ID.
      */
-    public static boolean isTrashId(IgniteUuid id) {
+    private static boolean isTrashId(IgniteUuid id) {
         assert id != null;
 
         UUID gid = id.globalId();
@@ -113,16 +123,6 @@ public class IgfsUtils {
     }
 
     /**
-     * Check whether provided ID is either root ID or trash ID.
-     *
-     * @param id ID.
-     * @return {@code True} if this is root ID or trash ID.
-     */
-    public static boolean isRootOrTrashId(IgniteUuid id) {
-        return ROOT_ID.equals(id) || isTrashId(id);
-    }
-
-    /**
      * Converts any passed exception to IGFS exception.
      *
      * @param err Initial exception.


[26/35] ignite git commit: IGNITE-2810: IGFS: Striped trash directory to reduce contention during removals.

Posted by nt...@apache.org.
IGNITE-2810: IGFS: Striped trash directory to reduce contention during removals.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/2d14842a
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/2d14842a
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/2d14842a

Branch: refs/heads/ignite-2791
Commit: 2d14842aba6a53d689d4b4f90a907e2c0f15d291
Parents: c53ffa9
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Mar 14 11:49:03 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Mar 14 11:49:03 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsDeleteWorker.java       |  28 ++--
 .../internal/processors/igfs/IgfsFileInfo.java  |   9 +-
 .../internal/processors/igfs/IgfsImpl.java      |   5 +-
 .../processors/igfs/IgfsMetaManager.java        | 146 ++++++++++---------
 .../processors/igfs/IgfsOutputStreamImpl.java   |   2 +-
 .../internal/processors/igfs/IgfsUtils.java     |  72 +++++++++
 .../processors/igfs/IgfsAbstractSelfTest.java   |  21 ++-
 .../igfs/IgfsDataManagerSelfTest.java           |   6 +-
 .../igfs/IgfsMetaManagerSelfTest.java           |   2 +-
 .../processors/igfs/IgfsProcessorSelfTest.java  |   9 +-
 .../processors/igfs/IgfsSizeSelfTest.java       |  13 +-
 11 files changed, 199 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2d14842a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
index 95a6a5d..ffddd3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
@@ -39,7 +39,6 @@ import org.apache.ignite.lang.IgniteUuid;
 
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_PURGED;
 import static org.apache.ignite.internal.GridTopic.TOPIC_IGFS;
-import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.TRASH_ID;
 
 /**
  * IGFS worker for removal from the trash directory.
@@ -150,13 +149,23 @@ public class IgfsDeleteWorker extends IgfsThread {
     }
 
     /**
-     * Perform cleanup of the trash directory.
+     * Perform cleanup of trash directories.
      */
     private void delete() {
+        for (int i = 0; i < IgfsUtils.TRASH_CONCURRENCY; i++)
+            delete(IgfsUtils.trashId(i));
+    }
+
+    /**
+     * Perform cleanup of concrete trash directory.
+     *
+     * @param trashId Trash ID.
+     */
+    private void delete(IgniteUuid trashId) {
         IgfsFileInfo info = null;
 
         try {
-            info = meta.info(TRASH_ID);
+            info = meta.info(trashId);
         }
         catch(ClusterTopologyServerNotFoundException e) {
             LT.warn(log, e, "Server nodes not found.");
@@ -174,7 +183,7 @@ public class IgfsDeleteWorker extends IgfsThread {
 
                 try {
                     if (!cancelled) {
-                        if (delete(entry.getKey(), fileId)) {
+                        if (delete(trashId, entry.getKey(), fileId)) {
                             if (log.isDebugEnabled())
                                 log.debug("Sending delete confirmation message [name=" + entry.getKey() +
                                     ", fileId=" + fileId + ']');
@@ -200,12 +209,13 @@ public class IgfsDeleteWorker extends IgfsThread {
     /**
      * Remove particular entry from the TRASH directory.
      *
+     * @param trashId ID of the trash directory.
      * @param name Entry name.
      * @param id Entry ID.
      * @return {@code True} in case the entry really was deleted form the file system by this call.
      * @throws IgniteCheckedException If failed.
      */
-    private boolean delete(String name, IgniteUuid id) throws IgniteCheckedException {
+    private boolean delete(IgniteUuid trashId, String name, IgniteUuid id) throws IgniteCheckedException {
         assert name != null;
         assert id != null;
 
@@ -214,10 +224,10 @@ public class IgfsDeleteWorker extends IgfsThread {
 
             if (info != null) {
                 if (info.isDirectory()) {
-                    if (!deleteDirectoryContents(TRASH_ID, id))
+                    if (!deleteDirectoryContents(trashId, id))
                         return false;
 
-                    if (meta.delete(TRASH_ID, name, id))
+                    if (meta.delete(trashId, name, id))
                         return true;
                 }
                 else {
@@ -235,7 +245,7 @@ public class IgfsDeleteWorker extends IgfsThread {
                     // In case this node crashes, other node will re-delete the file.
                     data.delete(lockedInfo).get();
 
-                    boolean ret = meta.delete(TRASH_ID, name, id);
+                    boolean ret = meta.delete(trashId, name, id);
 
                     if (info.path() != null)
                         IgfsUtils.sendEvents(igfsCtx.kernalContext(), info.path(), EVT_IGFS_FILE_PURGED);
@@ -299,7 +309,7 @@ public class IgfsDeleteWorker extends IgfsThread {
                                 // File is already locked:
                                 failedFiles++;
                             else {
-                                assert IgfsMetaManager.DELETE_LOCK_ID.equals(lockedInfo.lockId());
+                                assert IgfsUtils.DELETE_LOCK_ID.equals(lockedInfo.lockId());
 
                                 fut.add(data.delete(lockedInfo));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d14842a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
index a69920b..0a85657 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
@@ -23,7 +23,6 @@ import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.util.Collections;
 import java.util.Map;
-import java.util.UUID;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.util.GridLeanMap;
@@ -41,12 +40,6 @@ public final class IgfsFileInfo implements Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** ID for the root directory. */
-    public static final IgniteUuid ROOT_ID = new IgniteUuid(new UUID(0, 0), 0);
-
-    /** ID of the trash directory. */
-    public static final IgniteUuid TRASH_ID = new IgniteUuid(new UUID(0, 1), 0);
-
     /** Special access time value, indicating that the modification time value should be taken. */
     private static final long ACCESS_TIME_TAKE_MODIFICATION_TIME = -1L;
 
@@ -94,7 +87,7 @@ public final class IgfsFileInfo implements Externalizable {
      * {@link Externalizable} support.
      */
     public IgfsFileInfo() {
-        this(ROOT_ID);
+        this(IgfsUtils.ROOT_ID);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d14842a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index c1e47a8..0e52927 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -111,7 +111,6 @@ import static org.apache.ignite.igfs.IgfsMode.PRIMARY;
 import static org.apache.ignite.igfs.IgfsMode.PROXY;
 import static org.apache.ignite.internal.GridTopic.TOPIC_IGFS;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGFS;
-import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.ROOT_ID;
 
 /**
  * Cache-based IGFS implementation.
@@ -1206,7 +1205,7 @@ public final class IgfsImpl implements IgfsEx {
             @Override public IgfsMetrics call() throws Exception {
                 IgfsPathSummary sum = new IgfsPathSummary();
 
-                summary0(ROOT_ID, sum);
+                summary0(IgfsUtils.ROOT_ID, sum);
 
                 long secondarySpaceSize = 0;
 
@@ -1279,7 +1278,7 @@ public final class IgfsImpl implements IgfsEx {
 
         if (info != null) {
             if (info.isDirectory()) {
-                if (!ROOT_ID.equals(info.id()))
+                if (!IgfsUtils.ROOT_ID.equals(info.id()))
                     sum.directoriesCount(sum.directoriesCount() + 1);
 
                 for (IgfsListingEntry entry : info.listing().values())

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d14842a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index c120b9d..89ddd02 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -37,7 +37,6 @@ import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeMap;
 import java.util.TreeSet;
-import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorException;
@@ -91,8 +90,6 @@ import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_RENAMED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CREATED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_RENAMED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_WRITE;
-import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.ROOT_ID;
-import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.TRASH_ID;
 import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.builder;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
@@ -102,9 +99,6 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
  */
 @SuppressWarnings("all")
 public class IgfsMetaManager extends IgfsManager {
-    /** Lock Id used to lock files being deleted from TRASH. This is a global constant. */
-    static final IgniteUuid DELETE_LOCK_ID = new IgniteUuid(new UUID(0L, 0L), 0L);
-
     /** Comparator for Id sorting. */
     private static final Comparator<IgniteUuid> PATH_ID_SORTING_COMPARATOR
             = new Comparator<IgniteUuid>() {
@@ -359,9 +353,9 @@ public class IgfsMetaManager extends IgfsManager {
         // Collection of file IDs for components of specified path.
         List<IgniteUuid> ids = new ArrayList<>(components.size() + 1);
 
-        ids.add(ROOT_ID); // Always add root ID.
+        ids.add(IgfsUtils.ROOT_ID); // Always add root ID.
 
-        IgniteUuid fileId = ROOT_ID;
+        IgniteUuid fileId = IgfsUtils.ROOT_ID;
 
         for (String s : components) {
             assert !s.isEmpty();
@@ -416,8 +410,8 @@ public class IgfsMetaManager extends IgfsManager {
                 IgfsFileInfo info = id2InfoPrj.get(fileId);
 
                 // Force root ID always exist in cache.
-                if (info == null && ROOT_ID.equals(fileId))
-                    id2InfoPrj.putIfAbsent(ROOT_ID, info = new IgfsFileInfo());
+                if (info == null && IgfsUtils.ROOT_ID.equals(fileId))
+                    id2InfoPrj.putIfAbsent(IgfsUtils.ROOT_ID, info = new IgfsFileInfo());
 
                 return info;
             }
@@ -448,14 +442,14 @@ public class IgfsMetaManager extends IgfsManager {
                 Map<IgniteUuid, IgfsFileInfo> map = id2InfoPrj.getAll(fileIds);
 
                 // Force root ID always exist in cache.
-                if (fileIds.contains(ROOT_ID) && !map.containsKey(ROOT_ID)) {
+                if (fileIds.contains(IgfsUtils.ROOT_ID) && !map.containsKey(IgfsUtils.ROOT_ID)) {
                     IgfsFileInfo info = new IgfsFileInfo();
 
-                    id2InfoPrj.putIfAbsent(ROOT_ID, info);
+                    id2InfoPrj.putIfAbsent(IgfsUtils.ROOT_ID, info);
 
                     map = new GridLeanMap<>(map);
 
-                    map.put(ROOT_ID, info);
+                    map.put(IgfsUtils.ROOT_ID, info);
                 }
 
                 return map;
@@ -545,7 +539,7 @@ public class IgfsMetaManager extends IgfsManager {
      */
     private IgniteUuid composeLockId(boolean isDeleteLock) {
         if (isDeleteLock)
-            return DELETE_LOCK_ID;
+            return IgfsUtils.DELETE_LOCK_ID;
 
         return IgniteUuid.fromUuid(locNode.id());
     }
@@ -675,36 +669,38 @@ public class IgfsMetaManager extends IgfsManager {
         if (log.isDebugEnabled())
             log.debug("Locked file ids: " + fileIds);
 
-        // Force root & trash IDs always exist in cache.
-        addInfoIfNeeded(fileIds, map, ROOT_ID);
-        addInfoIfNeeded(fileIds, map, TRASH_ID);
+        for (IgniteUuid fileId : fileIds) {
+            if (IgfsUtils.isRootOrTrashId(fileId)) {
+                if (!map.containsKey(fileId))
+                    map.put(fileId, createSystemEntryIfAbsent(fileId));
+            }
+        }
 
         // Returns detail's map for locked IDs.
         return map;
     }
 
     /**
-     * Adds FileInfo into the cache if it is requested in fileIds and is not present in the map.
+     * create system entry if it is absent.
      *
-     * @param fileIds A list that may contain the id.
-     * @param map The map that may not contain the id.
-     * @param id The id to check.
+     * @param id System entry ID.
+     * @return Value of created or existing system entry.
      * @throws IgniteCheckedException On error.
      */
-    private void addInfoIfNeeded(Collection<IgniteUuid> fileIds, Map<IgniteUuid, IgfsFileInfo> map, IgniteUuid id)
+    private IgfsFileInfo createSystemEntryIfAbsent(IgniteUuid id)
         throws IgniteCheckedException {
         assert validTxState(true);
 
-        if (fileIds.contains(id) && !map.containsKey(id)) {
-            IgfsFileInfo info = new IgfsFileInfo(id);
+        assert IgfsUtils.isRootOrTrashId(id);
 
-            IgfsFileInfo anotherInfo = id2InfoPrj.getAndPutIfAbsent(id, info);
+        IgfsFileInfo info = new IgfsFileInfo(id);
 
-            if (anotherInfo != null)
-                info = anotherInfo;
+        IgfsFileInfo oldInfo = id2InfoPrj.getAndPutIfAbsent(id, info);
 
-            map.put(id, info);
-        }
+        if (oldInfo != null)
+            info = oldInfo;
+
+        return info;
     }
 
     /**
@@ -737,7 +733,7 @@ public class IgfsMetaManager extends IgfsManager {
     public IgfsFileInfo fileForFragmentizer(Collection<IgniteUuid> exclude) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
-                return fileForFragmentizer0(ROOT_ID, exclude);
+                return fileForFragmentizer0(IgfsUtils.ROOT_ID, exclude);
             }
             finally {
                 busyLock.leaveBusy();
@@ -1112,11 +1108,13 @@ public class IgfsMetaManager extends IgfsManager {
             try {
                 assert validTxState(false);
 
+                IgniteUuid trashId = IgfsUtils.randomTrashId();
+
                 final IgniteInternalTx tx = startTx();
 
                 try {
                     // NB: We may lock root because its id is less than any other id:
-                    final IgfsFileInfo rootInfo = lockIds(ROOT_ID, TRASH_ID).get(ROOT_ID);
+                    final IgfsFileInfo rootInfo = lockIds(IgfsUtils.ROOT_ID, trashId).get(IgfsUtils.ROOT_ID);
 
                     assert rootInfo != null;
 
@@ -1135,12 +1133,12 @@ public class IgfsMetaManager extends IgfsManager {
                     id2InfoPrj.put(newInfo.id(), newInfo);
 
                     // Add new info to trash listing.
-                    id2InfoPrj.invoke(TRASH_ID, new ListingAdd(newInfo.id().toString(),
+                    id2InfoPrj.invoke(trashId, new ListingAdd(newInfo.id().toString(),
                         new IgfsListingEntry(newInfo)));
 
                     // Remove listing entries from root.
                     // Note that root directory properties and other attributes are preserved:
-                    id2InfoPrj.put(ROOT_ID, new IgfsFileInfo(null/*listing*/, rootInfo));
+                    id2InfoPrj.put(IgfsUtils.ROOT_ID, new IgfsFileInfo(null/*listing*/, rootInfo));
 
                     tx.commit();
 
@@ -1182,15 +1180,16 @@ public class IgfsMetaManager extends IgfsManager {
 
                 final IgniteUuid victimId = pathIdList.get(pathIdList.size() - 1);
 
-                assert !TRASH_ID.equals(victimId) : "TRASH does not have path, it cannot ever be deletion victim.";
-                assert !ROOT_ID.equals(victimId); // root deletion is prevented in earlier stages.
+                assert !IgfsUtils.isRootOrTrashId(victimId) : "Cannot delete root or trash directories.";
 
                 allIds.addAll(pathIdList);
 
                 if (allIds.remove(null))
                     return null; // A fragment of the path no longer exists.
 
-                boolean added = allIds.add(TRASH_ID);
+                IgniteUuid trashId = IgfsUtils.randomTrashId();
+
+                boolean added = allIds.add(trashId);
                 assert added;
 
                 final IgniteInternalTx tx = startTx();
@@ -1209,7 +1208,7 @@ public class IgfsMetaManager extends IgfsManager {
                         throw new IgfsDirectoryNotEmptyException("Failed to remove directory (directory is not " +
                             "empty and recursive flag is not set).");
 
-                    IgfsFileInfo destInfo = infoMap.get(TRASH_ID);
+                    IgfsFileInfo destInfo = infoMap.get(trashId);
 
                     assert destInfo != null;
 
@@ -1237,7 +1236,7 @@ public class IgfsMetaManager extends IgfsManager {
                     id2InfoPrj.invoke(srcParentId, new ListingRemove(srcFileName, srcEntry.fileId()));
 
                     // Add listing entry into the destination parent listing.
-                    id2InfoPrj.invoke(TRASH_ID, new ListingAdd(destFileName, srcEntry));
+                    id2InfoPrj.invoke(trashId, new ListingAdd(destFileName, srcEntry));
 
                     if (victimInfo.isFile())
                         // Update a file info of the removed file with a file path,
@@ -1269,10 +1268,12 @@ public class IgfsMetaManager extends IgfsManager {
      * @param parentId Parent ID.
      * @param name Path name.
      * @param id Path ID.
+     * @param trashId Trash ID.
      * @return ID of an entry located directly under the trash directory.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable private IgniteUuid softDeleteNonTx(@Nullable IgniteUuid parentId, @Nullable String name, IgniteUuid id)
+    @Nullable private IgniteUuid softDeleteNonTx(@Nullable IgniteUuid parentId, @Nullable String name, IgniteUuid id,
+        IgniteUuid trashId)
         throws IgniteCheckedException {
         assert validTxState(true);
 
@@ -1280,16 +1281,15 @@ public class IgfsMetaManager extends IgfsManager {
 
         if (parentId == null) {
             // Handle special case when we deleting root directory.
-            assert ROOT_ID.equals(id);
+            assert IgfsUtils.ROOT_ID.equals(id);
 
-            IgfsFileInfo rootInfo = id2InfoPrj.get(ROOT_ID);
+            IgfsFileInfo rootInfo = id2InfoPrj.get(IgfsUtils.ROOT_ID);
 
             if (rootInfo == null)
                 return null; // Root was never created.
 
             // Ensure trash directory existence.
-            if (id2InfoPrj.get(TRASH_ID) == null)
-                id2InfoPrj.getAndPut(TRASH_ID, new IgfsFileInfo(TRASH_ID));
+            createSystemEntryIfAbsent(trashId);
 
             Map<String, IgfsListingEntry> rootListing = rootInfo.listing();
 
@@ -1314,12 +1314,12 @@ public class IgfsMetaManager extends IgfsManager {
                 id2InfoPrj.getAndPut(newInfo.id(), newInfo);
 
                 // Add new info to trash listing.
-                id2InfoPrj.invoke(TRASH_ID, new ListingAdd(newInfo.id().toString(),
+                id2InfoPrj.invoke(trashId, new ListingAdd(newInfo.id().toString(),
                     new IgfsListingEntry(newInfo)));
 
                 // Remove listing entries from root.
                 for (Map.Entry<String, IgfsListingEntry> entry : transferListing.entrySet())
-                    id2InfoPrj.invoke(ROOT_ID, new ListingRemove(entry.getKey(), entry.getValue().fileId()));
+                    id2InfoPrj.invoke(IgfsUtils.ROOT_ID, new ListingRemove(entry.getKey(), entry.getValue().fileId()));
 
                 resId = newInfo.id();
             }
@@ -1328,10 +1328,9 @@ public class IgfsMetaManager extends IgfsManager {
         }
         else {
             // Ensure trash directory existence.
-            if (id2InfoPrj.get(TRASH_ID) == null)
-                id2InfoPrj.getAndPut(TRASH_ID, new IgfsFileInfo(TRASH_ID));
+            createSystemEntryIfAbsent(trashId);
 
-            moveNonTx(id, name, parentId, id.toString(), TRASH_ID);
+            moveNonTx(id, name, parentId, id.toString(), trashId);
 
             resId = id;
         }
@@ -1390,7 +1389,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                             if (entryInfo != null) {
                                 // File must be locked for deletion:
-                                assert entryInfo.isDirectory() || DELETE_LOCK_ID.equals(entryInfo.lockId());
+                                assert entryInfo.isDirectory() || IgfsUtils.DELETE_LOCK_ID.equals(entryInfo.lockId());
 
                                 // Delete only files or empty folders.
                                 if (entryInfo.isFile() || entryInfo.isDirectory() && entryInfo.listing().isEmpty()) {
@@ -1457,8 +1456,8 @@ public class IgfsMetaManager extends IgfsManager {
                     if (victim == null)
                         return res;
 
-                    assert victim.isDirectory() || DELETE_LOCK_ID.equals(victim.lockId()) :
-                            " isDir: " + victim.isDirectory() + ", lockId: " + victim.lockId();
+                    assert victim.isDirectory() || IgfsUtils.DELETE_LOCK_ID.equals(victim.lockId()) :
+                        " isDir: " + victim.isDirectory() + ", lockId: " + victim.lockId();
 
                     // Proceed only in case both parent and child exist.
                     if (infos.containsKey(parentId) && infos.containsKey(id)) {
@@ -1504,21 +1503,24 @@ public class IgfsMetaManager extends IgfsManager {
     public Collection<IgniteUuid> pendingDeletes() throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
-                IgfsFileInfo trashInfo = id2InfoPrj.get(TRASH_ID);
+                Collection<IgniteUuid> ids = new HashSet<>();
 
-                if (trashInfo != null) {
-                    Map<String, IgfsListingEntry> listing = trashInfo.listing();
+                for (int i = 0; i < IgfsUtils.TRASH_CONCURRENCY; i++) {
+                    IgniteUuid trashId = IgfsUtils.trashId(i);
 
-                    if (listing != null && !listing.isEmpty()) {
-                        return F.viewReadOnly(listing.values(), new IgniteClosure<IgfsListingEntry, IgniteUuid>() {
-                            @Override public IgniteUuid apply(IgfsListingEntry e) {
-                                return e.fileId();
-                            }
-                        });
+                    IgfsFileInfo trashInfo = id2InfoPrj.get(trashId);
+
+                    if (trashInfo != null) {
+                        Map<String, IgfsListingEntry> listing = trashInfo.listing();
+
+                        if (listing != null && !listing.isEmpty()) {
+                            for (IgfsListingEntry entry : listing.values())
+                                ids.add(entry.fileId());
+                        }
                     }
                 }
 
-                return Collections.emptySet();
+                return ids;
             }
             finally {
                 busyLock.leaveBusy();
@@ -2455,6 +2457,8 @@ public class IgfsMetaManager extends IgfsManager {
                 assert fs != null;
                 assert path != null;
 
+                final IgniteUuid trashId = IgfsUtils.randomTrashId();
+
                 SynchronizationTask<Boolean> task = new SynchronizationTask<Boolean>() {
                     @Override public Boolean onSuccess(Map<IgfsPath, IgfsFileInfo> infos) throws Exception {
                         IgfsFileInfo info = infos.get(path);
@@ -2468,12 +2472,12 @@ public class IgfsMetaManager extends IgfsManager {
                         if (path.parent() != null) {
                             assert infos.containsKey(path.parent());
 
-                            softDeleteNonTx(infos.get(path.parent()).id(), path.name(), info.id());
+                            softDeleteNonTx(infos.get(path.parent()).id(), path.name(), info.id(), trashId);
                         }
                         else {
-                            assert ROOT_ID.equals(info.id());
+                            assert IgfsUtils.ROOT_ID.equals(info.id());
 
-                            softDeleteNonTx(null, path.name(), info.id());
+                            softDeleteNonTx(null, path.name(), info.id(), trashId);
                         }
 
                         // Update the deleted file info with path information for delete worker.
@@ -2491,7 +2495,7 @@ public class IgfsMetaManager extends IgfsManager {
                     }
                 };
 
-                Boolean res = synchronizeAndExecute(task, fs, false, Collections.singleton(TRASH_ID), path);
+                Boolean res = synchronizeAndExecute(task, fs, false, Collections.singleton(trashId), path);
 
                 delWorker.signal();
 
@@ -3441,6 +3445,8 @@ public class IgfsMetaManager extends IgfsManager {
 
         DirectoryChainBuilder b = null;
 
+        IgniteUuid trashId = IgfsUtils.randomTrashId();
+
         while (true) {
             if (busyLock.enterBusy()) {
                 try {
@@ -3461,11 +3467,11 @@ public class IgfsMetaManager extends IgfsManager {
                         if (overwrite)
                             // Lock also the TRASH directory because in case of overwrite we
                             // may need to delete the old file:
-                            b.idSet.add(TRASH_ID);
+                            b.idSet.add(trashId);
 
                         final Map<IgniteUuid, IgfsFileInfo> lockedInfos = lockIds(b.idSet);
 
-                        assert !overwrite || lockedInfos.get(TRASH_ID) != null; // TRASH must exist at this point.
+                        assert !overwrite || lockedInfos.get(trashId) != null; // TRASH must exist at this point.
 
                         // If the path was changed, we close the current Tx and repeat the procedure again
                         // starting from taking the path ids.
@@ -3536,8 +3542,8 @@ public class IgfsMetaManager extends IgfsManager {
                                         id2InfoPrj.invoke(parentId, new ListingRemove(name, deletedEntry.fileId()));
 
                                         // Add listing entry into the destination parent listing.
-                                        id2InfoPrj.invoke(TRASH_ID, new ListingAdd(
-                                                lowermostExistingInfo.id().toString(), deletedEntry));
+                                        id2InfoPrj.invoke(trashId,
+                                            new ListingAdd(lowermostExistingInfo.id().toString(), deletedEntry));
 
                                         // Update a file info of the removed file with a file path,
                                         // which will be used by delete worker for event notifications.
@@ -3697,7 +3703,7 @@ public class IgfsMetaManager extends IgfsManager {
                 idIdx++;
             }
 
-            assert idSet.contains(ROOT_ID);
+            assert idSet.contains(IgfsUtils.ROOT_ID);
 
             this.lowermostExistingId = lowermostExistingId;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d14842a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
index c9225ae..83056af 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
@@ -121,7 +121,7 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
         if (fileInfo.lockId() == null)
             throw new IgfsException("Failed to acquire file lock (concurrently modified?): " + path);
 
-        assert !IgfsMetaManager.DELETE_LOCK_ID.equals(fileInfo.lockId());
+        assert !IgfsUtils.DELETE_LOCK_ID.equals(fileInfo.lockId());
 
         this.igfsCtx = igfsCtx;
         meta = igfsCtx.meta();

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d14842a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
index 07fdda4..6f8960a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
@@ -18,6 +18,9 @@
 package org.apache.ignite.internal.processors.igfs;
 
 import java.lang.reflect.Constructor;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
@@ -36,6 +39,7 @@ import org.apache.ignite.internal.util.lang.IgniteOutClosureX;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.transactions.Transaction;
 import org.jetbrains.annotations.Nullable;
 
@@ -47,10 +51,78 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
  * Common IGFS utility methods.
  */
 public class IgfsUtils {
+    /** ID for the root directory. */
+    public static final IgniteUuid ROOT_ID = new IgniteUuid(new UUID(0, 0), 0);
+
+    /** Lock Id used to lock files being deleted from TRASH. This is a global constant. */
+    public static final IgniteUuid DELETE_LOCK_ID = new IgniteUuid(new UUID(0, 0), 0);
+
+    /** Constant trash concurrency level. */
+    public static final int TRASH_CONCURRENCY = 16;
+
+    /** Trash directory IDs. */
+    private static final IgniteUuid[] TRASH_IDS;
+
     /** Maximum number of file unlock transaction retries when topology changes. */
     private static final int MAX_CACHE_TX_RETRIES = IgniteSystemProperties.getInteger(IGNITE_CACHE_RETRIES_COUNT, 100);
 
     /**
+     * Static initializer.
+     */
+    static {
+        TRASH_IDS = new IgniteUuid[TRASH_CONCURRENCY];
+
+        for (int i = 0; i < TRASH_CONCURRENCY; i++)
+            TRASH_IDS[i] = new IgniteUuid(new UUID(0, i + 1), 0);
+    }
+
+    /**
+     * Get random trash ID.
+     *
+     * @return Trash ID.
+     */
+    public static IgniteUuid randomTrashId() {
+        return TRASH_IDS[ThreadLocalRandom.current().nextInt(TRASH_CONCURRENCY)];
+    }
+
+    /**
+     * Get trash ID for the given index.
+     *
+     * @param idx Index.
+     * @return Trahs ID.
+     */
+    public static IgniteUuid trashId(int idx) {
+        assert idx >= 0 && idx < TRASH_CONCURRENCY;
+
+        return TRASH_IDS[idx];
+    }
+
+    /**
+     * Check whether provided ID is trash ID.
+     *
+     * @param id ID.
+     * @return {@code True} if this is trash ID.
+     */
+    public static boolean isTrashId(IgniteUuid id) {
+        assert id != null;
+
+        UUID gid = id.globalId();
+
+        return id.localId() == 0 && gid.getMostSignificantBits() == 0 &&
+            gid.getLeastSignificantBits() > 0 && gid.getLeastSignificantBits() <= TRASH_CONCURRENCY;
+    }
+
+    /**
+     * Check whether provided ID is either root ID or trash ID.
+     *
+     * @param id ID.
+     * @return {@code True} if this is root ID or trash ID.
+     */
+    public static boolean isRootOrTrashId(IgniteUuid id) {
+        return ROOT_ID.equals(id) || isTrashId(id);
+    }
+
+    /**
      * Converts any passed exception to IGFS exception.
      *
      * @param err Initial exception.

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d14842a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
index 015b992..52d8bd5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
@@ -3138,14 +3138,22 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         igfs.format();
 
         int prevDifferentSize = Integer.MAX_VALUE; // Previous different size.
-        int size;
         int constCnt = 0, totalCnt = 0;
         final int constThreshold = 20;
         final long sleepPeriod = 500L;
         final long totalThreshold = CACHE_EMPTY_TIMEOUT / sleepPeriod;
 
         while (true) {
-            size = sumCacheSize(igfs);
+            int metaSize = 0;
+
+            for (IgniteUuid metaId : getMetaCache(igfs).keySet()) {
+                if (!IgfsUtils.isRootOrTrashId(metaId))
+                    metaSize++;
+            }
+
+            int dataSize = getDataCache(igfs).size();
+
+            int size = metaSize + dataSize;
 
             if (size <= 2)
                 return; // Caches are cleared, we're done. (2 because ROOT & TRASH always exist).
@@ -3206,15 +3214,6 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
-     * Gets summary IGFS cache size.
-     * @param igfs The IGFS to measure.
-     * @return data cache size + meta cache size.
-     */
-    private static int sumCacheSize(IgniteFileSystem igfs) {
-        return getMetaCache(igfs).size() + getDataCache(igfs).size();
-    }
-
-    /**
      * Clear particular {@link UniversalFileSystemAdapter}.
      *
      * @param uni IGFS.

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d14842a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
index 8d5ae7a..06ae40b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
@@ -168,7 +168,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
             long t = System.currentTimeMillis();
 
-            IgfsFileInfo info = new IgfsFileInfo(200, 0L, null, IgfsMetaManager.DELETE_LOCK_ID,
+            IgfsFileInfo info = new IgfsFileInfo(200, 0L, null, IgfsUtils.DELETE_LOCK_ID,
                     false, null, t, t);
 
             assertNull(mgr.dataBlock(info, path, 0, null).get());
@@ -253,7 +253,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
             long t = System.currentTimeMillis();
 
-            IgfsFileInfo info = new IgfsFileInfo(blockSize, 0L, null, IgfsMetaManager.DELETE_LOCK_ID,
+            IgfsFileInfo info = new IgfsFileInfo(blockSize, 0L, null, IgfsUtils.DELETE_LOCK_ID,
                 false, null, t, t);
 
             assertNull(mgr.dataBlock(info, path, 0, null).get());
@@ -345,7 +345,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
             long t = System.currentTimeMillis();
 
             IgfsFileInfo info =
-                new IgfsFileInfo(blockSize, 0L, null, IgfsMetaManager.DELETE_LOCK_ID,
+                new IgfsFileInfo(blockSize, 0L, null, IgfsUtils.DELETE_LOCK_ID,
                     false, null, t, t);
 
             IgfsFileAffinityRange range = new IgfsFileAffinityRange();

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d14842a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
index df519ed..f3d35f4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
@@ -42,7 +42,7 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.ROOT_ID;
+import static org.apache.ignite.internal.processors.igfs.IgfsUtils.ROOT_ID;
 import static org.apache.ignite.testframework.GridTestUtils.assertThrows;
 import static org.apache.ignite.testframework.GridTestUtils.assertThrowsInherited;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d14842a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
index c6853ae..f567099 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
@@ -778,7 +778,14 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
         assert !igfs.exists(path(dirPath));
         assert !igfs.exists(path(filePath));
 
-        assert grid(0).cachex(igfs.configuration().getMetaCacheName()).size() == 2; // ROOT + TRASH.
+        int metaSize = 0;
+
+        for (Object metaId : grid(0).cachex(igfs.configuration().getMetaCacheName()).keySet()) {
+            if (!IgfsUtils.isRootOrTrashId((IgniteUuid)metaId))
+                metaSize++;
+        }
+
+        assert metaSize == 0;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d14842a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
index 9c90534..fcf4b3b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
@@ -59,8 +59,6 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
-import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.ROOT_ID;
-import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.TRASH_ID;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
@@ -538,26 +536,27 @@ public class IgfsSizeSelfTest extends IgfsCommonAbstractTest {
             }).start();
 
             // Now add file ID to trash listing so that delete worker could "see" it.
+            IgniteUuid trashId = IgfsUtils.randomTrashId();
 
             try (Transaction tx = metaCache.unwrap(Ignite.class).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
                 Map<String, IgfsListingEntry> listing = Collections.singletonMap(path.name(),
                     new IgfsListingEntry(info));
 
                 // Clear root listing.
-                metaCache.put(ROOT_ID, new IgfsFileInfo(ROOT_ID));
+                metaCache.put(IgfsUtils.ROOT_ID, new IgfsFileInfo(IgfsUtils.ROOT_ID));
 
                 // Add file to trash listing.
-                IgfsFileInfo trashInfo = metaCache.get(TRASH_ID);
+                IgfsFileInfo trashInfo = metaCache.get(trashId);
 
                 if (trashInfo == null)
-                    metaCache.put(TRASH_ID, new IgfsFileInfo(listing, new IgfsFileInfo(TRASH_ID)));
+                    metaCache.put(trashId, new IgfsFileInfo(listing, new IgfsFileInfo(trashId)));
                 else
-                    metaCache.put(TRASH_ID, new IgfsFileInfo(listing, trashInfo));
+                    metaCache.put(trashId, new IgfsFileInfo(listing, trashInfo));
 
                 tx.commit();
             }
 
-            assert metaCache.get(TRASH_ID) != null;
+            assert metaCache.get(trashId) != null;
 
             // Now the file is locked and is located in trash, try adding some more data.
             os = igfs.create(otherPath, false);


[12/35] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by nt...@apache.org.
Merge remote-tracking branch 'origin/master'


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/8cf71d46
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/8cf71d46
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/8cf71d46

Branch: refs/heads/ignite-2791
Commit: 8cf71d463d9f2baa7651286a6524ec5d6d286b5b
Parents: b46ed10 648b10c
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Mar 10 14:05:53 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Mar 10 14:05:53 2016 +0300

----------------------------------------------------------------------
 .../install-package.cmd                         | 11 ++-
 .../Apache.Ignite.Core.nuspec                   |  6 +-
 .../NuGet/LINQPad/ComputeExample.linq           |  2 +-
 .../NuGet/LINQPad/PutGetExample.linq            |  2 +-
 .../NuGet/LINQPad/QueryExample.linq             |  2 +-
 modules/web/ignite-weblogic-test/pom.xml        | 74 +++++++-------------
 .../src/main/webapp/WEB-INF/web.xml             | 17 -----
 .../src/main/webapp/index.jsp                   | 17 -----
 8 files changed, 40 insertions(+), 91 deletions(-)
----------------------------------------------------------------------



[14/35] ignite git commit: Rat fix (cherry picked from commit 52ae798)

Posted by nt...@apache.org.
Rat fix
(cherry picked from commit 52ae798)


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/6231bb96
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/6231bb96
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/6231bb96

Branch: refs/heads/ignite-2791
Commit: 6231bb963ce9750bbf1582c04f68f96febc0137b
Parents: 70abaa3
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Mar 10 17:55:46 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Mar 10 17:56:18 2016 +0300

----------------------------------------------------------------------
 .../main/webapp/META-INF/config/default-config.xml | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6231bb96/modules/web/ignite-weblogic-test/src/main/webapp/META-INF/config/default-config.xml
----------------------------------------------------------------------
diff --git a/modules/web/ignite-weblogic-test/src/main/webapp/META-INF/config/default-config.xml b/modules/web/ignite-weblogic-test/src/main/webapp/META-INF/config/default-config.xml
index daa466d..1a479a6 100644
--- a/modules/web/ignite-weblogic-test/src/main/webapp/META-INF/config/default-config.xml
+++ b/modules/web/ignite-weblogic-test/src/main/webapp/META-INF/config/default-config.xml
@@ -1,5 +1,22 @@
 <?xml version="1.0" encoding="UTF-8"?>
 
+<!--
+  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.
+-->
+
 <beans xmlns="http://www.springframework.org/schema/beans"
        xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
        xsi:schemaLocation="http://www.springframework.org/schema/beans


[04/35] ignite git commit: IGNITE-2776 - An option to enlist into JTA transaction using sync callback instead of XA resource

Posted by nt...@apache.org.
IGNITE-2776 - An option to enlist into JTA transaction using sync callback instead of XA resource


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/334ce4c9
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/334ce4c9
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/334ce4c9

Branch: refs/heads/ignite-2791
Commit: 334ce4c92dc398b0a9870d41493cb14c4ddf891a
Parents: 157ef53
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Wed Mar 9 16:18:36 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Mar 9 16:18:36 2016 -0800

----------------------------------------------------------------------
 .../configuration/TransactionConfiguration.java |  32 ++
 .../HibernateL2CacheTransactionalSelfTest.java  |   9 +
 ...nateL2CacheTransactionalUseSyncSelfTest.java |  31 ++
 .../testsuites/IgniteHibernateTestSuite.java    |   4 +-
 .../processors/cache/jta/CacheJtaManager.java   |  26 +-
 .../processors/cache/jta/CacheJtaResource.java  | 304 +++++++++++++++++++
 .../cache/jta/GridCacheXAResource.java          | 251 ---------------
 .../cache/AbstarctCacheJtaSelfTest.java         | 183 -----------
 .../cache/AbstractCacheJtaSelfTest.java         | 183 +++++++++++
 .../GridPartitionedCacheJtaFactorySelfTest.java |   2 +-
 ...rtitionedCacheJtaFactoryUseSyncSelfTest.java |  32 ++
 ...titionedCacheJtaLookupClassNameSelfTest.java |   2 +-
 ...eplicatedCacheJtaFactoryUseSyncSelfTest.java |  32 ++
 .../ignite/testsuites/IgniteJtaTestSuite.java   |   5 +
 14 files changed, 649 insertions(+), 447 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/334ce4c9/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java
index b3d294d..95050a7 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java
@@ -70,6 +70,12 @@ public class TransactionConfiguration implements Serializable {
     private Factory txManagerFactory;
 
     /**
+     * Whether to use JTA {@code javax.transaction.Synchronization}
+     * instead of {@code javax.transaction.xa.XAResource}.
+     */
+    private boolean useJtaSync;
+
+    /**
      * Empty constructor.
      */
     public TransactionConfiguration() {
@@ -88,6 +94,7 @@ public class TransactionConfiguration implements Serializable {
         txSerEnabled = cfg.isTxSerializableEnabled();
         tmLookupClsName = cfg.getTxManagerLookupClassName();
         txManagerFactory = cfg.getTxManagerFactory();
+        useJtaSync = cfg.isUseJtaSynchronization();
     }
 
     /**
@@ -243,6 +250,7 @@ public class TransactionConfiguration implements Serializable {
      *
      * @param <T> Instance of {@code javax.transaction.TransactionManager}.
      * @return Transaction manager factory.
+     * @see #isUseJtaSynchronization()
      */
     @SuppressWarnings("unchecked")
     public <T> Factory<T> getTxManagerFactory() {
@@ -269,8 +277,32 @@ public class TransactionConfiguration implements Serializable {
      *
      * @param factory Transaction manager factory.
      * @param <T> Instance of {@code javax.transaction.TransactionManager}.
+     * @see #setUseJtaSynchronization(boolean)
      */
     public <T> void setTxManagerFactory(Factory<T> factory) {
         txManagerFactory = factory;
     }
+
+    /**
+     * @return Whether to use JTA {@code javax.transaction.Synchronization}
+     *      instead of {@code javax.transaction.xa.XAResource}.
+     * @see #getTxManagerFactory()
+     */
+    public boolean isUseJtaSynchronization() {
+        return useJtaSync;
+    }
+
+    /**
+     * Sets the flag that defines whether to use lightweight JTA synchronization callback to enlist
+     * into JTA transaction instead of creating a separate XA resource. In some cases this can give
+     * performance improvement, but keep in mind that most of the transaction managers do not allow
+     * to add more that one callback to a single transaction.
+     *
+     * @param useJtaSync Whether to use JTA {@code javax.transaction.Synchronization}
+     *      instead of {@code javax.transaction.xa.XAResource}.
+     * @see #setTxManagerFactory(Factory)
+     */
+    public void setUseJtaSynchronization(boolean useJtaSync) {
+        this.useJtaSync = useJtaSync;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/334ce4c9/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalSelfTest.java b/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalSelfTest.java
index 9141be2..e6117c0 100644
--- a/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalSelfTest.java
+++ b/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.cache.hibernate;
 
 import java.util.Collections;
 import javax.cache.configuration.Factory;
+import javax.transaction.Synchronization;
 import javax.transaction.TransactionManager;
 import javax.transaction.UserTransaction;
 import org.apache.commons.dbcp.managed.BasicManagedDataSource;
@@ -94,6 +95,7 @@ public class HibernateL2CacheTransactionalSelfTest extends HibernateL2CacheSelfT
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
         cfg.getTransactionConfiguration().setTxManagerFactory(new TestTmFactory());
+        cfg.getTransactionConfiguration().setUseJtaSynchronization(useJtaSynchronization());
 
         return cfg;
     }
@@ -142,4 +144,11 @@ public class HibernateL2CacheTransactionalSelfTest extends HibernateL2CacheSelfT
     @Override protected AccessType[] accessTypes() {
         return new AccessType[]{AccessType.TRANSACTIONAL};
     }
+
+    /**
+     * @return Whether to use {@link Synchronization}.
+     */
+    protected boolean useJtaSynchronization() {
+        return false;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/334ce4c9/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalUseSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalUseSyncSelfTest.java b/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalUseSyncSelfTest.java
new file mode 100644
index 0000000..44899f9
--- /dev/null
+++ b/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalUseSyncSelfTest.java
@@ -0,0 +1,31 @@
+/*
+ * 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.hibernate;
+
+import javax.transaction.Synchronization;
+
+/**
+ * Tests Hibernate L2 cache with TRANSACTIONAL access mode and {@link Synchronization}
+ * instead of XA resource.
+ */
+public class HibernateL2CacheTransactionalUseSyncSelfTest extends HibernateL2CacheTransactionalSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean useJtaSynchronization() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/334ce4c9/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java b/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java
index 309ed3b..99fea56 100644
--- a/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java
+++ b/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java
@@ -21,6 +21,7 @@ import junit.framework.TestSuite;
 import org.apache.ignite.cache.hibernate.HibernateL2CacheConfigurationSelfTest;
 import org.apache.ignite.cache.hibernate.HibernateL2CacheSelfTest;
 import org.apache.ignite.cache.hibernate.HibernateL2CacheTransactionalSelfTest;
+import org.apache.ignite.cache.hibernate.HibernateL2CacheTransactionalUseSyncSelfTest;
 import org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreNodeRestartTest;
 import org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreSelfTest;
 import org.apache.ignite.cache.store.hibernate.CacheHibernateStoreFactorySelfTest;
@@ -40,6 +41,7 @@ public class IgniteHibernateTestSuite extends TestSuite {
         // Hibernate L2 cache.
         suite.addTestSuite(HibernateL2CacheSelfTest.class);
         suite.addTestSuite(HibernateL2CacheTransactionalSelfTest.class);
+        suite.addTestSuite(HibernateL2CacheTransactionalUseSyncSelfTest.class);
         suite.addTestSuite(HibernateL2CacheConfigurationSelfTest.class);
 
         suite.addTestSuite(CacheHibernateBlobStoreSelfTest.class);
@@ -52,4 +54,4 @@ public class IgniteHibernateTestSuite extends TestSuite {
 
         return suite;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/334ce4c9/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
index a65a4f8..f581ebb 100644
--- a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
+++ b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
@@ -36,7 +36,7 @@ import org.jetbrains.annotations.Nullable;
  */
 public class CacheJtaManager extends CacheJtaManagerAdapter {
     /** */
-    private final ThreadLocal<GridCacheXAResource> xaRsrc = new ThreadLocal<>();
+    private final ThreadLocal<CacheJtaResource> rsrc = new ThreadLocal<>();
 
     /** */
     private TransactionManager jtaTm;
@@ -47,6 +47,9 @@ public class CacheJtaManager extends CacheJtaManagerAdapter {
     /** */
     private Factory<TransactionManager> tmFactory;
 
+    /** */
+    private boolean useJtaSync;
+
     /** {@inheritDoc} */
     @Override protected void start0() throws IgniteCheckedException {
         super.start0();
@@ -80,14 +83,15 @@ public class CacheJtaManager extends CacheJtaManagerAdapter {
                         + tmFactory + ", txMgr=" + txMgr + "]");
 
                 jtaTm = (TransactionManager)txMgr;
-
-                return;
             }
+            else {
+                String txLookupClsName = cctx.txConfig().getTxManagerLookupClassName();
 
-            String txLookupClsName = cctx.txConfig().getTxManagerLookupClassName();
+                if (txLookupClsName != null)
+                    tmLookupRef.set(createTmLookup(txLookupClsName));
+            }
 
-            if (txLookupClsName != null)
-                tmLookupRef.set(createTmLookup(txLookupClsName));
+            useJtaSync = cctx.txConfig().isUseJtaSynchronization();
         }
     }
 
@@ -140,7 +144,7 @@ public class CacheJtaManager extends CacheJtaManagerAdapter {
         }
 
         if (jtaTm != null) {
-            GridCacheXAResource rsrc = xaRsrc.get();
+            CacheJtaResource rsrc = this.rsrc.get();
 
             if (rsrc == null || rsrc.isFinished()) {
                 try {
@@ -165,12 +169,14 @@ public class CacheJtaManager extends CacheJtaManagerAdapter {
                             );
                         }
 
-                        rsrc = new GridCacheXAResource(tx, cctx.kernalContext());
+                        rsrc = new CacheJtaResource(tx, cctx.kernalContext());
 
-                        if (!jtaTx.enlistResource(rsrc))
+                        if (useJtaSync)
+                            jtaTx.registerSynchronization(rsrc);
+                        else if (!jtaTx.enlistResource(rsrc))
                             throw new IgniteCheckedException("Failed to enlist XA resource to JTA user transaction.");
 
-                        xaRsrc.set(rsrc);
+                        this.rsrc.set(rsrc);
                     }
                 }
                 catch (SystemException e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/334ce4c9/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaResource.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaResource.java b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaResource.java
new file mode 100644
index 0000000..f43981e
--- /dev/null
+++ b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaResource.java
@@ -0,0 +1,304 @@
+/*
+ * 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.jta;
+
+import java.util.concurrent.atomic.AtomicReference;
+import javax.cache.CacheException;
+import javax.transaction.Status;
+import javax.transaction.Synchronization;
+import javax.transaction.xa.XAException;
+import javax.transaction.xa.XAResource;
+import javax.transaction.xa.Xid;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.transactions.TransactionState;
+
+import static org.apache.ignite.transactions.TransactionState.ACTIVE;
+import static org.apache.ignite.transactions.TransactionState.COMMITTED;
+import static org.apache.ignite.transactions.TransactionState.PREPARED;
+import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK;
+
+/**
+ * Cache {@link XAResource} and {@link Synchronization} implementation.
+ */
+final class CacheJtaResource implements XAResource, Synchronization {
+    /** Logger reference. */
+    private static final AtomicReference<IgniteLogger> logRef = new AtomicReference<>();
+
+    /** */
+    private static IgniteLogger log;
+
+    /** */
+    private static final Xid[] NO_XID = new Xid[] {};
+
+    /** Cache transaction. */
+    private IgniteInternalTx cacheTx;
+
+    /** */
+    private Xid xid;
+
+    /**
+     * @param cacheTx Cache jta.
+     * @param ctx Kernal context.
+     */
+    public CacheJtaResource(IgniteInternalTx cacheTx, GridKernalContext ctx) {
+        assert cacheTx != null;
+        assert ctx != null;
+
+        this.cacheTx = cacheTx;
+
+        if (log == null)
+            log = U.logger(ctx, logRef, CacheJtaResource.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start(Xid xid, int flags) {
+        if (log.isDebugEnabled())
+            log.debug("XA resource start(...) [xid=" + xid + ", flags=<" + flags(flags) + ">]");
+
+        // Simply save global transaction id.
+        this.xid = xid;
+    }
+
+    /**
+     * @param msg Message.
+     * @param cause Cause.
+     * @throws XAException XA exception.
+     */
+    private void throwException(String msg, Throwable cause) throws XAException {
+        XAException ex = new XAException(msg);
+
+        ex.initCause(cause);
+
+        throw ex;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void rollback(Xid xid) throws XAException {
+        assert this.xid.equals(xid);
+
+        if (log.isDebugEnabled())
+            log.debug("XA resource rollback(...) [xid=" + xid + "]");
+
+        try {
+            cacheTx.rollback();
+        }
+        catch (IgniteCheckedException e) {
+            throwException("Failed to rollback cache transaction: " + e.getMessage(), e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public int prepare(Xid xid) throws XAException {
+        assert this.xid.equals(xid);
+
+        if (log.isDebugEnabled())
+            log.debug("XA resource prepare(...) [xid=" + xid + "]");
+
+        if (cacheTx.state() != ACTIVE)
+            throw new XAException("Cache transaction is not in active state.");
+
+        try {
+            cacheTx.prepare();
+        }
+        catch (IgniteCheckedException e) {
+            throwException("Failed to prepare cache transaction.", e);
+        }
+
+        return XA_OK;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void end(Xid xid, int flags) {
+        assert this.xid.equals(xid);
+
+        if (log.isDebugEnabled())
+            log.debug("XA resource end(...) [xid=" + xid + ", flags=<" + flags(flags) + ">]");
+
+        if ((flags & TMFAIL) > 0)
+            cacheTx.setRollbackOnly();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void commit(Xid xid, boolean onePhase) throws XAException {
+        assert this.xid.equals(xid);
+
+        if (log.isDebugEnabled())
+            log.debug("XA resource commit(...) [xid=" + xid + ", onePhase=" + onePhase + "]");
+
+        try {
+            cacheTx.commit();
+        }
+        catch (IgniteCheckedException e) {
+            throwException("Failed to commit cache transaction: " + e.getMessage(), e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void forget(Xid xid) throws XAException {
+        assert this.xid.equals(xid);
+
+        if (log.isDebugEnabled())
+            log.debug("XA resource forget(...) [xid=" + xid + "]");
+
+        try {
+            cacheTx.invalidate(true);
+
+            cacheTx.commit();
+        }
+        catch (IgniteCheckedException e) {
+            throwException("Failed to forget cache transaction: " + e.getMessage(), e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Xid[] recover(int i) {
+        if (cacheTx.state() == PREPARED)
+            return new Xid[] { xid };
+
+        return NO_XID;
+    }
+
+    /**
+     * @param flags JTA Flags.
+     * @return Comma-separated flags string.
+     */
+    private String flags(int flags) {
+        StringBuilder res = new StringBuilder();
+
+        addFlag(res, flags, TMENDRSCAN, "TMENDRSCAN");
+        addFlag(res, flags, TMFAIL, "TMFAIL");
+        addFlag(res, flags, TMJOIN, "TMJOIN");
+        addFlag(res, flags, TMNOFLAGS, "TMNOFLAGS");
+        addFlag(res, flags, TMONEPHASE, "TMONEPHASE");
+        addFlag(res, flags, TMRESUME, "TMRESUME");
+        addFlag(res, flags, TMSTARTRSCAN, "TMSTARTRSCAN");
+        addFlag(res, flags, TMSUCCESS, "TMSUCCESS");
+        addFlag(res, flags, TMSUSPEND, "TMSUSPEND");
+
+        return res.toString();
+    }
+
+    /**
+     * @param sb String builder.
+     * @param flags Flags bit set.
+     * @param mask Bit mask.
+     * @param flagName String name of the flag specified by given mask.
+     * @return String builder appended by flag if it's presented in bit set.
+     */
+    private StringBuilder addFlag(StringBuilder sb, int flags, int mask, String flagName) {
+        if ((flags & mask) > 0)
+            sb.append(sb.length() > 0 ? "," : "").append(flagName);
+
+        return sb;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getTransactionTimeout() {
+        return (int)(cacheTx.timeout() / 1000);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean setTransactionTimeout(int i) {
+        cacheTx.timeout(i * 1000);
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isSameRM(XAResource xar) {
+        if (xar == this)
+            return true;
+
+        if (!(xar instanceof CacheJtaResource))
+            return false;
+
+        CacheJtaResource other = (CacheJtaResource)xar;
+
+        return cacheTx == other.cacheTx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCompletion() {
+        if (log.isDebugEnabled())
+            log.debug("Synchronization.beforeCompletion() [xid=" + cacheTx.xid() + "]");
+
+        if (cacheTx.state() != ACTIVE)
+            throw new CacheException("Cache transaction is not in active state.");
+
+        try {
+            cacheTx.prepare();
+        }
+        catch (IgniteCheckedException e) {
+            throw new CacheException("Failed to prepare cache transaction.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void afterCompletion(int status) {
+        switch (status) {
+            case Status.STATUS_COMMITTED:
+                if (log.isDebugEnabled())
+                    log.debug("Synchronization.afterCompletion(STATUS_COMMITTED) [xid=" + cacheTx.xid() + "]");
+
+                try {
+                    cacheTx.commit();
+                }
+                catch (IgniteCheckedException e) {
+                    throw new CacheException("Failed to commit cache transaction.", e);
+                }
+
+                break;
+
+            case Status.STATUS_ROLLEDBACK:
+                if (log.isDebugEnabled())
+                    log.debug("Synchronization.afterCompletion(STATUS_ROLLEDBACK) [xid=" + cacheTx.xid() + "]");
+
+                try {
+                    cacheTx.rollback();
+                }
+                catch (IgniteCheckedException e) {
+                    throw new CacheException("Failed to rollback cache transaction.", e);
+                }
+
+                break;
+
+            default:
+                throw new IllegalArgumentException("Unknown transaction status: " + status);
+        }
+    }
+
+    /**
+     *
+     * @return {@code true} if jta was already committed or rolled back.
+     */
+    public boolean isFinished() {
+        TransactionState state = cacheTx.state();
+
+        return state == COMMITTED || state == ROLLED_BACK;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CacheJtaResource.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/334ce4c9/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
deleted file mode 100644
index 2cb4695..0000000
--- a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
+++ /dev/null
@@ -1,251 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.jta;
-
-import java.util.concurrent.atomic.AtomicReference;
-import javax.transaction.xa.XAException;
-import javax.transaction.xa.XAResource;
-import javax.transaction.xa.Xid;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.transactions.TransactionState;
-
-import static org.apache.ignite.transactions.TransactionState.ACTIVE;
-import static org.apache.ignite.transactions.TransactionState.COMMITTED;
-import static org.apache.ignite.transactions.TransactionState.PREPARED;
-import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK;
-
-/**
- * Cache XA resource implementation.
- */
-public final class GridCacheXAResource implements XAResource {
-    /** Logger reference. */
-    private static final AtomicReference<IgniteLogger> logRef = new AtomicReference<>();
-
-    /** */
-    private static IgniteLogger log;
-
-    /** */
-    private static final Xid[] NO_XID = new Xid[] {};
-
-    /** Cache transaction. */
-    private IgniteInternalTx cacheTx;
-
-    /** */
-    private Xid xid;
-
-    /**
-     * @param cacheTx Cache jta.
-     * @param ctx Kernal context.
-     */
-    public GridCacheXAResource(IgniteInternalTx cacheTx, GridKernalContext ctx) {
-        assert cacheTx != null;
-        assert ctx != null;
-
-        this.cacheTx = cacheTx;
-
-        if (log == null)
-            log = U.logger(ctx, logRef, GridCacheXAResource.class);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void start(Xid xid, int flags) {
-        if (log.isDebugEnabled())
-            log.debug("XA resource start(...) [xid=" + xid + ", flags=<" + flags(flags) + ">]");
-
-        // Simply save global transaction id.
-        this.xid = xid;
-    }
-
-    /**
-     * @param msg Message.
-     * @param cause Cause.
-     * @throws XAException XA exception.
-     */
-    private void throwException(String msg, Throwable cause) throws XAException {
-        XAException ex = new XAException(msg);
-
-        ex.initCause(cause);
-
-        throw ex;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void rollback(Xid xid) throws XAException {
-        assert this.xid.equals(xid);
-
-        if (log.isDebugEnabled())
-            log.debug("XA resource rollback(...) [xid=" + xid + "]");
-
-        try {
-            cacheTx.rollback();
-        }
-        catch (IgniteCheckedException e) {
-            throwException("Failed to rollback cache transaction: " + e.getMessage(), e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public int prepare(Xid xid) throws XAException {
-        assert this.xid.equals(xid);
-
-        if (log.isDebugEnabled())
-            log.debug("XA resource prepare(...) [xid=" + xid + "]");
-
-        if (cacheTx.state() != ACTIVE)
-            throw new XAException("Cache transaction is not in active state.");
-
-        try {
-            cacheTx.prepare();
-        }
-        catch (IgniteCheckedException e) {
-            throwException("Failed to prepare cache transaction.", e);
-        }
-
-        return XA_OK;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void end(Xid xid, int flags) {
-        assert this.xid.equals(xid);
-
-        if (log.isDebugEnabled())
-            log.debug("XA resource end(...) [xid=" + xid + ", flags=<" + flags(flags) + ">]");
-
-        if ((flags & TMFAIL) > 0)
-            cacheTx.setRollbackOnly();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void commit(Xid xid, boolean onePhase) throws XAException {
-        assert this.xid.equals(xid);
-
-        if (log.isDebugEnabled())
-            log.debug("XA resource commit(...) [xid=" + xid + ", onePhase=" + onePhase + "]");
-
-        try {
-            cacheTx.commit();
-        }
-        catch (IgniteCheckedException e) {
-            throwException("Failed to commit cache transaction: " + e.getMessage(), e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void forget(Xid xid) throws XAException {
-        assert this.xid.equals(xid);
-
-        if (log.isDebugEnabled())
-            log.debug("XA resource forget(...) [xid=" + xid + "]");
-
-        try {
-            cacheTx.invalidate(true);
-
-            cacheTx.commit();
-        }
-        catch (IgniteCheckedException e) {
-            throwException("Failed to forget cache transaction: " + e.getMessage(), e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Xid[] recover(int i) {
-        if (cacheTx.state() == PREPARED)
-            return new Xid[] { xid };
-
-        return NO_XID;
-    }
-
-    /**
-     * @param flags JTA Flags.
-     * @return Comma-separated flags string.
-     */
-    private String flags(int flags) {
-        StringBuilder res = new StringBuilder();
-
-        addFlag(res, flags, TMENDRSCAN, "TMENDRSCAN");
-        addFlag(res, flags, TMFAIL, "TMFAIL");
-        addFlag(res, flags, TMJOIN, "TMJOIN");
-        addFlag(res, flags, TMNOFLAGS, "TMNOFLAGS");
-        addFlag(res, flags, TMONEPHASE, "TMONEPHASE");
-        addFlag(res, flags, TMRESUME, "TMRESUME");
-        addFlag(res, flags, TMSTARTRSCAN, "TMSTARTRSCAN");
-        addFlag(res, flags, TMSUCCESS, "TMSUCCESS");
-        addFlag(res, flags, TMSUSPEND, "TMSUSPEND");
-
-        return res.toString();
-    }
-
-    /**
-     * @param sb String builder.
-     * @param flags Flags bit set.
-     * @param mask Bit mask.
-     * @param flagName String name of the flag specified by given mask.
-     * @return String builder appended by flag if it's presented in bit set.
-     */
-    private StringBuilder addFlag(StringBuilder sb, int flags, int mask, String flagName) {
-        if ((flags & mask) > 0)
-            sb.append(sb.length() > 0 ? "," : "").append(flagName);
-
-        return sb;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getTransactionTimeout() {
-        return (int)(cacheTx.timeout() / 1000);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean setTransactionTimeout(int i) {
-        cacheTx.timeout(i * 1000);
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isSameRM(XAResource xar) {
-        if (xar == this)
-            return true;
-
-        if (!(xar instanceof GridCacheXAResource))
-            return false;
-
-        GridCacheXAResource other = (GridCacheXAResource)xar;
-
-        return cacheTx == other.cacheTx;
-    }
-
-    /**
-     *
-     * @return {@code true} if jta was already committed or rolled back.
-     */
-    public boolean isFinished() {
-        TransactionState state = cacheTx.state();
-
-        return state == COMMITTED || state == ROLLED_BACK;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridCacheXAResource.class, this);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/334ce4c9/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/AbstarctCacheJtaSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/AbstarctCacheJtaSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/AbstarctCacheJtaSelfTest.java
deleted file mode 100644
index 41c4565..0000000
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/AbstarctCacheJtaSelfTest.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import javax.transaction.Status;
-import javax.transaction.UserTransaction;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.transactions.Transaction;
-import org.objectweb.jotm.Jotm;
-
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-import static org.apache.ignite.transactions.TransactionState.ACTIVE;
-
-/**
- * Abstract class for cache tests.
- */
-public abstract class AbstarctCacheJtaSelfTest extends GridCacheAbstractSelfTest {
-    /** */
-    private static final int GRID_CNT = 1;
-
-    /** Java Open Transaction Manager facade. */
-    protected static Jotm jotm;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        jotm = new Jotm(true, false);
-
-        super.beforeTestsStarted();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        super.afterTestsStopped();
-
-        jotm.stop();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return GRID_CNT;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        configureJta(cfg);
-
-        CacheConfiguration cfg1 = cacheConfiguration(gridName);
-
-        CacheConfiguration cfg2 = cacheConfiguration(gridName);
-
-        cfg2.setName("cache-2");
-
-        cfg.setCacheConfiguration(cfg1, cfg2);
-
-        return cfg;
-    }
-
-    /**
-     * @param cfg Ignite Configuration.
-     */
-    protected abstract void configureJta(IgniteConfiguration cfg);
-
-    /**
-     * JUnit.
-     *
-     * @throws Exception If failed.
-     */
-    public void testJta() throws Exception {
-        UserTransaction jtaTx = jotm.getUserTransaction();
-
-        IgniteCache<String, Integer> cache = jcache();
-
-        assert ignite(0).transactions().tx() == null;
-
-        jtaTx.begin();
-
-        try {
-            assert ignite(0).transactions().tx() == null;
-
-            assert cache.getAndPut("key", 1) == null;
-
-            Transaction tx = ignite(0).transactions().tx();
-
-            assert tx != null;
-            assert tx.state() == ACTIVE;
-
-            Integer one = 1;
-
-            assertEquals(one, cache.get("key"));
-
-            tx = ignite(0).transactions().tx();
-
-            assert tx != null;
-            assert tx.state() == ACTIVE;
-
-            jtaTx.commit();
-
-            assert ignite(0).transactions().tx() == null;
-        }
-        finally {
-            if (jtaTx.getStatus() == Status.STATUS_ACTIVE)
-                jtaTx.rollback();
-        }
-
-        assertEquals((Integer)1, cache.get("key"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("ConstantConditions")
-    public void testJtaTwoCaches() throws Exception {
-        UserTransaction jtaTx = jotm.getUserTransaction();
-
-        IgniteEx ignite = grid(0);
-
-        IgniteCache<String, Integer> cache1 = jcache();
-
-        IgniteCache<Object, Object> cache2 = ignite.cache("cache-2");
-
-        assertNull(ignite.transactions().tx());
-
-        jtaTx.begin();
-
-        try {
-            cache1.put("key", 0);
-            cache2.put("key", 0);
-            cache1.put("key1", 1);
-            cache2.put("key2", 2);
-
-            assertEquals(0, (int)cache1.get("key"));
-            assertEquals(0, (int)cache1.get("key"));
-            assertEquals(1, (int)cache1.get("key1"));
-            assertEquals(2, (int)cache2.get("key2"));
-
-            assertEquals(ignite.transactions().tx().state(), ACTIVE);
-
-            jtaTx.commit();
-
-            assertNull(ignite.transactions().tx());
-
-            assertEquals(0, (int)cache1.get("key"));
-            assertEquals(0, (int)cache2.get("key"));
-            assertEquals(1, (int)cache1.get("key1"));
-            assertEquals(2, (int)cache2.get("key2"));
-        }
-        finally {
-            if (jtaTx.getStatus() == Status.STATUS_ACTIVE)
-                jtaTx.rollback();
-        }
-
-        assertEquals(0, (int)cache1.get("key"));
-        assertEquals(0, (int)cache2.get("key"));
-        assertEquals(1, (int)cache1.get("key1"));
-        assertEquals(2, (int)cache2.get("key2"));
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/334ce4c9/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/AbstractCacheJtaSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/AbstractCacheJtaSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/AbstractCacheJtaSelfTest.java
new file mode 100644
index 0000000..96e3258
--- /dev/null
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/AbstractCacheJtaSelfTest.java
@@ -0,0 +1,183 @@
+/*
+ * 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;
+
+import javax.transaction.Status;
+import javax.transaction.UserTransaction;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.transactions.Transaction;
+import org.objectweb.jotm.Jotm;
+
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.transactions.TransactionState.ACTIVE;
+
+/**
+ * Abstract class for cache tests.
+ */
+public abstract class AbstractCacheJtaSelfTest extends GridCacheAbstractSelfTest {
+    /** */
+    private static final int GRID_CNT = 1;
+
+    /** Java Open Transaction Manager facade. */
+    protected static Jotm jotm;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        jotm = new Jotm(true, false);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        jotm.stop();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return GRID_CNT;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        configureJta(cfg);
+
+        CacheConfiguration cfg1 = cacheConfiguration(gridName);
+
+        CacheConfiguration cfg2 = cacheConfiguration(gridName);
+
+        cfg2.setName("cache-2");
+
+        cfg.setCacheConfiguration(cfg1, cfg2);
+
+        return cfg;
+    }
+
+    /**
+     * @param cfg Ignite Configuration.
+     */
+    protected abstract void configureJta(IgniteConfiguration cfg);
+
+    /**
+     * JUnit.
+     *
+     * @throws Exception If failed.
+     */
+    public void testJta() throws Exception {
+        UserTransaction jtaTx = jotm.getUserTransaction();
+
+        IgniteCache<String, Integer> cache = jcache();
+
+        assert ignite(0).transactions().tx() == null;
+
+        jtaTx.begin();
+
+        try {
+            assert ignite(0).transactions().tx() == null;
+
+            assert cache.getAndPut("key", 1) == null;
+
+            Transaction tx = ignite(0).transactions().tx();
+
+            assert tx != null;
+            assert tx.state() == ACTIVE;
+
+            Integer one = 1;
+
+            assertEquals(one, cache.get("key"));
+
+            tx = ignite(0).transactions().tx();
+
+            assert tx != null;
+            assert tx.state() == ACTIVE;
+
+            jtaTx.commit();
+
+            assert ignite(0).transactions().tx() == null;
+        }
+        finally {
+            if (jtaTx.getStatus() == Status.STATUS_ACTIVE)
+                jtaTx.rollback();
+        }
+
+        assertEquals((Integer)1, cache.get("key"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("ConstantConditions")
+    public void testJtaTwoCaches() throws Exception {
+        UserTransaction jtaTx = jotm.getUserTransaction();
+
+        IgniteEx ignite = grid(0);
+
+        IgniteCache<String, Integer> cache1 = jcache();
+
+        IgniteCache<Object, Object> cache2 = ignite.cache("cache-2");
+
+        assertNull(ignite.transactions().tx());
+
+        jtaTx.begin();
+
+        try {
+            cache1.put("key", 0);
+            cache2.put("key", 0);
+            cache1.put("key1", 1);
+            cache2.put("key2", 2);
+
+            assertEquals(0, (int)cache1.get("key"));
+            assertEquals(0, (int)cache1.get("key"));
+            assertEquals(1, (int)cache1.get("key1"));
+            assertEquals(2, (int)cache2.get("key2"));
+
+            assertEquals(ignite.transactions().tx().state(), ACTIVE);
+
+            jtaTx.commit();
+
+            assertNull(ignite.transactions().tx());
+
+            assertEquals(0, (int)cache1.get("key"));
+            assertEquals(0, (int)cache2.get("key"));
+            assertEquals(1, (int)cache1.get("key1"));
+            assertEquals(2, (int)cache2.get("key2"));
+        }
+        finally {
+            if (jtaTx.getStatus() == Status.STATUS_ACTIVE)
+                jtaTx.rollback();
+        }
+
+        assertEquals(0, (int)cache1.get("key"));
+        assertEquals(0, (int)cache2.get("key"));
+        assertEquals(1, (int)cache1.get("key1"));
+        assertEquals(2, (int)cache2.get("key2"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/334ce4c9/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaFactorySelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaFactorySelfTest.java
index 8d53d7f..f079974 100644
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaFactorySelfTest.java
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaFactorySelfTest.java
@@ -25,7 +25,7 @@ import org.objectweb.transaction.jta.TransactionManager;
 /**
  * Factory JTA integration test using PARTITIONED cache.
  */
-public class GridPartitionedCacheJtaFactorySelfTest extends AbstarctCacheJtaSelfTest {
+public class GridPartitionedCacheJtaFactorySelfTest extends AbstractCacheJtaSelfTest {
     /** {@inheritDoc} */
     @Override protected void configureJta(IgniteConfiguration cfg) {
         TransactionConfiguration txCfg = cfg.getTransactionConfiguration();

http://git-wip-us.apache.org/repos/asf/ignite/blob/334ce4c9/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaFactoryUseSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaFactoryUseSyncSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaFactoryUseSyncSelfTest.java
new file mode 100644
index 0000000..5e6deee
--- /dev/null
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaFactoryUseSyncSelfTest.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;
+
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ * Factory JTA integration test using PARTITIONED cache.
+ */
+public class GridPartitionedCacheJtaFactoryUseSyncSelfTest extends GridPartitionedCacheJtaFactorySelfTest {
+    /** {@inheritDoc} */
+    @Override protected void configureJta(IgniteConfiguration cfg) {
+        super.configureJta(cfg);
+
+        cfg.getTransactionConfiguration().setUseJtaSynchronization(true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/334ce4c9/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaLookupClassNameSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaLookupClassNameSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaLookupClassNameSelfTest.java
index ccebb9f..2e322f8 100644
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaLookupClassNameSelfTest.java
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaLookupClassNameSelfTest.java
@@ -30,7 +30,7 @@ import org.apache.ignite.testframework.GridTestUtils;
 /**
  * Lookup class name based JTA integration test using PARTITIONED cache.
  */
-public class GridPartitionedCacheJtaLookupClassNameSelfTest extends AbstarctCacheJtaSelfTest {
+public class GridPartitionedCacheJtaLookupClassNameSelfTest extends AbstractCacheJtaSelfTest {
     /** {@inheritDoc} */
     @Override protected void configureJta(IgniteConfiguration cfg) {
         cfg.getTransactionConfiguration().setTxManagerLookupClassName(TestTmLookup.class.getName());

http://git-wip-us.apache.org/repos/asf/ignite/blob/334ce4c9/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridReplicatedCacheJtaFactoryUseSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridReplicatedCacheJtaFactoryUseSyncSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridReplicatedCacheJtaFactoryUseSyncSelfTest.java
new file mode 100644
index 0000000..e25f5e8
--- /dev/null
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridReplicatedCacheJtaFactoryUseSyncSelfTest.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;
+
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ * Factory JTA integration test using REPLICATED cache.
+ */
+public class GridReplicatedCacheJtaFactoryUseSyncSelfTest extends GridReplicatedCacheJtaFactorySelfTest {
+    /** {@inheritDoc} */
+    @Override protected void configureJta(IgniteConfiguration cfg) {
+        super.configureJta(cfg);
+
+        cfg.getTransactionConfiguration().setUseJtaSynchronization(true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/334ce4c9/modules/jta/src/test/java/org/apache/ignite/testsuites/IgniteJtaTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/testsuites/IgniteJtaTestSuite.java b/modules/jta/src/test/java/org/apache/ignite/testsuites/IgniteJtaTestSuite.java
index 6e0c096..60c20dd 100644
--- a/modules/jta/src/test/java/org/apache/ignite/testsuites/IgniteJtaTestSuite.java
+++ b/modules/jta/src/test/java/org/apache/ignite/testsuites/IgniteJtaTestSuite.java
@@ -22,8 +22,10 @@ import org.apache.ignite.internal.processors.cache.CacheJndiTmFactorySelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheJtaConfigurationValidationSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheJtaFactoryConfigValidationSelfTest;
 import org.apache.ignite.internal.processors.cache.GridPartitionedCacheJtaFactorySelfTest;
+import org.apache.ignite.internal.processors.cache.GridPartitionedCacheJtaFactoryUseSyncSelfTest;
 import org.apache.ignite.internal.processors.cache.GridPartitionedCacheJtaLookupClassNameSelfTest;
 import org.apache.ignite.internal.processors.cache.GridReplicatedCacheJtaFactorySelfTest;
+import org.apache.ignite.internal.processors.cache.GridReplicatedCacheJtaFactoryUseSyncSelfTest;
 import org.apache.ignite.internal.processors.cache.GridReplicatedCacheJtaLookupClassNameSelfTest;
 import org.apache.ignite.internal.processors.cache.GridJtaLifecycleAwareSelfTest;
 
@@ -44,6 +46,9 @@ public class IgniteJtaTestSuite extends TestSuite {
         suite.addTestSuite(GridPartitionedCacheJtaLookupClassNameSelfTest.class);
         suite.addTestSuite(GridReplicatedCacheJtaLookupClassNameSelfTest.class);
 
+        suite.addTestSuite(GridPartitionedCacheJtaFactoryUseSyncSelfTest.class);
+        suite.addTestSuite(GridReplicatedCacheJtaFactoryUseSyncSelfTest.class);
+
         suite.addTestSuite(GridJtaLifecycleAwareSelfTest.class);
         suite.addTestSuite(GridCacheJtaConfigurationValidationSelfTest.class);
         suite.addTestSuite(GridCacheJtaFactoryConfigValidationSelfTest.class);


[32/35] ignite git commit: test commit (cherry picked from commit 1ef0854)

Posted by nt...@apache.org.
test commit
(cherry picked from commit 1ef0854)


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/402cbabd
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/402cbabd
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/402cbabd

Branch: refs/heads/ignite-2791
Commit: 402cbabdcc74cc4ebe57a625a18e159b42087224
Parents: c9b4a11
Author: Anton Vinogradov <av...@apache.org>
Authored: Mon Mar 14 17:57:41 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Mon Mar 14 19:25:02 2016 +0300

----------------------------------------------------------------------
 pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/402cbabd/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index bde8a2c..af44209 100644
--- a/pom.xml
+++ b/pom.xml
@@ -922,3 +922,4 @@
         </plugins>
     </build>
 </project>
+


[09/35] ignite git commit: compilation fix (cherry picked from commit 95a4175)

Posted by nt...@apache.org.
compilation fix
(cherry picked from commit 95a4175)


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/70abaa36
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/70abaa36
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/70abaa36

Branch: refs/heads/ignite-2791
Commit: 70abaa36ad4a801d00a8e5088dd803b3dc9dbf25
Parents: a8d16bd
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Mar 10 11:11:56 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Mar 10 12:03:09 2016 +0300

----------------------------------------------------------------------
 modules/web/ignite-weblogic-test/pom.xml        | 74 +++++++-------------
 .../src/main/webapp/WEB-INF/web.xml             | 17 -----
 .../src/main/webapp/index.jsp                   | 17 -----
 3 files changed, 25 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/70abaa36/modules/web/ignite-weblogic-test/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/ignite-weblogic-test/pom.xml b/modules/web/ignite-weblogic-test/pom.xml
index 4746f1f..622b798 100644
--- a/modules/web/ignite-weblogic-test/pom.xml
+++ b/modules/web/ignite-weblogic-test/pom.xml
@@ -1,38 +1,4 @@
 <!--
-  ~ 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.
-  -->
-
-<!--
-  ~ 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.
-  -->
-
-<!--
   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.
@@ -55,46 +21,56 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
     <modelVersion>4.0.0</modelVersion>
-    <groupId>org.apache.ignite</groupId>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../../parent</relativePath>
+    </parent>
+
     <artifactId>ignite-weblogic-test</artifactId>
     <packaging>war</packaging>
-    <version>1.0-SNAPSHOT</version>
-    <name>ignite-weblogic-test Maven Webapp</name>
-    <url>http://maven.apache.org</url>
-    <dependencies>
-        <dependency>
-            <groupId>junit</groupId>
-            <artifactId>junit</artifactId>
-            <version>3.8.1</version>
-            <scope>test</scope>
-        </dependency>
+    <version>1.6.0-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
 
+    <dependencies>
         <dependency>
             <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-core</artifactId>
-            <version>1.6.0-SNAPSHOT</version>
+            <version>${project.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-web</artifactId>
-            <version>1.6.0-SNAPSHOT</version>
+            <version>${project.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-log4j</artifactId>
-            <version>1.6.0-SNAPSHOT</version>
+            <version>${project.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-spring</artifactId>
-            <version>1.6.0-SNAPSHOT</version>
+            <version>${project.version}</version>
         </dependency>
     </dependencies>
 
     <build>
         <finalName>ignite-weblogic-test</finalName>
+
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-deploy-plugin</artifactId>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
+        </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/70abaa36/modules/web/ignite-weblogic-test/src/main/webapp/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/modules/web/ignite-weblogic-test/src/main/webapp/WEB-INF/web.xml b/modules/web/ignite-weblogic-test/src/main/webapp/WEB-INF/web.xml
index 054eab4..6fe7358 100644
--- a/modules/web/ignite-weblogic-test/src/main/webapp/WEB-INF/web.xml
+++ b/modules/web/ignite-weblogic-test/src/main/webapp/WEB-INF/web.xml
@@ -15,23 +15,6 @@
   ~ limitations under the License.
   -->
 
-<!--
-  ~ 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.
-  -->
-
 <!DOCTYPE web-app PUBLIC
  "-//Sun Microsystems, Inc.//DTD Web Application 2.3//EN"
  "http://java.sun.com/dtd/web-app_2_3.dtd" >

http://git-wip-us.apache.org/repos/asf/ignite/blob/70abaa36/modules/web/ignite-weblogic-test/src/main/webapp/index.jsp
----------------------------------------------------------------------
diff --git a/modules/web/ignite-weblogic-test/src/main/webapp/index.jsp b/modules/web/ignite-weblogic-test/src/main/webapp/index.jsp
index 0cc090d..4ca0be1 100644
--- a/modules/web/ignite-weblogic-test/src/main/webapp/index.jsp
+++ b/modules/web/ignite-weblogic-test/src/main/webapp/index.jsp
@@ -15,23 +15,6 @@
   ~ limitations under the License.
   --%>
 
-<%--
-  ~ 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.
-  --%>
-
 <%@ page import="org.apache.ignite.Ignition" %>
 <%@ page import="java.util.UUID" %>
 <html>


[02/35] ignite git commit: IGNITE-2702: .NET: Implemented compact footers optimization for binary serialization. This closes #523.

Posted by nt...@apache.org.
IGNITE-2702: .NET: Implemented compact footers optimization for binary serialization. This closes #523.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/c6c93892
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/c6c93892
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/c6c93892

Branch: refs/heads/ignite-2791
Commit: c6c93892f13c55fb6e48e542970e30b26d014511
Parents: 1f328e4
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Wed Mar 9 17:15:19 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Mar 9 17:15:19 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/binary/BinarySchema.java    |  11 +-
 .../processors/platform/PlatformContext.java    |  11 +-
 .../platform/PlatformContextImpl.java           |  72 ++++-
 .../platform/cluster/PlatformClusterGroup.java  |  12 +
 .../PlatformDotNetConfigurationClosure.java     |   5 -
 .../utils/PlatformConfigurationUtils.java       |  14 +
 .../ignite/platform/PlatformSqlQueryTask.java   | 117 +++++++++
 .../Apache.Ignite.Core.Tests.csproj             |   8 +-
 .../Binary/BinaryBuilderSelfTest.cs             |  33 ++-
 .../Binary/BinaryBuilderSelfTestFullFooter.cs   |  31 +++
 .../Binary/BinaryCompactFooterInteropTest.cs    | 129 +++++++++
 .../Binary/BinarySelfTest.cs                    |  23 +-
 .../Binary/BinarySelfTestFullFooter.cs          |  35 +++
 .../Compute/ComputeApiTest.cs                   |  57 +++-
 .../Compute/ComputeApiTestFullFooter.cs         |  65 +++++
 .../Config/Compute/compute-grid1.xml            |  20 +-
 .../Config/Compute/compute-grid2.xml            |   2 +-
 .../Config/Compute/compute-grid3.xml            |   2 +-
 .../Apache.Ignite.Core.Tests/Config/binary.xml  |  56 ----
 .../IgniteConfigurationSerializerTest.cs        |   3 +-
 .../IgniteConfigurationTest.cs                  |   2 +
 .../Services/ServicesTest.cs                    |  26 +-
 .../Services/ServicesTestFullFooter.cs          |  33 +++
 .../Apache.Ignite.Core.csproj                   |   1 +
 .../Binary/BinaryConfiguration.cs               |  44 +++-
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  15 ++
 .../IgniteConfigurationSection.xsd              |   1 +
 .../Apache.Ignite.Core/Impl/Binary/Binary.cs    |   8 +-
 .../Impl/Binary/BinaryObject.cs                 |   9 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |   9 +-
 .../Impl/Binary/BinaryObjectHeader.cs           | 176 +------------
 .../Impl/Binary/BinaryObjectSchema.cs           |  20 ++
 .../Impl/Binary/BinaryObjectSchemaHolder.cs     |  18 +-
 .../Impl/Binary/BinaryObjectSchemaSerializer.cs | 262 +++++++++++++++++++
 .../Impl/Binary/BinaryReader.cs                 |  46 +++-
 .../Impl/Binary/BinaryWriter.cs                 |  20 +-
 .../Impl/Binary/Marshaller.cs                   |  12 +-
 .../Impl/Binary/Metadata/BinaryType.cs          |  28 +-
 .../Impl/Cluster/ClusterGroupImpl.cs            |  15 ++
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  19 ++
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  47 +++-
 41 files changed, 1204 insertions(+), 313 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/core/src/main/java/org/apache/ignite/internal/binary/BinarySchema.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinarySchema.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinarySchema.java
index 04124e0..125719e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinarySchema.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinarySchema.java
@@ -86,7 +86,7 @@ public class BinarySchema implements Externalizable {
      * @param schemaId Schema ID.
      * @param fieldIds Field IDs.
      */
-    private BinarySchema(int schemaId, List<Integer> fieldIds) {
+    public BinarySchema(int schemaId, List<Integer> fieldIds) {
         assert fieldIds != null;
 
         this.schemaId = schemaId;
@@ -261,6 +261,15 @@ public class BinarySchema implements Externalizable {
     }
 
     /**
+     * Gets field ids array.
+     *
+     * @return Field ids.
+     */
+    public int[] fieldIds() {
+        return ids;
+    }
+
+    /**
      * Parse values.
      *
      * @param vals Values.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
index e88d57b..da92c6c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
@@ -143,6 +143,15 @@ public interface PlatformContext {
     public void writeAllMetadata(BinaryRawWriterEx writer);
 
     /**
+     * Write schema for the given type ID and schema ID.
+     *
+     * @param writer Writer.
+     * @param typeId Type ID.
+     * @param schemaId Schema ID.
+     */
+    public void writeSchema(BinaryRawWriterEx writer, int typeId, int schemaId);
+
+    /**
      * Write cluster metrics.
      *
      * @param writer Writer.
@@ -279,4 +288,4 @@ public interface PlatformContext {
      * @return Current platform name.
      */
     public String platform();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
index b45414a..d89176b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
@@ -34,9 +34,13 @@ import org.apache.ignite.events.JobEvent;
 import org.apache.ignite.events.SwapSpaceEvent;
 import org.apache.ignite.events.TaskEvent;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.binary.BinaryMetadata;
 import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinarySchema;
+import org.apache.ignite.internal.binary.BinarySchemaRegistry;
 import org.apache.ignite.internal.binary.BinaryTypeImpl;
 import org.apache.ignite.internal.binary.GridBinaryMarshaller;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
@@ -73,11 +77,13 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.Nullable;
 
 import java.sql.Timestamp;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
@@ -86,6 +92,7 @@ import java.util.concurrent.ConcurrentHashMap;
 /**
  * Implementation of platform context.
  */
+@SuppressWarnings("TypeMayBeWeakened")
 public class PlatformContextImpl implements PlatformContext {
     /** Supported event types. */
     private static final Set<Integer> evtTyps;
@@ -365,13 +372,36 @@ public class PlatformContextImpl implements PlatformContext {
 
                     boolean isEnum = reader.readBoolean();
 
-                    return new Metadata(typeId, typeName, affKey, fields, isEnum);
+                    // Read schemas
+                    int schemaCnt = reader.readInt();
+
+                    List<BinarySchema> schemas = null;
+
+                    if (schemaCnt > 0) {
+                        schemas = new ArrayList<>(schemaCnt);
+
+                        for (int i = 0; i < schemaCnt; i++) {
+                            int id = reader.readInt();
+                            int fieldCnt = reader.readInt();
+                            List<Integer> fieldIds = new ArrayList<>(fieldCnt);
+
+                            for (int j = 0; j < fieldCnt; j++)
+                                fieldIds.add(reader.readInt());
+
+                            schemas.add(new BinarySchema(id, fieldIds));
+                        }
+                    }
+
+                    return new Metadata(typeId, typeName, affKey, fields, isEnum, schemas);
                 }
             }
         );
 
+        BinaryContext binCtx = cacheObjProc.binaryContext();
+
         for (Metadata meta : metas)
-            cacheObjProc.updateMetadata(meta.typeId, meta.typeName, meta.affKey, meta.fields, meta.isEnum);
+            binCtx.updateMetadata(meta.typeId, new BinaryMetadata(meta.typeId,
+                meta.typeName, meta.fields, meta.affKey, meta.schemas, meta.isEnum));
     }
 
     /** {@inheritDoc} */
@@ -389,6 +419,30 @@ public class PlatformContextImpl implements PlatformContext {
             writeMetadata0(writer, cacheObjProc.typeId(m.typeName()), m);
     }
 
+    /** {@inheritDoc} */
+    @Override public void writeSchema(BinaryRawWriterEx writer, int typeId, int schemaId) {
+        BinarySchemaRegistry schemaReg = cacheObjProc.binaryContext().schemaRegistry(typeId);
+        BinarySchema schema = schemaReg.schema(schemaId);
+
+        if (schema == null) {
+            BinaryTypeImpl meta = (BinaryTypeImpl)cacheObjProc.metadata(typeId);
+
+            for (BinarySchema typeSchema : meta.metadata().schemas()) {
+                if (schemaId == typeSchema.schemaId()) {
+                    schema = typeSchema;
+                    break;
+                }
+            }
+
+            if (schema != null)
+                schemaReg.addSchema(schemaId, schema);
+        }
+
+        int[] fieldIds = schema == null ? null : schema.fieldIds();
+
+        writer.writeIntArray(fieldIds);
+    }
+
     /**
      * Write binary metadata.
      *
@@ -402,7 +456,8 @@ public class PlatformContextImpl implements PlatformContext {
         else {
             writer.writeBoolean(true);
 
-            Map<String, Integer> fields = ((BinaryTypeImpl)meta).metadata().fieldsMap();
+            BinaryMetadata meta0 = ((BinaryTypeImpl) meta).metadata();
+            Map<String, Integer> fields = meta0.fieldsMap();
 
             writer.writeInt(typeId);
             writer.writeString(meta.typeName());
@@ -651,21 +706,26 @@ public class PlatformContextImpl implements PlatformContext {
         /** Enum flag. */
         private final boolean isEnum;
 
+        /** Schemas. */
+        private final List<BinarySchema> schemas;
+
         /**
          * Constructor.
-         *
-         * @param typeId Type ID.
+         *  @param typeId Type ID.
          * @param typeName Type name.
          * @param affKey Affinity key.
          * @param fields Fields.
          * @param isEnum Enum flag.
+         * @param schemas Schemas.
          */
-        public Metadata(int typeId, String typeName, String affKey, Map<String, Integer> fields, boolean isEnum) {
+        private Metadata(int typeId, String typeName, String affKey, Map<String, Integer> fields, boolean isEnum,
+            List<BinarySchema> schemas) {
             this.typeId = typeId;
             this.typeName = typeName;
             this.affKey = affKey;
             this.fields = fields;
             this.isEnum = isEnum;
+            this.schemas = schemas;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
index f60766b..d80079c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
@@ -77,6 +77,9 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     /** */
     private static final int OP_TOPOLOGY = 14;
 
+    /** */
+    private static final int OP_SCHEMA = 15;
+
     /** Projection. */
     private final ClusterGroupEx prj;
 
@@ -188,6 +191,15 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
                 break;
             }
 
+            case OP_SCHEMA: {
+                int typeId = reader.readInt();
+                int schemaId = reader.readInt();
+
+                platformCtx.writeSchema(writer, typeId, schemaId);
+
+                break;
+            }
+
             default:
                 super.processInStreamOutStream(type, reader, writer);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
index 8728d77..db2fa4d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
@@ -109,7 +109,6 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
         if (bCfg == null) {
             bCfg = new BinaryConfiguration();
 
-            bCfg.setCompactFooter(false);
             bCfg.setNameMapper(new BinaryBasicNameMapper(true));
             bCfg.setIdMapper(new BinaryBasicIdMapper(true));
 
@@ -142,10 +141,6 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
             }
         }
 
-        if (bCfg.isCompactFooter())
-            throw new IgniteException("Unsupported " + BinaryMarshaller.class.getName() +
-                " \"compactFooter\" flag: must be false when running Apache Ignite.NET.");
-
         // Set Ignite home so that marshaller context works.
         String ggHome = igniteCfg.getIgniteHome();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index c0e9f1b..50728a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -28,6 +28,7 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.binary.*;
@@ -259,6 +260,13 @@ import java.util.Map;
 
         readCacheConfigurations(in, cfg);
         readDiscoveryConfiguration(in, cfg);
+
+        if (in.readBoolean()) {
+            if (cfg.getBinaryConfiguration() == null)
+                cfg.setBinaryConfiguration(new BinaryConfiguration());
+
+            cfg.getBinaryConfiguration().setCompactFooter(in.readBoolean());
+        }
     }
 
     /**
@@ -544,6 +552,12 @@ import java.util.Map;
 
         writeDiscoveryConfiguration(w, cfg.getDiscoverySpi());
 
+        BinaryConfiguration bc = cfg.getBinaryConfiguration();
+        w.writeBoolean(bc != null);
+
+        if (bc != null)
+            w.writeBoolean(bc.isCompactFooter());
+
         w.writeString(cfg.getIgniteHome());
 
         w.writeLong(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getInit());

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/core/src/test/java/org/apache/ignite/platform/PlatformSqlQueryTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformSqlQueryTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformSqlQueryTask.java
new file mode 100644
index 0000000..41d83aa
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformSqlQueryTask.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.platform;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.SqlQuery;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeTaskAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.jetbrains.annotations.Nullable;
+
+import javax.cache.Cache;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Task that performs an SQL query and returns results.
+ */
+public class PlatformSqlQueryTask extends ComputeTaskAdapter<String, Object> {
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        @Nullable String arg) throws IgniteException {
+        return Collections.singletonMap(new SqlQueryJob(arg), F.first(subgrid));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object reduce(List<ComputeJobResult> results) throws IgniteException {
+        ComputeJobResult res = results.get(0);
+
+        if (res.getException() != null)
+            throw res.getException();
+        else
+            return results.get(0).getData();
+    }
+
+    /**
+     * Job.
+     */
+    private static class SqlQueryJob extends ComputeJobAdapter implements Externalizable {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** Argument. */
+        private String arg;
+
+        /**
+         * Constructor.
+         */
+        public SqlQueryJob() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param arg Argument.
+         */
+        private SqlQueryJob(String arg) {
+            this.arg = arg;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Object execute() {
+            IgniteCache<Integer, PlatformComputeBinarizable> cache = ignite.cache(null);
+
+            SqlQuery<Integer, PlatformComputeBinarizable> qry = new SqlQuery<>("PlatformComputeBinarizable", arg);
+
+            List<Cache.Entry<Integer, PlatformComputeBinarizable>> qryRes = cache.query(qry).getAll();
+
+            Collection<PlatformComputeBinarizable> res = new ArrayList<>(qryRes.size());
+
+            for (Cache.Entry<Integer, PlatformComputeBinarizable> e : qryRes)
+                res.add(e.getValue());
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeObject(arg);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            arg = (String)in.readObject();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index bb56a3d..8c266d7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -61,6 +61,9 @@
     <Reference Include="System.XML" />
   </ItemGroup>
   <ItemGroup>
+    <Compile Include="Binary\BinaryBuilderSelfTestFullFooter.cs" />
+    <Compile Include="Binary\BinaryCompactFooterInteropTest.cs" />
+    <Compile Include="Binary\BinarySelfTestFullFooter.cs" />
     <Compile Include="Cache\CacheAffinityFieldTest.cs" />
     <Compile Include="Cache\CacheConfigurationTest.cs" />
     <Compile Include="Cache\CacheDynamicStartTest.cs" />
@@ -91,6 +94,7 @@
     <Compile Include="Cache\Store\CacheTestParallelLoadStore.cs" />
     <Compile Include="Cache\Store\CacheTestStore.cs" />
     <Compile Include="Compute\CancellationTest.cs" />
+    <Compile Include="Compute\ComputeApiTestFullFooter.cs" />
     <Compile Include="Compute\Forked\ForkedBinarizableClosureTaskTest.cs" />
     <Compile Include="Compute\Forked\ForkedResourceTaskTest.cs" />
     <Compile Include="Compute\Forked\ForkedSerializableClosureTaskTest.cs" />
@@ -137,6 +141,7 @@
     <Compile Include="ReconnectTest.cs" />
     <Compile Include="SerializationTest.cs" />
     <Compile Include="IgniteStartStopTest.cs" />
+    <Compile Include="Services\ServicesTestFullFooter.cs" />
     <Compile Include="TestUtils.cs" />
     <Compile Include="Memory\InteropMemoryTest.cs" />
     <Compile Include="Binary\BinaryBuilderSelfTest.cs" />
@@ -239,9 +244,6 @@
     <Content Include="Config\native-client-test-cache.xml">
       <CopyToOutputDirectory>Always</CopyToOutputDirectory>
     </Content>
-    <Content Include="Config\binary.xml">
-      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
-    </Content>
     <Content Include="Config\start-test-grid1.xml">
       <CopyToOutputDirectory>Always</CopyToOutputDirectory>
     </Content>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
index d442fb1..e2f7d8a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
@@ -24,6 +24,8 @@ namespace Apache.Ignite.Core.Tests.Binary
     using System.Collections.Generic;
     using System.Linq;
     using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Discovery.Tcp;
+    using Apache.Ignite.Core.Discovery.Tcp.Static;
     using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Binary;
     using NUnit.Framework;
@@ -83,11 +85,18 @@ namespace Apache.Ignite.Core.Tests.Binary
                         new BinaryTypeConfiguration(TypeEmpty),
                         new BinaryTypeConfiguration(typeof(TestEnumRegistered))
                     },
-                    DefaultIdMapper = new IdMapper()
+                    DefaultIdMapper = new IdMapper(),
+                    CompactFooter = GetCompactFooter()
                 },
                 JvmClasspath = TestUtils.CreateTestClasspath(),
                 JvmOptions = TestUtils.TestJavaOptions(),
-                SpringConfigUrl = "config\\binary.xml"
+                DiscoverySpi = new TcpDiscoverySpi
+                {
+                    IpFinder = new TcpDiscoveryStaticIpFinder
+                    {
+                        Endpoints = new[] { "127.0.0.1:47500", "127.0.0.1:47501" }
+                    }
+                }
             };
 
             _grid = (Ignite) Ignition.Start(cfg);
@@ -96,10 +105,18 @@ namespace Apache.Ignite.Core.Tests.Binary
         }
 
         /// <summary>
+        /// Gets the compact footer setting.
+        /// </summary>
+        protected virtual bool GetCompactFooter()
+        {
+            return true;
+        }
+
+        /// <summary>
         /// Tear down routine.
         /// </summary>
         [TestFixtureTearDown]
-        public virtual void TearDown()
+        public void TearDown()
         {
             if (_grid != null)
                 Ignition.Stop(_grid.Name, true);
@@ -1418,6 +1435,16 @@ namespace Apache.Ignite.Core.Tests.Binary
                 Assert.AreEqual((TestEnumRegistered)val, binEnum.Deserialize<TestEnumRegistered>());
             }
         }
+
+        /// <summary>
+        /// Tests the compact footer setting.
+        /// </summary>
+        [Test]
+        public void TestCompactFooterSetting()
+        {
+            Assert.AreEqual(GetCompactFooter(), _marsh.CompactFooter);
+        }
+
     }
 
     /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTestFullFooter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTestFullFooter.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTestFullFooter.cs
new file mode 100644
index 0000000..b6a1a8a
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTestFullFooter.cs
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Binary
+{
+    /// <summary>
+    /// Binary builder self test with compact footers disabled.
+    /// </summary>
+    public class BinaryBuilderSelfTestFullFooter : BinaryBuilderSelfTest
+    {
+        /** <inheritdoc /> */
+        protected override bool GetCompactFooter()
+        {
+            return false;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryCompactFooterInteropTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryCompactFooterInteropTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryCompactFooterInteropTest.cs
new file mode 100644
index 0000000..b01b65e
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryCompactFooterInteropTest.cs
@@ -0,0 +1,129 @@
+/*
+ * 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.
+ */
+
+#pragma warning disable 618   // SpringConfigUrl
+namespace Apache.Ignite.Core.Tests.Binary
+{
+    using System.Collections;
+    using System.Linq;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Tests.Compute;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests compact footer mode interop with Java.
+    /// </summary>
+    public class BinaryCompactFooterInteropTest
+    {
+        /** */
+        private const string PlatformSqlQueryTask = "org.apache.ignite.platform.PlatformSqlQueryTask";
+
+        /** */
+        private IIgnite _grid;
+
+        /** */
+        private IIgnite _clientGrid;
+
+        /// <summary>
+        /// Sets up the test.
+        /// </summary>
+        [SetUp]
+        public void TestSetUp()
+        {
+            // Start fresh cluster for each test
+            _grid = Ignition.Start(Config("config\\compute\\compute-grid1.xml"));
+            _clientGrid = Ignition.Start(Config("config\\compute\\compute-grid3.xml"));
+        }
+
+        /// <summary>
+        /// Tears down the test.
+        /// </summary>
+        [TearDown]
+        public void TestTearDown()
+        {
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        /// Tests an object that comes from Java.
+        /// </summary>
+        [Test]
+        public void TestFromJava([Values(true, false)] bool client)
+        {
+            var grid = client ? _clientGrid : _grid;
+
+            var fromJava = grid.GetCompute().ExecuteJavaTask<PlatformComputeBinarizable>(ComputeApiTest.EchoTask, 
+                ComputeApiTest.EchoTypeBinarizable);
+
+            Assert.AreEqual(1, fromJava.Field);
+        }
+
+        /// <summary>
+        /// Tests an object that comes from .NET in Java.
+        /// </summary>
+        [Test]
+        public void TestFromDotNet([Values(true, false)] bool client)
+        {
+            var grid = client ? _clientGrid : _grid;
+
+            var compute = grid.GetCompute().WithKeepBinary();
+
+            var arg = new PlatformComputeNetBinarizable {Field = 100};
+
+            var res = compute.ExecuteJavaTask<int>(ComputeApiTest.BinaryArgTask, arg);
+
+            Assert.AreEqual(arg.Field, res);
+        }
+
+        /// <summary>
+        /// Tests the indexing.
+        /// </summary>
+        [Test]
+        public void TestIndexing([Values(true, false)] bool client)
+        {
+            var grid = client ? _clientGrid : _grid;
+
+            var cache = grid.GetCache<int, PlatformComputeBinarizable>(null);
+
+            // Populate cache in .NET
+            for (var i = 0; i < 100; i++)
+                cache[i] = new PlatformComputeBinarizable {Field = i};
+
+            // Run SQL query on Java side
+            var qryRes = grid.GetCompute().ExecuteJavaTask<IList>(PlatformSqlQueryTask, "Field < 10");
+
+            Assert.AreEqual(10, qryRes.Count);
+            Assert.IsTrue(qryRes.OfType<PlatformComputeBinarizable>().All(x => x.Field < 10));
+        }
+
+        /// <summary>
+        /// Gets the config.
+        /// </summary>
+        private static IgniteConfiguration Config(string springUrl)
+        {
+            return new IgniteConfiguration
+            {
+                SpringConfigUrl = springUrl,
+                JvmOptions = TestUtils.TestJavaOptions(),
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                BinaryConfiguration = new BinaryConfiguration(
+                    typeof (PlatformComputeBinarizable),
+                    typeof (PlatformComputeNetBinarizable))
+            };
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
index 24ce3c8..0fcb792 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
@@ -20,6 +20,7 @@
 // ReSharper disable PossibleInvalidOperationException
 // ReSharper disable UnusedAutoPropertyAccessor.Global
 // ReSharper disable MemberCanBePrivate.Global
+// ReSharper disable UnusedMember.Local
 namespace Apache.Ignite.Core.Tests.Binary
 {
     using System;
@@ -37,7 +38,7 @@ namespace Apache.Ignite.Core.Tests.Binary
     using BinaryWriter = Apache.Ignite.Core.Impl.Binary.BinaryWriter;
 
     /// <summary>
-    /// 
+    /// Binary tests.
     /// </summary>
     [TestFixture]
     public class BinarySelfTest { 
@@ -50,7 +51,16 @@ namespace Apache.Ignite.Core.Tests.Binary
         [TestFixtureSetUp]
         public void BeforeTest()
         {
-            _marsh = new Marshaller(null);
+            _marsh = new Marshaller(GetBinaryConfiguration());
+        }
+
+        /// <summary>
+        /// Gets the binary configuration.
+        /// </summary>
+        /// <returns></returns>
+        protected virtual BinaryConfiguration GetBinaryConfiguration()
+        {
+            return new BinaryConfiguration { CompactFooter = true };
         }
         
         /**
@@ -1418,6 +1428,15 @@ namespace Apache.Ignite.Core.Tests.Binary
             Assert.Throws<BinaryObjectException>(() => new Marshaller(cfg));
         }
 
+        /// <summary>
+        /// Tests the compact footer setting.
+        /// </summary>
+        [Test]
+        public void TestCompactFooterSetting()
+        {
+            Assert.AreEqual(GetBinaryConfiguration().CompactFooter, _marsh.CompactFooter);
+        }
+
         private static void CheckKeepSerialized(BinaryConfiguration cfg, bool expKeep)
         {
             if (cfg.TypeConfigurations == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTestFullFooter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTestFullFooter.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTestFullFooter.cs
new file mode 100644
index 0000000..06e43e1
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTestFullFooter.cs
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Binary
+{
+    using Apache.Ignite.Core.Binary;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Binary test with full footers.
+    /// </summary>
+    [TestFixture]
+    public class BinarySelfTestFullFooter : BinarySelfTest
+    {
+        /** <inheritdoc /> */
+        protected override BinaryConfiguration GetBinaryConfiguration()
+        {
+            return new BinaryConfiguration {CompactFooter = false};
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
index c33d095..45fb4b4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
@@ -16,6 +16,8 @@
  */
 
 // ReSharper disable SpecifyACultureInStringConversionExplicitly
+// ReSharper disable UnusedAutoPropertyAccessor.Global
+#pragma warning disable 618  // SpringConfigUrl
 namespace Apache.Ignite.Core.Tests.Compute
 {
     using System;
@@ -27,6 +29,7 @@ namespace Apache.Ignite.Core.Tests.Compute
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Resource;
     using NUnit.Framework;
@@ -40,7 +43,7 @@ namespace Apache.Ignite.Core.Tests.Compute
         public const string EchoTask = "org.apache.ignite.platform.PlatformComputeEchoTask";
 
         /** Binary argument task name. */
-        private const string BinaryArgTask = "org.apache.ignite.platform.PlatformComputeBinarizableArgTask";
+        public const string BinaryArgTask = "org.apache.ignite.platform.PlatformComputeBinarizableArgTask";
 
         /** Broadcast task name. */
         public const string BroadcastTask = "org.apache.ignite.platform.PlatformComputeBroadcastTask";
@@ -88,7 +91,7 @@ namespace Apache.Ignite.Core.Tests.Compute
         private const int EchoTypeMap = 11;
 
         /** Echo type: binarizable. */
-        private const int EchoTypeBinarizable = 12;
+        public const int EchoTypeBinarizable = 12;
 
         /** Echo type: binary (Java only). */
         private const int EchoTypeBinarizableJava = 13;
@@ -126,14 +129,35 @@ namespace Apache.Ignite.Core.Tests.Compute
         [TestFixtureSetUp]
         public void InitClient()
         {
-            //TestUtils.JVM_DEBUG = true;
             TestUtils.KillProcesses();
 
-            _grid1 = Ignition.Start(Configuration("config\\compute\\compute-grid1.xml"));
-            _grid2 = Ignition.Start(Configuration("config\\compute\\compute-grid2.xml"));
-            _grid3 = Ignition.Start(Configuration("config\\compute\\compute-grid3.xml"));
+            var configs = GetConfigs();
+
+            _grid1 = Ignition.Start(Configuration(configs.Item1));
+            _grid2 = Ignition.Start(Configuration(configs.Item2));
+            _grid3 = Ignition.Start(Configuration(configs.Item3));
+        }
+
+        /// <summary>
+        /// Gets the configs.
+        /// </summary>
+        protected virtual Tuple<string, string, string> GetConfigs()
+        {
+            return Tuple.Create(
+                "config\\compute\\compute-grid1.xml",
+                "config\\compute\\compute-grid2.xml",
+                "config\\compute\\compute-grid3.xml");
+        }
+
+        /// <summary>
+        /// Gets the expected compact footers setting.
+        /// </summary>
+        protected virtual bool CompactFooter
+        {
+            get { return true; }
         }
 
+
         [TestFixtureTearDown]
         public void StopClient()
         {
@@ -313,7 +337,7 @@ namespace Apache.Ignite.Core.Tests.Compute
 
             Assert.AreEqual(topVer + 1, _grid1.GetCluster().TopologyVersion);
 
-            _grid3 = Ignition.Start(Configuration("config\\compute\\compute-grid3.xml"));
+            _grid3 = Ignition.Start(Configuration(GetConfigs().Item3));
 
             Assert.AreEqual(topVer + 2, _grid1.GetCluster().TopologyVersion);
         }
@@ -354,7 +378,7 @@ namespace Apache.Ignite.Core.Tests.Compute
             }
             finally 
             {
-                _grid2 = Ignition.Start(Configuration("config\\compute\\compute-grid2.xml"));
+                _grid2 = Ignition.Start(Configuration(GetConfigs().Item2));
             }
         }
 
@@ -386,7 +410,7 @@ namespace Apache.Ignite.Core.Tests.Compute
 
             Assert.IsTrue(nodes.Count == 2);
 
-            _grid2 = Ignition.Start(Configuration("config\\compute\\compute-grid2.xml"));
+            _grid2 = Ignition.Start(Configuration(GetConfigs().Item2));
 
             nodes = _grid1.GetCluster().GetNodes();
 
@@ -1135,6 +1159,9 @@ namespace Apache.Ignite.Core.Tests.Compute
             Assert.AreEqual(_grid1.GetCompute().ClusterGroup.GetNodes().Count, res);
         }
 
+        /// <summary>
+        /// Tests the exceptions.
+        /// </summary>
         [Test]
         public void TestExceptions()
         {
@@ -1146,6 +1173,18 @@ namespace Apache.Ignite.Core.Tests.Compute
         }
 
         /// <summary>
+        /// Tests the footer setting.
+        /// </summary>
+        [Test]
+        public void TestFooterSetting()
+        {
+            Assert.AreEqual(CompactFooter, ((Ignite)_grid1).Marshaller.CompactFooter);
+
+            foreach (var g in new[] {_grid1, _grid2, _grid3})
+                Assert.AreEqual(CompactFooter, g.GetConfiguration().BinaryConfiguration.CompactFooter);
+        }
+
+        /// <summary>
         /// Create configuration.
         /// </summary>
         /// <param name="path">XML config path.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTestFullFooter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTestFullFooter.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTestFullFooter.cs
new file mode 100644
index 0000000..1b1f98a
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTestFullFooter.cs
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Compute
+{
+    using System;
+    using System.IO;
+
+    /// <summary>
+    /// Compute API test with compact footers disabled.
+    /// </summary>
+    public class ComputeApiTestFullFooter : ComputeApiTest
+    {
+        /// <summary>
+        /// Gets the expected compact footers setting.
+        /// </summary>
+        protected override bool CompactFooter
+        {
+            get { return false; }
+        }
+
+        /// <summary>
+        /// Gets the configs.
+        /// </summary>
+        protected override Tuple<string, string, string> GetConfigs()
+        {
+            var baseConfigs = base.GetConfigs();
+
+            return Tuple.Create(
+                ReplaceFooterSetting(baseConfigs.Item1),
+                ReplaceFooterSetting(baseConfigs.Item2),
+                ReplaceFooterSetting(baseConfigs.Item3));
+        }
+
+        /// <summary>
+        /// Replaces the footer setting.
+        /// </summary>
+        public static string ReplaceFooterSetting(string path)
+        {
+            var text = File.ReadAllText(path).Replace(
+                "property name=\"compactFooter\" value=\"true\"",
+                "property name=\"compactFooter\" value=\"false\"");
+
+            path += "_fullFooter";
+
+            File.WriteAllText(path, text);
+
+            return path;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
index 78a30a8..566d6ac 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
@@ -19,8 +19,11 @@
 
 <beans xmlns="http://www.springframework.org/schema/beans"
        xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
        xsi:schemaLocation="http://www.springframework.org/schema/beans
-        http://www.springframework.org/schema/beans/spring-beans.xsd">
+                           http://www.springframework.org/schema/beans/spring-beans.xsd
+                           http://www.springframework.org/schema/util
+                           http://www.springframework.org/schema/util/spring-util.xsd">
     <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
         <property name="localHost" value="127.0.0.1"/>
         <property name="connectorConfiguration"><null/></property>
@@ -40,6 +43,19 @@
             <list>
                 <bean class="org.apache.ignite.configuration.CacheConfiguration">
                     <property name="startSize" value="10"/>
+                    <property name="queryEntities">
+                        <list>
+                            <bean class="org.apache.ignite.cache.QueryEntity">
+                                <property name="keyType" value="java.lang.Integer"/>
+                                <property name="valueType" value="org.apache.ignite.platform.PlatformComputeBinarizable"/>
+                                <property name="fields">
+                                    <util:map map-class="java.util.LinkedHashMap">
+                                        <entry key="Field" value="java.lang.Integer"/>
+                                    </util:map>
+                                </property>
+                            </bean>
+                        </list>
+                    </property>
                 </bean>
                 <bean class="org.apache.ignite.configuration.CacheConfiguration">
                     <property name="name" value="cache1"/>
@@ -54,7 +70,7 @@
 
         <property name="binaryConfiguration">
             <bean class="org.apache.ignite.configuration.BinaryConfiguration">
-                <property name="compactFooter" value="false"/>
+                <property name="compactFooter" value="true"/>
                 <property name="typeConfigurations">
                     <list>
                         <bean class="org.apache.ignite.binary.BinaryTypeConfiguration">

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
index b1e8235..711c3e3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
@@ -47,7 +47,7 @@
 
         <property name="binaryConfiguration">
             <bean class="org.apache.ignite.configuration.BinaryConfiguration">
-                <property name="compactFooter" value="false"/>
+                <property name="compactFooter" value="true"/>
                 <property name="typeConfigurations">
                     <list>
                         <bean class="org.apache.ignite.binary.BinaryTypeConfiguration">

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml
index d1c96b6..64d14bb 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml
@@ -34,7 +34,7 @@
 
         <property name="binaryConfiguration">
             <bean class="org.apache.ignite.configuration.BinaryConfiguration">
-                <property name="compactFooter" value="false"/>
+                <property name="compactFooter" value="true"/>
                 <property name="typeConfigurations">
                     <list>
                         <bean class="org.apache.ignite.binary.BinaryTypeConfiguration">

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/binary.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/binary.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/binary.xml
deleted file mode 100644
index f013749..0000000
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/binary.xml
+++ /dev/null
@@ -1,56 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-
-<!--
-  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.
--->
-
-<beans xmlns="http://www.springframework.org/schema/beans"
-       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xsi:schemaLocation="http://www.springframework.org/schema/beans
-        http://www.springframework.org/schema/beans/spring-beans.xsd">
-    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
-        <property name="localHost" value="127.0.0.1"/>
-        <property name="connectorConfiguration"><null/></property>
-
-        <property name="gridName" value="grid"/>
-
-        <property name="metricsUpdateFrequency" value="1000"/>
-        <property name="metricsLogFrequency" value="0"/>
-
-        <property name="cacheConfiguration">
-            <list>
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="cache"/>
-                </bean>
-            </list>
-        </property>
-      
-        <property name="discoverySpi">
-            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
-                <property name="ipFinder">
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
-                        <property name="addresses">
-                            <list>
-                                <!-- In distributed environment, replace with actual host IP address. -->
-                                <value>127.0.0.1:47500..47502</value>
-                            </list>
-                        </property>
-                    </bean>
-                </property>
-            </bean>
-        </property>
-    </bean>
-</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
index d8c52ee..d944a04 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -54,7 +54,7 @@ namespace Apache.Ignite.Core.Tests
         {
             var xml = @"<igniteConfig workDirectory='c:' JvmMaxMemoryMb='1024' MetricsLogFrequency='0:0:10'>
                             <localhost>127.1.1.1</localhost>
-                            <binaryConfiguration>
+                            <binaryConfiguration compactFooter='false'>
                                 <defaultNameMapper type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+NameMapper, Apache.Ignite.Core.Tests' bar='testBar' />
                                 <types>
                                     <string>Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+FooClass, Apache.Ignite.Core.Tests</string>
@@ -112,6 +112,7 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(
                 "Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+FooClass, Apache.Ignite.Core.Tests",
                 cfg.BinaryConfiguration.Types.Single());
+            Assert.IsFalse(cfg.BinaryConfiguration.CompactFooter);
             Assert.AreEqual(new[] {42, EventType.TaskFailed, EventType.JobFinished}, cfg.IncludedEventTypes);
 
             Assert.AreEqual("secondCache", cfg.CacheConfiguration.Last().Name);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
index 3aa26d8..c1f8fcd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -22,6 +22,7 @@ namespace Apache.Ignite.Core.Tests
     using System.ComponentModel;
     using System.IO;
     using System.Linq;
+    using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Discovery.Tcp;
@@ -60,6 +61,7 @@ namespace Apache.Ignite.Core.Tests
         public void TestDefaultValueAttributes()
         {
             CheckDefaultValueAttributes(new IgniteConfiguration());
+            CheckDefaultValueAttributes(new BinaryConfiguration());
             CheckDefaultValueAttributes(new TcpDiscoverySpi());
             CheckDefaultValueAttributes(new CacheConfiguration());
             CheckDefaultValueAttributes(new TcpDiscoveryMulticastIpFinder());

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
index ffcdea8..c563a61 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+#pragma warning disable 618   // SpringConfigUrl
 namespace Apache.Ignite.Core.Tests.Services
 {
     using System;
@@ -24,8 +25,10 @@ namespace Apache.Ignite.Core.Tests.Services
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Resource;
     using Apache.Ignite.Core.Services;
+    using Apache.Ignite.Core.Tests.Compute;
     using NUnit.Framework;
 
     /// <summary>
@@ -507,6 +510,19 @@ namespace Apache.Ignite.Core.Tests.Services
         }
 
         /// <summary>
+        /// Tests the footer setting.
+        /// </summary>
+        [Test]
+        public void TestFooterSetting()
+        {
+            foreach (var grid in Grids)
+            {
+                Assert.AreEqual(CompactFooter, ((Ignite)grid).Marshaller.CompactFooter);
+                Assert.AreEqual(CompactFooter, grid.GetConfiguration().BinaryConfiguration.CompactFooter);
+            }
+        }
+
+        /// <summary>
         /// Starts the grids.
         /// </summary>
         private void StartGrids()
@@ -558,8 +574,11 @@ namespace Apache.Ignite.Core.Tests.Services
         /// <summary>
         /// Gets the Ignite configuration.
         /// </summary>
-        private static IgniteConfiguration Configuration(string springConfigUrl)
+        private IgniteConfiguration Configuration(string springConfigUrl)
         {
+            if (!CompactFooter)
+                springConfigUrl = ComputeApiTestFullFooter.ReplaceFooterSetting(springConfigUrl);
+
             return new IgniteConfiguration
             {
                 SpringConfigUrl = springConfigUrl,
@@ -598,6 +617,11 @@ namespace Apache.Ignite.Core.Tests.Services
         }
 
         /// <summary>
+        /// Gets a value indicating whether compact footers should be used.
+        /// </summary>
+        protected virtual bool CompactFooter { get { return true; } }
+
+        /// <summary>
         /// Test service interface for proxying.
         /// </summary>
         private interface ITestIgniteService

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTestFullFooter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTestFullFooter.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTestFullFooter.cs
new file mode 100644
index 0000000..b4b4f11
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTestFullFooter.cs
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Services
+{
+    /// <summary>
+    /// Services test with compact footers disabled.
+    /// </summary>
+    public class ServicesTestFullFooter : ServicesTest
+    {
+        /// <summary>
+        /// Gets a value indicating whether compact footers should be used.
+        /// </summary>
+        protected override bool CompactFooter
+        {
+            get { return false; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index f1511d9..dedf084 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -193,6 +193,7 @@
     <Compile Include="Ignition.cs" />
     <Compile Include="IIgnite.cs" />
     <Compile Include="Impl\Binary\BinaryEnum.cs" />
+    <Compile Include="Impl\Binary\BinaryObjectSchemaSerializer.cs" />
     <Compile Include="Impl\Binary\JavaTypes.cs" />
     <Compile Include="Impl\Cache\CacheAffinityImpl.cs" />
     <Compile Include="Impl\Cache\CacheEntry.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs
index fa2fb1c..3a9d86c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Binary
 {
     using System;
     using System.Collections.Generic;
+    using System.ComponentModel;
     using System.Diagnostics.CodeAnalysis;
     using System.Linq;
     using Apache.Ignite.Core.Impl.Common;
@@ -29,11 +30,24 @@ namespace Apache.Ignite.Core.Binary
     public class BinaryConfiguration
     {
         /// <summary>
+        /// Default <see cref="CompactFooter"/> setting.
+        /// </summary>
+        public const bool DefaultCompactFooter = true;
+
+        /// <summary>
+        /// Default <see cref="DefaultKeepDeserialized"/> setting.
+        /// </summary>
+        public const bool DefaultDefaultKeepDeserialized = true;
+
+        /** Footer setting. */
+        private bool? _compactFooter;
+
+        /// <summary>
         /// Initializes a new instance of the <see cref="BinaryConfiguration"/> class.
         /// </summary>
         public BinaryConfiguration()
         {
-            DefaultKeepDeserialized = true;
+            DefaultKeepDeserialized = DefaultDefaultKeepDeserialized;
         }
 
         /// <summary>
@@ -54,6 +68,8 @@ namespace Apache.Ignite.Core.Binary
                 : cfg.TypeConfigurations.Select(x => new BinaryTypeConfiguration(x)).ToList();
 
             Types = cfg.Types == null ? null : cfg.Types.ToList();
+
+            CompactFooter = cfg.CompactFooter;
         }
 
         /// <summary>
@@ -95,6 +111,32 @@ namespace Apache.Ignite.Core.Binary
         /// <summary>
         /// Default keep deserialized flag.
         /// </summary>
+        [DefaultValue(DefaultDefaultKeepDeserialized)]
         public bool DefaultKeepDeserialized { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether to write footers in compact form.
+        /// When enabled, Ignite will not write fields metadata when serializing objects, 
+        /// because internally metadata is distributed inside cluster.
+        /// This increases serialization performance.
+        /// <para/>
+        /// <b>WARNING!</b> This mode should be disabled when already serialized data can be taken from some external
+        /// sources (e.g.cache store which stores data in binary form, data center replication, etc.). 
+        /// Otherwise binary objects without any associated metadata could could not be deserialized.
+        /// </summary>
+        [DefaultValue(DefaultCompactFooter)]
+        public bool CompactFooter
+        {
+            get { return _compactFooter ?? DefaultCompactFooter; }
+            set { _compactFooter = value; }
+        }
+
+        /// <summary>
+        /// Gets the compact footer internal nullable value.
+        /// </summary>
+        internal bool? CompactFooterInternal
+        {
+            get { return _compactFooter; }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
index 1dd22ea..e8cc8ff 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
@@ -207,6 +207,14 @@
             }
             else
                 writer.WriteBoolean(false);
+
+            // Binary config
+            var isCompactFooterSet = BinaryConfiguration != null && BinaryConfiguration.CompactFooterInternal != null;
+
+            writer.WriteBoolean(isCompactFooterSet);
+
+            if (isCompactFooterSet)
+                writer.WriteBoolean(BinaryConfiguration.CompactFooter);
         }
 
         /// <summary>
@@ -237,6 +245,13 @@
 
             // Discovery config
             DiscoverySpi = r.ReadBoolean() ? new TcpDiscoverySpi(r) : null;
+
+            // Binary config
+            if (r.ReadBoolean())
+            {
+                BinaryConfiguration = BinaryConfiguration ?? new BinaryConfiguration();
+                BinaryConfiguration.CompactFooter = r.ReadBoolean();
+            }
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
index 5181217..12a4660 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -78,6 +78,7 @@
                             </xs:element>
                         </xs:all>
                         <xs:attribute name="defaultKeepDeserialized" type="xs:boolean" />
+                        <xs:attribute name="compactFooter" type="xs:boolean" />
                     </xs:complexType>
                 </xs:element>
                 <xs:element name="cacheConfiguration" minOccurs="0">

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
index efe1df4..1deac07 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
@@ -190,8 +190,12 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             const int len = BinaryObjectHeader.Size;
 
-            var hdr = new BinaryObjectHeader(desc.TypeId, 0, len, 0, len,
-                desc.UserType ? BinaryObjectHeader.Flag.UserType : BinaryObjectHeader.Flag.None);
+            var flags = desc.UserType ? BinaryObjectHeader.Flag.UserType : BinaryObjectHeader.Flag.None;
+
+            if (_marsh.CompactFooter && desc.UserType)
+                flags |= BinaryObjectHeader.Flag.CompactFooter;
+
+            var hdr = new BinaryObjectHeader(desc.TypeId, 0, len, 0, len, flags);
 
             using (var stream = new BinaryHeapStream(len))
             {

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
index 513333b..13d3133 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
@@ -197,7 +197,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             var desc = _marsh.GetDescriptor(true, _header.TypeId);
 
-            InitializeFields();
+            InitializeFields(desc);
 
             int fieldId = BinaryUtils.FieldId(_header.TypeId, fieldName, desc.NameMapper, desc.IdMapper);
 
@@ -207,16 +207,19 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Lazy fields initialization routine.
         /// </summary>
-        private void InitializeFields()
+        private void InitializeFields(IBinaryTypeDescriptor desc = null)
         {
             if (_fields != null) 
                 return;
 
+            desc = desc ?? _marsh.GetDescriptor(true, _header.TypeId);
+
             using (var stream = new BinaryHeapStream(_data))
             {
                 var hdr = BinaryObjectHeader.Read(stream, _offset);
 
-                _fields = hdr.ReadSchemaAsDictionary(stream, _offset) ?? EmptyFields;
+                _fields = BinaryObjectSchemaSerializer.ReadSchema(stream, _offset, hdr, desc.Schema,_marsh)
+                    .ToDictionary() ?? EmptyFields;
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
index 0f1c0bd..083f557 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
@@ -632,7 +632,8 @@ namespace Apache.Ignite.Core.Impl.Binary
                     else
                     {
                         // New object, write in full form.
-                        var inSchema = inHeader.ReadSchema(inStream, inStartPos);
+                        var inSchema = BinaryObjectSchemaSerializer.ReadSchema(inStream, inStartPos, inHeader, 
+                            _desc.Schema, _binary.Marshaller);
 
                         var outSchema = BinaryObjectSchemaHolder.Current;
                         var schemaIdx = outSchema.PushSchema();
@@ -709,6 +710,9 @@ namespace Apache.Ignite.Core.Impl.Binary
                             var schemaPos = outStream.Position;
                             int outSchemaId;
 
+                            if (inHeader.IsCompactFooter)
+                                flags |= BinaryObjectHeader.Flag.CompactFooter;
+
                             var hasSchema = outSchema.WriteSchema(outStream, schemaIdx, out outSchemaId, ref flags);
 
                             if (hasSchema)
@@ -719,6 +723,9 @@ namespace Apache.Ignite.Core.Impl.Binary
 
                                 if (inHeader.HasRaw)
                                     outStream.WriteInt(outRawOff);
+
+                                if (_desc.Schema.Get(outSchemaId) == null)
+                                    _desc.Schema.Add(outSchemaId, outSchema.GetSchema(schemaIdx));
                             }
 
                             var outLen = outStream.Position - outStartPos;

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
index 0cabd7d..2624d52 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
@@ -18,7 +18,6 @@
 namespace Apache.Ignite.Core.Impl.Binary
 {
     using System;
-    using System.Collections.Generic;
     using System.Diagnostics;
     using System.IO;
     using System.Runtime.InteropServices;
@@ -183,7 +182,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// </summary>
         public int SchemaFieldSize
         {
-            get { return SchemaFieldOffsetSize + 4; }
+            get { return IsCompactFooter ? SchemaFieldOffsetSize : SchemaFieldOffsetSize + 4; }
         }
 
         /// <summary>
@@ -198,6 +197,9 @@ namespace Apache.Ignite.Core.Impl.Binary
 
                 var schemaSize = Length - SchemaOffset;
 
+                if (HasRaw)
+                    schemaSize -= 4;
+
                 return schemaSize / SchemaFieldSize;
             }
         }
@@ -221,164 +223,6 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
-        /// Reads the schema as dictionary according to this header data.
-        /// </summary>
-        /// <param name="stream">The stream.</param>
-        /// <param name="position">The position.</param>
-        /// <returns>Schema.</returns>
-        public Dictionary<int, int> ReadSchemaAsDictionary(IBinaryStream stream, int position)
-        {
-            Debug.Assert(stream != null);
-
-            ThrowIfUnsupported();
-
-            var schemaSize = SchemaFieldCount;
-
-            if (schemaSize == 0)
-                return null;
-
-            stream.Seek(position + SchemaOffset, SeekOrigin.Begin);
-
-            var schema = new Dictionary<int, int>(schemaSize);
-
-            var offsetSize = SchemaFieldOffsetSize;
-
-            if (offsetSize == 1)
-            {
-                for (var i = 0; i < schemaSize; i++)
-                    schema.Add(stream.ReadInt(), stream.ReadByte());
-            }
-            else if (offsetSize == 2)
-            {
-                for (var i = 0; i < schemaSize; i++)
-                    schema.Add(stream.ReadInt(), stream.ReadShort());
-            }
-            else
-            {
-                for (var i = 0; i < schemaSize; i++)
-                    schema.Add(stream.ReadInt(), stream.ReadInt());
-            }
-
-            return schema;
-        }
-
-        /// <summary>
-        /// Reads the schema according to this header data.
-        /// </summary>
-        /// <param name="stream">The stream.</param>
-        /// <param name="position">The position.</param>
-        /// <returns>Schema.</returns>
-        public BinaryObjectSchemaField[] ReadSchema(IBinaryStream stream, int position)
-        {
-            Debug.Assert(stream != null);
-
-            ThrowIfUnsupported();
-
-            var schemaSize = SchemaFieldCount;
-
-            if (schemaSize == 0)
-                return null;
-
-            stream.Seek(position + SchemaOffset, SeekOrigin.Begin);
-
-            var schema = new BinaryObjectSchemaField[schemaSize];
-
-            var offsetSize = SchemaFieldOffsetSize;
-
-            if (offsetSize == 1)
-            {
-                for (var i = 0; i < schemaSize; i++)
-                    schema[i] = new BinaryObjectSchemaField(stream.ReadInt(), stream.ReadByte());
-            }
-            else if (offsetSize == 2)
-            {
-                for (var i = 0; i < schemaSize; i++)
-                    schema[i] = new BinaryObjectSchemaField(stream.ReadInt(), stream.ReadShort());
-            }
-            else
-            {
-                for (var i = 0; i < schemaSize; i++)
-                    schema[i] = new BinaryObjectSchemaField(stream.ReadInt(), stream.ReadInt());
-            }
-
-            return schema;
-        }
-
-        /// <summary>
-        /// Writes an array of fields to a stream.
-        /// </summary>
-        /// <param name="fields">Fields.</param>
-        /// <param name="stream">Stream.</param>
-        /// <param name="offset">Offset in the array.</param>
-        /// <param name="count">Field count to write.</param>
-        /// <returns>
-        /// Flags according to offset sizes: <see cref="Flag.OffsetOneByte" />,
-        /// <see cref="Flag.OffsetTwoBytes" />, or 0.
-        /// </returns>
-        public static unsafe Flag WriteSchema(BinaryObjectSchemaField[] fields, IBinaryStream stream, int offset,
-            int count)
-        {
-            Debug.Assert(fields != null);
-            Debug.Assert(stream != null);
-            Debug.Assert(count > 0);
-            Debug.Assert(offset >= 0);
-            Debug.Assert(offset < fields.Length);
-
-            unchecked
-            {
-                // Last field is the farthest in the stream
-                var maxFieldOffset = fields[offset + count - 1].Offset;
-
-                if (maxFieldOffset <= byte.MaxValue)
-                {
-                    for (int i = offset; i < count + offset; i++)
-                    {
-                        var field = fields[i];
-
-                        stream.WriteInt(field.Id);
-                        stream.WriteByte((byte)field.Offset);
-                    }
-
-                    return Flag.OffsetOneByte;
-                }
-
-                if (maxFieldOffset <= ushort.MaxValue)
-                {
-                    for (int i = offset; i < count + offset; i++)
-                    {
-                        var field = fields[i];
-
-                        stream.WriteInt(field.Id);
-
-                        stream.WriteShort((short)field.Offset);
-                    }
-
-                    return Flag.OffsetTwoBytes;
-                }
-
-                if (BitConverter.IsLittleEndian)
-                {
-                    fixed (BinaryObjectSchemaField* ptr = &fields[offset])
-                    {
-                        stream.Write((byte*)ptr, count / BinaryObjectSchemaField.Size);
-                    }
-                }
-                else
-                {
-                    for (int i = offset; i < count + offset; i++)
-                    {
-                        var field = fields[i];
-
-                        stream.WriteInt(field.Id);
-                        stream.WriteInt(field.Offset);
-                    }
-                }
-
-                return Flag.None;
-            }
-        }
-
-        /// <summary>
         /// Writes specified header to a stream.
         /// </summary>
         /// <param name="header">The header.</param>
@@ -424,8 +268,6 @@ namespace Apache.Ignite.Core.Impl.Binary
             else
                 hdr = new BinaryObjectHeader(stream);
 
-            hdr.ThrowIfUnsupported();
-
             // Only one of the flags can be set
             var f = hdr.Flags;
             Debug.Assert((f & (Flag.OffsetOneByte | Flag.OffsetTwoBytes)) !=
@@ -482,15 +324,5 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             return !left.Equals(right);
         }
-
-        /// <summary>
-        /// Throws an exception if current header represents unsupported mode.
-        /// </summary>
-        private void ThrowIfUnsupported()
-        {
-            // Compact schema is not supported
-            if (IsCompactFooter)
-                throw new NotSupportedException("Compact binary object footer is not supported in Ignite.NET.");
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchema.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchema.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchema.cs
index a3467b8..9b12fa7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchema.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchema.cs
@@ -94,5 +94,25 @@ namespace Apache.Ignite.Core.Impl.Binary
                 }
             }
         }
+
+        /// <summary>
+        /// Gets all schemas.
+        /// </summary>
+        public IEnumerable<KeyValuePair<int, int[]>> GetAll()
+        {
+            if (_schema1 == null)
+                yield break;
+
+            yield return new KeyValuePair<int, int[]>(_schemaId1, _schema1);
+
+            if (_schema2 == null)
+                yield break;
+
+            yield return new KeyValuePair<int, int[]>(_schemaId2, _schema2);
+
+            if (_schemas != null)
+                foreach (var pair in _schemas)
+                    yield return pair;
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
index 65b6fc0..c95746a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
@@ -96,12 +96,28 @@ namespace Apache.Ignite.Core.Impl.Binary
             if (count == 0) 
                 return false;
 
-            flags |= BinaryObjectHeader.WriteSchema(_fields, stream, schemaOffset, count);
+            flags |= BinaryObjectSchemaSerializer.WriteSchema(_fields, stream, schemaOffset, count, 
+                (flags & BinaryObjectHeader.Flag.CompactFooter) == BinaryObjectHeader.Flag.CompactFooter);
 
             for (var i = schemaOffset; i < _idx; i++)
                 schemaId = Fnv1Hash.Update(schemaId, _fields[i].Id);
 
             return true;
         }
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        /// <param name="schemaOffset">The schema offset.</param>
+        /// <returns>Current schema as an array of field ids.</returns>
+        public int[] GetSchema(int schemaOffset)
+        {
+            int[] result = new int[_idx - schemaOffset];
+
+            for (int i = 0; i < result.Length; i++)
+                result[i] = _fields[i + schemaOffset].Id;
+
+            return result;
+        }
     }
 }


[22/35] ignite git commit: IGNITE-2759: Now objects will be deserialized during conflicts only when "keepBinary" is set. This closes #542.

Posted by nt...@apache.org.
IGNITE-2759: Now objects will be deserialized during conflicts only when "keepBinary" is set. This closes #542.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/ed030e53
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/ed030e53
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/ed030e53

Branch: refs/heads/ignite-2791
Commit: ed030e53b532bbb031d2c49ce71cd521e94dd848
Parents: 687200d
Author: dkarachentsev <dk...@gridgain.com>
Authored: Fri Mar 11 17:36:20 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Mar 11 17:36:20 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/GridCacheEntryEx.java   |  3 ++-
 .../internal/processors/cache/GridCacheMapEntry.java  | 14 +++++++-------
 .../cache/transactions/IgniteTxAdapter.java           |  6 ++++--
 .../GridCacheStoreManagerDeserializationTest.java     |  1 +
 .../processors/cache/GridCacheTestEntryEx.java        |  2 +-
 5 files changed, 15 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ed030e53/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index 71bb034..3a7b5ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -692,11 +692,12 @@ public interface GridCacheEntryEx {
     /**
      * Create versioned entry for this cache entry.
      *
+     * @param keepBinary Keep binary flag.
      * @return Versioned entry.
      * @throws IgniteCheckedException In case of error.
      * @throws GridCacheEntryRemovedException If entry was removed.
      */
-    public <K, V> GridCacheVersionedEntryEx<K, V> versionedEntry()
+    public <K, V> GridCacheVersionedEntryEx<K, V> versionedEntry(final boolean keepBinary)
         throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ed030e53/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 16ff153..fb6aeef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -1924,7 +1924,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                             }
                             else {
                                 writeObj = oldVal;
-                                writeObj0 = CU.value(oldVal, cctx, false);
+                                writeObj0 = cctx.unwrapBinaryIfNeeded(oldVal, keepBinary, false);
                             }
 
                             key0 = entry.key();
@@ -1936,18 +1936,18 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                             invokeRes = new IgniteBiTuple(null, e);
 
                             writeObj = oldVal;
-                            writeObj0 = CU.value(oldVal, cctx, false);
+                            writeObj0 = cctx.unwrapBinaryIfNeeded(oldVal, keepBinary, false);
                         }
                     }
                     else
-                        writeObj0 = CU.value((CacheObject)writeObj, cctx, false);
+                        writeObj0 = cctx.unwrapBinaryIfNeeded(writeObj, keepBinary, false);
 
                     GridTuple3<Long, Long, Boolean> expiration = ttlAndExpireTime(expiryPlc,
                         explicitTtl,
                         explicitExpireTime);
 
                     // Prepare old and new entries for conflict resolution.
-                    GridCacheVersionedEntryEx oldEntry = versionedEntry();
+                    GridCacheVersionedEntryEx oldEntry = versionedEntry(keepBinary);
                     GridCacheVersionedEntryEx newEntry = new GridCachePlainVersionedEntry<>(
                         oldEntry.key(),
                         writeObj0,
@@ -3353,14 +3353,14 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     }
 
     /** {@inheritDoc} */
-    @Override public synchronized GridCacheVersionedEntryEx versionedEntry()
+    @Override public synchronized GridCacheVersionedEntryEx versionedEntry(final boolean keepBinary)
         throws IgniteCheckedException, GridCacheEntryRemovedException {
         boolean isNew = isStartVersion();
 
         CacheObject val = isNew ? unswap(true) : rawGetOrUnmarshalUnlocked(false);
 
-        return new GridCachePlainVersionedEntry<>(keyValue(true),
-            CU.value(val, cctx, true),
+        return new GridCachePlainVersionedEntry<>(cctx.unwrapBinaryIfNeeded(key, keepBinary, true),
+            cctx.unwrapBinaryIfNeeded(val, keepBinary, true),
             ttlExtras(),
             expireTimeExtras(),
             ver.conflictVersion(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/ed030e53/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index ed44c49..f6dfd32 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -1413,10 +1413,12 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement
         assert newExpireTime != CU.EXPIRE_TIME_CALCULATE;
 
         // Construct old entry info.
-        GridCacheVersionedEntryEx oldEntry = old.versionedEntry();
+        GridCacheVersionedEntryEx oldEntry = old.versionedEntry(txEntry.keepBinary());
 
         // Construct new entry info.
-        Object newVal0 = CU.value(newVal, txEntry.context(), false);
+        GridCacheContext entryCtx = txEntry.context();
+
+        Object newVal0 = entryCtx.unwrapBinaryIfNeeded(newVal, txEntry.keepBinary(), false);
 
         GridCacheVersionedEntryEx newEntry = new GridCachePlainVersionedEntry(
             oldEntry.key(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/ed030e53/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java
index 9fe60e0..d12f56a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java
@@ -188,6 +188,7 @@ public class GridCacheStoreManagerDeserializationTest extends GridCommonAbstract
             streamer.addData(key, key);
         }
 
+        streamer.flush();
         streamer.close();
 
         streamer.future().get();

http://git-wip-us.apache.org/repos/asf/ignite/blob/ed030e53/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index e627083..82b3f4b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -677,7 +677,7 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
     }
 
     /** @inheritDoc */
-    @Override public GridCacheVersionedEntryEx versionedEntry() throws IgniteCheckedException {
+    @Override public GridCacheVersionedEntryEx versionedEntry(final boolean keepBinary) throws IgniteCheckedException {
         return null;
     }
 


[18/35] ignite git commit: Added tests.

Posted by nt...@apache.org.
Added tests.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/67db2645
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/67db2645
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/67db2645

Branch: refs/heads/ignite-2791
Commit: 67db26452ba7fb79d939fddbae598a35513558ed
Parents: b63cee4
Author: sboikov <sb...@gridgain.com>
Authored: Fri Mar 11 09:43:08 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Mar 11 09:43:08 2016 +0300

----------------------------------------------------------------------
 .../IgniteDynamicClientCacheStartSelfTest.java  | 78 ++++++++++++++++++++
 1 file changed, 78 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/67db2645/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
index 0793dbe..cc7120e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.List;
 import java.util.concurrent.Callable;
 import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
@@ -29,6 +30,8 @@ import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridNoStorageCacheMap;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -36,6 +39,7 @@ import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_GRID_NAME;
 
 /**
  * Tests that cache specified in configuration start on client nodes.
@@ -226,6 +230,58 @@ public class IgniteDynamicClientCacheStartSelfTest extends GridCommonAbstractTes
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testCreateCloseClientCache1() throws Exception {
+        Ignite ignite0 = startGrid(0);
+
+        client = true;
+
+        Ignite clientNode = startGrid(1);
+
+        client = false;
+
+        ignite0.createCache(new CacheConfiguration<>());
+
+        clientNode.cache(null);
+
+        clientNode.cache(null).close();
+
+        clientNode.cache(null);
+
+        startGrid(2);
+
+        checkCache(clientNode, null, false, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCreateCloseClientCache2() throws Exception {
+        Ignite ignite0 = startGrid(0);
+
+        Ignite ignite1 = startGrid(1);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setNodeFilter(new CachePredicate(F.asList(ignite0.name())));
+
+        ignite1.createCache(ccfg);
+
+        assertNull(((IgniteKernal)ignite0).context().cache().internalCache(null));
+
+        ignite0.cache(null);
+
+        ignite0.cache(null).close();
+
+        ignite0.cache(null);
+
+        startGrid(2);
+
+        checkCache(ignite0, null, false, false);
+    }
+
+    /**
      * @param ignite Node.
      * @param cacheName Cache name
      * @param srv {@code True} if server cache is expected.
@@ -283,4 +339,26 @@ public class IgniteDynamicClientCacheStartSelfTest extends GridCommonAbstractTes
             assertFalse(disco.cacheNearNode(node, cacheName));
         }
     }
+
+    /**
+     *
+     */
+    static class CachePredicate implements IgnitePredicate<ClusterNode> {
+        /** */
+        private List<String> excludeNodes;
+
+        /**
+         * @param excludeNodes Nodes names.
+         */
+        public CachePredicate(List<String> excludeNodes) {
+            this.excludeNodes = excludeNodes;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(ClusterNode clusterNode) {
+            String name = clusterNode.attribute(ATTR_GRID_NAME).toString();
+
+            return !excludeNodes.contains(name);
+        }
+    }
 }
\ No newline at end of file


[11/35] ignite git commit: IGNITE-2753: Removed unnecessary deserialization of keys and values passed to store manager. This closes #537.

Posted by nt...@apache.org.
IGNITE-2753: Removed unnecessary deserialization of keys and values passed to store manager. This closes #537.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/b46ed10a
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/b46ed10a
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/b46ed10a

Branch: refs/heads/ignite-2791
Commit: b46ed10acfc46117dd597c1e7e986e4216bfd047
Parents: a8d16bd
Author: dkarachentsev <dk...@gridgain.com>
Authored: Thu Mar 10 14:05:40 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Mar 10 14:05:40 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     |  25 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   2 +-
 ...ridCacheStoreManagerDeserializationTest.java | 345 +++++++++++++++++++
 ...calCacheStoreManagerDeserializationTest.java | 101 ++++++
 .../testsuites/IgniteCacheTestSuite4.java       |  11 +-
 5 files changed, 468 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b46ed10a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 1658b85..16ff153 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -208,7 +208,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
         // In case we deal with IGFS cache, count updated data
         if (cctx.cache().isIgfsDataCache() &&
-            cctx.kernalContext().igfsHelper().isIgfsBlockKey(key.value(cctx.cacheObjectContext(), false))) {
+            cctx.kernalContext().igfsHelper().isIgfsBlockKey(keyValue(false))) {
             int newSize = valueLength0(val, null);
             int oldSize = valueLength0(this.val, (this.val == null && hasOffHeapPointer()) ? valueBytes0() : null);
 
@@ -591,7 +591,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 }
 
                 keyClsLdrId = cctx.deploy().getClassLoaderId(
-                    U.detectObjectClassLoader(key.value(cctx.cacheObjectContext(), false)));
+                    U.detectObjectClassLoader(keyValue(false)));
             }
 
             IgniteBiTuple<byte[], Byte> valBytes = valueBytes0();
@@ -1250,7 +1250,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         // Persist outside of synchronization. The correctness of the
         // value will be handled by current transaction.
         if (writeThrough)
-            cctx.store().put(tx, keyValue(false), CU.value(val, cctx, false), newVer);
+            cctx.store().put(tx, key, val, newVer);
 
         if (intercept)
             cctx.config().getInterceptor().onAfterPut(new CacheLazyEntry(cctx, key, key0, val, val0, keepBinary));
@@ -1718,7 +1718,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 if (writeThrough)
                     // Must persist inside synchronization in non-tx mode.
-                    cctx.store().put(null, keyValue(false), CU.value(updated, cctx, false), ver);
+                    cctx.store().put(null, key, updated, ver);
+
 
                 // Update index inside synchronization since it can be updated
                 // in load methods without actually holding entry lock.
@@ -1752,7 +1753,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             else {
                 if (writeThrough)
                     // Must persist inside synchronization in non-tx mode.
-                    cctx.store().remove(null, keyValue(false));
+                    cctx.store().remove(null, key);
 
                 boolean hasValPtr = hasOffHeapPointer();
 
@@ -1978,10 +1979,10 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                             if (val == null) {
                                 assert deletedUnlocked();
 
-                                cctx.store().remove(null, keyValue(false));
+                                cctx.store().remove(null, key);
                             }
                             else
-                                cctx.store().put(null, keyValue(false), CU.value(val, cctx, false), ver);
+                                cctx.store().put(null, key, val, ver);
                         }
 
                         return new GridCacheUpdateAtomicResult(false,
@@ -2031,10 +2032,10 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                             if (val == null) {
                                 assert deletedUnlocked();
 
-                                cctx.store().remove(null, keyValue(false));
+                                cctx.store().remove(null, key);
                             }
                             else
-                                cctx.store().put(null, keyValue(false), CU.value(val, cctx, false), ver);
+                                cctx.store().put(null, key, val, ver);
                         }
                         else {
                             if (log.isDebugEnabled())
@@ -3295,7 +3296,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 if (cctx.store().isLocal()) {
                     if (val != null)
-                        cctx.store().put(null, keyValue(false), CU.value(val, cctx, false), ver);
+                        cctx.store().put(null, key, val, ver);
                 }
 
                 return true;
@@ -3358,7 +3359,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
         CacheObject val = isNew ? unswap(true) : rawGetOrUnmarshalUnlocked(false);
 
-        return new GridCachePlainVersionedEntry<>(key.value(cctx.cacheObjectContext(), true),
+        return new GridCachePlainVersionedEntry<>(keyValue(true),
             CU.value(val, cctx, true),
             ttlExtras(),
             expireTimeExtras(),
@@ -4106,7 +4107,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                         }
 
                         keyClsLdrId = cctx.deploy().getClassLoaderId(
-                            U.detectObjectClassLoader(key.value(cctx.cacheObjectContext(), false)));
+                            U.detectObjectClassLoader(keyValue(false)));
                     }
 
                     IgniteBiTuple<byte[], Byte> valBytes = valueBytes0();

http://git-wip-us.apache.org/repos/asf/ignite/blob/b46ed10a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
index fae8219..ab51bdb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
@@ -590,7 +590,7 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
                 }
 
                 if (cctx.store().isLocal())
-                    cctx.store().remove(null, keyValue(false));
+                    cctx.store().remove(null, key);
 
                 rmv = true;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b46ed10a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java
new file mode 100644
index 0000000..9fe60e0
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.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;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.processors.cache.extras.GridCacheObsoleteEntryExtras;
+import org.apache.ignite.internal.processors.cache.store.CacheLocalStore;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jsr166.ConcurrentHashMap8;
+
+import javax.cache.Cache;
+import javax.cache.integration.CacheLoaderException;
+import javax.cache.integration.CacheWriterException;
+import java.io.Serializable;
+import java.util.Map;
+
+import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
+
+/**
+ * Checks whether storing to local store doesn't cause binary objects unmarshalling,
+ * and as a consequence {@link ClassNotFoundException} to be thrown.
+ *
+ * @see <a href="https://issues.apache.org/jira/browse/IGNITE-2753">
+ *     https://issues.apache.org/jira/browse/IGNITE-2753
+ *     </a>
+ */
+public class GridCacheStoreManagerDeserializationTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    protected static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Cache store. */
+    protected static final GridCacheLocalTestStore store = new GridCacheLocalTestStore();
+
+    /** Test cache name. */
+    protected static final String CACHE_NAME = "cache_name";
+
+    /** Cache mode. */
+    protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+
+    /** Cache write order mode. */
+    protected CacheAtomicWriteOrderMode cacheAtomicWriteOrderMode() {
+        return CacheAtomicWriteOrderMode.PRIMARY;
+    }
+
+    /** Cache synchronization mode. */
+    private CacheWriteSynchronizationMode cacheWriteSynchronizationMode() {
+        return CacheWriteSynchronizationMode.FULL_SYNC;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        if (gridName != null && gridName.toLowerCase().startsWith("binary"))
+            c.setMarshaller(new BinaryMarshaller());
+        else
+            c.setMarshaller(new OptimizedMarshaller());
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        c.setDiscoverySpi(disco);
+
+        c.setCacheConfiguration(cacheConfiguration());
+
+        return c;
+    }
+
+    /**
+     * @return Cache configuration.
+     */
+    @SuppressWarnings("unchecked")
+    protected CacheConfiguration cacheConfiguration() {
+        CacheConfiguration cc = defaultCacheConfiguration();
+
+        cc.setSwapEnabled(false);
+        cc.setRebalanceMode(SYNC);
+
+        cc.setCacheStoreFactory(singletonFactory(store));
+        cc.setReadThrough(true);
+        cc.setWriteThrough(true);
+        cc.setLoadPreviousValue(true);
+        cc.setStoreKeepBinary(true);
+
+        cc.setCacheMode(cacheMode());
+        cc.setAtomicWriteOrderMode(cacheAtomicWriteOrderMode());
+        cc.setWriteSynchronizationMode(cacheWriteSynchronizationMode());
+
+        cc.setBackups(0);
+
+        cc.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+
+        return cc;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        store.map.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * Check whether test objects are stored correctly via stream API.
+     *
+     * @throws Exception
+     */
+    public void testStream() throws Exception {
+        final Ignite grid = startGrid();
+
+        final IgniteCache<TestObj, TestObj> cache = grid.createCache(CACHE_NAME);
+
+        final TestObj testObj = streamData(grid);
+
+        cache.destroy();
+        cache.close();
+
+        assert store.map.containsKey(testObj);
+
+        final IgniteCache<TestObj, TestObj> cache2 = grid.createCache(CACHE_NAME);
+
+        assert testObj.equals(cache2.get(testObj));
+        assert store.map.containsKey(testObj);
+    }
+
+    /**
+     * Simulate case where is called
+     * {@link org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry#clearInternal(
+     * GridCacheVersion, boolean, GridCacheObsoleteEntryExtras)}
+     *
+     * @throws Exception
+     */
+    public void testPartitionMove() throws Exception {
+        final Ignite grid = startGrid("binaryGrid1");
+
+        grid.createCache(CACHE_NAME);
+
+        final BinaryObjectBuilder builder = grid.binary().builder("custom_type");
+
+        final IgniteDataStreamer<BinaryObject, BinaryObject> streamer = grid.dataStreamer(CACHE_NAME);
+
+        streamer.keepBinary(true);
+
+        final int itemsNum = 10_000;
+
+        for (int i = 0; i < itemsNum; i++) {
+            final BinaryObject key = builder.setField("id", i).build();
+
+            streamer.addData(key, key);
+        }
+
+        streamer.close();
+
+        streamer.future().get();
+
+        assert store.map.size() == itemsNum;
+
+        startGrid("binaryGrid2");
+        startGrid("binaryGrid3");
+        startGrid("binaryGrid4");
+
+        Thread.sleep(10_000);
+    }
+
+    /**
+     * Check whether binary objects are stored without unmarshalling via stream API.
+     *
+     * @throws Exception
+     */
+    public void testBinaryStream() throws Exception {
+        final Ignite grid = startGrid("binaryGrid");
+
+        final IgniteCache<BinaryObject, BinaryObject> cache = grid.createCache(CACHE_NAME).withKeepBinary();
+
+        final BinaryObject key = streamBinaryData(grid);
+
+        assert cache.containsKey(key);
+        assert store.map.containsKey(key);
+
+        cache.destroy();
+        cache.close();
+
+        assert store.map.containsKey(key);
+
+        final IgniteCache<BinaryObject, BinaryObject> cache2 = grid.createCache(CACHE_NAME).withKeepBinary();
+
+        final BinaryObject loaded = cache2.get(key);
+
+        assert loaded == key;
+        assert store.map.containsKey(key);
+    }
+
+    /**
+     * Create and add test data via Streamer API.
+     *
+     * @param grid to get streamer.
+     * @return test object (it is key and val).
+     */
+    private TestObj streamData(final Ignite grid) {
+        final IgniteDataStreamer<TestObj, TestObj> streamer = grid.dataStreamer(CACHE_NAME);
+
+        TestObj entity = null;
+
+        for (int i = 0; i < 1; i++) {
+            entity = new TestObj(i);
+
+            streamer.addData(entity, entity);
+        }
+
+        streamer.flush();
+        streamer.close();
+        streamer.future().get();
+
+        return entity;
+    }
+
+    /**
+     * Create and add binary data via Streamer API.
+     *
+     * @param grid to get streamer.
+     * @return test object (it is key and val).
+     */
+    private BinaryObject streamBinaryData(final Ignite grid) {
+        final IgniteDataStreamer<BinaryObject, BinaryObject> streamer = grid.dataStreamer(CACHE_NAME);
+
+        streamer.keepBinary(true);
+
+        final BinaryObjectBuilder builder = grid.binary().builder("custom_type");
+
+        BinaryObject entity = null;
+
+        for (int i = 0; i < 1; i++) {
+            builder.setField("id", i);
+
+            entity = builder.build();
+
+            streamer.addData(entity, entity);
+        }
+
+        streamer.flush();
+        streamer.close();
+        streamer.future().get();
+
+        return entity;
+    }
+
+    /**
+     * Local store mock.
+     *
+     * @param <K>
+     * @param <V>
+     */
+    @CacheLocalStore
+    protected static class GridCacheLocalTestStore<K, V> extends CacheStoreAdapter<K, V> {
+        /** */
+        public final Map<K, V> map = new ConcurrentHashMap8<>();
+
+        /** {@inheritDoc} */
+        @Override public V load(final K key) throws CacheLoaderException {
+            return map.get(key);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(final Cache.Entry<? extends K, ? extends V> entry) throws CacheWriterException {
+            map.put(entry.getKey(), entry.getValue());
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(final Object key) throws CacheWriterException {
+            map.remove(key);
+        }
+    }
+
+    /**
+     * Test object.
+     */
+    static class TestObj implements Serializable {
+        /** */
+        Integer val;
+
+        /** */
+        public TestObj() {
+        }
+
+        /** */
+        public TestObj(final Integer val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(final Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            final TestObj testObj = (TestObj) o;
+
+            return val != null ? val.equals(testObj.val) : testObj.val == null;
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val != null ? val.hashCode() : 0;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b46ed10a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridLocalCacheStoreManagerDeserializationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridLocalCacheStoreManagerDeserializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridLocalCacheStoreManagerDeserializationTest.java
new file mode 100644
index 0000000..827b3cf
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridLocalCacheStoreManagerDeserializationTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+
+import javax.cache.expiry.ExpiryPolicy;
+import java.util.UUID;
+
+/**
+ * Checks whether storing to local store doesn't cause binary objects unmarshalling,
+ * and as a consequence {@link ClassNotFoundException} to be thrown.
+ *
+ * @see <a href="https://issues.apache.org/jira/browse/IGNITE-2753">
+ *     https://issues.apache.org/jira/browse/IGNITE-2753
+ *     </a>
+ */
+public class GridLocalCacheStoreManagerDeserializationTest extends GridCacheStoreManagerDeserializationTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.LOCAL;
+    }
+
+    /**
+     * Checks no additional unmarshalling happens in calling
+     * {@link GridCacheMapEntry#innerUpdateLocal(GridCacheVersion, GridCacheOperation, Object, Object[],
+     * boolean, boolean, boolean, boolean, ExpiryPolicy, boolean, boolean, CacheEntryPredicate[],
+     * boolean, UUID, String)}.
+     *
+     * @throws Exception
+     */
+    public void testUpdate() throws Exception {
+        // Goal is to check correct saving to store (no exception must be thrown)
+
+        final Ignite grid = startGrid();
+
+        final IgniteCache<TestObj, TestObj> cache = grid.createCache(CACHE_NAME);
+
+        final TestObj testObj = new TestObj(0);
+
+        cache.put(testObj, testObj);
+
+        assert testObj.equals(cache.get(testObj));
+        assert store.map.containsKey(testObj);
+
+        cache.remove(testObj);
+
+        assert cache.get(testObj) == null;
+        assert !store.map.containsKey(testObj);
+    }
+
+    /**
+     * Checks no additional unmarshalling happens in calling
+     * {@link GridCacheMapEntry#innerUpdateLocal(GridCacheVersion, GridCacheOperation, Object, Object[],
+     * boolean, boolean, boolean, boolean, ExpiryPolicy, boolean, boolean, CacheEntryPredicate[],
+     * boolean, UUID, String)} for binary objects.
+     *
+     * @throws Exception
+     */
+    public void testBinaryUpdate() throws Exception {
+        // Goal is to check correct saving to store (no exception must be thrown)
+        final Ignite grid = startGrid("binaryGrid");
+
+        final IgniteCache<BinaryObject, BinaryObject> cache = grid.createCache(CACHE_NAME).withKeepBinary();
+
+        final BinaryObjectBuilder builder = grid.binary().builder("custom_type");
+
+        final BinaryObject entity = builder.setField("id", 0).build();
+
+        cache.put(entity, entity);
+
+        assert entity.equals(cache.get(entity));
+        assert store.map.containsKey(entity);
+
+        cache.remove(entity);
+
+        assert cache.get(entity) == null;
+        assert !store.map.containsKey(entity);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b46ed10a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index 45679dd..7aab990 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -19,15 +19,14 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcStoreSessionListenerSelfTest;
+import org.apache.ignite.internal.processors.GridCacheTxLoadFromStoreOnLockSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheClientStoreSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticReadCommittedSeltTest;
 import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticRepeatableReadSeltTest;
 import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticSerializableSeltTest;
 import org.apache.ignite.internal.processors.cache.CacheGetEntryPessimisticReadCommittedSeltTest;
 import org.apache.ignite.internal.processors.cache.CacheGetEntryPessimisticRepeatableReadSeltTest;
 import org.apache.ignite.internal.processors.cache.CacheGetEntryPessimisticSerializableSeltTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheGetCustomCollectionsSelfTest;
-import org.apache.ignite.internal.processors.GridCacheTxLoadFromStoreOnLockSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheClientStoreSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheOffheapMapEntrySelfTest;
 import org.apache.ignite.internal.processors.cache.CacheReadThroughAtomicRestartSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheReadThroughLocalAtomicRestartSelfTest;
@@ -49,7 +48,9 @@ import org.apache.ignite.internal.processors.cache.GridCacheMultinodeUpdateAtomi
 import org.apache.ignite.internal.processors.cache.GridCacheMultinodeUpdateNearEnabledNoBackupsSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheMultinodeUpdateNearEnabledSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheMultinodeUpdateSelfTest;
+import org.apache.ignite.internal.processors.cache.GridCacheStoreManagerDeserializationTest;
 import org.apache.ignite.internal.processors.cache.GridCacheVersionMultinodeTest;
+import org.apache.ignite.internal.processors.cache.GridLocalCacheStoreManagerDeserializationTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicCopyOnReadDisabledTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicLocalPeekModesTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicLocalStoreValueTest;
@@ -63,6 +64,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicStoreValueTe
 import org.apache.ignite.internal.processors.cache.IgniteCacheConfigurationDefaultTemplateTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheConfigurationTemplateTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDynamicStopSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheGetCustomCollectionsSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheInvokeReadThroughTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheLoadRebalanceEvictionSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheTxCopyOnReadDisabledTest;
@@ -266,6 +268,9 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(CacheStoreUsageMultinodeDynamicStartAtomicTest.class);
         suite.addTestSuite(CacheStoreUsageMultinodeDynamicStartTxTest.class);
 
+        suite.addTestSuite(GridCacheStoreManagerDeserializationTest.class);
+        suite.addTestSuite(GridLocalCacheStoreManagerDeserializationTest.class);
+
         suite.addTestSuite(IgniteStartCacheInTransactionSelfTest.class);
         suite.addTestSuite(IgniteStartCacheInTransactionAtomicSelfTest.class);
 


[06/35] ignite git commit: IGNITE-2756 - Fixed StreamVisitorExample

Posted by nt...@apache.org.
IGNITE-2756 - Fixed StreamVisitorExample


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/a579d454
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/a579d454
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/a579d454

Branch: refs/heads/ignite-2791
Commit: a579d45415b3063ac72069924a15e87beda0447c
Parents: c31f384
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Wed Mar 9 18:53:30 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Mar 9 18:53:30 2016 -0800

----------------------------------------------------------------------
 .../streaming/StreamVisitorExample.java         | 31 +++++++++++++-------
 1 file changed, 20 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a579d454/examples/src/main/java/org/apache/ignite/examples/streaming/StreamVisitorExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/streaming/StreamVisitorExample.java b/examples/src/main/java/org/apache/ignite/examples/streaming/StreamVisitorExample.java
index 29781bc..c3d8c64 100644
--- a/examples/src/main/java/org/apache/ignite/examples/streaming/StreamVisitorExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/streaming/StreamVisitorExample.java
@@ -35,12 +35,16 @@ import org.apache.ignite.examples.ExamplesUtils;
 import org.apache.ignite.stream.StreamVisitor;
 
 /**
- * Stream random numbers into the streaming cache.
- * To start the example, you should:
- * <ul>
- *     <li>Start a few nodes using {@link ExampleNodeStartup}.</li>
- *     <li>Start streaming using {@link StreamVisitorExample}.</li>
- * </ul>
+ * This examples demonstrates the stream visitor which allows to customize the processing
+ * of the streamed data on the server side. Instead of populating the cache for which the
+ * streamer is created, we will calculate aggregated data on the fly and save results in
+ * another cache.
+ * <p>
+ * Remote nodes should always be started with special configuration file which
+ * enables P2P class loading: {@code 'ignite.{sh|bat} examples/config/example-ignite.xml'}.
+ * <p>
+ * Alternatively you can run {@link ExampleNodeStartup} in another JVM which will
+ * start node with {@code examples/config/example-ignite.xml} configuration.
  */
 public class StreamVisitorExample {
     /** Random number generator. */
@@ -53,7 +57,7 @@ public class StreamVisitorExample {
     private static final double[] INITIAL_PRICES = {194.9, 893.49, 34.21, 23.24, 57.93, 45.03, 44.41, 28.44, 378.49, 69.50};
 
     /** Caches' names. */
-    private static final String CACHE_NAME = "instCache";
+    private static final String INSTRUMENTS_CACHE_NAME = "instCache";
     private static final String MARKET_TICKS_CACHE_NAME = "marketTicks";
 
     public static void main(String[] args) throws Exception {
@@ -65,7 +69,7 @@ public class StreamVisitorExample {
                 return;
 
             // Financial instrument cache configuration.
-            CacheConfiguration<String, Instrument> instCfg = new CacheConfiguration<>(CACHE_NAME);
+            CacheConfiguration<String, Instrument> instCfg = new CacheConfiguration<>(INSTRUMENTS_CACHE_NAME);
 
             // Index key and value for querying financial instruments.
             // Note that Instrument class has @QuerySqlField annotation for secondary field indexing.
@@ -77,6 +81,10 @@ public class StreamVisitorExample {
                 IgniteCache<String, Instrument> instCache = ignite.getOrCreateCache(instCfg)
             ) {
                 try (IgniteDataStreamer<String, Double> mktStmr = ignite.dataStreamer(mktCache.getName())) {
+                    // To achieve proper indexing we should use fully-qualified name
+                    // of the class as a type name when binary object is created.
+                    final String instTypeName = Instrument.class.getName();
+
                     // Note that we receive market data, but do not populate 'mktCache' (it remains empty).
                     // Instead we update the instruments in the 'instCache'.
                     // Since both, 'instCache' and 'mktCache' use the same key, updates are collocated.
@@ -85,14 +93,15 @@ public class StreamVisitorExample {
                             String symbol = e.getKey();
                             Double tick = e.getValue();
 
-                            IgniteCache<String, BinaryObject> binInstCache = ignite.cache("instCache").withKeepBinary();
+                            IgniteCache<String, BinaryObject> binInstCache =
+                                ignite.cache(INSTRUMENTS_CACHE_NAME).withKeepBinary();
 
                             BinaryObject inst = binInstCache.get(symbol);
 
                             BinaryObjectBuilder instBuilder;
 
                             if (inst == null) {
-                                instBuilder = ignite.binary().builder("Instrument");
+                                instBuilder = ignite.binary().builder(instTypeName);
 
                                 // Constructor logic.
                                 instBuilder.setField(
@@ -146,7 +155,7 @@ public class StreamVisitorExample {
             }
             finally {
                 // Distributed cache could be removed from cluster only by #destroyCache() call.
-                ignite.destroyCache(CACHE_NAME);
+                ignite.destroyCache(INSTRUMENTS_CACHE_NAME);
                 ignite.destroyCache(MARKET_TICKS_CACHE_NAME);
             }
         }


[33/35] ignite git commit: test commit (revert) (cherry picked from commit 1ef0854)

Posted by nt...@apache.org.
test commit (revert)
(cherry picked from commit 1ef0854)


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/18de768e
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/18de768e
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/18de768e

Branch: refs/heads/ignite-2791
Commit: 18de768ec9f0e5bf5f6c337f1917ad7acc138c51
Parents: 402cbab
Author: Anton Vinogradov <av...@apache.org>
Authored: Mon Mar 14 19:36:34 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Mon Mar 14 19:36:34 2016 +0300

----------------------------------------------------------------------
 pom.xml | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/18de768e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index af44209..bde8a2c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -922,4 +922,3 @@
         </plugins>
     </build>
 </project>
-


[07/35] ignite git commit: IGNITE-2749 - Fixed web session clustering in WebLogic

Posted by nt...@apache.org.
IGNITE-2749 - Fixed web session clustering in WebLogic


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/5377dde2
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/5377dde2
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/5377dde2

Branch: refs/heads/ignite-2791
Commit: 5377dde21e4aa32ca9da4963015b15da4d2e2b13
Parents: a579d45
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Wed Mar 9 20:16:49 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Mar 9 20:16:49 2016 -0800

----------------------------------------------------------------------
 modules/web/ignite-weblogic-test/pom.xml        | 100 +++++++++++++++++++
 .../webapp/META-INF/config/default-config.xml   |  20 ++++
 .../src/main/webapp/WEB-INF/web.xml             |  69 +++++++++++++
 .../src/main/webapp/index.jsp                   |  53 ++++++++++
 .../ignite/cache/websession/WebSession.java     |  26 ++---
 .../cache/websession/WebSessionFilter.java      |  17 ++--
 .../cache/websession/WebSessionListener.java    |   8 +-
 pom.xml                                         |   1 +
 8 files changed, 267 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5377dde2/modules/web/ignite-weblogic-test/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/ignite-weblogic-test/pom.xml b/modules/web/ignite-weblogic-test/pom.xml
new file mode 100644
index 0000000..4746f1f
--- /dev/null
+++ b/modules/web/ignite-weblogic-test/pom.xml
@@ -0,0 +1,100 @@
+<!--
+  ~ 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.
+  -->
+
+<!--
+  ~ 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.
+  -->
+
+<!--
+  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.
+-->
+
+<!--
+    POM file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <groupId>org.apache.ignite</groupId>
+    <artifactId>ignite-weblogic-test</artifactId>
+    <packaging>war</packaging>
+    <version>1.0-SNAPSHOT</version>
+    <name>ignite-weblogic-test Maven Webapp</name>
+    <url>http://maven.apache.org</url>
+    <dependencies>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>3.8.1</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+            <version>1.6.0-SNAPSHOT</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-web</artifactId>
+            <version>1.6.0-SNAPSHOT</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-log4j</artifactId>
+            <version>1.6.0-SNAPSHOT</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-spring</artifactId>
+            <version>1.6.0-SNAPSHOT</version>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <finalName>ignite-weblogic-test</finalName>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5377dde2/modules/web/ignite-weblogic-test/src/main/webapp/META-INF/config/default-config.xml
----------------------------------------------------------------------
diff --git a/modules/web/ignite-weblogic-test/src/main/webapp/META-INF/config/default-config.xml b/modules/web/ignite-weblogic-test/src/main/webapp/META-INF/config/default-config.xml
new file mode 100644
index 0000000..daa466d
--- /dev/null
+++ b/modules/web/ignite-weblogic-test/src/main/webapp/META-INF/config/default-config.xml
@@ -0,0 +1,20 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+                            http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="atomic"/>
+
+                    <property name="cacheMode" value="PARTITIONED"/>
+
+                    <property name="atomicityMode" value="ATOMIC"/>
+                </bean>
+            </list>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5377dde2/modules/web/ignite-weblogic-test/src/main/webapp/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/modules/web/ignite-weblogic-test/src/main/webapp/WEB-INF/web.xml b/modules/web/ignite-weblogic-test/src/main/webapp/WEB-INF/web.xml
new file mode 100644
index 0000000..054eab4
--- /dev/null
+++ b/modules/web/ignite-weblogic-test/src/main/webapp/WEB-INF/web.xml
@@ -0,0 +1,69 @@
+<!--
+  ~ 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.
+  -->
+
+<!--
+  ~ 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.
+  -->
+
+<!DOCTYPE web-app PUBLIC
+ "-//Sun Microsystems, Inc.//DTD Web Application 2.3//EN"
+ "http://java.sun.com/dtd/web-app_2_3.dtd" >
+
+<web-app>
+  <display-name>Archetype Created Web Application</display-name>
+
+    <listener>
+        <listener-class>org.apache.ignite.startup.servlet.ServletContextListenerStartup</listener-class>
+    </listener>
+
+    <filter>
+        <filter-name>IgniteWebSessionsFilter</filter-name>
+        <filter-class>org.apache.ignite.cache.websession.WebSessionFilter</filter-class>
+    </filter>
+
+    <!-- You can also specify a custom URL pattern. -->
+    <filter-mapping>
+        <filter-name>IgniteWebSessionsFilter</filter-name>
+        <url-pattern>/*</url-pattern>
+    </filter-mapping>
+
+    <!-- Specify Ignite configuration (relative to META-INF folder or Ignite_HOME). -->
+    <context-param>
+        <param-name>IgniteConfigurationFilePath</param-name>
+        <param-value>config/default-config.xml </param-value>
+    </context-param>
+
+    <!-- Specify the name of Ignite cache for web sessions. -->
+    <context-param>
+        <param-name>IgniteWebSessionsCacheName</param-name>
+        <param-value>atomic</param-value>
+    </context-param>
+
+</web-app>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5377dde2/modules/web/ignite-weblogic-test/src/main/webapp/index.jsp
----------------------------------------------------------------------
diff --git a/modules/web/ignite-weblogic-test/src/main/webapp/index.jsp b/modules/web/ignite-weblogic-test/src/main/webapp/index.jsp
new file mode 100644
index 0000000..0cc090d
--- /dev/null
+++ b/modules/web/ignite-weblogic-test/src/main/webapp/index.jsp
@@ -0,0 +1,53 @@
+<%--
+  ~ 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.
+  --%>
+
+<%--
+  ~ 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.
+  --%>
+
+<%@ page import="org.apache.ignite.Ignition" %>
+<%@ page import="java.util.UUID" %>
+<html>
+<body>
+<h2>Session ID</h2>
+<%= request.getSession().getId() %>
+<h2>Session content</h2>
+<%= Ignition.ignite().cache("atomic").get(request.getSession().getId()) %>
+<h2>Cache size</h2>
+<%= Ignition.ignite().cache("atomic").size() %>
+
+<%
+    HttpSession ses = request.getSession();
+
+    ses.setAttribute(UUID.randomUUID().toString(), "Value");
+%>
+
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5377dde2/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSession.java
----------------------------------------------------------------------
diff --git a/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSession.java b/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSession.java
index 7441a1a..8b944e5 100644
--- a/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSession.java
+++ b/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSession.java
@@ -100,12 +100,15 @@ class WebSession implements HttpSession, Externalizable {
     }
 
     /**
+     * @param id Session ID.
      * @param ses Session.
      */
-    WebSession(HttpSession ses) {
+    WebSession(String id, HttpSession ses) {
+        assert id != null;
         assert ses != null;
 
-        id = ses.getId();
+        this.id = id;
+
         createTime = ses.getCreationTime();
         accessTime = ses.getLastAccessedTime();
         maxInactiveInterval = ses.getMaxInactiveInterval();
@@ -125,23 +128,17 @@ class WebSession implements HttpSession, Externalizable {
     }
 
     /**
+     * @param id Session ID.
      * @param ses Session.
      * @param isNew Is new flag.
      */
-    WebSession(HttpSession ses, boolean isNew) {
-        this(ses);
+    WebSession(String id, HttpSession ses, boolean isNew) {
+        this(id, ses);
 
         this.isNew = isNew;
     }
 
     /**
-     * @param accessTime Last access time.
-     */
-    void accessTime(long accessTime) {
-        this.accessTime = accessTime;
-    }
-
-    /**
      * @param ctx Servlet context.
      */
     public void servletContext(ServletContext ctx) {
@@ -299,13 +296,6 @@ class WebSession implements HttpSession, Externalizable {
         isValid = false;
     }
 
-    /**
-     * @param isNew New session flag.
-     */
-    void setNew(boolean isNew) {
-        this.isNew = isNew;
-    }
-
     /** {@inheritDoc} */
     @Override public boolean isNew() {
         if (!isValid)

http://git-wip-us.apache.org/repos/asf/ignite/blob/5377dde2/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionFilter.java
----------------------------------------------------------------------
diff --git a/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionFilter.java b/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionFilter.java
index 6e6be33..3dedee3 100644
--- a/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionFilter.java
+++ b/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionFilter.java
@@ -166,7 +166,7 @@ public class WebSessionFilter implements Filter {
     /** Web sessions caching cache name parameter name. */
     public static final String WEB_SES_CACHE_NAME_PARAM = "IgniteWebSessionsCacheName";
 
-    /** Web sessions caching retry on fail parameter name (valid for ATOMIC */
+    /** Web sessions caching retry on fail parameter name (valid for ATOMIC cache only). */
     public static final String WEB_SES_MAX_RETRIES_ON_FAIL_NAME_PARAM = "IgniteWebSessionsMaximumRetriesOnFail";
 
     /** Default retry on fail flag value. */
@@ -338,6 +338,9 @@ public class WebSessionFilter implements Filter {
         String sesId = httpReq.getRequestedSessionId();
 
         if (sesId != null) {
+            if (sesIdTransformer != null)
+                sesId = sesIdTransformer.apply(sesId);
+
             cached = cache.get(sesId);
 
             if (cached != null) {
@@ -345,7 +348,7 @@ public class WebSessionFilter implements Filter {
                     log.debug("Using cached session for ID: " + sesId);
 
                 if (cached.isNew())
-                    cached = new WebSession(cached, false);
+                    cached = new WebSession(cached.getId(), cached, false);
             }
             else {
                 if (log.isDebugEnabled())
@@ -386,8 +389,10 @@ public class WebSessionFilter implements Filter {
         if (ses != null && ses instanceof WebSession) {
             Collection<T2<String, Object>> updates = ((WebSession)ses).updates();
 
-            if (updates != null)
-                lsnr.updateAttributes(ses.getId(), updates, ses.getMaxInactiveInterval());
+            if (updates != null) {
+                lsnr.updateAttributes(sesIdTransformer != null ? sesIdTransformer.apply(ses.getId()) : ses.getId(),
+                    updates, ses.getMaxInactiveInterval());
+            }
         }
 
         return sesId;
@@ -406,7 +411,7 @@ public class WebSessionFilter implements Filter {
         if (log.isDebugEnabled())
             log.debug("Session created: " + sesId);
 
-        WebSession cached = new WebSession(ses, true);
+        WebSession cached = new WebSession(sesId, ses, true);
 
         for (int i = 0; i < retries; i++) {
             try {
@@ -428,7 +433,7 @@ public class WebSessionFilter implements Filter {
                     cached = old;
 
                     if (cached.isNew())
-                        cached = new WebSession(cached, false);
+                        cached = new WebSession(cached.getId(), cached, false);
                 }
 
                 break;

http://git-wip-us.apache.org/repos/asf/ignite/blob/5377dde2/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionListener.java b/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionListener.java
index b826031..0d7c44e 100644
--- a/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionListener.java
+++ b/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionListener.java
@@ -46,7 +46,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS;
  */
 class WebSessionListener {
     /** */
-    private static final long RETRY_DELAY = 1;
+    private static final long RETRY_DELAY = 1000;
 
     /** Cache. */
     private final IgniteCache<String, WebSession> cache;
@@ -186,7 +186,9 @@ class WebSessionListener {
             if (!entry.exists())
                 return null;
 
-            WebSession ses = new WebSession(entry.getValue());
+            WebSession ses0 = entry.getValue();
+
+            WebSession ses = new WebSession(ses0.getId(), ses0);
 
             for (T2<String, Object> update : updates) {
                 String name = update.get1();
@@ -216,4 +218,4 @@ class WebSessionListener {
             updates = U.readCollection(in);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5377dde2/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ccfd471..bde8a2c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -84,6 +84,7 @@
         <module>modules/osgi-paxlogging</module>
         <module>modules/osgi-karaf</module>
         <module>modules/osgi</module>
+        <module>modules/web/ignite-weblogic-test</module>
     </modules>
 
     <profiles>


[23/35] ignite git commit: IGNITE-2765 WebSessionFilter doesn't survive client reconnect

Posted by nt...@apache.org.
IGNITE-2765 WebSessionFilter doesn't survive client reconnect


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/0ad4f395
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/0ad4f395
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/0ad4f395

Branch: refs/heads/ignite-2791
Commit: 0ad4f395a4b18a5cb1cba4926261cf5dd56e606d
Parents: ed030e5
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Mar 11 19:23:49 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Fri Mar 11 19:34:23 2016 +0300

----------------------------------------------------------------------
 .../config/websession/example-cache-base.xml    | 148 +++++++++++++++
 .../config/websession/example-cache-client.xml  |  33 ++++
 .../test/config/websession/example-cache.xml    | 128 +------------
 .../test/config/websession/example-cache2.xml   |  31 +++
 .../cache/websession/WebSessionFilter.java      | 188 ++++++++++++++-----
 .../cache/websession/WebSessionListener.java    |  60 +++---
 .../internal/websession/WebSessionSelfTest.java |  72 +++++++
 7 files changed, 452 insertions(+), 208 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0ad4f395/modules/core/src/test/config/websession/example-cache-base.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/websession/example-cache-base.xml b/modules/core/src/test/config/websession/example-cache-base.xml
new file mode 100644
index 0000000..d3d5b46
--- /dev/null
+++ b/modules/core/src/test/config/websession/example-cache-base.xml
@@ -0,0 +1,148 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  ~ 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.
+  -->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean abstract="true" id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <!-- Set to true to enable distributed class loading for examples, default is false. -->
+        <property name="peerClassLoadingEnabled" value="true"/>
+
+        <property name="marshaller">
+            <bean class="org.apache.ignite.marshaller.optimized.OptimizedMarshaller">
+                <!-- Set to false to allow non-serializable objects in examples, default is true. -->
+                <property name="requireSerializable" value="false"/>
+            </bean>
+        </property>
+
+        <!-- Enable cache events for examples. -->
+        <property name="includeEventTypes">
+            <util:constant static-field="org.apache.ignite.events.EventType.EVTS_CACHE"/>
+        </property>
+
+        <!-- Cache configurations (all properties are optional). -->
+        <property name="cacheConfiguration">
+            <list>
+                <!-- Partitioned cache example configuration (Atomic mode). -->
+                <bean parent="cache-template">
+                    <property name="name" value="partitioned"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="backups" value="1"/>
+
+                    <property name="indexedTypes">
+                        <list>
+                            <!-- Key and value type for SQL table Long. -->
+                            <value>java.lang.Integer</value>
+                            <value>java.lang.Long</value>
+                        </list>
+                    </property>
+                </bean>
+
+                <!-- Partitioned cache example configuration (Atomic mode, PRIMARY write order mode). -->
+                <bean parent="cache-template">
+                    <property name="name" value="partitioned_primary"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="atomicWriteOrderMode" value="PRIMARY"/>
+                    <property name="backups" value="1"/>
+                </bean>
+
+                <!-- Partitioned cache example configuration (Transactional mode). -->
+                <bean parent="cache-template">
+                    <property name="name" value="partitioned_tx"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="nearConfiguration">
+                        <bean class="org.apache.ignite.configuration.NearCacheConfiguration"/>
+                    </property>
+                    <property name="backups" value="1"/>
+                </bean>
+
+                <!-- Replicated cache example configuration (Atomic mode). -->
+                <bean parent="cache-template">
+                    <property name="name" value="replicated"/>
+                    <property name="cacheMode" value="REPLICATED"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="indexedTypes">
+                        <list>
+                            <!-- Key and value type for SQL table DimStore. -->
+                            <value>java.lang.Integer</value>
+                            <value>java.lang.Integer</value>
+                        </list>
+                    </property>
+                </bean>
+
+                <!-- Replicated cache example configuration (Transactional mode). -->
+                <bean parent="cache-template">
+                    <property name="name" value="replicated_tx"/>
+                    <property name="cacheMode" value="REPLICATED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                </bean>
+
+                <!-- Local cache example configuration (Atomic mode). -->
+                <bean parent="cache-template">
+                    <property name="name" value="local"/>
+                    <property name="cacheMode" value="LOCAL"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                </bean>
+
+                <!-- Local cache example configuration (Transactional mode). -->
+                <bean parent="cache-template">
+                    <property name="name" value="local_tx"/>
+                    <property name="cacheMode" value="LOCAL"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                </bean>
+            </list>
+        </property>
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47509</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+
+    <!-- Template for all example cache configurations. -->
+    <bean id="cache-template" abstract="true" class="org.apache.ignite.configuration.CacheConfiguration">
+        <!-- Initial cache size. -->
+        <property name="startSize" value="3000000"/>
+
+        <!-- Set synchronous rebalancing (default is asynchronous). -->
+        <property name="rebalanceMode" value="SYNC"/>
+
+        <!-- Set to FULL_SYNC for examples, default is PRIMARY_SYNC. -->
+        <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ad4f395/modules/core/src/test/config/websession/example-cache-client.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/websession/example-cache-client.xml b/modules/core/src/test/config/websession/example-cache-client.xml
new file mode 100644
index 0000000..9272e14
--- /dev/null
+++ b/modules/core/src/test/config/websession/example-cache-client.xml
@@ -0,0 +1,33 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <!-- Imports Ignite configuration -->
+    <import resource="example-cache-base.xml"/>
+
+    <bean parent="ignite.cfg">
+        <property name="clientMode" value="true"/>
+
+        <property name="gridName" value="client"/>
+    </bean>
+
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ad4f395/modules/core/src/test/config/websession/example-cache.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/websession/example-cache.xml b/modules/core/src/test/config/websession/example-cache.xml
index 0cc0e1e..1090220 100644
--- a/modules/core/src/test/config/websession/example-cache.xml
+++ b/modules/core/src/test/config/websession/example-cache.xml
@@ -28,130 +28,12 @@
 -->
 <beans xmlns="http://www.springframework.org/schema/beans"
        xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xmlns:util="http://www.springframework.org/schema/util"
-       xsi:schemaLocation="
-        http://www.springframework.org/schema/beans
-        http://www.springframework.org/schema/beans/spring-beans.xsd
-        http://www.springframework.org/schema/util
-        http://www.springframework.org/schema/util/spring-util.xsd">
-    <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
-        <!-- Set to true to enable distributed class loading for examples, default is false. -->
-        <property name="peerClassLoadingEnabled" value="true"/>
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <!-- Imports Ignite configuration -->
+    <import resource="example-cache-base.xml"/>
 
-        <property name="marshaller">
-            <bean class="org.apache.ignite.marshaller.optimized.OptimizedMarshaller">
-                <!-- Set to false to allow non-serializable objects in examples, default is true. -->
-                <property name="requireSerializable" value="false"/>
-            </bean>
-        </property>
-
-        <!-- Enable cache events for examples. -->
-        <property name="includeEventTypes">
-            <util:constant static-field="org.apache.ignite.events.EventType.EVTS_CACHE"/>
-        </property>
-
-        <!-- Cache configurations (all properties are optional). -->
-        <property name="cacheConfiguration">
-            <list>
-                <!-- Partitioned cache example configuration (Atomic mode). -->
-                <bean parent="cache-template">
-                    <property name="name" value="partitioned"/>
-                    <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="ATOMIC"/>
-                    <property name="backups" value="1"/>
-
-                    <property name="indexedTypes">
-                        <list>
-                            <!-- Key and value type for SQL table Long. -->
-                            <value>java.lang.Integer</value>
-                            <value>java.lang.Long</value>
-                        </list>
-                    </property>
-                </bean>
-
-                <!-- Partitioned cache example configuration (Atomic mode, PRIMARY write order mode). -->
-                <bean parent="cache-template">
-                    <property name="name" value="partitioned_primary"/>
-                    <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="ATOMIC"/>
-                    <property name="atomicWriteOrderMode" value="PRIMARY"/>
-                    <property name="backups" value="1"/>
-                </bean>
-
-                <!-- Partitioned cache example configuration (Transactional mode). -->
-                <bean parent="cache-template">
-                    <property name="name" value="partitioned_tx"/>
-                    <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                    <property name="nearConfiguration">
-                        <bean class="org.apache.ignite.configuration.NearCacheConfiguration"/>
-                    </property>
-                    <property name="backups" value="1"/>
-                </bean>
-
-                <!-- Replicated cache example configuration (Atomic mode). -->
-                <bean parent="cache-template">
-                    <property name="name" value="replicated"/>
-                    <property name="cacheMode" value="REPLICATED"/>
-                    <property name="atomicityMode" value="ATOMIC"/>
-                    <property name="indexedTypes">
-                        <list>
-                            <!-- Key and value type for SQL table DimStore. -->
-                            <value>java.lang.Integer</value>
-                            <value>java.lang.Integer</value>
-                        </list>
-                    </property>
-                </bean>
-
-                <!-- Replicated cache example configuration (Transactional mode). -->
-                <bean parent="cache-template">
-                    <property name="name" value="replicated_tx"/>
-                    <property name="cacheMode" value="REPLICATED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                </bean>
-
-                <!-- Local cache example configuration (Atomic mode). -->
-                <bean parent="cache-template">
-                    <property name="name" value="local"/>
-                    <property name="cacheMode" value="LOCAL"/>
-                    <property name="atomicityMode" value="ATOMIC"/>
-                </bean>
-
-                <!-- Local cache example configuration (Transactional mode). -->
-                <bean parent="cache-template">
-                    <property name="name" value="local_tx"/>
-                    <property name="cacheMode" value="LOCAL"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                </bean>
-            </list>
-        </property>
-
-        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
-        <property name="discoverySpi">
-            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
-                <property name="ipFinder">
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
-                        <property name="addresses">
-                            <list>
-                                <!-- In distributed environment, replace with actual host IP address. -->
-                                <value>127.0.0.1:47500..47509</value>
-                            </list>
-                        </property>
-                    </bean>
-                </property>
-            </bean>
-        </property>
+    <bean parent="ignite.cfg">
     </bean>
 
-    <!-- Template for all example cache configurations. -->
-    <bean id="cache-template" abstract="true" class="org.apache.ignite.configuration.CacheConfiguration">
-        <!-- Initial cache size. -->
-        <property name="startSize" value="3000000"/>
-
-        <!-- Set synchronous rebalancing (default is asynchronous). -->
-        <property name="rebalanceMode" value="SYNC"/>
-
-        <!-- Set to FULL_SYNC for examples, default is PRIMARY_SYNC. -->
-        <property name="writeSynchronizationMode" value="FULL_SYNC"/>
-    </bean>
 </beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ad4f395/modules/core/src/test/config/websession/example-cache2.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/websession/example-cache2.xml b/modules/core/src/test/config/websession/example-cache2.xml
new file mode 100644
index 0000000..59a244f
--- /dev/null
+++ b/modules/core/src/test/config/websession/example-cache2.xml
@@ -0,0 +1,31 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <!-- Imports Ignite configuration -->
+    <import resource="example-cache-base.xml"/>
+
+    <bean parent="ignite.cfg">
+        <property name="gridName" value="grid2"/>
+    </bean>
+
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ad4f395/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionFilter.java
----------------------------------------------------------------------
diff --git a/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionFilter.java b/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionFilter.java
index 42de43b..f718035 100644
--- a/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionFilter.java
+++ b/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionFilter.java
@@ -35,6 +35,7 @@ import javax.servlet.http.HttpServletRequestWrapper;
 import javax.servlet.http.HttpSession;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteClientDisconnectedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteTransactions;
@@ -123,6 +124,15 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
  *         </td>
  *         <td>{@code 3}</td>
  *     </tr>
+ *     <tr>
+ *         <td>IgniteWebSessionsRetriesTimeout</td>
+ *         <td>
+ *             Retry timeout. Related to IgniteWebSessionsMaximumRetriesOnFail param.
+ *             <p>
+ *             Further attempts will be cancelled in case timeout was exceeded.
+ *         </td>
+ *         <td>{@code 10000} (10 seconds)</td>
+ *     </tr>
  * </table>
  * These parameters are taken from either filter init parameter list or
  * servlet context parameters. You can specify filter init parameters as follows:
@@ -169,9 +179,15 @@ public class WebSessionFilter implements Filter {
     /** Web sessions caching retry on fail parameter name (valid for ATOMIC cache only). */
     public static final String WEB_SES_MAX_RETRIES_ON_FAIL_NAME_PARAM = "IgniteWebSessionsMaximumRetriesOnFail";
 
+    /** Web sessions caching retry on fail timeout parameter name. */
+    public static final String WEB_SES_RETRIES_TIMEOUT_NAME_PARAM = "IgniteWebSessionsRetriesTimeout";
+
     /** Default retry on fail flag value. */
     public static final int DFLT_MAX_RETRIES_ON_FAIL = 3;
 
+    /** Default retry on fail timeout flag value. */
+    public static final int DFLT_RETRIES_ON_FAIL_TIMEOUT = 10000;
+
     /** Cache. */
     private IgniteCache<String, WebSession> cache;
 
@@ -193,9 +209,18 @@ public class WebSessionFilter implements Filter {
     /** Transactions enabled flag. */
     private boolean txEnabled;
 
+    /** Node. */
+    private Ignite webSesIgnite;
+
+    /** Cache name. */
+    private String cacheName;
+
     /** */
     private int retries;
 
+    /** */
+    private int retriesTimeout;
+
     /** {@inheritDoc} */
     @Override public void init(FilterConfig cfg) throws ServletException {
         ctx = cfg.getServletContext();
@@ -204,7 +229,7 @@ public class WebSessionFilter implements Filter {
             cfg.getInitParameter(WEB_SES_NAME_PARAM),
             ctx.getInitParameter(WEB_SES_NAME_PARAM));
 
-        String cacheName = U.firstNotNull(
+        cacheName = U.firstNotNull(
             cfg.getInitParameter(WEB_SES_CACHE_NAME_PARAM),
             ctx.getInitParameter(WEB_SES_CACHE_NAME_PARAM));
 
@@ -219,7 +244,19 @@ public class WebSessionFilter implements Filter {
             throw new IgniteException("Maximum number of retries parameter is invalid: " + retriesStr, e);
         }
 
-        Ignite webSesIgnite = G.ignite(gridName);
+        String retriesTimeoutStr = U.firstNotNull(
+            cfg.getInitParameter(WEB_SES_RETRIES_TIMEOUT_NAME_PARAM),
+            ctx.getInitParameter(WEB_SES_RETRIES_TIMEOUT_NAME_PARAM));
+
+        try {
+            retriesTimeout = retriesTimeoutStr != null ?
+                Integer.parseInt(retriesTimeoutStr) : DFLT_RETRIES_ON_FAIL_TIMEOUT;
+        }
+        catch (NumberFormatException e) {
+            throw new IgniteException("Retries timeout parameter is invalid: " + retriesTimeoutStr, e);
+        }
+
+        webSesIgnite = G.ignite(gridName);
 
         if (webSesIgnite == null)
             throw new IgniteException("Grid for web sessions caching is not started (is it configured?): " +
@@ -229,35 +266,9 @@ public class WebSessionFilter implements Filter {
 
         log = webSesIgnite.log();
 
-        cache = webSesIgnite.cache(cacheName);
-
-        if (cache == null)
-            throw new IgniteException("Cache for web sessions is not started (is it configured?): " + cacheName);
-
-        CacheConfiguration cacheCfg = cache.getConfiguration(CacheConfiguration.class);
-
-        if (cacheCfg.getWriteSynchronizationMode() == FULL_ASYNC)
-            throw new IgniteException("Cache for web sessions cannot be in FULL_ASYNC mode: " + cacheName);
-
-        if (!cacheCfg.isEagerTtl())
-            throw new IgniteException("Cache for web sessions cannot operate with lazy TTL. " +
-                "Consider setting eagerTtl to true for cache: " + cacheName);
-
-        if (cacheCfg.getCacheMode() == LOCAL)
-            U.quietAndWarn(webSesIgnite.log(), "Using LOCAL cache for web sessions caching " +
-                "(this is only OK in test mode): " + cacheName);
+        initCache();
 
-        if (cacheCfg.getCacheMode() == PARTITIONED && cacheCfg.getAtomicityMode() != ATOMIC)
-            U.quietAndWarn(webSesIgnite.log(), "Using " + cacheCfg.getAtomicityMode() + " atomicity for web sessions " +
-                "caching (switch to ATOMIC mode for better performance)");
-
-        if (log.isInfoEnabled())
-            log.info("Started web sessions caching [gridName=" + gridName + ", cacheName=" + cacheName +
-                ", maxRetriesOnFail=" + retries + ']');
-
-        txEnabled = cacheCfg.getAtomicityMode() == TRANSACTIONAL;
-
-        lsnr = new WebSessionListener(webSesIgnite, cache, retries);
+        lsnr = new WebSessionListener(webSesIgnite, this, retries);
 
         String srvInfo = ctx.getServerInfo();
 
@@ -286,6 +297,46 @@ public class WebSessionFilter implements Filter {
                 }
             };
         }
+
+        if (log.isInfoEnabled())
+            log.info("Started web sessions caching [gridName=" + gridName + ", cacheName=" + cacheName +
+                ", maxRetriesOnFail=" + retries + ']');
+    }
+
+    /**
+     * @return Cache.
+     */
+    IgniteCache<String, WebSession> getCache(){
+        return cache;
+    }
+
+    /**
+     * Init cache.
+     */
+    void initCache() {
+        cache = webSesIgnite.cache(cacheName);
+
+        if (cache == null)
+            throw new IgniteException("Cache for web sessions is not started (is it configured?): " + cacheName);
+
+        CacheConfiguration cacheCfg = cache.getConfiguration(CacheConfiguration.class);
+
+        if (cacheCfg.getWriteSynchronizationMode() == FULL_ASYNC)
+            throw new IgniteException("Cache for web sessions cannot be in FULL_ASYNC mode: " + cacheName);
+
+        if (!cacheCfg.isEagerTtl())
+            throw new IgniteException("Cache for web sessions cannot operate with lazy TTL. " +
+                "Consider setting eagerTtl to true for cache: " + cacheName);
+
+        if (cacheCfg.getCacheMode() == LOCAL)
+            U.quietAndWarn(webSesIgnite.log(), "Using LOCAL cache for web sessions caching " +
+                "(this is only OK in test mode): " + cacheName);
+
+        if (cacheCfg.getCacheMode() == PARTITIONED && cacheCfg.getAtomicityMode() != ATOMIC)
+            U.quietAndWarn(webSesIgnite.log(), "Using " + cacheCfg.getAtomicityMode() + " atomicity for web sessions " +
+                "caching (switch to ATOMIC mode for better performance)");
+
+        txEnabled = cacheCfg.getAtomicityMode() == TRANSACTIONAL;
     }
 
     /** {@inheritDoc} */
@@ -333,7 +384,7 @@ public class WebSessionFilter implements Filter {
      */
     private String doFilter0(HttpServletRequest httpReq, ServletResponse res, FilterChain chain) throws IOException,
         ServletException, CacheException {
-        WebSession cached;
+        WebSession cached = null;
 
         String sesId = httpReq.getRequestedSessionId();
 
@@ -341,7 +392,24 @@ public class WebSessionFilter implements Filter {
             if (sesIdTransformer != null)
                 sesId = sesIdTransformer.apply(sesId);
 
-            cached = cache.get(sesId);
+            for (int i = 0; i < retries; i++) {
+                try {
+                    cached = cache.get(sesId);
+                }
+                catch (CacheException | IgniteException | IllegalStateException e) {
+                    if (log.isDebugEnabled())
+                        log.debug(e.getMessage());
+
+                    if (i == retries - 1)
+                        throw new IgniteException("Failed to handle request [session= " + sesId + "]", e);
+                    else {
+                        if (log.isDebugEnabled())
+                            log.debug("Failed to handle request (will retry): " + sesId);
+
+                        handleCacheOperationException(e);
+                    }
+                }
+            }
 
             if (cached != null) {
                 if (log.isDebugEnabled())
@@ -455,7 +523,7 @@ public class WebSessionFilter implements Filter {
 
                 break;
             }
-            catch (CacheException | IgniteException e) {
+            catch (CacheException | IgniteException | IllegalStateException e) {
                 if (log.isDebugEnabled())
                     log.debug(e.getMessage());
 
@@ -465,29 +533,49 @@ public class WebSessionFilter implements Filter {
                     if (log.isDebugEnabled())
                         log.debug("Failed to save session (will retry): " + sesId);
 
-                    IgniteFuture<?> retryFut = null;
+                    handleCacheOperationException(e);
+                }
+            }
+        }
 
-                    if (X.hasCause(e, ClusterTopologyException.class)) {
-                        ClusterTopologyException cause = X.cause(e, ClusterTopologyException.class);
+        return cached;
+    }
 
-                        assert cause != null : e;
+    /**
+     * Handles cache operation exception.
+     * @param e Exception
+     */
+    void handleCacheOperationException(Exception e){
+        IgniteFuture<?> retryFut = null;
 
-                        retryFut = cause.retryReadyFuture();
-                    }
+        if (e instanceof IllegalStateException) {
+            initCache();
 
-                    if (retryFut != null) {
-                        try {
-                            retryFut.get();
-                        }
-                        catch (IgniteException retryErr) {
-                            throw new IgniteException("Failed to save session: " + sesId, retryErr);
-                        }
-                    }
-                }
-            }
+            return;
         }
+        else if (X.hasCause(e, IgniteClientDisconnectedException.class)) {
+            IgniteClientDisconnectedException cause = X.cause(e, IgniteClientDisconnectedException.class);
 
-        return cached;
+            assert cause != null : e;
+
+            retryFut = cause.reconnectFuture();
+        }
+        else if (X.hasCause(e, ClusterTopologyException.class)) {
+            ClusterTopologyException cause = X.cause(e, ClusterTopologyException.class);
+
+            assert cause != null : e;
+
+            retryFut = cause.retryReadyFuture();
+        }
+
+        if (retryFut != null) {
+            try {
+                retryFut.get(retriesTimeout);
+            }
+            catch (IgniteException retryErr) {
+                throw new IgniteException("Failed to wait for retry: " + retryErr);
+            }
+        }
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ad4f395/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionListener.java b/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionListener.java
index 0d7c44e..0d8ffec 100644
--- a/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionListener.java
+++ b/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionListener.java
@@ -32,12 +32,9 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cluster.ClusterTopologyException;
 import org.apache.ignite.internal.util.typedef.T2;
-import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteFuture;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 
@@ -45,11 +42,8 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS;
  * Session listener for web sessions caching.
  */
 class WebSessionListener {
-    /** */
-    private static final long RETRY_DELAY = 1000;
-
-    /** Cache. */
-    private final IgniteCache<String, WebSession> cache;
+    /** Filter. */
+    private final WebSessionFilter filter;
 
     /** Maximum retries. */
     private final int retries;
@@ -59,14 +53,14 @@ class WebSessionListener {
 
     /**
      * @param ignite Grid.
-     * @param cache Cache.
+     * @param filter Filter.
      * @param retries Maximum retries.
      */
-    WebSessionListener(Ignite ignite, IgniteCache<String, WebSession> cache, int retries) {
+    WebSessionListener(Ignite ignite, WebSessionFilter filter, int retries) {
         assert ignite != null;
-        assert cache != null;
+        assert filter != null;
 
-        this.cache = cache;
+        this.filter = filter;
         this.retries = retries > 0 ? retries : 1;
 
         log = ignite.log();
@@ -77,13 +71,22 @@ class WebSessionListener {
      */
     public void destroySession(String sesId) {
         assert sesId != null;
+        for (int i = 0; i < retries; i++) {
+            try {
+                if (filter.getCache().remove(sesId) && log.isDebugEnabled())
+                    log.debug("Session destroyed: " + sesId);
+            }
+            catch (CacheException | IgniteException | IllegalStateException e) {
+                if (i == retries - 1) {
+                    U.warn(log, "Failed to remove session [sesId=" +
+                        sesId + ", retries=" + retries + ']');
+                }
+                else {
+                    U.warn(log, "Failed to remove session (will retry): " + sesId);
 
-        try {
-            if (cache.remove(sesId) && log.isDebugEnabled())
-                log.debug("Session destroyed: " + sesId);
-        }
-        catch (CacheException e) {
-            U.error(log, "Failed to remove session: " + sesId, e);
+                    filter.handleCacheOperationException(e);
+                }
+            }
         }
     }
 
@@ -110,16 +113,16 @@ class WebSessionListener {
 
                         ExpiryPolicy plc = new ModifiedExpiryPolicy(new Duration(MILLISECONDS, ttl));
 
-                        cache0 = cache.withExpiryPolicy(plc);
+                        cache0 = filter.getCache().withExpiryPolicy(plc);
                     }
                     else
-                        cache0 = cache;
+                        cache0 = filter.getCache();
 
                     cache0.invoke(sesId, new AttributesProcessor(updates));
 
                     break;
                 }
-                catch (CacheException | IgniteException e) {
+                catch (CacheException | IgniteException | IllegalStateException e) {
                     if (i == retries - 1) {
                         U.warn(log, "Failed to apply updates for session (maximum number of retries exceeded) [sesId=" +
                             sesId + ", retries=" + retries + ']');
@@ -127,20 +130,7 @@ class WebSessionListener {
                     else {
                         U.warn(log, "Failed to apply updates for session (will retry): " + sesId);
 
-                        IgniteFuture<?> retryFut = null;
-
-                        if (X.hasCause(e, ClusterTopologyException.class)) {
-                            ClusterTopologyException cause = X.cause(e, ClusterTopologyException.class);
-
-                            assert cause != null : e;
-
-                            retryFut = cause.retryReadyFuture();
-                        }
-
-                        if (retryFut != null)
-                            retryFut.get();
-                        else
-                            U.sleep(RETRY_DELAY);
+                        filter.handleCacheOperationException(e);
                     }
                 }
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ad4f395/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java
index 298baf1..8706e1f 100644
--- a/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java
+++ b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java
@@ -37,6 +37,7 @@ import javax.servlet.http.HttpSession;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.events.Event;
+import org.apache.ignite.Ignition;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.X;
@@ -83,6 +84,77 @@ public class WebSessionSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectRequest() throws Exception {
+        testClientReconnectRequest("/modules/core/src/test/config/websession/example-cache.xml",
+            "/modules/core/src/test/config/websession/example-cache2.xml",
+            "/modules/core/src/test/config/websession/example-cache-client.xml");
+    }
+
+    /**
+     * Tests single request to a server. Checks the presence of session in cache.
+     *
+     * @param srvCfg Server configuration.
+     * @param clientCfg Client configuration.
+     * @throws Exception If failed.
+     */
+    private void testClientReconnectRequest(String srvCfg, String srvCfg2, String clientCfg) throws Exception {
+        Server srv = null;
+
+        Ignite ignite = Ignition.start(srvCfg);
+
+        try {
+            srv = startServer(TEST_JETTY_PORT, clientCfg, "client", new SessionCreateServlet());
+
+            URL url = new URL("http://localhost:" + TEST_JETTY_PORT + "/ignitetest/test");
+
+            URLConnection conn = url.openConnection();
+
+            conn.connect();
+
+            try (BufferedReader rdr = new BufferedReader(new InputStreamReader(conn.getInputStream()))) {
+                String sesId = rdr.readLine();
+
+                assertNotNull(sesId);
+            }
+
+            stopGrid(ignite.name());
+
+            ignite = Ignition.start(srvCfg);
+
+            conn = url.openConnection();
+
+            conn.connect();
+
+            try (BufferedReader rdr = new BufferedReader(new InputStreamReader(conn.getInputStream()))) {
+                String sesId = rdr.readLine();
+
+                assertNotNull(sesId);
+            }
+
+            Ignite ignite2 = Ignition.start(srvCfg2);
+
+            stopGrid(ignite.name());
+
+            conn = url.openConnection();
+
+            conn.connect();
+
+            try (BufferedReader rdr = new BufferedReader(new InputStreamReader(conn.getInputStream()))) {
+                String sesId = rdr.readLine();
+
+                assertNotNull(sesId);
+            }
+        }
+        finally {
+            stopServer(srv);
+
+            stopAllGrids();
+        }
+    }
+
+    /**
      * Tests single request to a server. Checks the presence of session in cache.
      *
      * @param cfg Configuration.


[29/35] ignite git commit: IGNITE-2763 GridDhtPartitionDemander fails with assertion on partition move

Posted by nt...@apache.org.
IGNITE-2763 GridDhtPartitionDemander fails with assertion on partition move


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/ec04f385
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/ec04f385
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/ec04f385

Branch: refs/heads/ignite-2791
Commit: ec04f385a1ded518fd44ba9469372da037fff0d0
Parents: b1d9e8b
Author: Anton Vinogradov <av...@apache.org>
Authored: Mon Mar 14 15:26:18 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Mon Mar 14 15:27:34 2016 +0300

----------------------------------------------------------------------
 .../distributed/dht/preloader/GridDhtPartitionDemander.java  | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ec04f385/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index 1354d96..273b603 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -549,6 +549,8 @@ public class GridDhtPartitionDemander {
 
                     assert part != null;
 
+                    boolean last = supply.last().contains(p);
+
                     if (part.state() == MOVING) {
                         boolean reserved = part.reserve();
 
@@ -578,8 +580,6 @@ public class GridDhtPartitionDemander {
                                 }
                             }
 
-                            boolean last = supply.last().contains(p);
-
                             // If message was last for this partition,
                             // then we take ownership.
                             if (last) {
@@ -597,7 +597,9 @@ public class GridDhtPartitionDemander {
                         }
                     }
                     else {
-                        fut.partitionDone(id, p);
+                        if (last) {
+                            fut.partitionDone(id, p);
+                        }
 
                         if (log.isDebugEnabled())
                             log.debug("Skipping rebalancing partition (state is not MOVING): " + part);


[19/35] ignite git commit: Added tests.

Posted by nt...@apache.org.
Added tests.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/21cfc2bf
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/21cfc2bf
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/21cfc2bf

Branch: refs/heads/ignite-2791
Commit: 21cfc2bf613e66f7493012ed675bac1eac4d084e
Parents: 67db264
Author: sboikov <sb...@gridgain.com>
Authored: Fri Mar 11 12:01:42 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Mar 11 12:01:42 2016 +0300

----------------------------------------------------------------------
 .../IgniteDynamicClientCacheStartSelfTest.java  | 29 ++++++++++++++++----
 1 file changed, 24 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/21cfc2bf/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
index cc7120e..b661b52 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
@@ -257,7 +257,22 @@ public class IgniteDynamicClientCacheStartSelfTest extends GridCommonAbstractTes
     /**
      * @throws Exception If failed.
      */
-    public void testCreateCloseClientCache2() throws Exception {
+    public void testCreateCloseClientCache2_1() throws Exception {
+        createCloseClientCache2(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCreateCloseClientCache2_2() throws Exception {
+        createCloseClientCache2(true);
+    }
+
+    /**
+     * @param createFromCacheClient If {@code true} creates cache from cache client node.
+     * @throws Exception If failed.
+     */
+    private void createCloseClientCache2(boolean createFromCacheClient) throws Exception {
         Ignite ignite0 = startGrid(0);
 
         Ignite ignite1 = startGrid(1);
@@ -266,15 +281,19 @@ public class IgniteDynamicClientCacheStartSelfTest extends GridCommonAbstractTes
 
         ccfg.setNodeFilter(new CachePredicate(F.asList(ignite0.name())));
 
-        ignite1.createCache(ccfg);
+        if (createFromCacheClient)
+            ignite0.createCache(ccfg);
+        else {
+            ignite1.createCache(ccfg);
 
-        assertNull(((IgniteKernal)ignite0).context().cache().internalCache(null));
+            assertNull(((IgniteKernal)ignite0).context().cache().internalCache(null));
+        }
 
-        ignite0.cache(null);
+        assertNotNull(ignite0.cache(null));
 
         ignite0.cache(null).close();
 
-        ignite0.cache(null);
+        assertNotNull(ignite0.cache(null));
 
         startGrid(2);
 


[20/35] ignite git commit: IGNITE-2690: Format of uptime for metrics

Posted by nt...@apache.org.
IGNITE-2690: Format of uptime for metrics


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/06f0c915
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/06f0c915
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/06f0c915

Branch: refs/heads/ignite-2791
Commit: 06f0c915bcc76ebf1e2fed74c8ad17d4f8031a59
Parents: 21cfc2b
Author: Alper Tekinalp <al...@gmail.com>
Authored: Fri Mar 11 12:43:53 2016 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Fri Mar 11 12:43:53 2016 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/internal/util/typedef/X.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/06f0c915/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java b/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java
index 65b89bf..174ab86 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java
@@ -213,7 +213,7 @@ public final class X {
         return (t[3] < 10 ? "0" + t[3] : Long.toString(t[3])) + ':' +
             (t[2] < 10 ? "0" + t[2] : Long.toString(t[2])) + ':' +
             (t[1] < 10 ? "0" + t[1] : Long.toString(t[1])) + ':' +
-            (t[0] < 10 ? "0" + t[0] : Long.toString(t[0]));
+            (t[0] < 10 ? "00" + t[0] : ( t[0] < 100 ? "0" + t[0] : Long.toString(t[0])));
     }
 
     /**


[05/35] ignite git commit: IGNITE-2758 - Fixed ignite-aws dependencies

Posted by nt...@apache.org.
IGNITE-2758 - Fixed ignite-aws dependencies


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/c31f384d
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/c31f384d
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/c31f384d

Branch: refs/heads/ignite-2791
Commit: c31f384d56e5939b9ae8bf45fa30184f3a14bc50
Parents: 334ce4c
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Wed Mar 9 18:06:44 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Mar 9 18:06:44 2016 -0800

----------------------------------------------------------------------
 modules/aws/pom.xml | 24 +++++++++++++++++++++---
 1 file changed, 21 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c31f384d/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index 4de5506..481e897 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -43,23 +43,41 @@
 
         <dependency>
             <groupId>com.amazonaws</groupId>
-            <artifactId>aws-java-sdk</artifactId>
+            <artifactId>aws-java-sdk-core</artifactId>
             <version>${aws.sdk.version}</version>
         </dependency>
 
         <dependency>
             <groupId>com.amazonaws</groupId>
-            <artifactId>aws-java-sdk-core</artifactId>
+            <artifactId>aws-java-sdk-s3</artifactId>
             <version>${aws.sdk.version}</version>
         </dependency>
 
         <dependency>
             <groupId>com.amazonaws</groupId>
-            <artifactId>aws-java-sdk-s3</artifactId>
+            <artifactId>aws-java-sdk-kms</artifactId>
             <version>${aws.sdk.version}</version>
         </dependency>
 
         <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-core</artifactId>
+            <version>2.5.3</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-annotations</artifactId>
+            <version>2.5.3</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>2.5.3</version>
+        </dependency>
+
+        <dependency>
             <groupId>joda-time</groupId>
             <artifactId>joda-time</artifactId>
             <version>2.8.1</version>


[31/35] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by nt...@apache.org.
Merge remote-tracking branch 'origin/master'


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/c9b4a119
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/c9b4a119
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/c9b4a119

Branch: refs/heads/ignite-2791
Commit: c9b4a11908b964d1d7d317fbc290a3e1c8f4ff48
Parents: 09b2895 ec04f38
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Mar 14 15:57:43 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Mar 14 15:57:43 2016 +0300

----------------------------------------------------------------------
 .../distributed/dht/preloader/GridDhtPartitionDemander.java  | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[25/35] ignite git commit: IGNITE-2781: IGFS: Force "copyOnRead=false" for meta and data caches.

Posted by nt...@apache.org.
IGNITE-2781: IGFS: Force "copyOnRead=false" for meta and data caches.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/c53ffa98
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/c53ffa98
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/c53ffa98

Branch: refs/heads/ignite-2791
Commit: c53ffa98cf491a747a66d532501a806dcb561840
Parents: 2a3a565
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Mar 14 10:19:23 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Mar 14 10:19:23 2016 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/internal/IgnitionEx.java    | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c53ffa98/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 193e28e..4796581 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1949,6 +1949,15 @@ public class IgnitionEx {
             }
 
             cfg.setCacheConfiguration(cacheCfgs.toArray(new CacheConfiguration[cacheCfgs.size()]));
+
+            // Iterate over IGFS caches and set "copyOnRead" flag to "false". Note that we do this after cloning
+            // to leave user object unchanged.
+            assert cfg.getCacheConfiguration() != null;
+
+            for (CacheConfiguration ccfg : cfg.getCacheConfiguration()) {
+                if (CU.isIgfsCache(cfg, ccfg.getName()))
+                    ccfg.setCopyOnRead(false);
+            }
         }
 
         /**


[08/35] ignite git commit: IGNITE-2775: Fixed HttpRequest.changeSessionId() to create a new id. - Fixes #539.

Posted by nt...@apache.org.
IGNITE-2775: Fixed HttpRequest.changeSessionId() to create a new id. - Fixes #539.

Signed-off-by: shtykh_roman <rs...@yahoo.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/a8d16bd1
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/a8d16bd1
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/a8d16bd1

Branch: refs/heads/ignite-2791
Commit: a8d16bd1fbdd59d76454dbbce834f9ab87b54724
Parents: 5377dde
Author: shtykh_roman <rs...@yahoo.com>
Authored: Thu Mar 10 16:38:10 2016 +0900
Committer: shtykh_roman <rs...@yahoo.com>
Committed: Thu Mar 10 16:38:10 2016 +0900

----------------------------------------------------------------------
 .../ignite/cache/websession/WebSession.java     |  24 +++-
 .../cache/websession/WebSessionFilter.java      |  41 +++++-
 .../internal/websession/WebSessionSelfTest.java | 136 +++++++++++++++++++
 3 files changed, 193 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a8d16bd1/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSession.java
----------------------------------------------------------------------
diff --git a/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSession.java b/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSession.java
index 8b944e5..5e0d49b 100644
--- a/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSession.java
+++ b/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSession.java
@@ -90,7 +90,7 @@ class WebSession implements HttpSession, Externalizable {
     private transient Collection<T2<String, Object>> updates;
 
     /** Genuine http session. */
-    private transient HttpSession genuineSession;
+    private transient HttpSession genSes;
 
     /**
      * Required by {@link Externalizable}.
@@ -123,8 +123,6 @@ class WebSession implements HttpSession, Externalizable {
 
             attrs.put(name, ses.getAttribute(name));
         }
-
-        genuineSession = ses;
     }
 
     /**
@@ -139,6 +137,15 @@ class WebSession implements HttpSession, Externalizable {
     }
 
     /**
+     * Sets the genuine http session.
+     *
+     * @param genSes Genuine http session.
+     */
+    protected void genSes(HttpSession genSes) {
+        this.genSes = genSes;
+    }
+
+    /**
      * @param ctx Servlet context.
      */
     public void servletContext(ServletContext ctx) {
@@ -188,6 +195,15 @@ class WebSession implements HttpSession, Externalizable {
         return id;
     }
 
+    /**
+     * Sets a session id.
+     *
+     * @param id Session id.
+     */
+    protected void setId(String id) {
+        this.id = id;
+    }
+
     /** {@inheritDoc} */
     @Override public ServletContext getServletContext() {
         return ctx;
@@ -291,7 +307,7 @@ class WebSession implements HttpSession, Externalizable {
 
         lsnr.destroySession(id);
 
-        genuineSession.invalidate();
+        genSes.invalidate();
 
         isValid = false;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a8d16bd1/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionFilter.java
----------------------------------------------------------------------
diff --git a/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionFilter.java b/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionFilter.java
index 3dedee3..42de43b 100644
--- a/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionFilter.java
+++ b/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionFilter.java
@@ -379,6 +379,7 @@ public class WebSessionFilter implements Filter {
         cached.servletContext(ctx);
         cached.listener(lsnr);
         cached.resetUpdates();
+        cached.genSes(httpReq.getSession(false));
 
         httpReq = new RequestWrapper(httpReq, cached);
 
@@ -399,8 +400,10 @@ public class WebSessionFilter implements Filter {
     }
 
     /**
-     * @param httpReq HTTP request.
-     * @return Cached session.
+     * Creates a new session from http request.
+     *
+     * @param httpReq Request.
+     * @return New session.
      */
     @SuppressWarnings("unchecked")
     private WebSession createSession(HttpServletRequest httpReq) {
@@ -408,11 +411,25 @@ public class WebSessionFilter implements Filter {
 
         String sesId = sesIdTransformer != null ? sesIdTransformer.apply(ses.getId()) : ses.getId();
 
-        if (log.isDebugEnabled())
-            log.debug("Session created: " + sesId);
+        return createSession(ses, sesId);
+    }
 
+    /**
+     * Creates a new web session with the specified id.
+     *
+     * @param ses Base session.
+     * @param sesId Session id.
+     * @return New session.
+     */
+    @SuppressWarnings("unchecked")
+    private WebSession createSession(HttpSession ses, String sesId) {
         WebSession cached = new WebSession(sesId, ses, true);
 
+        cached.genSes(ses);
+
+        if (log.isDebugEnabled())
+            log.debug("Session created: " + sesId);
+
         for (int i = 0; i < retries; i++) {
             try {
                 IgniteCache<String, WebSession> cache0;
@@ -517,5 +534,21 @@ public class WebSessionFilter implements Filter {
         @Override public HttpSession getSession() {
             return getSession(true);
         }
+
+        /** {@inheritDoc} */
+        @Override public String changeSessionId() {
+            HttpServletRequest req = (HttpServletRequest)getRequest();
+
+            String newId = req.changeSessionId();
+
+            this.ses.setId(newId);
+
+            this.ses = createSession(ses, newId);
+            this.ses.servletContext(ctx);
+            this.ses.listener(lsnr);
+            this.ses.resetUpdates();
+
+            return newId;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a8d16bd1/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java
index e2fda37..298baf1 100644
--- a/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java
+++ b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java
@@ -202,6 +202,88 @@ public class WebSessionSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Tests session id change.
+     *
+     * @throws Exception Exception If failed.
+     */
+    public void testChangeSessionId() throws Exception {
+        String newWebSesId;
+        Server srv = null;
+
+        try {
+            srv = startServer(TEST_JETTY_PORT, "/modules/core/src/test/config/websession/example-cache.xml",
+                null, new SessionIdChangeServlet());
+
+            Ignite ignite = G.ignite();
+
+            URLConnection conn = new URL("http://localhost:" + TEST_JETTY_PORT + "/ignitetest/chngsesid").openConnection();
+
+            conn.connect();
+
+            try (BufferedReader rdr = new BufferedReader(new InputStreamReader(conn.getInputStream()))) {
+
+                // checks if the old session object is invalidated.
+                String oldId = rdr.readLine();
+
+                assertNotNull(oldId);
+
+                // id from genuine session
+                String newGenSesId = rdr.readLine();
+
+                assertNotNull(newGenSesId);
+
+                assertFalse(newGenSesId.equals(oldId));
+
+                // id from replicated session
+                newWebSesId = rdr.readLine();
+
+                assertNotNull(newWebSesId);
+
+                assertTrue(newGenSesId.equals(newWebSesId));
+
+                IgniteCache<String, HttpSession> cache = ignite.cache(getCacheName());
+
+                assertNotNull(cache);
+
+                Thread.sleep(1000);
+
+                HttpSession ses = cache.get(newWebSesId);
+
+                assertNotNull(ses);
+
+                assertEquals("val1", ses.getAttribute("key1"));
+            }
+
+            conn = new URL("http://localhost:" + TEST_JETTY_PORT + "/ignitetest/simple").openConnection();
+
+            conn.addRequestProperty("Cookie", "JSESSIONID=" + newWebSesId);
+
+            conn.connect();
+
+            try (BufferedReader rdr = new BufferedReader(new InputStreamReader(conn.getInputStream()))) {
+
+                // checks if it can be handled with the subsequent request.
+                String sesId = rdr.readLine();
+
+                assertTrue(newWebSesId.equals(sesId));
+
+                String attr = rdr.readLine();
+
+                assertEquals("val1", attr);
+
+                String reqSesValid = rdr.readLine();
+
+                assertEquals("true", reqSesValid);
+
+                assertEquals("invalidated", rdr.readLine());
+            }
+        }
+        finally {
+            stopServer(srv);
+        }
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testRestarts() throws Exception {
@@ -404,6 +486,9 @@ public class WebSessionSelfTest extends GridCommonAbstractTest {
                 ses.invalidate();
 
                 res.getWriter().println(ses.getId());
+
+                // invalidates again.
+                req.getSession().invalidate();
             }
             else if (req.getPathInfo().equals("/valid")) {
                 X.println(">>>", "Created session: " + ses.getId(), ">>>");
@@ -418,6 +503,57 @@ public class WebSessionSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Test session behavior on id change.
+     */
+    private static class SessionIdChangeServlet extends HttpServlet {
+        /** {@inheritDoc} */
+        @Override protected void doGet(HttpServletRequest req, HttpServletResponse res)
+            throws ServletException, IOException {
+            HttpSession ses = req.getSession();
+
+            assertNotNull(ses);
+
+            if (req.getPathInfo().equals("/chngsesid")) {
+
+                ses.setAttribute("key1", "val1");
+
+                X.println(">>>", "Created session: " + ses.getId(), ">>>");
+
+                res.getWriter().println(req.getSession().getId());
+
+                String newId = req.changeSessionId();
+
+                // new id from genuine session.
+                res.getWriter().println(newId);
+
+                // new id from WebSession.
+                res.getWriter().println(req.getSession().getId());
+
+                res.getWriter().flush();
+            }
+            else if (req.getPathInfo().equals("/simple")) {
+                res.getWriter().println(req.getSession().getId());
+
+                res.getWriter().println(req.getSession().getAttribute("key1"));
+
+                res.getWriter().println(req.isRequestedSessionIdValid());
+
+                try {
+                    req.getSession().invalidate();
+                    res.getWriter().println("invalidated");
+                }
+                catch (Exception e) {
+                    res.getWriter().println("failed");
+                }
+
+                res.getWriter().flush();
+            }
+            else
+                throw new ServletException("Nonexisting path: " + req.getPathInfo());
+        }
+    }
+
+    /**
      * Servlet for restarts test.
      */
     private static class RestartsTestServlet extends HttpServlet {


[16/35] ignite git commit: Upgrade Apache Commons Collections to v3.2.2

Posted by nt...@apache.org.
Upgrade Apache Commons Collections to v3.2.2


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/053af5de
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/053af5de
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/053af5de

Branch: refs/heads/ignite-2791
Commit: 053af5de5c30696a2a6060cc3c51d1573995a758
Parents: a895b8e
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Mar 10 18:41:37 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Mar 10 18:41:37 2016 +0300

----------------------------------------------------------------------
 parent/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/053af5de/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 9d6c39f..a52ac12 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -57,7 +57,7 @@
         <commons.beanutils.bundle.version>1.8.3_1</commons.beanutils.bundle.version>
         <commons.beanutils.version>1.8.3</commons.beanutils.version>
         <commons.codec.version>1.6</commons.codec.version>
-        <commons.collections.version>3.2.1</commons.collections.version>
+        <commons.collections.version>3.2.2</commons.collections.version>
         <commons.lang.version>2.6</commons.lang.version>
         <cron4j.version>2.2.5</cron4j.version>
         <curator.version>2.9.1</curator.version>


[03/35] ignite git commit: IGNITE-2621: Correct handling for tasks in mixed-platform cluster - minors

Posted by nt...@apache.org.
IGNITE-2621: Correct handling for tasks in mixed-platform cluster - minors


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/157ef53e
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/157ef53e
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/157ef53e

Branch: refs/heads/ignite-2791
Commit: 157ef53ea3d826db8c2455658d8694d60472e80d
Parents: c6c9389
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Wed Mar 9 18:12:39 2016 +0300
Committer: Pavel Tupitsyn <pt...@gridgain.com>
Committed: Wed Mar 9 18:12:39 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/platform/PlatformProcessorImpl.java   | 2 +-
 .../internal/processors/platform/compute/PlatformCompute.java | 7 +++++--
 2 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/157ef53e/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
index 95daa4d..ff7bbd7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
@@ -303,7 +303,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     @Override public PlatformTarget compute(PlatformTarget grp) {
         PlatformClusterGroup grp0 = (PlatformClusterGroup)grp;
 
-        return new PlatformCompute(platformCtx, grp0.projection());
+        return new PlatformCompute(platformCtx, grp0.projection(), PlatformUtils.ATTR_PLATFORM);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/157ef53e/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
index a1a82ae..9eb746c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
@@ -78,12 +78,15 @@ public class PlatformCompute extends PlatformAbstractTarget {
      * @param platformCtx Context.
      * @param grp Cluster group.
      */
-    public PlatformCompute(PlatformContext platformCtx, ClusterGroup grp) {
+    public PlatformCompute(PlatformContext platformCtx, ClusterGroup grp, String platformAttr) {
         super(platformCtx);
 
+        assert grp != null;
+        assert platformAttr != null;
+
         compute = (IgniteComputeImpl)grp.ignite().compute(grp);
 
-        ClusterGroup platformGrp = grp.forAttribute(PlatformUtils.ATTR_PLATFORM, platformCtx.platform());
+        ClusterGroup platformGrp = grp.forAttribute(platformAttr, platformCtx.platform());
 
         computeForPlatform = (IgniteComputeImpl)grp.ignite().compute(platformGrp);
     }


[24/35] ignite git commit: Fixed ClassNotFoundException for void.class

Posted by nt...@apache.org.
Fixed ClassNotFoundException for void.class


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/2a3a5651
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/2a3a5651
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/2a3a5651

Branch: refs/heads/ignite-2791
Commit: 2a3a5651e929121e493ceb49c93a933a38035b16
Parents: 0ad4f39
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Fri Mar 11 14:35:14 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Fri Mar 11 14:35:14 2016 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/internal/util/IgniteUtils.java | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2a3a5651/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 35722e9..7b2414f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -589,6 +589,7 @@ public abstract class IgniteUtils {
         primitiveMap.put("double", double.class);
         primitiveMap.put("char", char.class);
         primitiveMap.put("boolean", boolean.class);
+        primitiveMap.put("void", void.class);
 
         boxedClsMap.put(byte.class, Byte.class);
         boxedClsMap.put(short.class, Short.class);
@@ -598,6 +599,7 @@ public abstract class IgniteUtils {
         boxedClsMap.put(double.class, Double.class);
         boxedClsMap.put(char.class, Character.class);
         boxedClsMap.put(boolean.class, Boolean.class);
+        boxedClsMap.put(void.class, Void.class);
 
         try {
             OBJECT_CTOR = Object.class.getConstructor();


[13/35] ignite git commit: IGNITE-2700: Closures are now written using binary marshaller. This closes #518.

Posted by nt...@apache.org.
IGNITE-2700: Closures are now written using binary marshaller. This closes #518.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/215e8a1e
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/215e8a1e
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/215e8a1e

Branch: refs/heads/ignite-2791
Commit: 215e8a1e3143bf22d792fdefbd4e6a65b372ae24
Parents: 8cf71d4
Author: Ilya Lantukh <il...@gridgain.com>
Authored: Thu Mar 10 15:37:14 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Mar 10 15:37:14 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/binary/BinaryContext.java   |  29 +-
 .../internal/binary/BinaryFieldAccessor.java    |   1 +
 .../closure/GridClosureProcessor.java           | 375 ++++++++++++++++-
 .../resources/META-INF/classnames.properties    |   6 +
 .../ignite/internal/GridAffinitySelfTest.java   |   4 +-
 ...omputationBinarylizableClosuresSelfTest.java | 413 +++++++++++++++++++
 .../binary/BinaryMarshallerSelfTest.java        |  50 +++
 ...IgniteBinaryObjectsComputeGridTestSuite.java |   7 +-
 8 files changed, 858 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/215e8a1e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
index 4df9ba2..b9b633f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -61,6 +61,7 @@ import org.apache.ignite.cache.affinity.AffinityKeyMapped;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.binary.BinaryMetadataKey;
+import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
 import org.apache.ignite.internal.processors.datastructures.CollocatedQueueItemKey;
 import org.apache.ignite.internal.processors.datastructures.CollocatedSetItemKey;
 import org.apache.ignite.internal.util.IgniteUtils;
@@ -89,6 +90,23 @@ public class BinaryContext {
     static final BinaryInternalMapper SIMPLE_NAME_LOWER_CASE_MAPPER =
         new BinaryInternalMapper(new BinaryBasicNameMapper(true), new BinaryBasicIdMapper(true), false);
 
+    /** Set of system classes that should be marshalled with BinaryMarshaller. */
+    private static final Set<String> BINARYLIZABLE_SYS_CLSS;
+
+    /** Binarylizable system classes set initialization. */
+    static {
+        Set<String> sysClss = new HashSet<>();
+
+        sysClss.add(GridClosureProcessor.C1V2.class.getName());
+        sysClss.add(GridClosureProcessor.C1MLAV2.class.getName());
+        sysClss.add(GridClosureProcessor.C2V2.class.getName());
+        sysClss.add(GridClosureProcessor.C2MLAV2.class.getName());
+        sysClss.add(GridClosureProcessor.C4V2.class.getName());
+        sysClss.add(GridClosureProcessor.C4MLAV2.class.getName());
+
+        BINARYLIZABLE_SYS_CLSS = Collections.unmodifiableSet(sysClss);
+    }
+
     /** */
     private final ConcurrentMap<Class<?>, BinaryClassDescriptor> descByCls = new ConcurrentHashMap8<>();
 
@@ -255,7 +273,7 @@ public class BinaryContext {
     /**
      * @return Ignite configuration.
      */
-    public IgniteConfiguration configuration(){
+    public IgniteConfiguration configuration() {
         return igniteCfg;
     }
 
@@ -587,6 +605,11 @@ public class BinaryContext {
         String clsName = cls.getName();
 
         if (marshCtx.isSystemType(clsName)) {
+            BinarySerializer serializer = null;
+
+            if (BINARYLIZABLE_SYS_CLSS.contains(clsName))
+                serializer = new BinaryReflectiveSerializer();
+
             desc = new BinaryClassDescriptor(this,
                 cls,
                 false,
@@ -594,7 +617,7 @@ public class BinaryContext {
                 clsName,
                 null,
                 SIMPLE_NAME_LOWER_CASE_MAPPER,
-                null,
+                serializer,
                 false,
                 true /* registered */
             );
@@ -775,7 +798,7 @@ public class BinaryContext {
 
         if (prevMap != null && !mapper.equals(prevMap))
             throw new IgniteException("Different mappers [clsName=" + clsName + ", newMapper=" + mapper
-            + ", prevMap=" + prevMap + "]");
+                + ", prevMap=" + prevMap + "]");
 
         prevMap = typeId2Mapper.putIfAbsent(mapper.typeId(clsName), mapper);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/215e8a1e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java
index 8c8bf27..af33b63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java
@@ -619,6 +619,7 @@ public abstract class BinaryFieldAccessor {
 
                 case BINARY:
                 case OBJECT:
+                case PROXY:
                     writer.writeObjectField(val);
 
                     break;

http://git-wip-us.apache.org/repos/asf/ignite/blob/215e8a1e/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
index 043f754..c6883dc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
@@ -31,6 +31,12 @@ import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.TimeUnit;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.compute.ComputeJob;
 import org.apache.ignite.compute.ComputeJobMasterLeaveAware;
@@ -61,6 +67,7 @@ import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.internal.util.worker.GridWorkerFuture;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteClosure;
+import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.lang.IgniteReducer;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.resources.LoadBalancerResource;
@@ -76,6 +83,9 @@ import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKe
  *
  */
 public class GridClosureProcessor extends GridProcessorAdapter {
+    /** Ignite version in which binarylizable versions of closures were introduced. */
+    public static final IgniteProductVersion BINARYLIZABLE_CLOSURES_SINCE = IgniteProductVersion.fromString("1.6.0");
+
     /** */
     private final Executor sysPool;
 
@@ -254,7 +264,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                     case BROADCAST: {
                         for (ClusterNode n : nodes)
                             for (Runnable r : jobs)
-                                mapper.map(job(r), n);
+                                mapper.map(downgradeJobIfNeeded(job(r), n), n);
 
                         break;
                     }
@@ -263,7 +273,9 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                         for (Runnable r : jobs) {
                             ComputeJob job = job(r);
 
-                            mapper.map(job, lb.getBalancedNode(job, null));
+                            ClusterNode n = lb.getBalancedNode(job, null);
+
+                            mapper.map(downgradeJobIfNeeded(job, n), n);
                         }
 
                         break;
@@ -306,7 +318,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                     case BROADCAST: {
                         for (ClusterNode n : nodes)
                             for (Callable<R> c : jobs)
-                                mapper.map(job(c), n);
+                                mapper.map(downgradeJobIfNeeded(job(c), n), n);
 
                         break;
                     }
@@ -315,7 +327,9 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                         for (Callable<R> c : jobs) {
                             ComputeJob job = job(c);
 
-                            mapper.map(job, lb.getBalancedNode(job, null));
+                            ClusterNode n = lb.getBalancedNode(job, null);
+
+                            mapper.map(downgradeJobIfNeeded(job, n), n);
                         }
 
                         break;
@@ -1025,7 +1039,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
     private static <T, R> ComputeJob job(final IgniteClosure<T, R> job, @Nullable final T arg) {
         A.notNull(job, "job");
 
-        return job instanceof ComputeJobMasterLeaveAware ? new C1MLA<>(job, arg) : new C1<>(job, arg);
+        return job instanceof ComputeJobMasterLeaveAware ? new C1MLAV2<>(job, arg) : new C1V2<>(job, arg);
     }
 
     /**
@@ -1037,7 +1051,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
     private static <R> ComputeJob job(final Callable<R> c) {
         A.notNull(c, "job");
 
-        return c instanceof ComputeJobMasterLeaveAware ? new C2MLA<>(c) : new C2<>(c);
+        return c instanceof ComputeJobMasterLeaveAware ? new C2MLAV2<>(c) : new C2V2<>(c);
     }
 
     /**
@@ -1049,7 +1063,46 @@ public class GridClosureProcessor extends GridProcessorAdapter {
     private static ComputeJob job(final Runnable r) {
         A.notNull(r, "job");
 
-       return r instanceof ComputeJobMasterLeaveAware ? new C4MLA(r) : new C4(r);
+        return r instanceof ComputeJobMasterLeaveAware ? new C4MLAV2(r) : new C4V2(r);
+    }
+
+    /**
+     * Downgrades provided job to older version if target does not support it.
+     *
+     * @param job Job.
+     * @param node Node.
+     * @return Provided or downgraded job.
+     */
+    private static ComputeJob downgradeJobIfNeeded(ComputeJob job, ClusterNode node) {
+        A.notNull(job, "job");
+
+        assert node != null;
+
+        IgniteProductVersion nodeVer = node.version();
+
+        if (nodeVer.compareTo(BINARYLIZABLE_CLOSURES_SINCE) >= 0)
+            return job;
+
+        if (job instanceof C1V2) {
+            if (job instanceof C1MLAV2)
+                return new C1MLA<>(((C1MLAV2)job).job, ((C1MLAV2)job).arg);
+            else
+                return new C1<>(((C1V2)job).job, ((C1V2)job).arg);
+        }
+        else if (job instanceof C2V2) {
+            if (job instanceof C2MLAV2)
+                return new C2MLA<>(((C2MLAV2)job).c);
+            else
+                return new C2<>(((C2V2)job).c);
+        }
+        else if (job instanceof C4V2) {
+            if (job instanceof C4MLAV2)
+                return new C4MLA(((C4MLAV2)job).r);
+            else
+                return new C4(((C4V2)job).r);
+        }
+
+        return job;
     }
 
     /**
@@ -1294,9 +1347,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg) {
-            ComputeJob job = job(this.job);
-
-            return Collections.singletonMap(job, node);
+            return Collections.singletonMap(downgradeJobIfNeeded(job(this.job), node), node);
         }
 
         /** {@inheritDoc} */
@@ -1348,9 +1399,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg) {
-            ComputeJob job = job(this.job);
-
-            return Collections.singletonMap(job, node);
+            return Collections.singletonMap(downgradeJobIfNeeded(job(this.job), node), node);
         }
 
         /** {@inheritDoc} */
@@ -1488,7 +1537,9 @@ public class GridClosureProcessor extends GridProcessorAdapter {
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg) {
             ComputeJob job = job(this.job, this.arg);
 
-            return Collections.singletonMap(job, lb.getBalancedNode(job, null));
+            ClusterNode node = lb.getBalancedNode(job, null);
+
+            return Collections.singletonMap(downgradeJobIfNeeded(job, node), node);
         }
 
         /** {@inheritDoc} */
@@ -1537,7 +1588,9 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                 for (T jobArg : args) {
                     ComputeJob job = job(this.job, jobArg);
 
-                    mapper.map(job, lb.getBalancedNode(job, null));
+                    ClusterNode node = lb.getBalancedNode(job, null);
+
+                    mapper.map(downgradeJobIfNeeded(job, node), node);
                 }
 
                 return mapper.map();
@@ -1593,7 +1646,9 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                 for (T jobArg : args) {
                     ComputeJob job = job(this.job, jobArg);
 
-                    mapper.map(job, lb.getBalancedNode(job, null));
+                    ClusterNode node = lb.getBalancedNode(job, null);
+
+                    mapper.map(downgradeJobIfNeeded(job, node), node);
                 }
 
                 return mapper.map();
@@ -1607,7 +1662,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
         @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) {
             ComputeJobResultPolicy resPlc = super.result(res, rcvd);
 
-            if (res.getException() == null && resPlc != FAILOVER && !rdc.collect((R1) res.getData()))
+            if (res.getException() == null && resPlc != FAILOVER && !rdc.collect((R1)res.getData()))
                 resPlc = REDUCE; // If reducer returned false - reduce right away.
 
             return resPlc;
@@ -1647,7 +1702,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                 JobMapper mapper = new JobMapper(subgrid.size());
 
                 for (ClusterNode n : subgrid)
-                    mapper.map(job(job, arg), n);
+                    mapper.map(downgradeJobIfNeeded(job(job, arg), n), n);
 
                 return mapper.map();
             }
@@ -1680,7 +1735,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
         /**
          *
          */
-        public C1(){
+        public C1() {
             // No-op.
         }
 
@@ -1729,6 +1784,72 @@ public class GridClosureProcessor extends GridProcessorAdapter {
     /**
      *
      */
+    public static class C1V2<T, R> implements ComputeJob, Binarylizable, GridNoImplicitInjection,
+        GridInternalWrapper<IgniteClosure> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        protected IgniteClosure<T, R> job;
+
+        /** */
+        @GridToStringInclude
+        protected T arg;
+
+        /**
+         *
+         */
+        public C1V2() {
+            // No-op.
+        }
+
+        /**
+         * @param job Job.
+         * @param arg Argument.
+         */
+        C1V2(IgniteClosure<T, R> job, T arg) {
+            this.job = job;
+            this.arg = arg;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Object execute() {
+            return job.apply(arg);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cancel() {
+            // No-op.
+        }
+
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            BinaryRawWriter rawWriter = writer.rawWriter();
+
+            rawWriter.writeObject(job);
+            rawWriter.writeObject(arg);
+        }
+
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            BinaryRawReader rawReader = reader.rawReader();
+
+            job = rawReader.readObject();
+            arg = rawReader.readObject();
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteClosure userObject() {
+            return job;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(C1V2.class, this);
+        }
+    }
+
+    /**
+     *
+     */
     private static class C1MLA<T, R> extends C1<T, R> implements ComputeJobMasterLeaveAware {
         /** */
         private static final long serialVersionUID = 0L;
@@ -1762,6 +1883,39 @@ public class GridClosureProcessor extends GridProcessorAdapter {
     /**
      *
      */
+    public static class C1MLAV2<T, R> extends C1V2<T, R> implements ComputeJobMasterLeaveAware {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         *
+         */
+        public C1MLAV2() {
+            // No-op.
+        }
+
+        /**
+         * @param job Job.
+         * @param arg Argument.
+         */
+        private C1MLAV2(IgniteClosure<T, R> job, T arg) {
+            super(job, arg);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMasterNodeLeft(ComputeTaskSession ses) {
+            ((ComputeJobMasterLeaveAware)job).onMasterNodeLeft(ses);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(C1MLAV2.class, this, super.toString());
+        }
+    }
+
+    /**
+     *
+     */
     private static class C2<R> implements ComputeJob, Externalizable, GridNoImplicitInjection, GridInternalWrapper<Callable> {
         /** */
         private static final long serialVersionUID = 0L;
@@ -1772,7 +1926,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
         /**
          *
          */
-        public C2(){
+        public C2() {
             // No-op.
         }
 
@@ -1822,7 +1976,66 @@ public class GridClosureProcessor extends GridProcessorAdapter {
     /**
      *
      */
-    private static class C2MLA<R> extends C2<R> implements ComputeJobMasterLeaveAware{
+    public static class C2V2<R> implements ComputeJob, Binarylizable, GridNoImplicitInjection,
+        GridInternalWrapper<Callable> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        protected Callable<R> c;
+
+        /**
+         *
+         */
+        public C2V2() {
+            // No-op.
+        }
+
+        /**
+         * @param c Callable.
+         */
+        private C2V2(Callable<R> c) {
+            this.c = c;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object execute() {
+            try {
+                return c.call();
+            }
+            catch (Exception e) {
+                throw new IgniteException(e);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cancel() {
+            // No-op.
+        }
+
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            writer.rawWriter().writeObject(c);
+        }
+
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            c = reader.rawReader().readObject();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Callable userObject() {
+            return c;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(C2V2.class, this);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class C2MLA<R> extends C2<R> implements ComputeJobMasterLeaveAware {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -1852,6 +2065,38 @@ public class GridClosureProcessor extends GridProcessorAdapter {
     }
 
     /**
+     *
+     */
+    public static class C2MLAV2<R> extends C2V2<R> implements ComputeJobMasterLeaveAware {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         *
+         */
+        public C2MLAV2() {
+            // No-op.
+        }
+
+        /**
+         * @param c Callable.
+         */
+        private C2MLAV2(Callable<R> c) {
+            super(c);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMasterNodeLeft(ComputeTaskSession ses) {
+            ((ComputeJobMasterLeaveAware)c).onMasterNodeLeft(ses);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(C2MLAV2.class, this, super.toString());
+        }
+    }
+
+    /**
      */
     private static class C4 implements ComputeJob, Externalizable, GridNoImplicitInjection, GridInternalWrapper<Runnable> {
         /** */
@@ -1863,7 +2108,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
         /**
          *
          */
-        public C4(){
+        public C4() {
             // No-op.
         }
 
@@ -1908,6 +2153,60 @@ public class GridClosureProcessor extends GridProcessorAdapter {
     }
 
     /**
+     */
+    public static class C4V2 implements ComputeJob, Binarylizable, GridNoImplicitInjection, GridInternalWrapper<Runnable> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        protected Runnable r;
+
+        /**
+         *
+         */
+        public C4V2() {
+            // No-op.
+        }
+
+        /**
+         * @param r Runnable.
+         */
+        private C4V2(Runnable r) {
+            this.r = r;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Object execute() {
+            r.run();
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cancel() {
+            // No-op.
+        }
+
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            writer.rawWriter().writeObject(r);
+        }
+
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            r = reader.rawReader().readObject();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Runnable userObject() {
+            return r;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(C4V2.class, this);
+        }
+    }
+
+    /**
      *
      */
     private static class C4MLA extends C4 implements ComputeJobMasterLeaveAware {
@@ -1938,4 +2237,36 @@ public class GridClosureProcessor extends GridProcessorAdapter {
             return S.toString(C4MLA.class, this, super.toString());
         }
     }
+
+    /**
+     *
+     */
+    public static class C4MLAV2 extends C4V2 implements ComputeJobMasterLeaveAware {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         *
+         */
+        public C4MLAV2() {
+            // No-op.
+        }
+
+        /**
+         * @param r Runnable.
+         */
+        private C4MLAV2(Runnable r) {
+            super(r);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMasterNodeLeft(ComputeTaskSession ses) {
+            ((ComputeJobMasterLeaveAware)r).onMasterNodeLeft(ses);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(C4MLAV2.class, this, super.toString());
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/215e8a1e/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index 8c3ad88..9728d9c 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -900,11 +900,17 @@ org.apache.ignite.internal.processors.clock.GridClockDeltaSnapshotMessage
 org.apache.ignite.internal.processors.clock.GridClockDeltaVersion
 org.apache.ignite.internal.processors.closure.GridClosurePolicy
 org.apache.ignite.internal.processors.closure.GridClosureProcessor$C1
+org.apache.ignite.internal.processors.closure.GridClosureProcessor$C1V2
 org.apache.ignite.internal.processors.closure.GridClosureProcessor$C1MLA
+org.apache.ignite.internal.processors.closure.GridClosureProcessor$C1MLAV2
 org.apache.ignite.internal.processors.closure.GridClosureProcessor$C2
+org.apache.ignite.internal.processors.closure.GridClosureProcessor$C2V2
 org.apache.ignite.internal.processors.closure.GridClosureProcessor$C2MLA
+org.apache.ignite.internal.processors.closure.GridClosureProcessor$C2MLAV2
 org.apache.ignite.internal.processors.closure.GridClosureProcessor$C4
+org.apache.ignite.internal.processors.closure.GridClosureProcessor$C4V2
 org.apache.ignite.internal.processors.closure.GridClosureProcessor$C4MLA
+org.apache.ignite.internal.processors.closure.GridClosureProcessor$C4MLAV2
 org.apache.ignite.internal.processors.closure.GridClosureProcessor$T1
 org.apache.ignite.internal.processors.closure.GridClosureProcessor$T10
 org.apache.ignite.internal.processors.closure.GridClosureProcessor$T11

http://git-wip-us.apache.org/repos/asf/ignite/blob/215e8a1e/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java
index a75023f..9e438e9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java
@@ -84,13 +84,15 @@ public class GridAffinitySelfTest extends GridCommonAbstractTest {
     /**
      * @throws IgniteCheckedException If failed.
      */
-    public void testAffinity() throws IgniteCheckedException {
+    public void testAffinity() throws Exception {
         Ignite g1 = grid(1);
         Ignite g2 = grid(2);
 
         assert caches(g1).size() == 0;
         assert F.first(caches(g2)).getCacheMode() == PARTITIONED;
 
+        awaitPartitionMapExchange();
+
         Map<ClusterNode, Collection<String>> map = g1.<String>affinity(null).mapKeysToNodes(F.asList("1"));
 
         assertNotNull(map);

http://git-wip-us.apache.org/repos/asf/ignite/blob/215e8a1e/modules/core/src/test/java/org/apache/ignite/internal/GridComputationBinarylizableClosuresSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridComputationBinarylizableClosuresSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridComputationBinarylizableClosuresSelfTest.java
new file mode 100644
index 0000000..96f0277
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridComputationBinarylizableClosuresSelfTest.java
@@ -0,0 +1,413 @@
+/*
+ * 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;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.compute.ComputeJobMasterLeaveAware;
+import org.apache.ignite.compute.ComputeTaskSession;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.lang.IgniteClosure;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Test ensuring that correct closures are serialized.
+ */
+public class GridComputationBinarylizableClosuresSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        TestBinarylizableClosure.writeCalled.set(false);
+        TestBinarylizableClosure.readCalled.set(false);
+        TestBinarylizableClosure.executed.set(false);
+
+        TestBinarylizableMasterLeaveAwareClosure.writeCalled.set(false);
+        TestBinarylizableMasterLeaveAwareClosure.readCalled.set(false);
+
+        TestBinarylizableCallable.writeCalled.set(false);
+        TestBinarylizableCallable.readCalled.set(false);
+        TestBinarylizableCallable.executed.set(false);
+
+        TestBinarylizableMasterLeaveAwareCallable.writeCalled.set(false);
+        TestBinarylizableMasterLeaveAwareCallable.readCalled.set(false);
+
+        TestBinarylizableRunnable.writeCalled.set(false);
+        TestBinarylizableRunnable.readCalled.set(false);
+        TestBinarylizableRunnable.executed.set(false);
+
+        TestBinarylizableMasterLeaveAwareRunnable.writeCalled.set(false);
+        TestBinarylizableMasterLeaveAwareRunnable.readCalled.set(false);
+
+        TestBinarylizableObject.writeCalled.set(false);
+        TestBinarylizableObject.readCalled.set(false);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * Test that Binarylizable IgniteClosure is serialized using BinaryMarshaller.
+     *
+     * @throws Exception If failed.
+     */
+    public void testJob() throws Exception {
+        Ignite ignite = startGrid(1);
+        startGrid(2);
+
+        final TestBinarylizableClosure closure = new TestBinarylizableClosure();
+
+        ignite.compute(ignite.cluster().forRemotes()).apply(closure, new TestBinarylizableObject());
+
+        assert TestBinarylizableClosure.executed.get();
+        assert TestBinarylizableClosure.writeCalled.get();
+        assert TestBinarylizableClosure.readCalled.get();
+
+        assert TestBinarylizableObject.writeCalled.get();
+        assert TestBinarylizableObject.readCalled.get();
+    }
+
+    /**
+     * Test that Binarylizable IgniteClosure with ComputeJobMasterLeaveAware interface is serialized
+     * using BinaryMarshaller.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMasterLeaveAwareJob() throws Exception {
+        Ignite ignite = startGrid(1);
+        startGrid(2);
+
+        final TestBinarylizableMasterLeaveAwareClosure job = new TestBinarylizableMasterLeaveAwareClosure();
+
+        ignite.compute(ignite.cluster().forRemotes()).apply(job, new TestBinarylizableObject());
+
+        assert TestBinarylizableClosure.executed.get();
+        assert TestBinarylizableClosure.writeCalled.get();
+        assert TestBinarylizableClosure.readCalled.get();
+
+        assert TestBinarylizableMasterLeaveAwareClosure.writeCalled.get();
+        assert TestBinarylizableMasterLeaveAwareClosure.readCalled.get();
+
+        assert TestBinarylizableObject.writeCalled.get();
+        assert TestBinarylizableObject.readCalled.get();
+    }
+
+    /**
+     * Test that Binarylizable IgniteCallable is serialized using BinaryMarshaller.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCallable() throws Exception {
+        Ignite ignite = startGrid(1);
+        startGrid(2);
+
+        final TestBinarylizableCallable callable = new TestBinarylizableCallable();
+
+        ignite.compute(ignite.cluster().forRemotes()).call(callable);
+
+        assert TestBinarylizableCallable.executed.get();
+        assert TestBinarylizableCallable.writeCalled.get();
+        assert TestBinarylizableCallable.readCalled.get();
+    }
+
+    /**
+     * Test that Binarylizable IgniteCallable with ComputeJobMasterLeaveAware interface is serialized
+     * using BinaryMarshaller.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMasterLeaveAwareCallable() throws Exception {
+        Ignite ignite = startGrid(1);
+        startGrid(2);
+
+        final TestBinarylizableMasterLeaveAwareCallable callable = new TestBinarylizableMasterLeaveAwareCallable();
+
+        ignite.compute(ignite.cluster().forRemotes()).call(callable);
+
+        assert TestBinarylizableCallable.executed.get();
+        assert TestBinarylizableCallable.writeCalled.get();
+        assert TestBinarylizableCallable.readCalled.get();
+
+        assert TestBinarylizableMasterLeaveAwareCallable.writeCalled.get();
+        assert TestBinarylizableMasterLeaveAwareCallable.readCalled.get();
+    }
+
+    /**
+     * Test that Binarylizable IgniteRunnable is serialized using BinaryMarshaller.
+     *
+     * @throws Exception If failed.
+     */
+    public void testRunnable() throws Exception {
+        Ignite ignite = startGrid(1);
+        startGrid(2);
+
+        final TestBinarylizableRunnable runnable = new TestBinarylizableRunnable();
+
+        ignite.compute(ignite.cluster().forRemotes()).run(runnable);
+
+        assert TestBinarylizableRunnable.executed.get();
+        assert TestBinarylizableRunnable.writeCalled.get();
+        assert TestBinarylizableRunnable.readCalled.get();
+    }
+
+    /**
+     * Test that Binarylizable IgniteRunnable with ComputeJobMasterLeaveAware interface is serialized
+     * using BinaryMarshaller.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMasterLeaveAwareRunnable() throws Exception {
+        Ignite ignite = startGrid(1);
+        startGrid(2);
+
+        final TestBinarylizableMasterLeaveAwareRunnable runnable = new TestBinarylizableMasterLeaveAwareRunnable();
+
+        ignite.compute(ignite.cluster().forRemotes()).run(runnable);
+
+        assert TestBinarylizableRunnable.executed.get();
+        assert TestBinarylizableRunnable.writeCalled.get();
+        assert TestBinarylizableRunnable.readCalled.get();
+
+        assert TestBinarylizableMasterLeaveAwareRunnable.writeCalled.get();
+        assert TestBinarylizableMasterLeaveAwareRunnable.readCalled.get();
+    }
+
+    /**
+     * Test Binarylizable IgniteClosure.
+     */
+    private static class TestBinarylizableClosure implements IgniteClosure, Binarylizable {
+
+        /** Tracks {@link TestBinarylizableClosure::writeBinary(BinaryWriter writer)} calls. */
+        private static AtomicBoolean writeCalled = new AtomicBoolean();
+
+        /** Tracks {@link TestBinarylizableClosure::readBinary(BinaryReader reader)} calls. */
+        private static AtomicBoolean readCalled = new AtomicBoolean();
+
+        /** Tracks {@link TestBinarylizableClosure::apply(Object o)} calls. */
+        private static AtomicBoolean executed = new AtomicBoolean();
+
+        /** {@inheritDoc} */
+        @Override public Object apply(Object o) {
+            executed.set(true);
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            writeCalled.set(true);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            readCalled.set(true);
+        }
+    }
+
+    /**
+     * Test Binarylizable IgniteClosure with ComputeJobMasterLeaveAware interface.
+     */
+    private static class TestBinarylizableMasterLeaveAwareClosure extends TestBinarylizableClosure
+        implements ComputeJobMasterLeaveAware {
+
+        /** Tracks {@link TestBinarylizableMasterLeaveAwareClosure::writeBinary(BinaryWriter writer)} calls. */
+        private static AtomicBoolean writeCalled = new AtomicBoolean();
+
+        /** Tracks {@link TestBinarylizableMasterLeaveAwareClosure::readBinary(BinaryReader reader)} calls. */
+        private static AtomicBoolean readCalled = new AtomicBoolean();
+
+        /** {@inheritDoc} */
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            super.writeBinary(writer);
+            writeCalled.set(true);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            super.readBinary(reader);
+            readCalled.set(true);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteException {
+        }
+    }
+
+    /**
+     * Test Binarylizable object.
+     */
+    private static class TestBinarylizableObject implements Binarylizable {
+
+        /** Tracks {@link TestBinarylizableObject::writeBinary(BinaryWriter writer)} calls. */
+        private static AtomicBoolean writeCalled = new AtomicBoolean();
+
+        /** Tracks {@link TestBinarylizableObject::readBinary(BinaryReader reader)} calls. */
+        private static AtomicBoolean readCalled = new AtomicBoolean();
+
+        /** {@inheritDoc} */
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            writeCalled.set(true);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            readCalled.set(true);
+        }
+    }
+
+    /**
+     * Test Binarylizable Callable.
+     */
+    private static class TestBinarylizableCallable implements IgniteCallable, Binarylizable {
+
+        /** Tracks {@link TestBinarylizableCallable::writeBinary(BinaryWriter writer)} calls. */
+        private static AtomicBoolean writeCalled = new AtomicBoolean();
+
+        /** Tracks {@link TestBinarylizableCallable::readBinary(BinaryReader reader)} calls. */
+        private static AtomicBoolean readCalled = new AtomicBoolean();
+
+        /** Tracks {@link TestBinarylizableCallable::call()} calls. */
+        private static AtomicBoolean executed = new AtomicBoolean();
+
+        /** {@inheritDoc} */
+        @Override public Object call() throws Exception {
+            executed.set(true);
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            writeCalled.set(true);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            readCalled.set(true);
+        }
+    }
+
+    /**
+     * Test Binarylizable Callable with ComputeJobMasterLeaveAware interface.
+     */
+    private static class TestBinarylizableMasterLeaveAwareCallable extends TestBinarylizableCallable
+        implements ComputeJobMasterLeaveAware {
+
+        /** Tracks {@link TestBinarylizableMasterLeaveAwareCallable::writeBinary(BinaryWriter writer)} calls. */
+        private static AtomicBoolean writeCalled = new AtomicBoolean();
+
+        /** Tracks {@link TestBinarylizableMasterLeaveAwareCallable::readBinary(BinaryReader reader)} calls. */
+        private static AtomicBoolean readCalled = new AtomicBoolean();
+
+        /** {@inheritDoc} */
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            super.writeBinary(writer);
+            writeCalled.set(true);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            super.readBinary(reader);
+            readCalled.set(true);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteException {
+        }
+    }
+
+    /**
+     * Test Binarylizable Runnable.
+     */
+    private static class TestBinarylizableRunnable implements IgniteRunnable, Binarylizable {
+
+        /** Tracks {@link TestBinarylizableRunnable::writeBinary(BinaryWriter writer)} calls. */
+        private static AtomicBoolean writeCalled = new AtomicBoolean();
+
+        /** Tracks {@link TestBinarylizableRunnable::readBinary(BinaryReader reader)} calls. */
+        private static AtomicBoolean readCalled = new AtomicBoolean();
+
+        /** Tracks {@link TestBinarylizableRunnable::run()} calls. */
+        private static AtomicBoolean executed = new AtomicBoolean();
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            executed.set(true);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            writeCalled.set(true);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            readCalled.set(true);
+        }
+    }
+
+    /**
+     * Test Binarylizable Runnable with ComputeJobMasterLeaveAware interface.
+     */
+    private static class TestBinarylizableMasterLeaveAwareRunnable extends TestBinarylizableRunnable
+        implements ComputeJobMasterLeaveAware {
+
+        /** Tracks {@link TestBinarylizableMasterLeaveAwareRunnable::writeBinary(BinaryWriter writer)} calls. */
+        private static AtomicBoolean writeCalled = new AtomicBoolean();
+
+        /** Tracks {@link TestBinarylizableMasterLeaveAwareRunnable::readBinary(BinaryReader reader)} calls. */
+        private static AtomicBoolean readCalled = new AtomicBoolean();
+
+        /** {@inheritDoc} */
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            super.writeBinary(writer);
+            writeCalled.set(true);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            super.readBinary(reader);
+            readCalled.set(true);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteException {
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/215e8a1e/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
index 37b908a..eefe66c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
@@ -2690,6 +2690,35 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Test object with {@link Proxy} field.
+     *
+     * @throws Exception If fails.
+     */
+    public void testObjectContainingProxy() throws Exception {
+        BinaryMarshaller marsh = binaryMarshaller();
+
+        SomeItf inItf = (SomeItf)Proxy.newProxyInstance(
+            BinaryMarshallerSelfTest.class.getClassLoader(), new Class[] {SomeItf.class},
+            new InvocationHandler() {
+                private NonSerializable obj = new NonSerializable(null);
+
+                @Override public Object invoke(Object proxy, Method mtd, Object[] args) throws Throwable {
+                    if ("hashCode".equals(mtd.getName()))
+                        return obj.hashCode();
+
+                    obj.checkAfterUnmarshalled();
+
+                    return 17;
+                }
+            }
+        );
+
+        SomeItf outItf = marsh.unmarshal(marsh.marshal(inItf), null);
+
+        assertEquals(outItf.checkAfterUnmarshalled(), 17);
+    }
+
+    /**
      * Test duplicate fields.
      *
      * @throws Exception If failed.
@@ -4676,4 +4705,25 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
             rawValArr = rawReader.readDecimalArray();
         }
     }
+
+    /**
+     * Wrapper object.
+     */
+    private static class Wrapper {
+
+        /** Value. */
+        private final Object value;
+
+        /** Constructor. */
+        public Wrapper(Object value) {
+            this.value = value;
+        }
+
+        /**
+         * @return Value.
+         */
+        public Object getValue() {
+            return value;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/215e8a1e/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsComputeGridTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsComputeGridTestSuite.java
index 35be98d..e659966 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsComputeGridTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsComputeGridTestSuite.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
+import org.apache.ignite.internal.GridComputationBinarylizableClosuresSelfTest;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.testframework.config.GridTestProperties;
 
@@ -32,6 +33,10 @@ public class IgniteBinaryObjectsComputeGridTestSuite {
     public static TestSuite suite() throws Exception {
         GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
 
-        return IgniteComputeGridTestSuite.suite();
+        TestSuite suite = IgniteComputeGridTestSuite.suite();
+
+        suite.addTestSuite(GridComputationBinarylizableClosuresSelfTest.class);
+
+        return suite;
     }
 }


[21/35] ignite git commit: Assembly fix (cherry picked from commit cb6da11)

Posted by nt...@apache.org.
Assembly fix
(cherry picked from commit cb6da11)


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/687200da
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/687200da
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/687200da

Branch: refs/heads/ignite-2791
Commit: 687200dad41d0c27e401d46283699e4492a97542
Parents: 06f0c91
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Mar 11 13:08:34 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Fri Mar 11 13:09:28 2016 +0300

----------------------------------------------------------------------
 assembly/dependencies-fabric-lgpl.xml | 1 +
 assembly/dependencies-fabric.xml      | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/687200da/assembly/dependencies-fabric-lgpl.xml
----------------------------------------------------------------------
diff --git a/assembly/dependencies-fabric-lgpl.xml b/assembly/dependencies-fabric-lgpl.xml
index 8fa3c4e..75e2939 100644
--- a/assembly/dependencies-fabric-lgpl.xml
+++ b/assembly/dependencies-fabric-lgpl.xml
@@ -129,6 +129,7 @@
                 <exclude>org.apache.ignite:ignite-schema-import</exclude>
                 <exclude>org.apache.ignite:ignite-codegen</exclude>
                 <exclude>org.apache.ignite:ignite-apache-license-gen</exclude>
+                <exclude>org.apache.ignite:ignite-weblogic-test</exclude>
             </excludes>
             <sources>
                 <includeModuleDirectory>true</includeModuleDirectory>

http://git-wip-us.apache.org/repos/asf/ignite/blob/687200da/assembly/dependencies-fabric.xml
----------------------------------------------------------------------
diff --git a/assembly/dependencies-fabric.xml b/assembly/dependencies-fabric.xml
index 2330a3a..4add771 100644
--- a/assembly/dependencies-fabric.xml
+++ b/assembly/dependencies-fabric.xml
@@ -132,6 +132,7 @@
                 <exclude>org.apache.ignite:ignite-hibernate</exclude>
                 <exclude>org.apache.ignite:ignite-schedule</exclude>
                 <exclude>org.apache.ignite:ignite-geospatial</exclude>
+                <exclude>org.apache.ignite:ignite-weblogic-test</exclude>
             </excludes>
             <sources>
                 <includeModuleDirectory>true</includeModuleDirectory>


[10/35] ignite git commit: .NET: Fix NuGet package name to 'Apache.Ignite' instead of 'Apache.Ignite.NET'

Posted by nt...@apache.org.
.NET: Fix NuGet package name to 'Apache.Ignite' instead of 'Apache.Ignite.NET'


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/648b10c8
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/648b10c8
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/648b10c8

Branch: refs/heads/ignite-2791
Commit: 648b10c8cf0372dce552b8ff1506884044f86312
Parents: 70abaa3
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Thu Mar 10 12:39:38 2016 +0300
Committer: Pavel Tupitsyn <pt...@gridgain.com>
Committed: Thu Mar 10 12:39:38 2016 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests.NuGet/install-package.cmd   | 11 +++++++++--
 .../dotnet/Apache.Ignite.Core/Apache.Ignite.Core.nuspec  |  6 +++---
 .../Apache.Ignite.Core/NuGet/LINQPad/ComputeExample.linq |  2 +-
 .../Apache.Ignite.Core/NuGet/LINQPad/PutGetExample.linq  |  2 +-
 .../Apache.Ignite.Core/NuGet/LINQPad/QueryExample.linq   |  2 +-
 5 files changed, 15 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/648b10c8/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/install-package.cmd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/install-package.cmd b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/install-package.cmd
index e1b57a8..5af7607 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/install-package.cmd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/install-package.cmd
@@ -1,3 +1,10 @@
+rem Install NuGet package to a 'pkg' folder no matter what version it is
+
 rmdir pkg /S /Q
-nuget install Apache.Ignite.NET
-move Apache.Ignite.NET* pkg
\ No newline at end of file
+rmdir tmp1 /S /Q
+mkdir tmp1
+cd tmp1
+nuget install Apache.Ignite
+cd ..
+move tmp1\Apache.Ignite.* pkg
+rmdir tmp1 /S /Q
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/648b10c8/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.nuspec
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.nuspec b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.nuspec
index 54a4a9e..ff3ce34 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.nuspec
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.nuspec
@@ -29,8 +29,8 @@ Creating NuGet package:
 
 <package >
     <metadata>
-        <id>Apache.Ignite.NET</id>
-        <title>Apache Ignite.NET x64</title>
+        <id>Apache.Ignite</id>
+        <title>Apache Ignite x64</title>
         <!-- -->
         <version>$version$</version>
         <authors>Apache Ignite</authors>
@@ -40,7 +40,7 @@ Creating NuGet package:
         <iconUrl>https://ignite.apache.org/images/logo_ignite_32_32.png</iconUrl>
         <requireLicenseAcceptance>false</requireLicenseAcceptance>
         <description>
-Apache Ignite.NET In-Memory Data Fabric is a high-performance, integrated and distributed in-memory platform for computing and transacting on large-scale data sets in real-time, orders of magnitude faster than possible with traditional disk-based or flash technologies.
+Apache Ignite In-Memory Data Fabric is a high-performance, integrated and distributed in-memory platform for computing and transacting on large-scale data sets in real-time, orders of magnitude faster than possible with traditional disk-based or flash technologies.
             
 More info: https://apacheignite-net.readme.io/
             

http://git-wip-us.apache.org/repos/asf/ignite/blob/648b10c8/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/ComputeExample.linq
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/ComputeExample.linq b/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/ComputeExample.linq
index 194858c..921af3e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/ComputeExample.linq
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/ComputeExample.linq
@@ -1,5 +1,5 @@
 <Query Kind="Program">
-  <NuGetReference>Apache.Ignite.NET</NuGetReference>
+  <NuGetReference>Apache.Ignite</NuGetReference>
   <Namespace>Apache.Ignite.Core</Namespace>
   <Namespace>Apache.Ignite.Core.Binary</Namespace>
   <Namespace>Apache.Ignite.Core.Cache.Configuration</Namespace>

http://git-wip-us.apache.org/repos/asf/ignite/blob/648b10c8/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/PutGetExample.linq
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/PutGetExample.linq b/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/PutGetExample.linq
index 0569da6..2995f01 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/PutGetExample.linq
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/PutGetExample.linq
@@ -1,5 +1,5 @@
 <Query Kind="Program">
-    <NuGetReference>Apache.Ignite.NET</NuGetReference>
+    <NuGetReference>Apache.Ignite</NuGetReference>
     <Namespace>Apache.Ignite.Core</Namespace>
     <Namespace>Apache.Ignite.Core.Binary</Namespace>
 </Query>

http://git-wip-us.apache.org/repos/asf/ignite/blob/648b10c8/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/QueryExample.linq
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/QueryExample.linq b/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/QueryExample.linq
index b13e1e9..5a37f36 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/QueryExample.linq
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/QueryExample.linq
@@ -1,5 +1,5 @@
 <Query Kind="Program">
-  <NuGetReference>Apache.Ignite.NET</NuGetReference>
+  <NuGetReference>Apache.Ignite</NuGetReference>
   <Namespace>Apache.Ignite.Core</Namespace>
   <Namespace>Apache.Ignite.Core.Binary</Namespace>
   <Namespace>Apache.Ignite.Core.Cache.Configuration</Namespace>


[28/35] ignite git commit: IGNITE-2814: IGFS: File lock/unlock/reserve operations are no longer require put/replace on cache. Thin entry processors are used instead.

Posted by nt...@apache.org.
IGNITE-2814: IGFS: File lock/unlock/reserve operations are no longer require put/replace on cache. Thin entry processors are used instead.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/b1d9e8b6
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/b1d9e8b6
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/b1d9e8b6

Branch: refs/heads/ignite-2791
Commit: b1d9e8b6c7553133c4c4ca3820f6f30d202b7ea2
Parents: d83fa11
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Mar 14 13:17:58 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Mar 14 13:17:58 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsMetaManager.java        | 270 ++++++++++++++++---
 .../processors/igfs/IgfsOutputStreamImpl.java   |  76 +-----
 2 files changed, 228 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b1d9e8b6/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index 89ddd02..8bb9e92 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -466,10 +466,11 @@ public class IgfsMetaManager extends IgfsManager {
      * Lock the file explicitly outside of transaction.
      *
      * @param fileId File ID to lock.
+     * @param delete If file is being locked for delete.
      * @return Locked file info or {@code null} if file cannot be locked or doesn't exist.
      * @throws IgniteCheckedException If the file with such id does not exist, or on another failure.
      */
-    public @Nullable IgfsFileInfo lock(IgniteUuid fileId, boolean isDeleteLock) throws IgniteCheckedException {
+    public @Nullable IgfsFileInfo lock(IgniteUuid fileId, boolean delete) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
                 assert validTxState(false);
@@ -487,13 +488,11 @@ public class IgfsMetaManager extends IgfsManager {
                     if (oldInfo.lockId() != null)
                         return null; // The file is already locked, we cannot lock it.
 
-                    IgfsFileInfo newInfo = lockInfo(oldInfo, isDeleteLock);
+                    IgniteUuid lockId = createFileLockId(delete);
 
-                    boolean put = id2InfoPrj.replace(fileId, oldInfo, newInfo);
+                    id2InfoPrj.invoke(fileId, new FileLockProcessor(lockId));
 
-                    assert put : "Value was not stored in cache [fileId=" + fileId + ", newInfo=" + newInfo + ']';
-
-                    assert newInfo.id().equals(oldInfo.id()); // Same id.
+                    IgfsFileInfo newInfo = id2InfoPrj.get(fileId);
 
                     tx.commit();
 
@@ -515,30 +514,13 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Set lock on file info.
+     * Create file lock ID.
      *
-     * @param info File info.
-     * @return New file info with lock set, or null if the info passed in is already locked.
-     * @throws IgniteCheckedException In case lock is already set on that file.
+     * @param delete If lock ID is required for file deletion.
+     * @return Lock ID.
      */
-    private @Nullable IgfsFileInfo lockInfo(IgfsFileInfo info, boolean isDeleteLock) {
-         assert info != null;
-
-         if (info.lockId() != null)
-             return null; // Null return value indicates that the file is already locked.
-
-         return new IgfsFileInfo(info, composeLockId(isDeleteLock), info.modificationTime());
-    }
-
-    /**
-     * Gets a new lock id.
-     * The returned Id #globalId() method will return the Id of the node which locked the file.
-     *
-     * @param isDeleteLock if this is special delete lock.
-     * @return The new lock id.
-     */
-    private IgniteUuid composeLockId(boolean isDeleteLock) {
-        if (isDeleteLock)
+    private IgniteUuid createFileLockId(boolean delete) {
+        if (delete)
             return IgfsUtils.DELETE_LOCK_ID;
 
         return IgniteUuid.fromUuid(locNode.id());
@@ -584,12 +566,7 @@ public class IgfsMetaManager extends IgfsManager {
                                     "[fileId=" + fileId + ", lockId=" + info.lockId() + ", actualLockId=" +
                                     oldInfo.lockId() + ']');
 
-                            IgfsFileInfo newInfo = new IgfsFileInfo(oldInfo, null, modificationTime);
-
-                            boolean put = id2InfoPrj.put(fileId, newInfo);
-
-                            assert put : "Value was not stored in cache [fileId=" + fileId + ", newInfo=" + newInfo
-                                    + ']';
+                            id2InfoPrj.invoke(fileId, new FileUnlockProcessor(modificationTime));
 
                             return null;
                         }
@@ -1680,6 +1657,57 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
+     * Reserve space for file.
+     *
+     * @param path File path.
+     * @param fileId File ID.
+     * @param space Space.
+     * @param affRange Affinity range.
+     * @return New file info.
+     */
+    public IgfsFileInfo reserveSpace(IgfsPath path, IgniteUuid fileId, long space, IgfsFileAffinityRange affRange)
+        throws IgniteCheckedException {
+        assert validTxState(false);
+
+        if (busyLock.enterBusy()) {
+            try {
+                if (log.isDebugEnabled())
+                    log.debug("Reserve file space [path=" + path + ", id=" + fileId + ']');
+
+                IgniteInternalTx tx = startTx();
+
+                try {
+                    // Lock file ID for this transaction.
+                    IgfsFileInfo oldInfo = info(fileId);
+
+                    if (oldInfo == null)
+                        throw fsException("File has been deleted concurrently [path=" + path + ", id=" + fileId + ']');
+
+                    id2InfoPrj.invoke(fileId, new FileReserveSpaceProcessor(space, affRange));
+
+                    IgfsFileInfo newInfo = id2InfoPrj.get(fileId);
+
+                    tx.commit();
+
+                    return newInfo;
+                }
+                catch (GridClosureException e) {
+                    throw U.cast(e);
+                }
+                finally {
+                    tx.close();
+                }
+            }
+            finally {
+                busyLock.leaveBusy();
+            }
+        }
+        else
+            throw new IllegalStateException("Failed to reseve file space because Grid is stopping [path=" + path +
+                ", id=" + fileId + ']');
+    }
+
+    /**
      * Update file info in cache.
      *
      * @param fileId File ID to update information for.
@@ -1992,7 +2020,7 @@ public class IgfsMetaManager extends IgfsManager {
                                     "the secondary file system because the path points to a directory: " + path);
 
                             IgfsFileInfo newInfo = new IgfsFileInfo(status.blockSize(), status.length(), affKey,
-                                composeLockId(false), igfsCtx.igfs().evictExclude(path, false), status.properties(),
+                                createFileLockId(false), igfsCtx.igfs().evictExclude(path, false), status.properties(),
                                 status.accessTime(), status.modificationTime());
 
                             // Add new file info to the listing optionally removing the previous one.
@@ -2113,13 +2141,11 @@ public class IgfsMetaManager extends IgfsManager {
                             }
 
                             // Set lock and return.
-                            IgfsFileInfo lockedInfo = lockInfo(info, false);
-
-                            assert lockedInfo != null; // We checked the lock above.
+                            IgniteUuid lockId = createFileLockId(false);
 
-                            boolean put = id2InfoPrj.put(info.id(), lockedInfo);
+                            id2InfoPrj.invoke(info.id(), new FileLockProcessor(lockId));
 
-                            assert put;
+                            IgfsFileInfo lockedInfo = id2InfoPrj.get(info.id());
 
                             return new IgfsSecondaryOutputStreamDescriptor(infos.get(path.parent()).id(),
                                 lockedInfo, out);
@@ -3455,7 +3481,7 @@ public class IgfsMetaManager extends IgfsManager {
                         @Override protected IgfsFileInfo buildLeaf() {
                             long t = System.currentTimeMillis();
 
-                            return new IgfsFileInfo(blockSize, 0L, affKey, composeLockId(false),
+                            return new IgfsFileInfo(blockSize, 0L, affKey, createFileLockId(false),
                                  evictExclude, leafProps, t, t);
                         }
                     };
@@ -3505,7 +3531,7 @@ public class IgfsMetaManager extends IgfsManager {
                                                 + "[fileName=" + name + ", fileId=" + lowermostExistingInfo.id()
                                                 + ", lockId=" + lockId + ']');
 
-                                        IgniteUuid newLockId = composeLockId(false);
+                                        IgniteUuid newLockId = createFileLockId(false);
 
                                         EntryProcessorResult<IgfsFileInfo> result
                                             = id2InfoPrj.invoke(lowermostExistingInfo.id(),
@@ -3553,7 +3579,7 @@ public class IgfsMetaManager extends IgfsManager {
                                         long t = System.currentTimeMillis();
 
                                         final IgfsFileInfo newFileInfo = new IgfsFileInfo(cfg.getBlockSize(), 0L,
-                                            affKey, composeLockId(false), evictExclude, fileProps, t, t);
+                                            affKey, createFileLockId(false), evictExclude, fileProps, t, t);
 
                                         assert newFileInfo.lockId() != null; // locked info should be created.
 
@@ -3874,4 +3900,162 @@ public class IgfsMetaManager extends IgfsManager {
             return S.toString(LockFileProcessor.class, this);
         }
     }
+
+    /**
+     * File lock entry processor.
+     */
+    private static class FileLockProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, Void>, Externalizable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Lock Id. */
+        private IgniteUuid lockId;
+
+        /**
+         * Default constructor.
+         */
+        public FileLockProcessor() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param lockId Lock ID.
+         */
+        public FileLockProcessor(IgniteUuid lockId) {
+            this.lockId = lockId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Void process(MutableEntry<IgniteUuid, IgfsFileInfo> entry, Object... args)
+            throws EntryProcessorException {
+            IgfsFileInfo old = entry.getValue();
+
+            entry.setValue(new IgfsFileInfo(old, lockId, old.modificationTime()));
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            U.writeGridUuid(out, lockId);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            lockId = U.readGridUuid(in);
+        }
+    }
+
+    /**
+     * File unlock entry processor.
+     */
+    private static class FileUnlockProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, Void>,
+        Externalizable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Modification time. */
+        private long modificationTime;
+
+        /**
+         * Default constructor.
+         */
+        public FileUnlockProcessor() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param modificationTime Modification time.
+         */
+        public FileUnlockProcessor(long modificationTime) {
+            this.modificationTime = modificationTime;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Void process(MutableEntry<IgniteUuid, IgfsFileInfo> entry, Object... args)
+            throws EntryProcessorException {
+            IgfsFileInfo old = entry.getValue();
+
+            entry.setValue(new IgfsFileInfo(old, null, modificationTime));
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeLong(modificationTime);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            modificationTime = in.readLong();
+        }
+    }
+
+    /**
+     * File reserve space entry processor.
+     */
+    private static class FileReserveSpaceProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, Void>,
+        Externalizable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Space. */
+        private long space;
+
+        /** Affinity range. */
+        private IgfsFileAffinityRange affRange;
+
+        /**
+         * Default constructor.
+         */
+        public FileReserveSpaceProcessor() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param space Space.
+         * @param affRange
+         */
+        public FileReserveSpaceProcessor(long space, IgfsFileAffinityRange affRange) {
+            this.space = space;
+            this.affRange = affRange;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Void process(MutableEntry<IgniteUuid, IgfsFileInfo> entry, Object... args)
+            throws EntryProcessorException {
+            IgfsFileInfo oldInfo = entry.getValue();
+
+            IgfsFileMap newMap = new IgfsFileMap(oldInfo.fileMap());
+
+            newMap.addRange(affRange);
+
+            IgfsFileInfo newInfo = new IgfsFileInfo(oldInfo, oldInfo.length() + space);
+
+            newInfo.fileMap(newMap);
+
+            entry.setValue(newInfo);
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeLong(space);
+            out.writeObject(affRange);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            space = in.readLong();
+            affRange = (IgfsFileAffinityRange)in.readObject();
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b1d9e8b6/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
index 83056af..8c11073 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
@@ -18,10 +18,7 @@
 package org.apache.ignite.internal.processors.igfs;
 
 import java.io.DataInput;
-import java.io.Externalizable;
 import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
 import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.IgniteCheckedException;
@@ -30,11 +27,9 @@ import org.apache.ignite.igfs.IgfsMode;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.igfs.IgfsPathNotFoundException;
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
@@ -297,8 +292,7 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
             if (space > 0) {
                 data.awaitAllAcksReceived(fileInfo.id());
 
-                IgfsFileInfo fileInfo0 = meta.updateInfo(fileInfo.id(),
-                    new ReserveSpaceClosure(space, streamRange));
+                IgfsFileInfo fileInfo0 = meta.reserveSpace(path, fileInfo.id(), space, streamRange);
 
                 if (fileInfo0 == null)
                     throw new IOException("File was concurrently deleted: " + path);
@@ -446,72 +440,4 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
     @Override public String toString() {
         return S.toString(IgfsOutputStreamImpl.class, this);
     }
-
-    /**
-     * Helper closure to reserve specified space and update file's length
-     */
-    @GridInternal
-    private static final class ReserveSpaceClosure implements IgniteClosure<IgfsFileInfo, IgfsFileInfo>,
-        Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Space amount (bytes number) to increase file's length. */
-        private long space;
-
-        /** Affinity range for this particular update. */
-        private IgfsFileAffinityRange range;
-
-        /**
-         * Empty constructor required for {@link Externalizable}.
-         *
-         */
-        public ReserveSpaceClosure() {
-            // No-op.
-        }
-
-        /**
-         * Constructs the closure to reserve specified space and update file's length.
-         *
-         * @param space Space amount (bytes number) to increase file's length.
-         * @param range Affinity range specifying which part of file was colocated.
-         */
-        private ReserveSpaceClosure(long space, IgfsFileAffinityRange range) {
-            this.space = space;
-            this.range = range;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsFileInfo apply(IgfsFileInfo oldInfo) {
-            IgfsFileMap oldMap = oldInfo.fileMap();
-
-            IgfsFileMap newMap = new IgfsFileMap(oldMap);
-
-            newMap.addRange(range);
-
-            // Update file length.
-            IgfsFileInfo updated = new IgfsFileInfo(oldInfo, oldInfo.length() + space);
-
-            updated.fileMap(newMap);
-
-            return updated;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeLong(space);
-            out.writeObject(range);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            space = in.readLong();
-            range = (IgfsFileAffinityRange)in.readObject();
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(ReserveSpaceClosure.class, this);
-        }
-    }
 }
\ No newline at end of file


[30/35] ignite git commit: IGNITE-2828: IGFS: Introduced processor for "updateTimes" operation.

Posted by nt...@apache.org.
IGNITE-2828: IGFS: Introduced processor for "updateTimes" operation.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/09b28952
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/09b28952
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/09b28952

Branch: refs/heads/ignite-2791
Commit: 09b28952f1c95130fd0ebadbbd33e70b7d81245c
Parents: b1d9e8b
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Mar 14 15:57:28 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Mar 14 15:57:28 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsMetaManager.java        | 73 ++++++++++++++++----
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |  6 --
 2 files changed, 61 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/09b28952/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index 8bb9e92..d3aae58 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -411,7 +411,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                 // Force root ID always exist in cache.
                 if (info == null && IgfsUtils.ROOT_ID.equals(fileId))
-                    id2InfoPrj.putIfAbsent(IgfsUtils.ROOT_ID, info = new IgfsFileInfo());
+                    info = createSystemEntryIfAbsent(fileId);
 
                 return info;
             }
@@ -443,13 +443,9 @@ public class IgfsMetaManager extends IgfsManager {
 
                 // Force root ID always exist in cache.
                 if (fileIds.contains(IgfsUtils.ROOT_ID) && !map.containsKey(IgfsUtils.ROOT_ID)) {
-                    IgfsFileInfo info = new IgfsFileInfo();
-
-                    id2InfoPrj.putIfAbsent(IgfsUtils.ROOT_ID, info);
-
                     map = new GridLeanMap<>(map);
 
-                    map.put(IgfsUtils.ROOT_ID, info);
+                    map.put(IgfsUtils.ROOT_ID, createSystemEntryIfAbsent(IgfsUtils.ROOT_ID));
                 }
 
                 return map;
@@ -666,8 +662,6 @@ public class IgfsMetaManager extends IgfsManager {
      */
     private IgfsFileInfo createSystemEntryIfAbsent(IgniteUuid id)
         throws IgniteCheckedException {
-        assert validTxState(true);
-
         assert IgfsUtils.isRootOrTrashId(id);
 
         IgfsFileInfo info = new IgfsFileInfo(id);
@@ -2995,11 +2989,10 @@ public class IgfsMetaManager extends IgfsManager {
 
                     assert parentInfo.isDirectory();
 
-                    IgfsFileInfo updated = new IgfsFileInfo(fileInfo,
+                    id2InfoPrj.invoke(fileId, new UpdateTimesProcessor(
                         accessTime == -1 ? fileInfo.accessTime() : accessTime,
-                        modificationTime == -1 ? fileInfo.modificationTime() : modificationTime);
-
-                    id2InfoPrj.put(fileId, updated);
+                        modificationTime == -1 ? fileInfo.modificationTime() : modificationTime)
+                    );
 
                     id2InfoPrj.invoke(parentId, new UpdateListingEntry(fileId, fileName, 0, accessTime,
                         modificationTime));
@@ -4058,4 +4051,60 @@ public class IgfsMetaManager extends IgfsManager {
             affRange = (IgfsFileAffinityRange)in.readObject();
         }
     }
+
+    /**
+     * Update times entry processor.
+     */
+    private static class UpdateTimesProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, Void>,
+        Externalizable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Access time. */
+        private long accessTime;
+
+        /** Modification time. */
+        private long modificationTime;
+
+        /**
+         * Default constructor.
+         */
+        public UpdateTimesProcessor() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param accessTime Access time.
+         * @param modificationTime Modification time.
+         */
+        public UpdateTimesProcessor(long accessTime, long modificationTime) {
+            this.accessTime = accessTime;
+            this.modificationTime = modificationTime;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Void process(MutableEntry<IgniteUuid, IgfsFileInfo> entry, Object... args)
+            throws EntryProcessorException {
+
+            IgfsFileInfo oldInfo = entry.getValue();
+
+            entry.setValue(new IgfsFileInfo(oldInfo, accessTime, modificationTime));
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeLong(accessTime);
+            out.writeLong(modificationTime);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            accessTime = in.readLong();
+            modificationTime = in.readLong();
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/09b28952/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
index cc5e1ce..489088c 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
@@ -90,12 +90,6 @@ public class IgniteIgfsTestSuite extends TestSuite {
         suite.addTest(new TestSuite(IgfsModesSelfTest.class));
         suite.addTest(new TestSuite(IgfsMetricsSelfTest.class));
 
-        suite.addTest(new TestSuite(IgfsPrimarySelfTest.class));
-        suite.addTest(new TestSuite(IgfsPrimaryOffheapTieredSelfTest.class));
-        suite.addTest(new TestSuite(IgfsPrimaryOffheapValuesSelfTest.class));
-        suite.addTest(new TestSuite(IgfsDualSyncSelfTest.class));
-        suite.addTest(new TestSuite(IgfsDualAsyncSelfTest.class));
-
         suite.addTest(new TestSuite(IgfsClientCacheSelfTest.class));
         suite.addTest(new TestSuite(IgfsOneClientNodeTest.class));
 


[15/35] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by nt...@apache.org.
Merge remote-tracking branch 'origin/master'


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/a895b8eb
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/a895b8eb
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/a895b8eb

Branch: refs/heads/ignite-2791
Commit: a895b8eb5051794f84507b4f7c98ff9216fbf2db
Parents: 6231bb9 215e8a1
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Mar 10 17:56:37 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Mar 10 17:56:37 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/binary/BinaryContext.java   |  29 +-
 .../internal/binary/BinaryFieldAccessor.java    |   1 +
 .../processors/cache/GridCacheMapEntry.java     |  25 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   2 +-
 .../closure/GridClosureProcessor.java           | 375 ++++++++++++++++-
 .../resources/META-INF/classnames.properties    |   6 +
 .../ignite/internal/GridAffinitySelfTest.java   |   4 +-
 ...omputationBinarylizableClosuresSelfTest.java | 413 +++++++++++++++++++
 .../binary/BinaryMarshallerSelfTest.java        |  50 +++
 ...ridCacheStoreManagerDeserializationTest.java | 345 ++++++++++++++++
 ...calCacheStoreManagerDeserializationTest.java | 101 +++++
 ...IgniteBinaryObjectsComputeGridTestSuite.java |   7 +-
 .../testsuites/IgniteCacheTestSuite4.java       |  11 +-
 .../install-package.cmd                         |  11 +-
 .../Apache.Ignite.Core.nuspec                   |   6 +-
 .../NuGet/LINQPad/ComputeExample.linq           |   2 +-
 .../NuGet/LINQPad/PutGetExample.linq            |   2 +-
 .../NuGet/LINQPad/QueryExample.linq             |   2 +-
 18 files changed, 1341 insertions(+), 51 deletions(-)
----------------------------------------------------------------------



[17/35] ignite git commit: IGNITE-2735 - Interrupt all acquires on local node after ignite.close - Fixes #525.

Posted by nt...@apache.org.
IGNITE-2735 - Interrupt all acquires on local node after ignite.close - Fixes #525.

Signed-off-by: Valentin Kulichenko <va...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/b63cee45
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/b63cee45
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/b63cee45

Branch: refs/heads/ignite-2791
Commit: b63cee457007bfcf91bd4d0415c57a1b05647b7b
Parents: 053af5d
Author: Vladisav Jelisavcic <vl...@gmail.com>
Authored: Thu Mar 10 21:02:52 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Thu Mar 10 21:02:52 2016 -0800

----------------------------------------------------------------------
 .../datastructures/DataStructuresProcessor.java |   5 +
 .../datastructures/GridCacheSemaphoreEx.java    |   5 +
 .../datastructures/GridCacheSemaphoreImpl.java  | 161 +++++++++++++++++--
 ...eAbstractDataStructuresFailoverSelfTest.java |  44 +++++
 4 files changed, 206 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b63cee45/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 445fc3e..0b02abd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -259,6 +259,11 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
     @Override public void onKernalStop(boolean cancel) {
         super.onKernalStop(cancel);
 
+        for (GridCacheRemovable ds : dsMap.values()) {
+            if (ds instanceof GridCacheSemaphoreEx)
+                ((GridCacheSemaphoreEx)ds).stop();
+        }
+
         if (initLatch.getCount() > 0) {
             initFailed = true;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b63cee45/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreEx.java
index 4d39635..b49d6b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreEx.java
@@ -44,4 +44,9 @@ public interface GridCacheSemaphoreEx extends IgniteSemaphore, GridCacheRemovabl
      * @param nodeId Id of the node that left the grid.
      */
     public void onNodeRemoved(UUID nodeId);
+
+    /**
+     * Callback to notify local semaphore instance on node stop.
+     */
+    public void stop();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b63cee45/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java
index 37df9d5..2c60e8b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java
@@ -179,6 +179,29 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
         }
 
         /**
+         * Set a flag indicating that it is not safe to continue using this semaphore.
+         * This is the case only if one of two things happened:
+         * 1. A node that previously acquired on this semaphore failed and
+         * semaphore is created in non-failoversafe mode;
+         * 2. Local node failed (is closed), so any any threads on this node
+         * waiting to acquire are notified, and semaphore is not safe to be used anymore.
+         *
+         * @return True is semaphore is not safe to be used anymore.
+         */
+        protected boolean isBroken() {
+            return broken;
+        }
+
+        /** Flag indicating that a node failed and it is not safe to continue using this semaphore.
+         * Any attempt to acquire on broken semaphore will result in {@linkplain IgniteInterruptedException}.
+         *
+         * @param broken True if semaphore should not be used anymore.
+         * */
+        protected void setBroken(boolean broken) {
+            this.broken = broken;
+        }
+
+        /**
          * This method is used by the AQS to test if the current thread should block or not.
          *
          * @param acquires Number of permits to acquire.
@@ -186,6 +209,10 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
          */
         final int nonfairTryAcquireShared(int acquires) {
             for (;;) {
+                // If broken, return immediately, exception will be thrown anyway.
+                if(broken)
+                    return 1;
+
                 int available = getState();
 
                 int remaining = available - acquires;
@@ -209,6 +236,10 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
                 return true;
 
             for (;;) {
+                // If broken, return immediately, exception will be thrown anyway.
+                if(broken)
+                    return true;
+
                 int cur = getState();
 
                 int next = cur + releases;
@@ -228,6 +259,9 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
          */
         final int drainPermits() {
             for (;;) {
+                // If broken, return immediately, exception will be thrown anyway.
+                if(broken)
+                    return 1;
 
                 int current = getState();
 
@@ -504,7 +538,7 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
                 sync.releaseFailedNode(nodeId);
             else {
                 // Interrupt every waiting thread if this semaphore is not failover safe.
-                sync.broken = true;
+                sync.setBroken(true);
 
                 for (Thread t : sync.getSharedQueuedThreads())
                     t.interrupt();
@@ -515,6 +549,13 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
         }
     }
 
+    @Override public void stop() {
+        sync.setBroken(true);
+
+        // Try to notify any waiting threads.
+        sync.releaseShared(0);
+    }
+
     /** {@inheritDoc} */
     @Override public void needCheckNotRemoved() {
         // No-op.
@@ -527,15 +568,17 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
 
     /** {@inheritDoc} */
     @Override public void acquire(int permits) throws IgniteInterruptedException {
+        ctx.kernalContext().gateway().readLock();
+
         A.ensure(permits >= 0, "Number of permits must be non-negative.");
 
         try {
             initializeSemaphore();
 
-            if(isBroken())
-                Thread.currentThread().interrupt();
-
             sync.acquireSharedInterruptibly(permits);
+
+            if(isBroken())
+                throw new InterruptedException();
         }
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
@@ -543,10 +586,15 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
         catch (InterruptedException e) {
             throw new IgniteInterruptedException(e);
         }
+        finally {
+            ctx.kernalContext().gateway().readUnlock();
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void acquireUninterruptibly() {
+        ctx.kernalContext().gateway().readLock();
+
         try {
             initializeSemaphore();
 
@@ -555,10 +603,15 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
         }
+        finally {
+            ctx.kernalContext().gateway().readUnlock();
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void acquireUninterruptibly(int permits) {
+        ctx.kernalContext().gateway().readLock();
+
         A.ensure(permits >= 0, "Number of permits must be non-negative.");
 
         try {
@@ -569,10 +622,15 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
         }
+        finally {
+            ctx.kernalContext().gateway().readUnlock();
+        }
     }
 
     /** {@inheritDoc} */
     @Override public int availablePermits() {
+        ctx.kernalContext().gateway().readLock();
+
         int ret;
         try {
             initializeSemaphore();
@@ -603,12 +661,17 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
         }
+        finally {
+            ctx.kernalContext().gateway().readUnlock();
+        }
 
         return ret;
     }
 
     /** {@inheritDoc} */
     @Override public int drainPermits() {
+        ctx.kernalContext().gateway().readLock();
+
         try {
             initializeSemaphore();
 
@@ -617,26 +680,49 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
         }
+        finally {
+            ctx.kernalContext().gateway().readUnlock();
+        }
     }
 
     /** {@inheritDoc} */
     @Override public boolean tryAcquire() {
+        ctx.kernalContext().gateway().readLock();
+
         try {
             initializeSemaphore();
 
-            return sync.nonfairTryAcquireShared(1) >= 0;
+            boolean result = sync.nonfairTryAcquireShared(1) >= 0;
+
+            if(isBroken())
+                throw new InterruptedException();
+
+            return result;
         }
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
         }
+        catch (InterruptedException e) {
+            throw new IgniteInterruptedException(e);
+        }
+        finally {
+            ctx.kernalContext().gateway().readUnlock();
+        }
     }
 
     /** {@inheritDoc} */
     @Override public boolean tryAcquire(long timeout, TimeUnit unit) throws IgniteException {
+        ctx.kernalContext().gateway().readLock();
+
         try {
             initializeSemaphore();
 
-            return sync.tryAcquireSharedNanos(1, unit.toNanos(timeout));
+            boolean result = sync.tryAcquireSharedNanos(1, unit.toNanos(timeout));
+
+            if(isBroken())
+                throw new InterruptedException();
+
+            return result;
         }
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
@@ -644,6 +730,9 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
         catch (InterruptedException e) {
             throw new IgniteInterruptedException(e);
         }
+        finally {
+            ctx.kernalContext().gateway().readUnlock();
+        }
     }
 
     /** {@inheritDoc} */
@@ -653,6 +742,8 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
 
     /** {@inheritDoc} */
     @Override public void release(int permits) {
+        ctx.kernalContext().gateway().readLock();
+
         A.ensure(permits >= 0, "Number of permits must be non-negative.");
 
         try {
@@ -663,10 +754,15 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
         }
+        finally {
+            ctx.kernalContext().gateway().readUnlock();
+        }
     }
 
     /** {@inheritDoc} */
     @Override public boolean tryAcquire(int permits) {
+        ctx.kernalContext().gateway().readLock();
+
         A.ensure(permits >= 0, "Number of permits must be non-negative.");
 
         try {
@@ -677,15 +773,25 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
         }
+        finally {
+            ctx.kernalContext().gateway().readUnlock();
+        }
     }
 
     /** {@inheritDoc} */
     @Override public boolean tryAcquire(int permits, long timeout, TimeUnit unit) throws IgniteInterruptedException {
+        ctx.kernalContext().gateway().readLock();
+
         A.ensure(permits >= 0, "Number of permits must be non-negative.");
         try {
             initializeSemaphore();
 
-            return sync.tryAcquireSharedNanos(permits, unit.toNanos(timeout));
+            boolean result =  sync.tryAcquireSharedNanos(permits, unit.toNanos(timeout));
+
+            if(isBroken())
+                throw new InterruptedException();
+
+            return result;
         }
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
@@ -693,15 +799,32 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
         catch (InterruptedException e) {
             throw new IgniteInterruptedException(e);
         }
+        finally {
+            ctx.kernalContext().gateway().readUnlock();
+        }
     }
 
     /** {@inheritDoc} */
     @Override public boolean isFailoverSafe() {
-        return sync.failoverSafe;
+        ctx.kernalContext().gateway().readLock();
+
+        try {
+            initializeSemaphore();
+
+            return sync.failoverSafe;
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+        finally {
+            ctx.kernalContext().gateway().readUnlock();
+        }
     }
 
     /** {@inheritDoc} */
     @Override public boolean hasQueuedThreads() {
+        ctx.kernalContext().gateway().readLock();
+
         try {
             initializeSemaphore();
 
@@ -710,10 +833,15 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
         }
+        finally {
+            ctx.kernalContext().gateway().readUnlock();
+        }
     }
 
     /** {@inheritDoc} */
     @Override public int getQueueLength() {
+        ctx.kernalContext().gateway().readLock();
+
         try {
             initializeSemaphore();
 
@@ -722,11 +850,26 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Exter
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
         }
+        finally {
+            ctx.kernalContext().gateway().readUnlock();
+        }
     }
 
     /** {@inheritDoc} */
     @Override public boolean isBroken(){
-        return sync.broken;
+        ctx.kernalContext().gateway().readLock();
+
+        try {
+            initializeSemaphore();
+
+            return sync.isBroken();
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+        finally {
+            ctx.kernalContext().gateway().readUnlock();
+        }
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b63cee45/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
index e85468e..fbd72bf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
@@ -454,6 +454,50 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Ig
     /**
      * @throws Exception If failed.
      */
+    public void testSemaphoreSingleNodeFailure() throws Exception {
+        final Ignite i1 = grid(0);
+
+        IgniteSemaphore sem1 = i1.semaphore(STRUCTURE_NAME, 1, false, true);
+
+        sem1.acquire();
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                boolean failed = true;
+
+                IgniteSemaphore sem2 = i1.semaphore(STRUCTURE_NAME, 1, false, true);
+
+                try {
+                    sem2.acquire();
+                }
+                catch (Exception e){
+                    failed = false;
+                }
+                finally {
+                    assertFalse(failed);
+
+                    sem2.release();
+                }
+                return null;
+            }
+        });
+
+        while(!sem1.hasQueuedThreads()){
+            try {
+                Thread.sleep(1);
+            } catch (InterruptedException e) {
+                fail();
+            }
+        }
+
+        i1.close();
+
+        fut.get();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testSemaphoreConstantTopologyChangeFailoverSafe() throws Exception {
         doTestSemaphore(new ConstantTopologyChangeWorker(TOP_CHANGE_THREAD_CNT), true);
     }


[35/35] ignite git commit: Merge branch 'master' into ignite-2791

Posted by nt...@apache.org.
Merge branch 'master' into ignite-2791


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/ecb2d58e
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/ecb2d58e
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/ecb2d58e

Branch: refs/heads/ignite-2791
Commit: ecb2d58e0a718e1e726636b5faa21f91a9cff8b0
Parents: ef04253 7b7233e
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Mar 15 21:17:31 2016 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Mar 15 21:17:31 2016 +0300

----------------------------------------------------------------------
 assembly/dependencies-fabric-lgpl.xml           |   1 +
 assembly/dependencies-fabric.xml                |   1 +
 .../streaming/StreamVisitorExample.java         |  31 +-
 modules/aws/pom.xml                             |  24 +-
 .../configuration/TransactionConfiguration.java |  32 +
 .../org/apache/ignite/internal/IgnitionEx.java  |   9 +
 .../ignite/internal/binary/BinaryContext.java   |  29 +-
 .../internal/binary/BinaryFieldAccessor.java    |   1 +
 .../ignite/internal/binary/BinarySchema.java    |  11 +-
 .../processors/cache/GridCacheEntryEx.java      |   3 +-
 .../processors/cache/GridCacheMapEntry.java     |  37 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   2 +-
 .../dht/preloader/GridDhtPartitionDemander.java |   8 +-
 .../cache/transactions/IgniteTxAdapter.java     |   6 +-
 .../closure/GridClosureProcessor.java           | 375 ++++++++++-
 .../datastructures/DataStructuresProcessor.java |   5 +
 .../datastructures/GridCacheSemaphoreEx.java    |   5 +
 .../datastructures/GridCacheSemaphoreImpl.java  | 161 ++++-
 .../processors/igfs/IgfsDeleteWorker.java       |  28 +-
 .../internal/processors/igfs/IgfsFileImpl.java  |  29 -
 .../internal/processors/igfs/IgfsFileInfo.java  |  47 +-
 .../internal/processors/igfs/IgfsImpl.java      |  13 +-
 .../processors/igfs/IgfsListingEntry.java       | 129 +---
 .../processors/igfs/IgfsMetaManager.java        | 656 +++++++++++--------
 .../processors/igfs/IgfsOutputStreamImpl.java   |  80 +--
 .../internal/processors/igfs/IgfsUtils.java     |  72 ++
 .../processors/platform/PlatformContext.java    |  11 +-
 .../platform/PlatformContextImpl.java           |  72 +-
 .../platform/PlatformProcessorImpl.java         |   2 +-
 .../platform/cluster/PlatformClusterGroup.java  |  12 +
 .../platform/compute/PlatformCompute.java       |   7 +-
 .../PlatformDotNetConfigurationClosure.java     |   5 -
 .../utils/PlatformConfigurationUtils.java       |  14 +
 .../ignite/internal/util/IgniteUtils.java       |   2 +
 .../apache/ignite/internal/util/typedef/X.java  |   2 +-
 .../resources/META-INF/classnames.properties    |   6 +
 .../config/websession/example-cache-base.xml    | 148 +++++
 .../config/websession/example-cache-client.xml  |  33 +
 .../test/config/websession/example-cache.xml    | 128 +---
 .../test/config/websession/example-cache2.xml   |  31 +
 .../ignite/internal/GridAffinitySelfTest.java   |   4 +-
 ...omputationBinarylizableClosuresSelfTest.java | 413 ++++++++++++
 .../binary/BinaryMarshallerSelfTest.java        |  50 ++
 ...ridCacheStoreManagerDeserializationTest.java | 346 ++++++++++
 .../processors/cache/GridCacheTestEntryEx.java  |   2 +-
 ...calCacheStoreManagerDeserializationTest.java | 101 +++
 .../IgniteDynamicClientCacheStartSelfTest.java  |  97 +++
 ...eAbstractDataStructuresFailoverSelfTest.java |  44 ++
 .../processors/igfs/IgfsAbstractSelfTest.java   |  21 +-
 .../igfs/IgfsDataManagerSelfTest.java           |   6 +-
 .../igfs/IgfsMetaManagerSelfTest.java           |   2 +-
 .../processors/igfs/IgfsProcessorSelfTest.java  |   9 +-
 .../processors/igfs/IgfsSizeSelfTest.java       |  13 +-
 .../ignite/platform/PlatformSqlQueryTask.java   | 117 ++++
 ...IgniteBinaryObjectsComputeGridTestSuite.java |   7 +-
 .../testsuites/IgniteCacheTestSuite4.java       |  11 +-
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |   6 -
 .../HibernateL2CacheTransactionalSelfTest.java  |   9 +
 ...nateL2CacheTransactionalUseSyncSelfTest.java |  31 +
 .../testsuites/IgniteHibernateTestSuite.java    |   4 +-
 .../processors/cache/jta/CacheJtaManager.java   |  26 +-
 .../processors/cache/jta/CacheJtaResource.java  | 304 +++++++++
 .../cache/jta/GridCacheXAResource.java          | 251 -------
 .../cache/AbstarctCacheJtaSelfTest.java         | 183 ------
 .../cache/AbstractCacheJtaSelfTest.java         | 183 ++++++
 .../GridPartitionedCacheJtaFactorySelfTest.java |   2 +-
 ...rtitionedCacheJtaFactoryUseSyncSelfTest.java |  32 +
 ...titionedCacheJtaLookupClassNameSelfTest.java |   2 +-
 ...eplicatedCacheJtaFactoryUseSyncSelfTest.java |  32 +
 .../ignite/testsuites/IgniteJtaTestSuite.java   |   5 +
 .../install-package.cmd                         |  11 +-
 .../Apache.Ignite.Core.Tests.csproj             |   8 +-
 .../Binary/BinaryBuilderSelfTest.cs             |  33 +-
 .../Binary/BinaryBuilderSelfTestFullFooter.cs   |  31 +
 .../Binary/BinaryCompactFooterInteropTest.cs    | 129 ++++
 .../Binary/BinarySelfTest.cs                    |  23 +-
 .../Binary/BinarySelfTestFullFooter.cs          |  35 +
 .../Compute/ComputeApiTest.cs                   |  57 +-
 .../Compute/ComputeApiTestFullFooter.cs         |  65 ++
 .../Config/Compute/compute-grid1.xml            |  20 +-
 .../Config/Compute/compute-grid2.xml            |   2 +-
 .../Config/Compute/compute-grid3.xml            |   2 +-
 .../Apache.Ignite.Core.Tests/Config/binary.xml  |  56 --
 .../IgniteConfigurationSerializerTest.cs        |   3 +-
 .../IgniteConfigurationTest.cs                  |   2 +
 .../Services/ServicesTest.cs                    |  26 +-
 .../Services/ServicesTestFullFooter.cs          |  33 +
 .../Apache.Ignite.Core.csproj                   |   1 +
 .../Apache.Ignite.Core.nuspec                   |   6 +-
 .../Binary/BinaryConfiguration.cs               |  44 +-
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  15 +
 .../IgniteConfigurationSection.xsd              |   1 +
 .../Apache.Ignite.Core/Impl/Binary/Binary.cs    |   8 +-
 .../Impl/Binary/BinaryObject.cs                 |   9 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |   9 +-
 .../Impl/Binary/BinaryObjectHeader.cs           | 176 +----
 .../Impl/Binary/BinaryObjectSchema.cs           |  20 +
 .../Impl/Binary/BinaryObjectSchemaHolder.cs     |  18 +-
 .../Impl/Binary/BinaryObjectSchemaSerializer.cs | 262 ++++++++
 .../Impl/Binary/BinaryReader.cs                 |  46 +-
 .../Impl/Binary/BinaryWriter.cs                 |  20 +-
 .../Impl/Binary/Marshaller.cs                   |  12 +-
 .../Impl/Binary/Metadata/BinaryType.cs          |  28 +-
 .../Impl/Cluster/ClusterGroupImpl.cs            |  15 +
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  19 +
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  47 +-
 .../NuGet/LINQPad/ComputeExample.linq           |   2 +-
 .../NuGet/LINQPad/PutGetExample.linq            |   2 +-
 .../NuGet/LINQPad/QueryExample.linq             |   2 +-
 modules/web/ignite-weblogic-test/pom.xml        |  76 +++
 .../webapp/META-INF/config/default-config.xml   |  37 ++
 .../src/main/webapp/WEB-INF/web.xml             |  52 ++
 .../src/main/webapp/index.jsp                   |  36 +
 .../ignite/cache/websession/WebSession.java     |  42 +-
 .../cache/websession/WebSessionFilter.java      | 244 +++++--
 .../cache/websession/WebSessionListener.java    |  66 +-
 .../internal/websession/WebSessionSelfTest.java | 208 ++++++
 parent/pom.xml                                  |   2 +-
 pom.xml                                         |   1 +
 119 files changed, 4992 insertions(+), 1651 deletions(-)
----------------------------------------------------------------------