You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2017/10/03 14:03:15 UTC

[01/37] ignite git commit: IGNITE-6029 Record serializer refactoring and initial stuff for Record V2 serialization.

Repository: ignite
Updated Branches:
  refs/heads/ignite-2.3 b7bb79232 -> db69e9027


http://git-wip-us.apache.org/repos/asf/ignite/blob/00770767/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java
new file mode 100644
index 0000000..2b55c5f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java
@@ -0,0 +1,64 @@
+/*
+ * 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.persistence.wal.serializer;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
+import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput;
+import org.apache.ignite.internal.processors.cache.persistence.wal.RecordDataSerializer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord;
+
+/**
+ * Record data V2 serializer.
+ */
+public class RecordDataV2Serializer implements RecordDataSerializer {
+    /** V1 data serializer delegate. */
+    private final RecordDataV1Serializer delegateSerializer;
+
+    /**
+     * Create an instance of V2 data serializer.
+     *
+     * @param delegateSerializer V1 data serializer.
+     */
+    public RecordDataV2Serializer(RecordDataV1Serializer delegateSerializer) {
+        this.delegateSerializer = delegateSerializer;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int size(WALRecord record) throws IgniteCheckedException {
+        if (record instanceof HeaderRecord)
+            throw new UnsupportedOperationException("Getting size of header records is forbidden since version 2 of serializer");
+
+        return delegateSerializer.size(record);
+    }
+
+    /** {@inheritDoc} */
+    @Override public WALRecord readRecord(WALRecord.RecordType type, ByteBufferBackedDataInput in) throws IOException, IgniteCheckedException {
+        return delegateSerializer.readRecord(type, in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeRecord(WALRecord record, ByteBuffer buf) throws IgniteCheckedException {
+        if (record instanceof HeaderRecord)
+            throw new UnsupportedOperationException("Writing header records is forbidden since version 2 of serializer");
+
+        delegateSerializer.writeRecord(record, buf);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/00770767/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java
index ce6fdc7..c4e1bf2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java
@@ -21,84 +21,22 @@ import java.io.DataInput;
 import java.io.EOFException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteSystemProperties;
-import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.wal.WALPointer;
-import org.apache.ignite.internal.pagemem.wal.record.CacheState;
-import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord;
-import org.apache.ignite.internal.pagemem.wal.record.DataEntry;
-import org.apache.ignite.internal.pagemem.wal.record.DataRecord;
-import org.apache.ignite.internal.pagemem.wal.record.LazyDataEntry;
-import org.apache.ignite.internal.pagemem.wal.record.MemoryRecoveryRecord;
-import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
-import org.apache.ignite.internal.pagemem.wal.record.TxRecord;
 import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
 import org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType;
-import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageInsertFragmentRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageInsertRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageRemoveRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageSetFreeListPageRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageUpdateRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.FixCountRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.FixLeftmostChildRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.FixRemoveId;
-import org.apache.ignite.internal.pagemem.wal.record.delta.InitNewPageRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.InnerReplaceRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.InsertRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.MergeRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageAddRootRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageCutRootRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRootInlineRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRootRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdateLastAllocatedIndex;
-import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdateLastSuccessfulFullSnapshotId;
-import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdateLastSuccessfulSnapshotId;
-import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdateNextSnapshotId;
-import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdatePartitionDataRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.NewRootInitRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.PageListMetaResetCountRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.PagesListAddPageRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.PagesListInitNewPageRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.PagesListRemovePageRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.PagesListSetNextRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.PagesListSetPreviousRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.PartitionDestroyRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.PartitionMetaStateRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.RecycleRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.RemoveRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.ReplaceRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.SplitExistingPageRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.SplitForwardPageRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.TrackingPageDeltaRecord;
-import org.apache.ignite.internal.processors.cache.CacheObject;
-import org.apache.ignite.internal.processors.cache.CacheObjectContext;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheOperation;
-import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO;
 import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileInput;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer;
 import org.apache.ignite.internal.processors.cache.persistence.wal.RecordSerializer;
 import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentEofException;
+import org.apache.ignite.internal.processors.cache.persistence.wal.WalSegmentTailReachedException;
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32;
-import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord;
+import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.io.RecordIO;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.U;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_SKIP_CRC;
 
@@ -116,1328 +54,97 @@ public class RecordV1Serializer implements RecordSerializer {
     /** Length of Type */
     public static final int REC_TYPE_SIZE = 1;
 
-    /** Length of WAL Pointer */
-    public static final int FILE_WAL_POINTER_SIZE = 12;
+    /** Length of WAL Pointer: Index (8) + File offset (4). */
+    public static final int FILE_WAL_POINTER_SIZE = 8 + 4;
 
     /** Length of CRC value */
-    private static final int CRC_SIZE = 4;
+    public static final int CRC_SIZE = 4;
 
-    /** */
-    public static final int HEADER_RECORD_SIZE = REC_TYPE_SIZE + FILE_WAL_POINTER_SIZE + /*Magic*/8 + /*Version*/4 + CRC_SIZE;
+    /** Total length of HEADER record. */
+    public static final int HEADER_RECORD_SIZE = REC_TYPE_SIZE + FILE_WAL_POINTER_SIZE + CRC_SIZE + RecordDataV1Serializer.HEADER_RECORD_DATA_SIZE;
 
-    /** Cache shared context */
-    private GridCacheSharedContext cctx;
-
-    /** Size of page used for PageMemory regions */
-    private int pageSize;
-
-    /** Cache object processor to reading {@link DataEntry DataEntries} */
-    private IgniteCacheObjectProcessor co;
-
-    /** Skip CRC calculation/check flag */
-    private boolean skipCrc = IgniteSystemProperties.getBoolean(IGNITE_PDS_SKIP_CRC, false);
-
-    /** Write pointer. */
-    private final boolean writePointer;
-
-    /** Serializer of {@link TxRecord} records. */
-    private TxRecordSerializer txRecordSerializer;
-
-    /**
-     * @param cctx Cache shared context.
-     */
-    public RecordV1Serializer(GridCacheSharedContext cctx) {
-        this(cctx, false);
-    }
-
-    /**
-     * @param cctx Cache shared context.
-     */
-    public RecordV1Serializer(GridCacheSharedContext cctx, boolean writePointer) {
-        this.cctx = cctx;
-        this.writePointer = writePointer;
-
-        co = cctx.kernalContext().cacheObjects();
-        pageSize = cctx.database().pageSize();
-        txRecordSerializer = new TxRecordSerializer(cctx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int version() {
-        return 1;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean writePointer() {
-        return writePointer;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("CastConflictsWithInstanceof")
-    @Override public void writeRecord(WALRecord record, ByteBuffer buf) throws IgniteCheckedException {
-        assert record.size() > 0 && buf.remaining() >= record.size() : record.size();
-
-        int startPos = buf.position();
-
-        buf.put((byte)(record.type().ordinal() + 1));
-
-        putPosition(buf, (FileWALPointer)record.position());
-
-        switch (record.type()) {
-            case PAGE_RECORD:
-                PageSnapshot snap = (PageSnapshot)record;
-
-                buf.putInt(snap.fullPageId().groupId());
-                buf.putLong(snap.fullPageId().pageId());
-                buf.put(snap.pageData());
-
-                break;
-
-            case MEMORY_RECOVERY:
-                MemoryRecoveryRecord memoryRecoveryRecord = (MemoryRecoveryRecord)record;
-
-                buf.putLong(memoryRecoveryRecord.time());
-
-                break;
-
-            case PARTITION_DESTROY:
-                PartitionDestroyRecord partDestroy = (PartitionDestroyRecord)record;
-
-                buf.putInt(partDestroy.groupId());
-                buf.putInt(partDestroy.partitionId());
-
-                break;
-
-            case META_PAGE_INIT:
-                MetaPageInitRecord updRootsRec = (MetaPageInitRecord)record;
-
-                buf.putInt(updRootsRec.groupId());
-                buf.putLong(updRootsRec.pageId());
-
-                buf.putShort((short)updRootsRec.ioType());
-                buf.putShort((short)updRootsRec.ioVersion());
-                buf.putLong(updRootsRec.treeRoot());
-                buf.putLong(updRootsRec.reuseListRoot());
-
-                break;
-
-            case PARTITION_META_PAGE_UPDATE_COUNTERS:
-                MetaPageUpdatePartitionDataRecord partDataRec = (MetaPageUpdatePartitionDataRecord)record;
-
-                buf.putInt(partDataRec.groupId());
-                buf.putLong(partDataRec.pageId());
-
-                buf.putLong(partDataRec.updateCounter());
-                buf.putLong(partDataRec.globalRemoveId());
-                buf.putInt(partDataRec.partitionSize());
-                buf.putLong(partDataRec.countersPageId());
-                buf.put(partDataRec.state());
-                buf.putInt(partDataRec.allocatedIndexCandidate());
-
-                break;
-
-            case CHECKPOINT_RECORD:
-                CheckpointRecord cpRec = (CheckpointRecord)record;
-
-                assert cpRec.checkpointMark() == null || cpRec.checkpointMark() instanceof FileWALPointer :
-                    "Invalid WAL record: " + cpRec;
-
-                FileWALPointer walPtr = (FileWALPointer)cpRec.checkpointMark();
-                UUID cpId = cpRec.checkpointId();
-
-                buf.putLong(cpId.getMostSignificantBits());
-                buf.putLong(cpId.getLeastSignificantBits());
-
-                buf.put(walPtr == null ? (byte)0 : 1);
-
-                if (walPtr != null) {
-                    buf.putLong(walPtr.index());
-                    buf.putInt(walPtr.fileOffset());
-                    buf.putInt(walPtr.length());
-                }
-
-                putCacheStates(buf, cpRec.cacheGroupStates());
-
-                buf.put(cpRec.end() ? (byte)1 : 0);
-
-                break;
-
-            case DATA_RECORD:
-                DataRecord dataRec = (DataRecord)record;
-
-                buf.putInt(dataRec.writeEntries().size());
-
-                for (DataEntry dataEntry : dataRec.writeEntries())
-                    putDataEntry(buf, dataEntry);
-
-                break;
-
-            case HEADER_RECORD:
-                buf.putLong(HeaderRecord.MAGIC);
-
-                buf.putInt(((HeaderRecord)record).version());
-
-                break;
-
-            case DATA_PAGE_INSERT_RECORD:
-                DataPageInsertRecord diRec = (DataPageInsertRecord)record;
-
-                buf.putInt(diRec.groupId());
-                buf.putLong(diRec.pageId());
-
-                buf.putShort((short)diRec.payload().length);
-
-                buf.put(diRec.payload());
-
-                break;
-
-            case DATA_PAGE_UPDATE_RECORD:
-                DataPageUpdateRecord uRec = (DataPageUpdateRecord)record;
-
-                buf.putInt(uRec.groupId());
-                buf.putLong(uRec.pageId());
-                buf.putInt(uRec.itemId());
-
-                buf.putShort((short)uRec.payload().length);
-
-                buf.put(uRec.payload());
-
-                break;
-
-            case DATA_PAGE_INSERT_FRAGMENT_RECORD:
-                final DataPageInsertFragmentRecord difRec = (DataPageInsertFragmentRecord)record;
-
-                buf.putInt(difRec.groupId());
-                buf.putLong(difRec.pageId());
-
-                buf.putLong(difRec.lastLink());
-                buf.putInt(difRec.payloadSize());
-                buf.put(difRec.payload());
-
-                break;
-
-            case DATA_PAGE_REMOVE_RECORD:
-                DataPageRemoveRecord drRec = (DataPageRemoveRecord)record;
-
-                buf.putInt(drRec.groupId());
-                buf.putLong(drRec.pageId());
-
-                buf.put((byte)drRec.itemId());
-
-                break;
-
-            case DATA_PAGE_SET_FREE_LIST_PAGE:
-                DataPageSetFreeListPageRecord freeListRec = (DataPageSetFreeListPageRecord)record;
-
-                buf.putInt(freeListRec.groupId());
-                buf.putLong(freeListRec.pageId());
-
-                buf.putLong(freeListRec.freeListPage());
-
-                break;
-
-            case INIT_NEW_PAGE_RECORD:
-                InitNewPageRecord inpRec = (InitNewPageRecord)record;
-
-                buf.putInt(inpRec.groupId());
-                buf.putLong(inpRec.pageId());
-
-                buf.putShort((short)inpRec.ioType());
-                buf.putShort((short)inpRec.ioVersion());
-                buf.putLong(inpRec.newPageId());
-
-                break;
-
-            case BTREE_META_PAGE_INIT_ROOT:
-                MetaPageInitRootRecord imRec = (MetaPageInitRootRecord)record;
-
-                buf.putInt(imRec.groupId());
-                buf.putLong(imRec.pageId());
-
-                buf.putLong(imRec.rootId());
-
-                break;
-
-            case BTREE_META_PAGE_INIT_ROOT2:
-                MetaPageInitRootInlineRecord imRec2 = (MetaPageInitRootInlineRecord)record;
-
-                buf.putInt(imRec2.groupId());
-                buf.putLong(imRec2.pageId());
-
-                buf.putLong(imRec2.rootId());
-
-                buf.putShort((short)imRec2.inlineSize());
-                break;
-
-            case BTREE_META_PAGE_ADD_ROOT:
-                MetaPageAddRootRecord arRec = (MetaPageAddRootRecord)record;
-
-                buf.putInt(arRec.groupId());
-                buf.putLong(arRec.pageId());
-
-                buf.putLong(arRec.rootId());
-
-                break;
-
-            case BTREE_META_PAGE_CUT_ROOT:
-                MetaPageCutRootRecord crRec = (MetaPageCutRootRecord)record;
-
-                buf.putInt(crRec.groupId());
-                buf.putLong(crRec.pageId());
-
-                break;
-
-            case BTREE_INIT_NEW_ROOT:
-                NewRootInitRecord<?> riRec = (NewRootInitRecord<?>)record;
-
-                buf.putInt(riRec.groupId());
-                buf.putLong(riRec.pageId());
-
-                buf.putLong(riRec.rootId());
-                buf.putShort((short)riRec.io().getType());
-                buf.putShort((short)riRec.io().getVersion());
-                buf.putLong(riRec.leftId());
-                buf.putLong(riRec.rightId());
-
-                putRow(buf, riRec.rowBytes());
-
-                break;
-
-            case BTREE_PAGE_RECYCLE:
-                RecycleRecord recRec = (RecycleRecord)record;
-
-                buf.putInt(recRec.groupId());
-                buf.putLong(recRec.pageId());
-
-                buf.putLong(recRec.newPageId());
-
-                break;
-
-            case BTREE_PAGE_INSERT:
-                InsertRecord<?> inRec = (InsertRecord<?>)record;
-
-                buf.putInt(inRec.groupId());
-                buf.putLong(inRec.pageId());
-
-                buf.putShort((short)inRec.io().getType());
-                buf.putShort((short)inRec.io().getVersion());
-                buf.putShort((short)inRec.index());
-                buf.putLong(inRec.rightId());
-
-                putRow(buf, inRec.rowBytes());
-
-                break;
-
-            case BTREE_FIX_LEFTMOST_CHILD:
-                FixLeftmostChildRecord flRec = (FixLeftmostChildRecord)record;
-
-                buf.putInt(flRec.groupId());
-                buf.putLong(flRec.pageId());
-
-                buf.putLong(flRec.rightId());
-
-                break;
-
-            case BTREE_FIX_COUNT:
-                FixCountRecord fcRec = (FixCountRecord)record;
-
-                buf.putInt(fcRec.groupId());
-                buf.putLong(fcRec.pageId());
-
-                buf.putShort((short)fcRec.count());
-
-                break;
-
-            case BTREE_PAGE_REPLACE:
-                ReplaceRecord<?> rRec = (ReplaceRecord<?>)record;
-
-                buf.putInt(rRec.groupId());
-                buf.putLong(rRec.pageId());
-
-                buf.putShort((short)rRec.io().getType());
-                buf.putShort((short)rRec.io().getVersion());
-                buf.putShort((short)rRec.index());
-
-                putRow(buf, rRec.rowBytes());
-
-                break;
-
-            case BTREE_PAGE_REMOVE:
-                RemoveRecord rmRec = (RemoveRecord)record;
-
-                buf.putInt(rmRec.groupId());
-                buf.putLong(rmRec.pageId());
-
-                buf.putShort((short)rmRec.index());
-                buf.putShort((short)rmRec.count());
-
-                break;
-
-            case BTREE_PAGE_INNER_REPLACE:
-                InnerReplaceRecord<?> irRec = (InnerReplaceRecord<?>)record;
-
-                buf.putInt(irRec.groupId());
-                buf.putLong(irRec.pageId());
-
-                buf.putShort((short)irRec.destinationIndex());
-                buf.putLong(irRec.sourcePageId());
-                buf.putShort((short)irRec.sourceIndex());
-                buf.putLong(irRec.removeId());
-
-                break;
-
-            case BTREE_FORWARD_PAGE_SPLIT:
-                SplitForwardPageRecord sfRec = (SplitForwardPageRecord)record;
-
-                buf.putInt(sfRec.groupId());
-                buf.putLong(sfRec.pageId());
-
-                buf.putLong(sfRec.forwardId());
-                buf.putShort((short)sfRec.ioType());
-                buf.putShort((short)sfRec.ioVersion());
-                buf.putLong(sfRec.sourcePageId());
-                buf.putShort((short)sfRec.middleIndex());
-                buf.putShort((short)sfRec.count());
-
-                break;
-
-            case BTREE_EXISTING_PAGE_SPLIT:
-                SplitExistingPageRecord seRec = (SplitExistingPageRecord)record;
-
-                buf.putInt(seRec.groupId());
-                buf.putLong(seRec.pageId());
-
-                buf.putShort((short)seRec.middleIndex());
-                buf.putLong(seRec.forwardId());
-
-                break;
-
-            case BTREE_PAGE_MERGE:
-                MergeRecord<?> mRec = (MergeRecord<?>)record;
-
-                buf.putInt(mRec.groupId());
-                buf.putLong(mRec.pageId());
-
-                buf.putLong(mRec.parentId());
-                buf.putShort((short)mRec.parentIndex());
-                buf.putLong(mRec.rightId());
-                buf.put((byte)(mRec.isEmptyBranch() ? 1 : 0));
-
-                break;
-
-            case PAGES_LIST_SET_NEXT:
-                PagesListSetNextRecord plNextRec = (PagesListSetNextRecord)record;
-
-                buf.putInt(plNextRec.groupId());
-                buf.putLong(plNextRec.pageId());
-
-                buf.putLong(plNextRec.nextPageId());
-
-                break;
-
-            case PAGES_LIST_SET_PREVIOUS:
-                PagesListSetPreviousRecord plPrevRec = (PagesListSetPreviousRecord)record;
-
-                buf.putInt(plPrevRec.groupId());
-                buf.putLong(plPrevRec.pageId());
-
-                buf.putLong(plPrevRec.previousPageId());
-
-                break;
-
-            case PAGES_LIST_INIT_NEW_PAGE:
-                PagesListInitNewPageRecord plNewRec = (PagesListInitNewPageRecord)record;
-
-                buf.putInt(plNewRec.groupId());
-                buf.putLong(plNewRec.pageId());
-                buf.putInt(plNewRec.ioType());
-                buf.putInt(plNewRec.ioVersion());
-                buf.putLong(plNewRec.newPageId());
-
-                buf.putLong(plNewRec.previousPageId());
-                buf.putLong(plNewRec.dataPageId());
-
-                break;
-
-            case PAGES_LIST_ADD_PAGE:
-                PagesListAddPageRecord plAddRec = (PagesListAddPageRecord)record;
-
-                buf.putInt(plAddRec.groupId());
-                buf.putLong(plAddRec.pageId());
-
-                buf.putLong(plAddRec.dataPageId());
-
-                break;
-
-            case PAGES_LIST_REMOVE_PAGE:
-                PagesListRemovePageRecord plRmvRec = (PagesListRemovePageRecord)record;
-
-                buf.putInt(plRmvRec.groupId());
-                buf.putLong(plRmvRec.pageId());
-
-                buf.putLong(plRmvRec.removedPageId());
-
-                break;
-
-            case BTREE_FIX_REMOVE_ID:
-                FixRemoveId frRec = (FixRemoveId)record;
-
-                buf.putInt(frRec.groupId());
-                buf.putLong(frRec.pageId());
-
-                buf.putLong(frRec.removeId());
-
-                break;
-
-            case TRACKING_PAGE_DELTA:
-                TrackingPageDeltaRecord tpDelta = (TrackingPageDeltaRecord)record;
-
-                buf.putInt(tpDelta.groupId());
-                buf.putLong(tpDelta.pageId());
-
-                buf.putLong(tpDelta.pageIdToMark());
-                buf.putLong(tpDelta.nextSnapshotId());
-                buf.putLong(tpDelta.lastSuccessfulSnapshotId());
-
-                break;
-
-            case META_PAGE_UPDATE_NEXT_SNAPSHOT_ID:
-                MetaPageUpdateNextSnapshotId mpUpdateNextSnapshotId = (MetaPageUpdateNextSnapshotId)record;
-
-                buf.putInt(mpUpdateNextSnapshotId.groupId());
-                buf.putLong(mpUpdateNextSnapshotId.pageId());
-
-                buf.putLong(mpUpdateNextSnapshotId.nextSnapshotId());
-
-                break;
-
-            case META_PAGE_UPDATE_LAST_SUCCESSFUL_FULL_SNAPSHOT_ID:
-                MetaPageUpdateLastSuccessfulFullSnapshotId mpUpdateLastSuccFullSnapshotId =
-                    (MetaPageUpdateLastSuccessfulFullSnapshotId)record;
-
-                buf.putInt(mpUpdateLastSuccFullSnapshotId.groupId());
-                buf.putLong(mpUpdateLastSuccFullSnapshotId.pageId());
-
-                buf.putLong(mpUpdateLastSuccFullSnapshotId.lastSuccessfulFullSnapshotId());
-
-                break;
-
-            case META_PAGE_UPDATE_LAST_SUCCESSFUL_SNAPSHOT_ID:
-                MetaPageUpdateLastSuccessfulSnapshotId mpUpdateLastSuccSnapshotId =
-                    (MetaPageUpdateLastSuccessfulSnapshotId)record;
-
-                buf.putInt(mpUpdateLastSuccSnapshotId.groupId());
-                buf.putLong(mpUpdateLastSuccSnapshotId.pageId());
-
-                buf.putLong(mpUpdateLastSuccSnapshotId.lastSuccessfulSnapshotId());
-                buf.putLong(mpUpdateLastSuccSnapshotId.lastSuccessfulSnapshotTag());
-
-                break;
-
-            case META_PAGE_UPDATE_LAST_ALLOCATED_INDEX:
-                MetaPageUpdateLastAllocatedIndex mpUpdateLastAllocatedIdx =
-                        (MetaPageUpdateLastAllocatedIndex) record;
-
-                buf.putInt(mpUpdateLastAllocatedIdx.groupId());
-                buf.putLong(mpUpdateLastAllocatedIdx.pageId());
-
-                buf.putInt(mpUpdateLastAllocatedIdx.lastAllocatedIndex());
-
-                break;
-
-            case PART_META_UPDATE_STATE:
-                PartitionMetaStateRecord partMetaStateRecord = (PartitionMetaStateRecord) record;
-
-                buf.putInt(partMetaStateRecord.groupId());
-
-                buf.putInt(partMetaStateRecord.partitionId());
-
-                buf.put(partMetaStateRecord.state());
-
-                buf.putLong(partMetaStateRecord.updateCounter());
-
-                break;
-
-            case PAGE_LIST_META_RESET_COUNT_RECORD:
-                PageListMetaResetCountRecord pageListMetaResetCntRecord = (PageListMetaResetCountRecord) record;
-
-                buf.putInt(pageListMetaResetCntRecord.groupId());
-                buf.putLong(pageListMetaResetCntRecord.pageId());
-
-                break;
-
-            case TX_RECORD:
-                txRecordSerializer.writeTxRecord((TxRecord) record, buf);
-
-                break;
-
-            default:
-                throw new UnsupportedOperationException("Type: " + record.type());
-        }
-
-        if (!skipCrc) {
-            int curPos = buf.position();
-
-            buf.position(startPos);
-
-            // This call will move buffer position to the end of the record again.
-            int crcVal = PureJavaCrc32.calcCrc32(buf, curPos - startPos);
-
-            buf.putInt(crcVal);
-        }
-        else
-            buf.putInt(0);
-    }
-
-    /** {@inheritDoc} */
-    @Override public WALRecord readRecord(FileInput in0, WALPointer expPtr) throws  IOException, IgniteCheckedException {
-        long startPos = -1;
-
-        try (FileInput.Crc32CheckingFileInput in = in0.startRead(skipCrc)) {
-            startPos = in0.position();
-
-            WALRecord res = readRecord(in, expPtr);
-
-            assert res != null;
-
-            int size = (int) (in0.position() - startPos + CRC_SIZE);
-
-            res.size(size); // Account for CRC which will be read afterwards.
-
-            if (writePointer && expPtr instanceof FileWALPointer) {
-                FileWALPointer ptr = (FileWALPointer) expPtr;
-
-                res.position(new FileWALPointer(ptr.index(), ptr.fileOffset(), size));
-            }
-
-            return res;
-        }
-        catch (EOFException | SegmentEofException e) {
-            throw e;
-        }
-        catch (Exception e) {
-            throw new IgniteCheckedException("Failed to read WAL record at position: " + startPos, e);
-        }
-    }
-
-    /**
-     * Loads record from input, does not read CRC value
-     *
-     * @param in Input to read record from
-     * @param expPtr expected WAL pointer for record. Used to validate actual position against expected from the file
-     * @throws SegmentEofException if end of WAL segment reached
-     */
-    private WALRecord readRecord(ByteBufferBackedDataInput in, WALPointer expPtr) throws IOException, IgniteCheckedException {
-        int type = in.readUnsignedByte();
-
-        if (type == WALRecord.RecordType.STOP_ITERATION_RECORD_TYPE)
-            throw new SegmentEofException("Reached logical end of the segment", null);
-
-        FileWALPointer ptr = readPosition(in);
-
-        if (!F.eq(ptr, expPtr))
-            throw new SegmentEofException("WAL segment rollover detected (will end iteration) [expPtr=" + expPtr +
-                ", readPtr=" + ptr + ']', null);
-
-        RecordType recType = RecordType.fromOrdinal(type - 1);
-
-        if (recType == null)
-            throw new IOException("Unknown record type: " + type);
-
-        WALRecord res;
-
-        switch (recType) {
-            case PAGE_RECORD:
-                byte[] arr = new byte[pageSize];
-
-                int cacheId = in.readInt();
-                long pageId = in.readLong();
-
-                in.readFully(arr);
-
-                res = new PageSnapshot(new FullPageId(pageId, cacheId), arr);
-
-                break;
-
-            case CHECKPOINT_RECORD:
-                long msb = in.readLong();
-                long lsb = in.readLong();
-                boolean hasPtr = in.readByte() != 0;
-                int idx = hasPtr ? in.readInt() : 0;
-                int offset = hasPtr ? in.readInt() : 0;
-                int len = hasPtr ? in.readInt() : 0;
-
-                Map<Integer, CacheState> states = readPartitionStates(in);
-
-                boolean end = in.readByte() != 0;
-
-                FileWALPointer walPtr = hasPtr ? new FileWALPointer(idx, offset, len) : null;
-
-                CheckpointRecord cpRec = new CheckpointRecord(new UUID(msb, lsb), walPtr, end);
-
-                cpRec.cacheGroupStates(states);
-
-                res = cpRec;
-
-                break;
-
-            case META_PAGE_INIT:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                int ioType = in.readUnsignedShort();
-                int ioVer = in.readUnsignedShort();
-                long treeRoot = in.readLong();
-                long reuseListRoot = in.readLong();
-
-                res = new MetaPageInitRecord(cacheId, pageId, ioType, ioVer, treeRoot, reuseListRoot);
-
-                break;
-
-            case PARTITION_META_PAGE_UPDATE_COUNTERS:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                long updCntr = in.readLong();
-                long rmvId = in.readLong();
-                int partSize = in.readInt();
-                long countersPageId = in.readLong();
-                byte state = in.readByte();
-                int allocatedIdxCandidate = in.readInt();
-
-                res = new MetaPageUpdatePartitionDataRecord(cacheId, pageId, updCntr, rmvId, partSize, countersPageId, state, allocatedIdxCandidate);
-
-                break;
-
-            case MEMORY_RECOVERY:
-                long ts = in.readLong();
-
-                res = new MemoryRecoveryRecord(ts);
-
-                break;
-
-            case PARTITION_DESTROY:
-                cacheId = in.readInt();
-                int partId = in.readInt();
-
-                res = new PartitionDestroyRecord(cacheId, partId);
-
-                break;
-
-            case DATA_RECORD:
-                int entryCnt = in.readInt();
-
-                List<DataEntry> entries = new ArrayList<>(entryCnt);
-
-                for (int i = 0; i < entryCnt; i++)
-                    entries.add(readDataEntry(in));
-
-                res = new DataRecord(entries);
-
-                break;
-
-            case HEADER_RECORD:
-                long magic = in.readLong();
-
-                if (magic != HeaderRecord.MAGIC)
-                    throw new EOFException("Magic is corrupted [exp=" + U.hexLong(HeaderRecord.MAGIC) +
-                        ", actual=" + U.hexLong(magic) + ']');
-
-                int ver = in.readInt();
-
-                res = new HeaderRecord(ver);
-
-                break;
-
-            case DATA_PAGE_INSERT_RECORD: {
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                int size = in.readUnsignedShort();
-
-                in.ensure(size);
-
-                byte[] payload = new byte[size];
-
-                in.readFully(payload);
-
-                res = new DataPageInsertRecord(cacheId, pageId, payload);
-
-                break;
-            }
-
-            case DATA_PAGE_UPDATE_RECORD: {
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                int itemId = in.readInt();
-
-                int size = in.readUnsignedShort();
-
-                in.ensure(size);
-
-                byte[] payload = new byte[size];
-
-                in.readFully(payload);
-
-                res = new DataPageUpdateRecord(cacheId, pageId, itemId, payload);
-
-                break;
-            }
-
-            case DATA_PAGE_INSERT_FRAGMENT_RECORD: {
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                final long lastLink = in.readLong();
-                final int payloadSize = in.readInt();
-
-                final byte[] payload = new byte[payloadSize];
-
-                in.readFully(payload);
-
-                res = new DataPageInsertFragmentRecord(cacheId, pageId, payload, lastLink);
-
-                break;
-            }
-
-            case DATA_PAGE_REMOVE_RECORD:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                int itemId = in.readUnsignedByte();
-
-                res = new DataPageRemoveRecord(cacheId, pageId, itemId);
-
-                break;
-
-            case DATA_PAGE_SET_FREE_LIST_PAGE:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                long freeListPage = in.readLong();
-
-                res = new DataPageSetFreeListPageRecord(cacheId, pageId, freeListPage);
-
-                break;
-
-            case INIT_NEW_PAGE_RECORD:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                ioType = in.readUnsignedShort();
-                ioVer = in.readUnsignedShort();
-                long virtualPageId = in.readLong();
-
-                res = new InitNewPageRecord(cacheId, pageId, ioType, ioVer, virtualPageId);
-
-                break;
-
-            case BTREE_META_PAGE_INIT_ROOT:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                long rootId = in.readLong();
-
-                res = new MetaPageInitRootRecord(cacheId, pageId, rootId);
-
-                break;
-
-            case BTREE_META_PAGE_INIT_ROOT2:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                long rootId2 = in.readLong();
-                int inlineSize = in.readShort();
-
-                res = new MetaPageInitRootInlineRecord(cacheId, pageId, rootId2, inlineSize);
-
-                break;
-
-            case BTREE_META_PAGE_ADD_ROOT:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                rootId = in.readLong();
-
-                res = new MetaPageAddRootRecord(cacheId, pageId, rootId);
-
-                break;
-
-            case BTREE_META_PAGE_CUT_ROOT:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                res = new MetaPageCutRootRecord(cacheId, pageId);
-
-                break;
-
-            case BTREE_INIT_NEW_ROOT:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                rootId = in.readLong();
-                ioType = in.readUnsignedShort();
-                ioVer = in.readUnsignedShort();
-                long leftId = in.readLong();
-                long rightId = in.readLong();
-
-                BPlusIO<?> io = BPlusIO.getBPlusIO(ioType, ioVer);
-
-                byte[] rowBytes = new byte[io.getItemSize()];
-
-                in.readFully(rowBytes);
-
-                res = new NewRootInitRecord<>(cacheId, pageId, rootId, (BPlusInnerIO<?>)io, leftId, rowBytes, rightId);
-
-                break;
-
-            case BTREE_PAGE_RECYCLE:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                long newPageId = in.readLong();
-
-                res = new RecycleRecord(cacheId, pageId, newPageId);
-
-                break;
-
-            case BTREE_PAGE_INSERT:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                ioType = in.readUnsignedShort();
-                ioVer = in.readUnsignedShort();
-                int itemIdx = in.readUnsignedShort();
-                rightId = in.readLong();
-
-                io = BPlusIO.getBPlusIO(ioType, ioVer);
-
-                rowBytes = new byte[io.getItemSize()];
-
-                in.readFully(rowBytes);
-
-                res = new InsertRecord<>(cacheId, pageId, io, itemIdx, rowBytes, rightId);
-
-                break;
-
-            case BTREE_FIX_LEFTMOST_CHILD:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                rightId = in.readLong();
-
-                res = new FixLeftmostChildRecord(cacheId, pageId, rightId);
-
-                break;
-
-            case BTREE_FIX_COUNT:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                int cnt = in.readUnsignedShort();
-
-                res = new FixCountRecord(cacheId, pageId, cnt);
-
-                break;
-
-            case BTREE_PAGE_REPLACE:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                ioType = in.readUnsignedShort();
-                ioVer = in.readUnsignedShort();
-                itemIdx = in.readUnsignedShort();
-
-                io = BPlusIO.getBPlusIO(ioType, ioVer);
-
-                rowBytes = new byte[io.getItemSize()];
-
-                in.readFully(rowBytes);
-
-                res = new ReplaceRecord<>(cacheId, pageId, io, rowBytes, itemIdx);
-
-                break;
-
-            case BTREE_PAGE_REMOVE:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                itemIdx = in.readUnsignedShort();
-                cnt = in.readUnsignedShort();
-
-                res = new RemoveRecord(cacheId, pageId, itemIdx, cnt);
-
-                break;
-
-            case BTREE_PAGE_INNER_REPLACE:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                int dstIdx = in.readUnsignedShort();
-                long srcPageId = in.readLong();
-                int srcIdx = in.readUnsignedShort();
-                rmvId = in.readLong();
-
-                res = new InnerReplaceRecord<>(cacheId, pageId, dstIdx, srcPageId, srcIdx, rmvId);
-
-                break;
-
-            case BTREE_FORWARD_PAGE_SPLIT:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                long fwdId = in.readLong();
-                ioType = in.readUnsignedShort();
-                ioVer = in.readUnsignedShort();
-                srcPageId = in.readLong();
-                int mid = in.readUnsignedShort();
-                cnt = in.readUnsignedShort();
-
-                res = new SplitForwardPageRecord(cacheId, pageId, fwdId, ioType, ioVer, srcPageId, mid, cnt);
-
-                break;
-
-            case BTREE_EXISTING_PAGE_SPLIT:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                mid = in.readUnsignedShort();
-                fwdId = in.readLong();
-
-                res = new SplitExistingPageRecord(cacheId, pageId, mid, fwdId);
-
-                break;
-
-            case BTREE_PAGE_MERGE:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                long prntId = in.readLong();
-                int prntIdx = in.readUnsignedShort();
-                rightId = in.readLong();
-                boolean emptyBranch = in.readBoolean();
-
-                res = new MergeRecord<>(cacheId, pageId, prntId, prntIdx, rightId, emptyBranch);
-
-                break;
-
-            case BTREE_FIX_REMOVE_ID:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                rmvId = in.readLong();
-
-                res = new FixRemoveId(cacheId, pageId, rmvId);
-
-                break;
-
-            case PAGES_LIST_SET_NEXT:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-                long nextPageId = in.readLong();
-
-                res = new PagesListSetNextRecord(cacheId, pageId, nextPageId);
-
-                break;
-
-            case PAGES_LIST_SET_PREVIOUS:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-                long prevPageId = in.readLong();
-
-                res = new PagesListSetPreviousRecord(cacheId, pageId, prevPageId);
-
-                break;
-
-            case PAGES_LIST_INIT_NEW_PAGE:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-                ioType = in.readInt();
-                ioVer = in.readInt();
-                newPageId = in.readLong();
-                prevPageId = in.readLong();
-                long addDataPageId = in.readLong();
-
-                res = new PagesListInitNewPageRecord(cacheId, pageId, ioType, ioVer, newPageId, prevPageId, addDataPageId);
-
-                break;
-
-            case PAGES_LIST_ADD_PAGE:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-                long dataPageId = in.readLong();
-
-                res = new PagesListAddPageRecord(cacheId, pageId, dataPageId);
-
-                break;
-
-            case PAGES_LIST_REMOVE_PAGE:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-                long rmvdPageId = in.readLong();
-
-                res = new PagesListRemovePageRecord(cacheId, pageId, rmvdPageId);
-
-                break;
-
-            case TRACKING_PAGE_DELTA:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                long pageIdToMark = in.readLong();
-                long nextSnapshotId0 = in.readLong();
-                long lastSuccessfulSnapshotId0 = in.readLong();
-
-                res = new TrackingPageDeltaRecord(cacheId, pageId, pageIdToMark, nextSnapshotId0, lastSuccessfulSnapshotId0);
-
-                break;
-
-            case META_PAGE_UPDATE_NEXT_SNAPSHOT_ID:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                long nextSnapshotId = in.readLong();
-
-                res = new MetaPageUpdateNextSnapshotId(cacheId, pageId, nextSnapshotId);
-
-                break;
-
-            case META_PAGE_UPDATE_LAST_SUCCESSFUL_FULL_SNAPSHOT_ID:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                long lastSuccessfulFullSnapshotId = in.readLong();
-
-                res = new MetaPageUpdateLastSuccessfulFullSnapshotId(cacheId, pageId, lastSuccessfulFullSnapshotId);
-
-                break;
-
-            case META_PAGE_UPDATE_LAST_SUCCESSFUL_SNAPSHOT_ID:
-                cacheId = in.readInt();
-                pageId = in.readLong();
-
-                long lastSuccessfulSnapshotId = in.readLong();
-                long lastSuccessfulSnapshotTag = in.readLong();
-
-                res = new MetaPageUpdateLastSuccessfulSnapshotId(cacheId, pageId, lastSuccessfulSnapshotId, lastSuccessfulSnapshotTag);
+    /** Skip CRC calculation/check flag */
+    public static boolean SKIP_CRC = IgniteSystemProperties.getBoolean(IGNITE_PDS_SKIP_CRC, false);
 
-                break;
+    /** V1 data serializer. */
+    private final RecordDataV1Serializer dataSerializer;
 
-            case META_PAGE_UPDATE_LAST_ALLOCATED_INDEX:
-                cacheId = in.readInt();
-                pageId = in.readLong();
+    /** Write pointer. */
+    private final boolean writePointer;
 
-                int lastAllocatedIdx = in.readInt();
+    /** Record read/write functional interface. */
+    private final RecordIO recordIO = new RecordIO() {
 
-                res = new MetaPageUpdateLastAllocatedIndex(cacheId, pageId, lastAllocatedIdx);
+        /** {@inheritDoc} */
+        @Override public int sizeWithHeaders(WALRecord record) throws IgniteCheckedException {
+            return dataSerializer.size(record) + REC_TYPE_SIZE + FILE_WAL_POINTER_SIZE + CRC_SIZE;
+        }
 
-                break;
+        /** {@inheritDoc} */
+        @Override public WALRecord readWithHeaders(ByteBufferBackedDataInput in, WALPointer expPtr) throws IOException, IgniteCheckedException {
+            RecordType recType = readRecordType(in);
 
-            case PART_META_UPDATE_STATE:
-                cacheId = in.readInt();
-                partId = in.readInt();
+            if (recType == RecordType.SWITCH_SEGMENT_RECORD)
+                throw new SegmentEofException("Reached end of segment", null);
 
-                state = in.readByte();
+            FileWALPointer ptr = readPosition(in);
 
-                long updateCounter = in.readLong();
+            if (!F.eq(ptr, expPtr))
+                throw new SegmentEofException("WAL segment rollover detected (will end iteration) [expPtr=" + expPtr +
+                        ", readPtr=" + ptr + ']', null);
 
-                res = new PartitionMetaStateRecord(cacheId, partId, GridDhtPartitionState.fromOrdinal(state), updateCounter);
+            return dataSerializer.readRecord(recType, in);
+        }
 
-                break;
+        /** {@inheritDoc} */
+        @Override public void writeWithHeaders(WALRecord record, ByteBuffer buf) throws IgniteCheckedException {
+            // Write record type.
+            putRecordType(buf, record);
 
-            case PAGE_LIST_META_RESET_COUNT_RECORD:
-                cacheId = in.readInt();
-                pageId = in.readLong();
+            // Write record file position.
+            putPositionOfRecord(buf, record);
 
-                res = new PageListMetaResetCountRecord(cacheId, pageId);
-                break;
+            // Write record data.
+            dataSerializer.writeRecord(record, buf);
+        }
+    };
 
-            case SWITCH_SEGMENT_RECORD:
-                throw new EOFException("END OF SEGMENT");
+    /**
+     * Create an instance of V1 serializer.
+     *
+     * @param dataSerializer V1 data serializer.
+     * @param writePointer Write pointer.
+     */
+    public RecordV1Serializer(RecordDataV1Serializer dataSerializer, boolean writePointer) {
+        this.dataSerializer = dataSerializer;
+        this.writePointer = writePointer;
+    }
 
-            case TX_RECORD:
-                res = txRecordSerializer.readTxRecord(in);
+    /** {@inheritDoc} */
+    @Override public int version() {
+        return 1;
+    }
 
-                break;
+    /** {@inheritDoc} */
+    @Override public boolean writePointer() {
+        return writePointer;
+    }
 
-            default:
-                throw new UnsupportedOperationException("Type: " + recType);
-        }
+    /** {@inheritDoc} */
+    @SuppressWarnings("CastConflictsWithInstanceof")
+    @Override public void writeRecord(WALRecord record, ByteBuffer buf) throws IgniteCheckedException {
+        writeWithCrc(record, buf, recordIO);
+    }
 
-        return res;
+    /** {@inheritDoc} */
+    @Override public WALRecord readRecord(FileInput in0, WALPointer expPtr) throws  IOException, IgniteCheckedException {
+        return readWithCrc(in0, expPtr, recordIO);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("CastConflictsWithInstanceof")
     @Override public int size(WALRecord record) throws IgniteCheckedException {
-        int commonFields = REC_TYPE_SIZE + FILE_WAL_POINTER_SIZE + CRC_SIZE;
-
-        switch (record.type()) {
-            case PAGE_RECORD:
-                assert record instanceof PageSnapshot;
-
-                PageSnapshot pageRec = (PageSnapshot)record;
-
-                return commonFields + pageRec.pageData().length + 12;
-
-            case CHECKPOINT_RECORD:
-                CheckpointRecord cpRec = (CheckpointRecord)record;
-
-                assert cpRec.checkpointMark() == null || cpRec.checkpointMark() instanceof FileWALPointer :
-                    "Invalid WAL record: " + cpRec;
-
-                int cacheStatesSize = cacheStatesSize(cpRec.cacheGroupStates());
-
-                FileWALPointer walPtr = (FileWALPointer)cpRec.checkpointMark();
-
-                return commonFields + 18 + cacheStatesSize + (walPtr == null ? 0 : 16);
-
-            case META_PAGE_INIT:
-                return commonFields + /*cache ID*/4 + /*page ID*/8 + /*ioType*/2  + /*ioVer*/2 +  /*tree root*/8 + /*reuse root*/8;
-
-            case PARTITION_META_PAGE_UPDATE_COUNTERS:
-                return commonFields + /*cache ID*/4 + /*page ID*/8 + /*upd cntr*/8 + /*rmv id*/8 + /*part size*/4 + /*counters page id*/8 + /*state*/ 1
-                    + /*allocatedIdxCandidate*/ 4;
-
-            case MEMORY_RECOVERY:
-                return commonFields + 8;
-
-            case PARTITION_DESTROY:
-                return commonFields + /*cacheId*/4 + /*partId*/4;
-
-            case DATA_RECORD:
-                DataRecord dataRec = (DataRecord)record;
-
-                return commonFields + 4 + dataSize(dataRec);
-
-            case HEADER_RECORD:
-                return HEADER_RECORD_SIZE;
-
-            case DATA_PAGE_INSERT_RECORD:
-                DataPageInsertRecord diRec = (DataPageInsertRecord)record;
-
-                return commonFields + 4 + 8 + 2 + diRec.payload().length;
-
-            case DATA_PAGE_UPDATE_RECORD:
-                DataPageUpdateRecord uRec = (DataPageUpdateRecord)record;
-
-                return commonFields + 4 + 8 + 2 + 4 +
-                    uRec.payload().length;
-
-            case DATA_PAGE_INSERT_FRAGMENT_RECORD:
-                final DataPageInsertFragmentRecord difRec = (DataPageInsertFragmentRecord)record;
-
-                return commonFields + 4 + 8 + 8 + 4 + difRec.payloadSize();
-
-            case DATA_PAGE_REMOVE_RECORD:
-                return commonFields + 4 + 8 + 1;
-
-            case DATA_PAGE_SET_FREE_LIST_PAGE:
-                return commonFields + 4 + 8 + 8;
-
-            case INIT_NEW_PAGE_RECORD:
-                return commonFields + 4 + 8 + 2 + 2 + 8;
-
-            case BTREE_META_PAGE_INIT_ROOT:
-                return commonFields + 4 + 8 + 8;
-
-            case BTREE_META_PAGE_INIT_ROOT2:
-                return commonFields + 4 + 8 + 8 + 2;
-
-            case BTREE_META_PAGE_ADD_ROOT:
-                return commonFields + 4 + 8 + 8;
-
-            case BTREE_META_PAGE_CUT_ROOT:
-                return commonFields + 4 + 8;
-
-            case BTREE_INIT_NEW_ROOT:
-                NewRootInitRecord<?> riRec = (NewRootInitRecord<?>)record;
-
-                return commonFields + 4 + 8 + 8 + 2 + 2 + 8 + 8 + riRec.io().getItemSize();
-
-            case BTREE_PAGE_RECYCLE:
-                return commonFields + 4 + 8 + 8;
-
-            case BTREE_PAGE_INSERT:
-                InsertRecord<?> inRec = (InsertRecord<?>)record;
-
-                return commonFields + 4 + 8 + 2 + 2 + 2 + 8 + inRec.io().getItemSize();
-
-            case BTREE_FIX_LEFTMOST_CHILD:
-                return commonFields + 4 + 8 + 8;
-
-            case BTREE_FIX_COUNT:
-                return commonFields + 4 + 8 + 2;
-
-            case BTREE_PAGE_REPLACE:
-                ReplaceRecord<?> rRec = (ReplaceRecord<?>)record;
-
-                return commonFields + 4 + 8 + 2 + 2 + 2 + rRec.io().getItemSize();
-
-            case BTREE_PAGE_REMOVE:
-                return commonFields + 4 + 8 + 2 + 2;
-
-            case BTREE_PAGE_INNER_REPLACE:
-                return commonFields + 4 + 8 + 2 + 8 + 2 + 8;
-
-            case BTREE_FORWARD_PAGE_SPLIT:
-                return commonFields + 4 + 8 + 8 + 2 + 2 + 8 + 2 + 2;
-
-            case BTREE_EXISTING_PAGE_SPLIT:
-                return commonFields + 4 + 8 + 2 + 8;
-
-            case BTREE_PAGE_MERGE:
-                return commonFields + 4 + 8 + 8 + 2 + 8 + 1;
-
-            case BTREE_FIX_REMOVE_ID:
-                return commonFields + 4 + 8 + 8;
-
-            case PAGES_LIST_SET_NEXT:
-                return commonFields + 4 + 8 + 8;
-
-            case PAGES_LIST_SET_PREVIOUS:
-                return commonFields + 4 + 8 + 8;
-
-            case PAGES_LIST_INIT_NEW_PAGE:
-                return commonFields + 4 + 8 + 4 + 4 + 8 + 8 + 8;
-
-            case PAGES_LIST_ADD_PAGE:
-                return commonFields + 4 + 8 + 8;
-
-            case PAGES_LIST_REMOVE_PAGE:
-                return commonFields + 4 + 8 + 8;
-
-            case TRACKING_PAGE_DELTA:
-                return commonFields + 4 + 8 + 8 + 8 + 8;
-
-            case META_PAGE_UPDATE_LAST_SUCCESSFUL_SNAPSHOT_ID:
-                return commonFields + 4 + 8 + 8 + 8;
-
-            case META_PAGE_UPDATE_LAST_SUCCESSFUL_FULL_SNAPSHOT_ID:
-                return commonFields + 4 + 8 + 8;
-
-            case META_PAGE_UPDATE_NEXT_SNAPSHOT_ID:
-                return commonFields + 4 + 8 + 8;
-
-            case META_PAGE_UPDATE_LAST_ALLOCATED_INDEX:
-                return commonFields + 4 + 8 + 4;
-
-            case PART_META_UPDATE_STATE:
-                return commonFields + /*cacheId*/ 4 + /*partId*/ 4 + /*State*/1 + /*Update Counter*/ 8;
-
-            case PAGE_LIST_META_RESET_COUNT_RECORD:
-                return commonFields + /*cacheId*/ 4 + /*pageId*/ 8;
-
-            case SWITCH_SEGMENT_RECORD:
-                return commonFields - CRC_SIZE; //CRC is not loaded for switch segment, exception is thrown instead
-
-            case TX_RECORD:
-                return commonFields + txRecordSerializer.sizeOfTxRecord((TxRecord) record);
-
-            default:
-                throw new UnsupportedOperationException("Type: " + record.type());
-        }
+        return recordIO.sizeWithHeaders(record);
     }
 
     /**
@@ -1463,213 +170,106 @@ public class RecordV1Serializer implements RecordSerializer {
     }
 
     /**
-     * @param dataRec Data record to serialize.
-     * @return Full data record size.
-     * @throws IgniteCheckedException If failed to obtain the length of one of the entries.
-     */
-    private int dataSize(DataRecord dataRec) throws IgniteCheckedException {
-        int sz = 0;
-
-        for (DataEntry entry : dataRec.writeEntries())
-            sz += entrySize(entry);
-
-        return sz;
-    }
-
-    /**
-     * @param entry Entry to get size for.
-     * @return Entry size.
-     * @throws IgniteCheckedException If failed to get key or value bytes length.
+     * Writes record file position to given {@code buf}.
+     *
+     * @param buf Buffer to write record file position.
+     * @param record WAL record.
      */
-    private int entrySize(DataEntry entry) throws IgniteCheckedException {
-        GridCacheContext cctx = this.cctx.cacheContext(entry.cacheId());
-        CacheObjectContext coCtx = cctx.cacheObjectContext();
-
-        return
-            /*cache ID*/4 +
-            /*key*/entry.key().valueBytesLength(coCtx) +
-            /*value*/(entry.value() == null ? 4 : entry.value().valueBytesLength(coCtx)) +
-            /*op*/1 +
-            /*near xid ver*/CacheVersionIO.size(entry.nearXidVersion(), true) +
-            /*write ver*/CacheVersionIO.size(entry.writeVersion(), false) +
-            /*part ID*/4 +
-            /*expire Time*/8 +
-            /*part cnt*/8;
+    public static void putPositionOfRecord(ByteBuffer buf, WALRecord record) {
+        putPosition(buf, (FileWALPointer) record.position());
     }
 
     /**
-     * @param states Partition states.
-     * @return Size required to write partition states.
+     * Writes record type to given {@code buf}.
+     *
+     * @param buf Buffer to write record type.
+     * @param record WAL record.
      */
-    private int cacheStatesSize(Map<Integer, CacheState> states) {
-        // Need 4 bytes for the number of caches.
-        int size = 2;
-
-        for (Map.Entry<Integer, CacheState> entry : states.entrySet()) {
-            // Cache ID.
-            size += 4;
-
-            // Need 2 bytes for the number of partitions.
-            size += 2;
-
-            CacheState state = entry.getValue();
-
-            // 2 bytes partition ID, size and counter per partition.
-            size += 18 * state.size();
-        }
-
-        return size;
+    public static void putRecordType(ByteBuffer buf, WALRecord record) {
+        buf.put((byte)(record.type().ordinal() + 1));
     }
 
     /**
-     * @param buf Buffer to write to.
-     * @param entry Data entry.
+     * Reads record type from given {@code in}.
+     *
+     * @param in Buffer to read record type.
+     * @return Record type.
+     * @throws IgniteCheckedException If logical end of segment is reached.
+     * @throws IOException In case of I/O problems.
      */
-    private void putDataEntry(ByteBuffer buf, DataEntry entry) throws IgniteCheckedException {
-        buf.putInt(entry.cacheId());
-
-        if (!entry.key().putValue(buf))
-            throw new AssertionError();
+    public static RecordType readRecordType(DataInput in) throws IgniteCheckedException, IOException {
+        int type = in.readUnsignedByte();
 
-        if (entry.value() == null)
-            buf.putInt(-1);
-        else if (!entry.value().putValue(buf))
-            throw new AssertionError();
+        if (type == WALRecord.RecordType.STOP_ITERATION_RECORD_TYPE)
+            throw new SegmentEofException("Reached logical end of the segment", null);
 
-        buf.put((byte)entry.op().ordinal());
+        RecordType recType = RecordType.fromOrdinal(type - 1);
 
-        putVersion(buf, entry.nearXidVersion(), true);
-        putVersion(buf, entry.writeVersion(), false);
+        if (recType == null)
+            throw new IOException("Unknown record type: " + type);
 
-        buf.putInt(entry.partitionId());
-        buf.putLong(entry.partitionCounter());
-        buf.putLong(entry.expireTime());
+        return recType;
     }
 
     /**
-     * @param states Cache states.
+     * Reads record from file {@code in0} and validates CRC of record.
+     *
+     * @param in0 File input.
+     * @param expPtr Expected WAL pointer for record. Used to validate actual position against expected from the file.
+     * @param reader Record reader I/O interface.
+     * @return WAL record.
+     * @throws EOFException In case of end of file.
+     * @throws IgniteCheckedException If it's unable to read record.
      */
-    private void putCacheStates(ByteBuffer buf, Map<Integer, CacheState> states) {
-        buf.putShort((short)states.size());
+    public static WALRecord readWithCrc(FileInput in0, WALPointer expPtr, RecordIO reader) throws EOFException, IgniteCheckedException {
+        long startPos = -1;
 
-        for (Map.Entry<Integer, CacheState> entry : states.entrySet()) {
-            buf.putInt(entry.getKey());
+        try (FileInput.Crc32CheckingFileInput in = in0.startRead(SKIP_CRC)) {
+            startPos = in0.position();
 
-            CacheState state = entry.getValue();
+            WALRecord res = reader.readWithHeaders(in, expPtr);
 
-            // Need 2 bytes for the number of partitions.
-            buf.putShort((short)state.size());
+            assert res != null;
 
-            for (int i = 0; i < state.size(); i++) {
-                buf.putShort((short)state.partitionByIndex(i));
+            res.size((int)(in0.position() - startPos + CRC_SIZE)); // Account for CRC which will be read afterwards.
 
-                buf.putLong(state.partitionSizeByIndex(i));
-                buf.putLong(state.partitionCounterByIndex(i));
-            }
+            return res;
         }
-    }
-
-    /**
-     * @param in Input to read from.
-     * @return Read entry.
-     */
-    private DataEntry readDataEntry(ByteBufferBackedDataInput in) throws IOException, IgniteCheckedException {
-        int cacheId = in.readInt();
-
-        int keySize = in.readInt();
-        byte keyType = in.readByte();
-        byte[] keyBytes = new byte[keySize];
-        in.readFully(keyBytes);
-
-        int valSize = in.readInt();
-
-        byte valType = 0;
-        byte[] valBytes = null;
-
-        if (valSize >= 0) {
-            valType = in.readByte();
-            valBytes = new byte[valSize];
-            in.readFully(valBytes);
+        catch (EOFException | SegmentEofException | WalSegmentTailReachedException e) {
+            throw e;
         }
-
-        byte ord = in.readByte();
-
-        GridCacheOperation op = GridCacheOperation.fromOrdinal(ord & 0xFF);
-
-        GridCacheVersion nearXidVer = readVersion(in, true);
-        GridCacheVersion writeVer = readVersion(in, false);
-
-        int partId = in.readInt();
-        long partCntr = in.readLong();
-        long expireTime = in.readLong();
-
-        GridCacheContext cacheCtx = cctx.cacheContext(cacheId);
-
-        if (cacheCtx != null) {
-            CacheObjectContext coCtx = cacheCtx.cacheObjectContext();
-
-            KeyCacheObject key = co.toKeyCacheObject(coCtx, keyType, keyBytes);
-            CacheObject val = valBytes != null ? co.toCacheObject(coCtx, valType, valBytes) : null;
-
-            return new DataEntry(
-                cacheId,
-                key,
-                val,
-                op,
-                nearXidVer,
-                writeVer,
-                expireTime,
-                partId,
-                partCntr
-            );
+        catch (Exception e) {
+            throw new IgniteCheckedException("Failed to read WAL record at position: " + startPos, e);
         }
-        else
-            return new LazyDataEntry(
-                cctx,
-                cacheId,
-                keyType,
-                keyBytes,
-                valType,
-                valBytes,
-                op,
-                nearXidVer,
-                writeVer,
-                expireTime,
-                partId,
-                partCntr);
     }
 
     /**
-     * @param buf Buffer to read from.
-     * @return Read map.
+     * Writes record with calculated CRC to buffer {@code buf}.
+     *
+     * @param record WAL record.
+     * @param buf Buffer to write.
+     * @param writer Record write I/O interface.
+     * @throws IgniteCheckedException If it's unable to write record.
      */
-    private Map<Integer, CacheState> readPartitionStates(DataInput buf) throws IOException {
-        int caches = buf.readShort() & 0xFFFF;
-
-        if (caches == 0)
-            return Collections.emptyMap();
+    public static void writeWithCrc(WALRecord record, ByteBuffer buf, RecordIO writer) throws IgniteCheckedException {
+        assert record.size() >= 0 && buf.remaining() >= record.size() : record.size();
 
-        Map<Integer, CacheState> states = new HashMap<>(caches, 1.0f);
-
-        for (int i = 0; i < caches; i++) {
-            int cacheId = buf.readInt();
+        int startPos = buf.position();
 
-            int parts = buf.readShort() & 0xFFFF;
+        writer.writeWithHeaders(record, buf);
 
-            CacheState state = new CacheState(parts);
+        if (!SKIP_CRC) {
+            int curPos = buf.position();
 
-            for (int p = 0; p < parts; p++) {
-                int partId = buf.readShort() & 0xFFFF;
-                long size = buf.readLong();
-                long partCntr = buf.readLong();
+            buf.position(startPos);
 
-                state.addPartitionState(partId, size, partCntr);
-            }
+            // This call will move buffer position to the end of the record again.
+            int crcVal = PureJavaCrc32.calcCrc32(buf, curPos - startPos);
 
-            states.put(cacheId, state);
+            buf.putInt(crcVal);
         }
-
-        return states;
+        else
+            buf.putInt(0);
     }
 
     /**
@@ -1703,15 +303,4 @@ public class RecordV1Serializer implements RecordSerializer {
             throw new IOException(e);
         }
     }
-
-    /**
-     * @param buf Buffer.
-     * @param rowBytes Row bytes.
-     */
-    @SuppressWarnings("unchecked")
-    private static void putRow(ByteBuffer buf, byte[] rowBytes) {
-        assert rowBytes.length > 0;
-
-        buf.put(rowBytes);
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/00770767/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java
new file mode 100644
index 0000000..0a5bf01
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java
@@ -0,0 +1,170 @@
+/*
+ * 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.persistence.wal.serializer;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.wal.WALPointer;
+import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
+import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput;
+import org.apache.ignite.internal.processors.cache.persistence.wal.FileInput;
+import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.RecordSerializer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentEofException;
+import org.apache.ignite.internal.processors.cache.persistence.wal.WalSegmentTailReachedException;
+import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.io.RecordIO;
+import org.apache.ignite.internal.util.typedef.F;
+
+import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.CRC_SIZE;
+import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.REC_TYPE_SIZE;
+
+/**
+ * Record V2 serializer.
+ * Stores records in following format:
+ * <ul>
+ * <li>Record type from {@link WALRecord.RecordType#ordinal()} incremented by 1</li>
+ * <li>WAL pointer to double check consistency</li>
+ * <li>Record length</li>
+ * <li>Data</li>
+ * <li>CRC or zero padding</li>
+ * </ul>
+ */
+public class RecordV2Serializer implements RecordSerializer {
+    /** Length of WAL Pointer: Index (8) + File offset (4) + Record length (4) */
+    public static final int FILE_WAL_POINTER_SIZE = 8 + 4 + 4;
+
+    /** V2 data serializer. */
+    private final RecordDataV2Serializer dataSerializer;
+
+    /** Write pointer. */
+    private final boolean writePointer;
+
+    /** Record read/write functional interface. */
+    private final RecordIO recordIO = new RecordIO() {
+
+        /** {@inheritDoc} */
+        @Override public int sizeWithHeaders(WALRecord record) throws IgniteCheckedException {
+            return dataSerializer.size(record) + REC_TYPE_SIZE + FILE_WAL_POINTER_SIZE + CRC_SIZE;
+        }
+
+        /** {@inheritDoc} */
+        @Override public WALRecord readWithHeaders(
+            ByteBufferBackedDataInput in,
+            WALPointer expPtr
+        ) throws IOException, IgniteCheckedException {
+            WALRecord.RecordType recType = RecordV1Serializer.readRecordType(in);
+
+            if (recType == WALRecord.RecordType.SWITCH_SEGMENT_RECORD)
+                throw new SegmentEofException("Reached end of segment", null);
+
+            FileWALPointer ptr = readPositionAndCheckPoint(in, expPtr);
+
+            return dataSerializer.readRecord(recType, in);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeWithHeaders(
+            WALRecord record,
+            ByteBuffer buf
+        ) throws IgniteCheckedException {
+            // Write record type.
+            RecordV1Serializer.putRecordType(buf, record);
+
+            // Write record file position.
+            putPositionOfRecord(buf, record);
+
+            // Write record data.
+            dataSerializer.writeRecord(record, buf);
+        }
+    };
+
+    /**
+     * Create an instance of Record V2 serializer.
+     *
+     * @param dataSerializer V2 data serializer.
+     */
+    public RecordV2Serializer(RecordDataV2Serializer dataSerializer, boolean writePointer) {
+        this.dataSerializer = dataSerializer;
+        this.writePointer = writePointer;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int version() {
+        return 2;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writePointer() {
+        return writePointer;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int size(WALRecord record) throws IgniteCheckedException {
+        return recordIO.sizeWithHeaders(record);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeRecord(WALRecord record, ByteBuffer buf) throws IgniteCheckedException {
+        RecordV1Serializer.writeWithCrc(record, buf, recordIO);
+    }
+
+    /** {@inheritDoc} */
+    @Override public WALRecord readRecord(FileInput in, WALPointer expPtr) throws IOException, IgniteCheckedException {
+        return RecordV1Serializer.readWithCrc(in, expPtr, recordIO);
+    }
+
+    /**
+     * @param in Data input to read pointer from.
+     * @return Read file WAL pointer.
+     * @throws IOException If failed to write.
+     */
+    public static FileWALPointer readPositionAndCheckPoint(
+        DataInput in,
+        WALPointer expPtr
+    ) throws IgniteCheckedException, IOException {
+        long idx = in.readLong();
+        int fileOffset = in.readInt();
+        int length = in.readInt();
+
+        FileWALPointer p = (FileWALPointer)expPtr;
+
+        if (!F.eq(idx, p.index()) || !F.eq(fileOffset, p.fileOffset()))
+            throw new WalSegmentTailReachedException(
+                "WAL segment tail is reached. [ " +
+                        "Expected next state: {Index=" + p.index() + ",Offset=" + p.fileOffset() + "}, " +
+                        "Actual state : {Index=" + idx + ",Offset=" + fileOffset + "} ]", null);
+
+        return new FileWALPointer(idx, fileOffset, length);
+    }
+
+    /**
+     * Writes record file position to given {@code buf}.
+     *
+     * @param buf Buffer to write record file position.
+     * @param record WAL record.
+     */
+    public static void putPositionOfRecord(ByteBuffer buf, WALRecord record) {
+        FileWALPointer p = (FileWALPointer)record.position();
+
+        buf.putLong(p.index());
+        buf.putInt(p.fileOffset());
+        buf.putInt(record.size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/00770767/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/io/RecordIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/io/RecordIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/io/RecordIO.java
new file mode 100644
index 0000000..d609e61
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/io/RecordIO.java
@@ -0,0 +1,60 @@
+/*
+ * 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.persistence.wal.serializer.io;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.wal.WALPointer;
+import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
+import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput;
+
+/**
+ * Internal interface to provide size, read and write operations of WAL records
+ * including record header and data.
+ */
+public interface RecordIO {
+    /**
+     * Calculates and returns size of record data and headers.
+     *
+     * @param record WAL record.
+     * @return Size in bytes.
+     * @throws IgniteCheckedException If it's unable to calculate size of record.
+     */
+    int sizeWithHeaders(WALRecord record) throws IgniteCheckedException;
+
+    /**
+     * Reads record data with headers from {@code in}.
+     *
+     * @param in Buffer to read.
+     * @param expPtr Expected WAL pointer for record. Used to validate actual position against expected from the file.
+     * @return WAL record.
+     * @throws IOException In case of I/O problems.
+     * @throws IgniteCheckedException If it's unable to read record.
+     */
+    WALRecord readWithHeaders(ByteBufferBackedDataInput in, WALPointer expPtr) throws IOException, IgniteCheckedException;
+
+    /**
+     * Writes record data with headers to {@code buf}.
+     *
+     * @param record WAL record.
+     * @param buf Buffer to write.
+     * @throws IgniteCheckedException If it's unable to write record.
+     */
+    void writeWithHeaders(WALRecord record, ByteBuffer buf) throws IgniteCheckedException;
+}


[24/37] ignite git commit: IGNITE-6520: Using actual AffinityReadyFuture result

Posted by vo...@apache.org.
IGNITE-6520: Using actual AffinityReadyFuture result


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

Branch: refs/heads/ignite-2.3
Commit: 9160d5eaf75d07ef16f6fe02b09aaa2ed76132b5
Parents: 013d7db
Author: Andrey Kuznetsov <st...@gmail.com>
Authored: Mon Oct 2 11:28:32 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Oct 2 11:28:32 2017 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/GridPartitionedGetFuture.java        | 6 ++----
 .../cache/distributed/dht/GridPartitionedSingleGetFuture.java  | 6 ++----
 .../processors/cache/distributed/near/GridNearGetFuture.java   | 6 ++----
 3 files changed, 6 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9160d5ea/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
index 73c0ea5..015eb82 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
@@ -713,17 +713,15 @@ public class GridPartitionedGetFuture<K, V> extends CacheDistributedGetFutureAda
                 onDone(Collections.<K, V>emptyMap());
             }
             else {
-                final AffinityTopologyVersion updTopVer =
+                AffinityTopologyVersion updTopVer =
                     new AffinityTopologyVersion(Math.max(topVer.topologyVersion() + 1, cctx.discovery().topologyVersion()));
 
                 cctx.affinity().affinityReadyFuture(updTopVer).listen(
                     new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                         @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
                             try {
-                                fut.get();
-
                                 // Remap.
-                                map(keys.keySet(), F.t(node, keys), updTopVer);
+                                map(keys.keySet(), F.t(node, keys), fut.get());
 
                                 onDone(Collections.<K, V>emptyMap());
                             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9160d5ea/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
index 234ee91..61489e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
@@ -703,16 +703,14 @@ public class GridPartitionedSingleGetFuture extends GridCacheFutureAdapter<Objec
             return false;
 
         if (canRemap) {
-            final AffinityTopologyVersion updTopVer = new AffinityTopologyVersion(
+            AffinityTopologyVersion updTopVer = new AffinityTopologyVersion(
                 Math.max(topVer.topologyVersion() + 1, cctx.discovery().topologyVersion()));
 
             cctx.affinity().affinityReadyFuture(updTopVer).listen(
                 new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                     @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
                         try {
-                            fut.get();
-
-                            remap(updTopVer);
+                            remap(fut.get());
                         }
                         catch (IgniteCheckedException e) {
                             onDone(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/9160d5ea/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index 807270d..2361507 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -917,17 +917,15 @@ public final class GridNearGetFuture<K, V> extends CacheDistributedGetFutureAdap
                 onDone(Collections.<K, V>emptyMap());
             }
             else {
-                final AffinityTopologyVersion updTopVer =
+                AffinityTopologyVersion updTopVer =
                     new AffinityTopologyVersion(Math.max(topVer.topologyVersion() + 1, cctx.discovery().topologyVersion()));
 
                 cctx.affinity().affinityReadyFuture(updTopVer).listen(
                     new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                         @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
                             try {
-                                fut.get();
-
                                 // Remap.
-                                map(keys.keySet(), F.t(node, keys), updTopVer);
+                                map(keys.keySet(), F.t(node, keys), fut.get());
 
                                 onDone(Collections.<K, V>emptyMap());
                             }


[06/37] ignite git commit: IGNITE-6046: JDBC thin driver: allowed multiple statements execution (e.g. "SELECT something; SELECT something_else". This closes #2618.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 076468b..aaa8e57 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -134,6 +134,7 @@ import org.apache.ignite.internal.processors.query.IgniteSqlSegmentedIndexMultiN
 import org.apache.ignite.internal.processors.query.IgniteSqlSegmentedIndexSelfTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest;
 import org.apache.ignite.internal.processors.query.LazyQuerySelfTest;
+import org.apache.ignite.internal.processors.query.MultipleStatementsSqlQuerySelfTest;
 import org.apache.ignite.internal.processors.query.SqlSchemaSelfTest;
 import org.apache.ignite.internal.processors.query.h2.GridH2IndexingInMemSelfTest;
 import org.apache.ignite.internal.processors.query.h2.GridH2IndexingOffheapSelfTest;
@@ -161,6 +162,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(ClientConnectorConfigurationValidationSelfTest.class);
 
         suite.addTestSuite(SqlSchemaSelfTest.class);
+        suite.addTestSuite(MultipleStatementsSqlQuerySelfTest.class);
 
         // Misc tests.
         // TODO: Enable when IGNITE-1094 is fixed.


[08/37] ignite git commit: ignite-6445 Deadlock detection triggering fixed

Posted by vo...@apache.org.
ignite-6445 Deadlock detection triggering fixed

Signed-off-by: Andrey Gura <ag...@apache.org>


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

Branch: refs/heads/ignite-2.3
Commit: 78688431e6fb183235e310f6830c9b499d0f7d5e
Parents: 405749a
Author: Vitaliy Biryukov <Bi...@gmail.com>
Authored: Thu Sep 28 16:27:52 2017 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Thu Sep 28 16:27:52 2017 +0300

----------------------------------------------------------------------
 .../distributed/dht/GridDhtTxPrepareFuture.java |  4 +-
 .../TxOptimisticDeadlockDetectionTest.java      | 78 +++++++++++++++++---
 2 files changed, 69 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/78688431/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 00ce1d5..6873890 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -973,7 +973,7 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
         if (last || tx.isSystemInvalidate())
             tx.state(PREPARED);
 
-        if (super.onDone(res, err)) {
+        if (super.onDone(res, res == null ? err : null)) {
             // Don't forget to clean up.
             cctx.mvcc().removeVersionedFuture(this);
 
@@ -992,7 +992,7 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
     public void complete() {
         GridNearTxPrepareResponse res = new GridNearTxPrepareResponse();
 
-        res.error(new IgniteCheckedException("Failed to prepare transaction."));
+        res.error(err != null ? err : new IgniteCheckedException("Failed to prepare transaction."));
 
         onComplete(res);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/78688431/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
index 3414227..91a5742 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.cache.transactions;
 
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -45,6 +45,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPr
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -165,6 +166,16 @@ public class TxOptimisticDeadlockDetectionTest extends AbstractDeadlockDetection
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testDeadlocksPartitionedNearTxOnPrimary() throws Exception {
+        for (CacheWriteSynchronizationMode syncMode : CacheWriteSynchronizationMode.values()) {
+            doTestDeadlocksTxOnPrimary(createCache(PARTITIONED, syncMode, true),  ORDINAL_START_KEY);
+            doTestDeadlocksTxOnPrimary(createCache(PARTITIONED, syncMode, true),  CUSTOM_START_KEY);
+        }
+    }
+
+    /**
      * @param cacheMode Cache mode.
      * @param syncMode Write sync mode.
      * @param near Near.
@@ -224,16 +235,55 @@ public class TxOptimisticDeadlockDetectionTest extends AbstractDeadlockDetection
     }
 
     /**
+     * @param cache Cache.
+     * @param startKey Start key.
+     */
+    private void doTestDeadlocksTxOnPrimary(IgniteCache cache, Object startKey) {
+        try {
+            awaitPartitionMapExchange();
+
+            doTestDeadlock(3, false, false, startKey, true);
+
+            doTestDeadlock(4, false, false, startKey, true);
+        }
+        catch (Throwable e) {
+            U.error(log, "Unexpected exception: ", e);
+
+            //TODO "if" statement will be removed after fixing https://issues.apache.org/jira/browse/IGNITE-6445
+            if (!e.getMessage().equals("Failed to detect deadlock"))
+                fail();
+        }
+        finally {
+            if (cache != null)
+                cache.destroy();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void doTestDeadlock(
+        int txCnt,
+        boolean lockPrimaryFirst,
+        boolean clientTx,
+        Object startKey
+    ) throws Exception {
+        doTestDeadlock(txCnt, lockPrimaryFirst, clientTx, startKey, false);
+    }
+
+    /**
      * @throws Exception If failed.
      */
     private void doTestDeadlock(
         final int txCnt,
         boolean lockPrimaryFirst,
         final boolean clientTx,
-        Object startKey
+        Object startKey,
+        boolean txOnPrimary
     ) throws Exception {
+
         log.info(">>> Test deadlock [txCnt=" + txCnt + ", lockPrimaryFirst=" + lockPrimaryFirst +
-            ", clientTx=" + clientTx + ", startKey=" + startKey + ']');
+            ", clientTx=" + clientTx + ", startKey=" + startKey + ", txOnPrimary=" + txOnPrimary + ']');
 
         TestCommunicationSpi.init(txCnt);
 
@@ -241,7 +291,7 @@ public class TxOptimisticDeadlockDetectionTest extends AbstractDeadlockDetection
 
         final AtomicReference<TransactionDeadlockException> deadlockErr = new AtomicReference<>();
 
-        final List<List<Object>> keySets = generateKeys(txCnt, startKey, !lockPrimaryFirst);
+        final List<List<Object>> keySets = generateKeys(txCnt, startKey, !lockPrimaryFirst, txOnPrimary);
 
         final Set<Object> involvedKeys = new GridConcurrentHashSet<>();
         final Set<Object> involvedLockedKeys = new GridConcurrentHashSet<>();
@@ -283,14 +333,12 @@ public class TxOptimisticDeadlockDetectionTest extends AbstractDeadlockDetection
                         ((IgniteCacheProxy)cache).context().affinity().primaryByKey(key, NONE);
 
                     List<Object> primaryKeys = primaryKeys(
-                        grid(primaryNode).cache(CACHE_NAME), 5, incrementKey(key , (100 * threadNum)));
+                        grid(primaryNode).cache(CACHE_NAME), 5, incrementKey(key, (100 * threadNum)));
 
-                    Map<Object, Integer> entries = new HashMap<>();
+                    Map<Object, Integer> entries = new LinkedHashMap<>();
 
                     involvedKeys.add(key);
 
-                    entries.put(key, 0);
-
                     for (Object o : primaryKeys) {
                         involvedKeys.add(o);
 
@@ -303,6 +351,8 @@ public class TxOptimisticDeadlockDetectionTest extends AbstractDeadlockDetection
                         entries.put(k, 2);
                     }
 
+                    entries.put(key, 0);
+
                     log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode().id() +
                         ", tx=" + tx.xid() + ", entries=" + entries + ']');
 
@@ -313,7 +363,12 @@ public class TxOptimisticDeadlockDetectionTest extends AbstractDeadlockDetection
                 catch (Throwable e) {
                     log.info("Expected exception: " + e);
 
-                    e.printStackTrace(System.out);
+                    String stackTrace = X.getFullStackTrace(e);
+
+                    log.info(stackTrace);
+
+                    assertTrue("DeadlockDetection hasn't executed at "+ (threadNum - 1) + " node.",
+                        stackTrace.contains(TxDeadlockDetection.class.getName()));
 
                     // At least one stack trace should contain TransactionDeadlockException.
                     if (hasCause(e, TransactionTimeoutException.class) &&
@@ -364,13 +419,14 @@ public class TxOptimisticDeadlockDetectionTest extends AbstractDeadlockDetection
     /**
      * @param nodesCnt Nodes count.
      */
-    private <T> List<List<T>> generateKeys(int nodesCnt, T startKey, boolean reverse) throws IgniteCheckedException {
+    private <T> List<List<T>> generateKeys(int nodesCnt, T startKey, boolean reverse,
+        boolean txOnPrimary) throws IgniteCheckedException {
         List<List<T>> keySets = new ArrayList<>();
 
         for (int i = 0; i < nodesCnt; i++) {
             List<T> keys = new ArrayList<>(2);
 
-            int n1 = i + 1;
+            int n1 = txOnPrimary ? i : i + 1;
             int n2 = n1 + 1;
 
             int i1 = n1 < nodesCnt ? n1 : n1 - nodesCnt;


[30/37] ignite git commit: IGNITE-6517 .NET: DataStreamer DefaultPerNodeBufferSize, DefaultParallelOpsMultiplier, Timeout

Posted by vo...@apache.org.
IGNITE-6517 .NET: DataStreamer DefaultPerNodeBufferSize, DefaultParallelOpsMultiplier, Timeout

This closes #2785


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

Branch: refs/heads/ignite-2.3
Commit: 5764960e802e91b87956f4515e289eaf0003a2de
Parents: 5ca7909
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Oct 2 16:48:23 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Oct 2 16:48:23 2017 +0300

----------------------------------------------------------------------
 .../datastreamer/PlatformDataStreamer.java      | 14 ++++++
 .../Dataload/DataStreamerTest.cs                | 50 +++++++++++++++++---
 .../Apache.Ignite.Core.csproj                   |  1 +
 .../Datastream/DataStreamerDefaults.cs          | 46 ++++++++++++++++++
 .../Datastream/IDataStreamer.cs                 | 21 +++++++-
 .../Impl/Binary/BinaryReaderExtensions.cs       | 10 +---
 .../Impl/Binary/BinaryUtils.cs                  | 14 ++++++
 .../Impl/Datastream/DataStreamerImpl.cs         | 43 ++++++++++++++++-
 8 files changed, 179 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5764960e/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
index fba0a4c..8cd14c7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
@@ -86,6 +86,12 @@ public class PlatformDataStreamer extends PlatformAbstractTarget {
     /** */
     private static final int OP_LISTEN_TOPOLOGY = 11;
 
+    /** */
+    private static final int OP_GET_TIMEOUT = 12;
+
+    /** */
+    private static final int OP_SET_TIMEOUT = 13;
+
     /** Cache name. */
     private final String cacheName;
 
@@ -230,6 +236,14 @@ public class PlatformDataStreamer extends PlatformAbstractTarget {
 
             case OP_PER_NODE_PARALLEL_OPS:
                 return ldr.perNodeParallelOperations();
+
+            case OP_GET_TIMEOUT:
+                return ldr.timeout();
+
+            case OP_SET_TIMEOUT:
+                ldr.timeout(val);
+
+                return TRUE;
         }
 
         return super.processInLongOutLong(type, val);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5764960e/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs
index fe5955f..60a1067 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs
@@ -95,25 +95,40 @@ namespace Apache.Ignite.Core.Tests.Dataload
         {
             using (IDataStreamer<int, int> ldr = _grid.GetDataStreamer<int, int>(CacheName))
             {
+                Assert.AreEqual(CacheName, ldr.CacheName);
+                Assert.AreEqual(0, ldr.AutoFlushFrequency);
+
+                Assert.IsFalse(ldr.AllowOverwrite);
                 ldr.AllowOverwrite = true;
                 Assert.IsTrue(ldr.AllowOverwrite);
                 ldr.AllowOverwrite = false;
                 Assert.IsFalse(ldr.AllowOverwrite);
 
+                Assert.IsFalse(ldr.SkipStore);
                 ldr.SkipStore = true;
                 Assert.IsTrue(ldr.SkipStore);
                 ldr.SkipStore = false;
                 Assert.IsFalse(ldr.SkipStore);
 
+                Assert.AreEqual(DataStreamerDefaults.DefaultPerNodeBufferSize, ldr.PerNodeBufferSize);
                 ldr.PerNodeBufferSize = 1;
                 Assert.AreEqual(1, ldr.PerNodeBufferSize);
                 ldr.PerNodeBufferSize = 2;
                 Assert.AreEqual(2, ldr.PerNodeBufferSize);
 
-                ldr.PerNodeParallelOperations = 1;
-                Assert.AreEqual(1, ldr.PerNodeParallelOperations);
+                Assert.AreEqual(0, ldr.PerNodeParallelOperations);
+                var ops = DataStreamerDefaults.DefaultParallelOperationsMultiplier *
+                          IgniteConfiguration.DefaultThreadPoolSize;
+                ldr.PerNodeParallelOperations = ops;
+                Assert.AreEqual(ops, ldr.PerNodeParallelOperations);
                 ldr.PerNodeParallelOperations = 2;
                 Assert.AreEqual(2, ldr.PerNodeParallelOperations);
+
+                Assert.AreEqual(DataStreamerDefaults.DefaultTimeout, ldr.Timeout);
+                ldr.Timeout = TimeSpan.MaxValue;
+                Assert.AreEqual(TimeSpan.MaxValue, ldr.Timeout);
+                ldr.Timeout = TimeSpan.FromSeconds(1.5);
+                Assert.AreEqual(1.5, ldr.Timeout.TotalSeconds);
             }
         }
 
@@ -123,28 +138,37 @@ namespace Apache.Ignite.Core.Tests.Dataload
         [Test]        
         public void TestAddRemove()
         {
-            using (IDataStreamer<int, int> ldr = _grid.GetDataStreamer<int, int>(CacheName))
+            IDataStreamer<int, int> ldr;
+
+            using (ldr = _grid.GetDataStreamer<int, int>(CacheName))
             {
+                Assert.IsFalse(ldr.Task.IsCompleted);
+
                 ldr.AllowOverwrite = true;
 
                 // Additions.
-                ldr.AddData(1, 1);
+                var task = ldr.AddData(1, 1);
                 ldr.Flush();                
                 Assert.AreEqual(1, _cache.Get(1));
+                Assert.IsTrue(task.IsCompleted);
+                Assert.IsFalse(ldr.Task.IsCompleted);
 
-                ldr.AddData(new KeyValuePair<int, int>(2, 2));
+                task = ldr.AddData(new KeyValuePair<int, int>(2, 2));
                 ldr.Flush();
                 Assert.AreEqual(2, _cache.Get(2));
+                Assert.IsTrue(task.IsCompleted);
 
-                ldr.AddData(new List<KeyValuePair<int, int>> { new KeyValuePair<int, int>(3, 3), new KeyValuePair<int, int>(4, 4) });
+                task = ldr.AddData(new [] { new KeyValuePair<int, int>(3, 3), new KeyValuePair<int, int>(4, 4) });
                 ldr.Flush();
                 Assert.AreEqual(3, _cache.Get(3));
                 Assert.AreEqual(4, _cache.Get(4));
+                Assert.IsTrue(task.IsCompleted);
 
                 // Removal.
-                ldr.RemoveData(1);
+                task = ldr.RemoveData(1);
                 ldr.Flush();
                 Assert.IsFalse(_cache.ContainsKey(1));
+                Assert.IsTrue(task.IsCompleted);
 
                 // Mixed.
                 ldr.AddData(5, 5);                
@@ -165,6 +189,8 @@ namespace Apache.Ignite.Core.Tests.Dataload
                 for (int i = 5; i < 13; i++)
                     Assert.AreEqual(i, _cache.Get(i));
             }
+
+            Assert.IsTrue(ldr.Task.IsCompleted);
         }
 
         /// <summary>
@@ -517,6 +543,16 @@ namespace Apache.Ignite.Core.Tests.Dataload
 
                 for (var i = 0; i < 100; i++)
                     Assert.AreEqual(i + 1, cache.Get(i).Val);
+
+                // Repeating WithKeepBinary call: valid args.
+                Assert.AreSame(ldr, ldr.WithKeepBinary<int, IBinaryObject>());
+
+                // Invalid type args.
+                var ex = Assert.Throws<InvalidOperationException>(() => ldr.WithKeepBinary<string, IBinaryObject>());
+
+                Assert.AreEqual(
+                    "Can't change type of binary streamer. WithKeepBinary has been called on an instance of " +
+                    "binary streamer with incompatible generic arguments.", ex.Message);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5764960e/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 67c540c..58abd26 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -104,6 +104,7 @@
     <Compile Include="Common\ExceptionFactory.cs" />
     <Compile Include="Configuration\Package-Info.cs" />
     <Compile Include="Configuration\ClientConnectorConfiguration.cs" />
+    <Compile Include="Datastream\DataStreamerDefaults.cs" />
     <Compile Include="Impl\Binary\BinaryTypeId.cs" />
     <Compile Include="Impl\Client\Cache\CacheFlags.cs" />
     <Compile Include="Impl\Client\Cache\Query\ClientQueryCursor.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/5764960e/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/DataStreamerDefaults.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/DataStreamerDefaults.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/DataStreamerDefaults.cs
new file mode 100644
index 0000000..315ae7f
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/DataStreamerDefaults.cs
@@ -0,0 +1,46 @@
+/*
+ * 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.Datastream
+{
+    using System;
+
+    /// <summary>
+    /// Data streamer configuration defaults.
+    /// </summary>
+    public static class DataStreamerDefaults
+    {
+        /// <summary>
+        /// The default per node buffer size, see <see cref="IDataStreamer{TK,TV}.PerNodeBufferSize"/>.
+        /// </summary>
+        public const int DefaultPerNodeBufferSize = 512;
+
+        /// <summary>
+        /// Default multiplier for parallel operations per node:
+        /// <see cref="IDataStreamer{TK,TV}.PerNodeParallelOperations"/> = 
+        /// <see cref="IgniteConfiguration.DataStreamerThreadPoolSize"/> * 
+        /// <see cref="DefaultParallelOperationsMultiplier"/>.
+        /// </summary>
+        public const int DefaultParallelOperationsMultiplier = 8;
+
+        /// <summary>
+        /// The default timeout (see <see cref="IDataStreamer{TK,TV}.Timeout"/>).
+        /// Negative value means no timeout.
+        /// </summary>
+        public static readonly TimeSpan DefaultTimeout = TimeSpan.FromMilliseconds(-1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5764960e/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/IDataStreamer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/IDataStreamer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/IDataStreamer.cs
index 222f6c3..277130c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/IDataStreamer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/IDataStreamer.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Datastream
 {
     using System;
     using System.Collections.Generic;
+    using System.ComponentModel;
     using System.Threading.Tasks;
     using Apache.Ignite.Core.Cache.Store;
 
@@ -110,8 +111,9 @@ namespace Apache.Ignite.Core.Datastream
         /// <para />
         /// Setter must be called before any add/remove operation.
         /// <para />
-        /// Default is <c>1024</c>.
+        /// Default is <see cref="DataStreamerDefaults.DefaultPerNodeBufferSize"/>.
         /// </summary>
+        [DefaultValue(DataStreamerDefaults.DefaultPerNodeBufferSize)]
         int PerNodeBufferSize { get; set; }
 
         /// <summary>
@@ -119,7 +121,9 @@ namespace Apache.Ignite.Core.Datastream
         /// <para />
         /// Setter must be called before any add/remove operation.
         /// <para />
-        /// Default is <c>16</c>.
+        /// Default is 0, which means Ignite calculates this automatically as 
+        /// <see cref="IgniteConfiguration.DataStreamerThreadPoolSize"/> * 
+        /// <see cref="DataStreamerDefaults.DefaultParallelOperationsMultiplier"/>.
         /// </summary>
         int PerNodeParallelOperations { get; set; }
 
@@ -208,5 +212,18 @@ namespace Apache.Ignite.Core.Datastream
         /// <typeparam name="TV1">Value type in binary mode.</typeparam>
         /// <returns>Streamer instance with binary mode enabled.</returns>
         IDataStreamer<TK1, TV1> WithKeepBinary<TK1, TV1>();
+
+        /// <summary>
+        /// Gets or sets the timeout. Negative values mean no timeout.
+        /// Default is <see cref="DataStreamerDefaults.DefaultTimeout"/>.
+        /// <para />
+        /// Timeout is used in the following cases:
+        /// <li>Any data addition method can be blocked when all per node parallel operations are exhausted.
+        /// The timeout defines the max time you will be blocked waiting for a permit to add a chunk of data
+        /// into the streamer;</li> 
+        /// <li>Total timeout time for <see cref="Flush"/> operation;</li>
+        /// <li>Total timeout time for <see cref="Close"/> operation.</li>
+        /// </summary>
+        TimeSpan Timeout { get; set; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5764960e/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReaderExtensions.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReaderExtensions.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReaderExtensions.cs
index 7556c41..da87d21 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReaderExtensions.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReaderExtensions.cs
@@ -58,15 +58,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <returns>TimeSpan.</returns>
         public static TimeSpan ReadLongAsTimespan(this IBinaryRawReader reader)
         {
-            long ms = reader.ReadLong();
-
-            if (ms >= TimeSpan.MaxValue.TotalMilliseconds)
-                return TimeSpan.MaxValue;
-
-            if (ms <= TimeSpan.MinValue.TotalMilliseconds)
-                return TimeSpan.MinValue;
-
-            return TimeSpan.FromMilliseconds(ms);
+            return BinaryUtils.LongToTimeSpan(reader.ReadLong());
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5764960e/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
index 46e6752..139783d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
@@ -1664,6 +1664,20 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
+        /// Converts long to timespan.
+        /// </summary>
+        public static TimeSpan LongToTimeSpan(long ms)
+        {
+            if (ms >= TimeSpan.MaxValue.TotalMilliseconds)
+                return TimeSpan.MaxValue;
+
+            if (ms <= TimeSpan.MinValue.TotalMilliseconds)
+                return TimeSpan.MinValue;
+
+            return TimeSpan.FromMilliseconds(ms);
+        }
+
+        /// <summary>
         /// Creates and instance from the type name in reader.
         /// </summary>
         private static T CreateInstance<T>(BinaryReader reader)

http://git-wip-us.apache.org/repos/asf/ignite/blob/5764960e/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
index 96b24ab..555c6e6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
@@ -93,6 +93,12 @@ namespace Apache.Ignite.Core.Impl.Datastream
         /** */
         private const int OpListenTopology = 11;
 
+        /** */
+        private const int OpGetTimeout = 12;
+
+        /** */
+        private const int OpSetTimeout = 13;
+
         /** Cache name. */
         private readonly string _cacheName;
 
@@ -356,8 +362,6 @@ namespace Apache.Ignite.Core.Impl.Datastream
         {
             get
             {
-                ThrowIfDisposed();
-
                 return _closeFut.Task;
             }
         }
@@ -549,6 +553,41 @@ namespace Apache.Ignite.Core.Impl.Datastream
         }
 
         /** <inheritDoc /> */
+        public TimeSpan Timeout
+        {
+            get
+            {
+                _rwLock.EnterReadLock();
+
+                try
+                {
+                    ThrowIfDisposed();
+
+                    return BinaryUtils.LongToTimeSpan(DoOutInOp(OpGetTimeout));
+                }
+                finally
+                {
+                    _rwLock.ExitReadLock();
+                }
+            }
+            set
+            {
+                _rwLock.EnterWriteLock();
+
+                try
+                {
+                    ThrowIfDisposed();
+
+                    DoOutInOp(OpSetTimeout, (long) value.TotalMilliseconds);
+                }
+                finally
+                {
+                    _rwLock.ExitWriteLock();
+                }
+            }
+        }
+
+        /** <inheritDoc /> */
         [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes")]
         protected override void Dispose(bool disposing)
         {


[13/37] ignite git commit: .NET: Fix API docs link

Posted by vo...@apache.org.
.NET: Fix API docs link


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

Branch: refs/heads/ignite-2.3
Commit: b8c4863c9676b4ff83fdbffc6ff05dfdb7ddb875
Parents: afaa5e0
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Sep 29 15:03:17 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Sep 29 15:03:17 2017 +0300

----------------------------------------------------------------------
 modules/platforms/dotnet/docfx/index.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b8c4863c/modules/platforms/dotnet/docfx/index.md
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/docfx/index.md b/modules/platforms/dotnet/docfx/index.md
index d240c0b..4472cff 100644
--- a/modules/platforms/dotnet/docfx/index.md
+++ b/modules/platforms/dotnet/docfx/index.md
@@ -6,5 +6,5 @@ Apache Ignite In-Memory Data Fabric is designed to deliver uncompromised perform
 [high performance computing](https://ignite.apache.org/features.html), to the industry most advanced [data grid](https://ignite.apache.org/features.html), 
 highly available [service grid](https://ignite.apache.org/features.html), and [streaming](https://ignite.apache.org/features.html).
 
-* [API Documentation](/api)
+* [API Documentation](api/)
 * [Gettting Started](https://apacheignite-net.readme.io/docs/getting-started)
\ No newline at end of file


[28/37] ignite git commit: IGNITE-6231 .NET: Do not start a new thread explicitly in Flusher.RunThread

Posted by vo...@apache.org.
IGNITE-6231 .NET: Do not start a new thread explicitly in Flusher.RunThread


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

Branch: refs/heads/ignite-2.3
Commit: 7aaacd8f92645558430b5343a1cb8bf7e908529a
Parents: f57b677
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Oct 2 16:09:04 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Oct 2 16:09:04 2017 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7aaacd8f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
index fb2df01..96b24ab 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
@@ -858,7 +858,7 @@ namespace Apache.Ignite.Core.Impl.Datastream
             /// </summary>
             public void RunThread()
             {
-                new Thread(Run).Start();
+                Task.Factory.StartNew(Run);
             }
         }
 


[12/37] ignite git commit: IGNITE-6473: Introduced the constant of default persistence directory name. This closes #2724.

Posted by vo...@apache.org.
IGNITE-6473: Introduced the constant of default persistence directory name. This closes #2724.


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

Branch: refs/heads/ignite-2.3
Commit: afaa5e03d8298f138f4d9ff8a8e80dc5934374eb
Parents: c2cdbd9
Author: Vyacheslav Daradur <da...@gmail.com>
Authored: Fri Sep 29 12:15:40 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Sep 29 12:15:40 2017 +0300

----------------------------------------------------------------------
 .../IgnitePersistenceCompatibilityAbstractTest.java     |  4 +++-
 .../cache/persistence/file/FilePageStoreManager.java    |  5 ++++-
 .../IgnitePdsCacheRebalancingAbstractTest.java          |  5 +++--
 .../persistence/IgnitePdsClientNearCachePutGetTest.java |  6 ++++--
 .../persistence/IgnitePdsContinuousRestartTest.java     |  4 +++-
 .../cache/persistence/IgnitePdsDynamicCacheTest.java    |  4 +++-
 .../IgnitePdsExchangeDuringCheckpointTest.java          |  6 ++++--
 .../IgnitePdsMultiNodePutGetRestartTest.java            |  6 ++++--
 .../cache/persistence/IgnitePdsPageSizesTest.java       |  6 ++++--
 .../IgnitePdsRecoveryAfterFileCorruptionTest.java       |  4 +++-
 .../IgnitePdsRemoveDuringRebalancingTest.java           | 12 +++++++-----
 .../IgnitePdsSingleNodePutGetPersistenceTest.java       |  6 ++++--
 .../db/IgnitePdsMultiNodePutGetRestartTest.java         |  6 ++++--
 .../IgnitePdsPageEvictionDuringPartitionClearTest.java  |  6 ++++--
 .../cache/persistence/db/IgnitePdsPageEvictionTest.java |  6 ++++--
 .../db/IgnitePdsRebalancingOnNotStableTopologyTest.java |  6 ++++--
 .../persistence/db/IgnitePdsTransactionsHangTest.java   |  5 +++--
 .../db/IgnitePdsWholeClusterRestartTest.java            |  4 +++-
 .../file/DefaultPageSizeBackwardsCompatibilityTest.java |  4 +++-
 .../db/file/IgnitePdsCacheIntegrationTest.java          |  6 ++++--
 ...tePdsCheckpointSimulationWithRealCpDisabledTest.java |  4 +++-
 .../persistence/db/file/IgnitePdsEvictionTest.java      |  4 +++-
 .../db/file/IgnitePdsNoActualWalHistoryTest.java        |  6 ++++--
 .../db/file/IgnitePdsThreadInterruptionTest.java        |  4 +++-
 .../cache/persistence/db/wal/IgnitePdsWalTlbTest.java   |  5 +++--
 .../persistence/db/wal/IgniteWalFlushFailoverTest.java  |  3 ++-
 .../db/wal/IgniteWalHistoryReservationsTest.java        |  3 ++-
 .../db/wal/IgniteWalRecoverySeveralRestartsTest.java    |  6 ++++--
 .../cache/persistence/db/wal/IgniteWalRecoveryTest.java |  8 +++++---
 .../db/wal/WalRecoveryTxLogicalRecordsTest.java         |  6 ++++--
 .../persistence/db/wal/reader/IgniteWalReaderTest.java  |  7 ++++---
 .../pagemem/PagesWriteThrottleSandboxTest.java          |  4 +++-
 .../pagemem/PagesWriteThrottleSmokeTest.java            |  3 ++-
 .../standbycluster/AbstractNodeJoinTemplate.java        |  5 +++--
 .../standbycluster/IgniteStandByClusterTest.java        |  6 ++++--
 .../IgniteAbstractStandByClientReconnectTest.java       |  5 +++--
 .../processors/database/IgniteDbAbstractTest.java       |  5 +++--
 .../org/apache/ignite/testframework/GridTestUtils.java  |  3 ++-
 ...ePdsSingleNodeWithIndexingPutGetPersistenceTest.java |  6 ++++--
 .../processors/cache/index/LongIndexNameTest.java       |  4 +++-
 .../IgniteDbSingleNodeWithIndexingWalRestoreTest.java   |  7 ++++---
 ...istentStoreQueryWithMultipleClassesPerCacheTest.java |  3 ++-
 .../database/IgnitePersistentStoreSchemaLoadTest.java   |  3 ++-
 43 files changed, 147 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java
index 87c24cb..d76b862 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java
@@ -25,6 +25,8 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.compatibility.testframework.junits.IgniteCompatibilityAbstractTest;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  * Super class for all persistence compatibility tests.
  */
@@ -55,7 +57,7 @@ public abstract class IgnitePersistenceCompatibilityAbstractTest extends IgniteC
      * @see #isDefaultDBWorkDirectoryEmpty()
      */
     protected Path getDefaultDbWorkPath() throws IgniteCheckedException {
-        return Paths.get(U.defaultWorkDirectory() + File.separator + "db");
+        return Paths.get(U.defaultWorkDirectory() + File.separator + DFLT_STORE_DIR);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
index b3eb74a..a20bda1 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
@@ -78,6 +78,9 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     /** */
     public static final String CACHE_DATA_FILENAME = "cache_data.dat";
 
+    /** */
+    public static final String DFLT_STORE_DIR = "db";
+
     /** Marshaller. */
     private static final Marshaller marshaller = new JdkMarshaller();
 
@@ -134,7 +137,7 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
         else
             storeWorkDir = new File(U.resolveWorkDirectory(
                 igniteCfg.getWorkDirectory(),
-                "db",
+                DFLT_STORE_DIR,
                 false
             ), consId);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java
index 96f832b..705156f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java
@@ -55,6 +55,7 @@ import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 import static org.apache.ignite.testframework.GridTestUtils.runMultiThreadedAsync;
 
 /**
@@ -162,14 +163,14 @@ public abstract class IgnitePdsCacheRebalancingAbstractTest extends GridCommonAb
     @Override protected void beforeTestsStarted() throws Exception {
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsClientNearCachePutGetTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsClientNearCachePutGetTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsClientNearCachePutGetTest.java
index 17dad10..2c15416 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsClientNearCachePutGetTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsClientNearCachePutGetTest.java
@@ -23,6 +23,8 @@ import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.internal.processors.database.IgniteDbClientNearCachePutGetTest;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -41,7 +43,7 @@ public class IgnitePdsClientNearCachePutGetTest extends IgniteDbClientNearCacheP
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
 
         super.beforeTest();
     }
@@ -50,6 +52,6 @@ public class IgnitePdsClientNearCachePutGetTest extends IgniteDbClientNearCacheP
     @Override protected void afterTest() throws Exception {
         super.afterTest();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java
index df62f3a..a363e8c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java
@@ -42,6 +42,8 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -128,7 +130,7 @@ public class IgnitePdsContinuousRestartTest extends GridCommonAbstractTest {
      * @throws IgniteCheckedException If failed.
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDynamicCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDynamicCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDynamicCacheTest.java
index 5873a35..0325e12 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDynamicCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDynamicCacheTest.java
@@ -36,6 +36,8 @@ import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.internal.processors.database.IgniteDbDynamicCacheSelfTest;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -206,7 +208,7 @@ public class IgnitePdsDynamicCacheTest extends IgniteDbDynamicCacheSelfTest {
      * @throws IgniteCheckedException If failed.
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java
index 3969fb6..94b8f53 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java
@@ -31,6 +31,8 @@ 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 static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -124,12 +126,12 @@ public class IgnitePdsExchangeDuringCheckpointTest extends GridCommonAbstractTes
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         stopAllGrids();
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         stopAllGrids();
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsMultiNodePutGetRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsMultiNodePutGetRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsMultiNodePutGetRestartTest.java
index df01bf6..0437251 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsMultiNodePutGetRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsMultiNodePutGetRestartTest.java
@@ -42,6 +42,8 @@ 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 static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -116,7 +118,7 @@ public class IgnitePdsMultiNodePutGetRestartTest extends GridCommonAbstractTest
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
 
         super.beforeTest();
     }
@@ -125,7 +127,7 @@ public class IgnitePdsMultiNodePutGetRestartTest extends GridCommonAbstractTest
     @Override protected void afterTest() throws Exception {
         super.afterTest();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPageSizesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPageSizesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPageSizesTest.java
index aa480ed..1d6ba4b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPageSizesTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPageSizesTest.java
@@ -33,6 +33,8 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -77,12 +79,12 @@ public class IgnitePdsPageSizesTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java
index 11d5eef..6e2752d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java
@@ -53,6 +53,8 @@ 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 static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -359,6 +361,6 @@ public class IgnitePdsRecoveryAfterFileCorruptionTest extends GridCommonAbstract
      *
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRemoveDuringRebalancingTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRemoveDuringRebalancingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRemoveDuringRebalancingTest.java
index 544ef82..78da14d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRemoveDuringRebalancingTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRemoveDuringRebalancingTest.java
@@ -40,6 +40,8 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -69,7 +71,7 @@ public class IgnitePdsRemoveDuringRebalancingTest extends GridCommonAbstractTest
         memPlcCfg.setName("dfltMemPlc");
         memPlcCfg.setInitialSize(100 * 1024 * 1024);
         memPlcCfg.setMaxSize(100 * 1024 * 1024);
-        memPlcCfg.setSwapFilePath("db");
+        memPlcCfg.setSwapFilePath(DFLT_STORE_DIR);
 
         dbCfg.setMemoryPolicies(memPlcCfg);
         dbCfg.setDefaultMemoryPolicyName("dfltMemPlc");
@@ -93,18 +95,18 @@ public class IgnitePdsRemoveDuringRebalancingTest extends GridCommonAbstractTest
     @Override protected void beforeTestsStarted() throws Exception {
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
 
-        U.delete(new File(U.getIgniteHome(), "db"));
+        U.delete(new File(U.getIgniteHome(), DFLT_STORE_DIR));
     }
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         G.stopAll(true);
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
 
-        U.delete(new File(U.getIgniteHome(), "db"));
+        U.delete(new File(U.getIgniteHome(), DFLT_STORE_DIR));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSingleNodePutGetPersistenceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSingleNodePutGetPersistenceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSingleNodePutGetPersistenceTest.java
index 1f861c7..4add384 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSingleNodePutGetPersistenceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSingleNodePutGetPersistenceTest.java
@@ -23,6 +23,8 @@ import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.internal.processors.database.IgniteDbSingleNodePutGetTest;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -41,7 +43,7 @@ public class IgnitePdsSingleNodePutGetPersistenceTest extends IgniteDbSingleNode
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
 
         super.beforeTest();
     }
@@ -50,6 +52,6 @@ public class IgnitePdsSingleNodePutGetPersistenceTest extends IgniteDbSingleNode
     @Override protected void afterTest() throws Exception {
         super.afterTest();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsMultiNodePutGetRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsMultiNodePutGetRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsMultiNodePutGetRestartTest.java
index 5bc2f2f..b8db802 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsMultiNodePutGetRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsMultiNodePutGetRestartTest.java
@@ -42,6 +42,8 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.configuration.MemoryConfiguration;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -100,7 +102,7 @@ public class IgnitePdsMultiNodePutGetRestartTest extends GridCommonAbstractTest
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
 
         super.beforeTest();
     }
@@ -109,7 +111,7 @@ public class IgnitePdsMultiNodePutGetRestartTest extends GridCommonAbstractTest
     @Override protected void afterTest() throws Exception {
         super.afterTest();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionDuringPartitionClearTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionDuringPartitionClearTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionDuringPartitionClearTest.java
index 38b1d55..c1bec35 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionDuringPartitionClearTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionDuringPartitionClearTest.java
@@ -37,6 +37,8 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -102,7 +104,7 @@ public class IgnitePdsPageEvictionDuringPartitionClearTest extends GridCommonAbs
      */
     public void testPageEvictionOnNodeStart() throws Exception {
         for (int r = 0; r < 3; r++) {
-            deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+            deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
 
             startGrids(2);
 
@@ -151,7 +153,7 @@ public class IgnitePdsPageEvictionDuringPartitionClearTest extends GridCommonAbs
             finally {
                 stopAllGrids();
 
-                deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+                deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionTest.java
index d266125..13cd8b4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionTest.java
@@ -39,6 +39,8 @@ 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 static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -96,7 +98,7 @@ public class IgnitePdsPageEvictionTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
 
         stopAllGrids();
 
@@ -107,7 +109,7 @@ public class IgnitePdsPageEvictionTest extends GridCommonAbstractTest {
     @Override protected void afterTest() throws Exception {
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsRebalancingOnNotStableTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsRebalancingOnNotStableTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsRebalancingOnNotStableTopologyTest.java
index 967c8f0..546a87a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsRebalancingOnNotStableTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsRebalancingOnNotStableTopologyTest.java
@@ -36,6 +36,8 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.testframework.junits.multijvm.IgniteProcessProxy;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  * We start writing to unstable cluster.
  * After that we start killing node.
@@ -200,14 +202,14 @@ public class IgnitePdsRebalancingOnNotStableTopologyTest extends GridCommonAbstr
     @Override protected void beforeTest() throws Exception {
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsTransactionsHangTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsTransactionsHangTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsTransactionsHangTest.java
index a927454..7e8cfac 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsTransactionsHangTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsTransactionsHangTest.java
@@ -48,6 +48,7 @@ import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.jsr166.LongAdder8;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
@@ -89,12 +90,12 @@ public class IgnitePdsTransactionsHangTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java
index 1cdfaea..df5bfdf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java
@@ -37,6 +37,8 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.checkpoint.noop.NoopCheckpointSpi;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -111,7 +113,7 @@ public class IgnitePdsWholeClusterRestartTest extends GridCommonAbstractTest {
      * @throws IgniteCheckedException If failed.
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/DefaultPageSizeBackwardsCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/DefaultPageSizeBackwardsCompatibilityTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/DefaultPageSizeBackwardsCompatibilityTest.java
index e2951e2..e577886 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/DefaultPageSizeBackwardsCompatibilityTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/DefaultPageSizeBackwardsCompatibilityTest.java
@@ -33,6 +33,8 @@ 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 static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -142,6 +144,6 @@ public class DefaultPageSizeBackwardsCompatibilityTest extends GridCommonAbstrac
      * @throws IgniteCheckedException If failed.
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheIntegrationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheIntegrationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheIntegrationTest.java
index 80d3743..d36894f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheIntegrationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCacheIntegrationTest.java
@@ -46,6 +46,8 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -112,14 +114,14 @@ public class IgnitePdsCacheIntegrationTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         super.afterTest();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java
index 2278981..b4c32d8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java
@@ -77,6 +77,8 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  * Test simulated chekpoints,
  * Disables integrated check pointer thread
@@ -981,6 +983,6 @@ public class IgnitePdsCheckpointSimulationWithRealCpDisabledTest extends GridCom
      *
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java
index 8544cf1..a9b0892 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java
@@ -41,6 +41,8 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  * Test for page evictions.
  */
@@ -295,6 +297,6 @@ public class IgnitePdsEvictionTest extends GridCommonAbstractTest {
      * @throws IgniteCheckedException If fail.
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsNoActualWalHistoryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsNoActualWalHistoryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsNoActualWalHistoryTest.java
index 2edc375..1779fce 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsNoActualWalHistoryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsNoActualWalHistoryTest.java
@@ -38,6 +38,8 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -86,14 +88,14 @@ public class IgnitePdsNoActualWalHistoryTest extends GridCommonAbstractTest {
     @Override protected void beforeTest() throws Exception {
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java
index 6f11d4d..2a00768 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java
@@ -33,6 +33,8 @@ import org.jsr166.ThreadLocalRandom8;
 
 import java.util.concurrent.atomic.AtomicReference;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  * Test what interruptions of writing threads do not affect PDS.
  */
@@ -200,6 +202,6 @@ public class IgnitePdsThreadInterruptionTest extends GridCommonAbstractTest {
      * @throws IgniteCheckedException If fail.
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java
index 8b3cbf8..a06d587 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java
@@ -32,6 +32,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 
 /**
  *
@@ -89,7 +90,7 @@ public class IgnitePdsWalTlbTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
 
         stopAllGrids();
 
@@ -100,7 +101,7 @@ public class IgnitePdsWalTlbTest extends GridCommonAbstractTest {
     @Override protected void afterTest() throws Exception {
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java
index 048e8bf..12ec6ef 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java
@@ -47,6 +47,7 @@ import java.nio.file.OpenOption;
 import static java.nio.file.StandardOpenOption.CREATE;
 import static java.nio.file.StandardOpenOption.READ;
 import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 
 /**
  *
@@ -163,7 +164,7 @@ public class IgniteWalFlushFailoverTest extends GridCommonAbstractTest {
      * @throws IgniteCheckedException
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java
index 94045ff..5d5458e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java
@@ -42,6 +42,7 @@ import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 
 /**
  *
@@ -454,7 +455,7 @@ public class IgniteWalHistoryReservationsTest extends GridCommonAbstractTest {
      * @throws IgniteCheckedException If failed.
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoverySeveralRestartsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoverySeveralRestartsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoverySeveralRestartsTest.java
index a6e65d6..9497dc6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoverySeveralRestartsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoverySeveralRestartsTest.java
@@ -39,6 +39,8 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -111,7 +113,7 @@ public class IgniteWalRecoverySeveralRestartsTest extends GridCommonAbstractTest
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
 
         super.beforeTest();
     }
@@ -120,7 +122,7 @@ public class IgniteWalRecoverySeveralRestartsTest extends GridCommonAbstractTest
     @Override protected void afterTest() throws Exception {
         super.afterTest();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
index 718a9a8..c160f60 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
@@ -91,6 +91,8 @@ import org.apache.ignite.transactions.TransactionIsolation;
 import org.junit.Assert;
 import sun.nio.ch.DirectBuffer;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -179,7 +181,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
     @Override protected void beforeTest() throws Exception {
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
 
         cacheName = "partitioned";
     }
@@ -190,7 +192,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
 
         logOnly = false;
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**
@@ -606,7 +608,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
     private File cacheDir(final String cacheName, String consId) throws IgniteCheckedException {
         consId = consId.replaceAll("[\\.:]", "_");
 
-        final File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false);
+        final File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false);
 
         assert dbDir.exists();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java
index a810ecd..f5d46e2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java
@@ -64,6 +64,8 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.junit.Assert;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -133,12 +135,12 @@ public class WalRecoveryTxLogicalRecordsTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
index ebb80a1..e2f58bd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
@@ -72,6 +72,7 @@ import org.apache.ignite.transactions.Transaction;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.events.EventType.EVT_WAL_SEGMENT_ARCHIVED;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 
 /**
  * Test suite for WAL segments reader and event generator.
@@ -171,7 +172,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
         if (fillWalBeforeTest)
-            deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+            deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**
@@ -196,7 +197,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
             consistentId = "127_0_0_1_47500";
 
         final String workDir = U.defaultWorkDirectory();
-        final File db = U.resolveWorkDirectory(workDir, "db", false);
+        final File db = U.resolveWorkDirectory(workDir, DFLT_STORE_DIR, false);
         final File wal = new File(db, "wal");
         final File walArchive = new File(wal, "archive");
 
@@ -499,7 +500,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
         @Nullable final BiConsumer<Object, Object> objConsumer,
         @Nullable final Consumer<DataRecord> dataRecordHnd) throws IgniteCheckedException {
 
-        final File db = U.resolveWorkDirectory(workDir, "db", false);
+        final File db = U.resolveWorkDirectory(workDir, DFLT_STORE_DIR, false);
         final File wal = new File(db, "wal");
         final File walArchive = new File(wal, "archive");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java
index 409ab84..e3de493 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java
@@ -45,6 +45,8 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  * Test to visualize and debug {@link PagesWriteThrottle}.
  * Prints puts/gets rate, number of dirty pages, pages written in current checkpoint and pages in checkpoint buffer.
@@ -258,7 +260,7 @@ public class PagesWriteThrottleSandboxTest extends GridCommonAbstractTest {
      * @throws IgniteCheckedException If failed.
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
         deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "snapshot", false));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
index f4f21ef..70a1df8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
@@ -54,6 +54,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import static java.nio.file.StandardOpenOption.CREATE;
 import static java.nio.file.StandardOpenOption.READ;
 import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 
 /**
  *
@@ -277,7 +278,7 @@ public class PagesWriteThrottleSmokeTest extends GridCommonAbstractTest {
      * @throws IgniteCheckedException If failed.
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
         deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "snapshot", false));
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java
index 675aca5..436db1c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java
@@ -41,6 +41,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Assert;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 import static org.apache.ignite.internal.util.IgniteUtils.field;
 
 /**
@@ -212,7 +213,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
 
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /** {@inheritDoc} */
@@ -221,7 +222,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
 
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteStandByClusterTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteStandByClusterTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteStandByClusterTest.java
index c2bece0..77f89ba 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteStandByClusterTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteStandByClusterTest.java
@@ -52,6 +52,8 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 import org.junit.Assert;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -491,7 +493,7 @@ public class IgniteStandByClusterTest extends GridCommonAbstractTest {
 
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", true));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, true));
     }
 
     /**
@@ -502,6 +504,6 @@ public class IgniteStandByClusterTest extends GridCommonAbstractTest {
 
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", true));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, true));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteAbstractStandByClientReconnectTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteAbstractStandByClientReconnectTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteAbstractStandByClientReconnectTest.java
index 2f229ce..59dcce2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteAbstractStandByClientReconnectTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteAbstractStandByClientReconnectTest.java
@@ -43,6 +43,7 @@ import org.jetbrains.annotations.Nullable;
 import org.junit.Assert;
 
 import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 
 /**
  *
@@ -322,7 +323,7 @@ public abstract class IgniteAbstractStandByClientReconnectTest extends GridCommo
 
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", true));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, true));
     }
 
     @Override protected void afterTest() throws Exception {
@@ -330,7 +331,7 @@ public abstract class IgniteAbstractStandByClientReconnectTest extends GridCommo
 
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", true));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, true));
     }
 
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
index 16fcb45..c9e583f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
@@ -36,6 +36,7 @@ import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 
 /**
  *
@@ -163,7 +164,7 @@ public abstract class IgniteDbAbstractTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
 
         startGrids(gridCount());
 
@@ -188,7 +189,7 @@ public abstract class IgniteDbAbstractTest extends GridCommonAbstractTest {
 
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index abae97c..380284f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@ -101,6 +101,7 @@ import org.apache.ignite.testframework.config.GridTestProperties;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 import static org.springframework.util.FileSystemUtils.deleteRecursively;
 
 /**
@@ -1929,6 +1930,6 @@ public final class GridTestUtils {
      * @throws Exception If failed.
      */
     public static void deleteDbFiles() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgnitePdsSingleNodeWithIndexingPutGetPersistenceTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgnitePdsSingleNodeWithIndexingPutGetPersistenceTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgnitePdsSingleNodeWithIndexingPutGetPersistenceTest.java
index 7e7b9e6..998e1e4 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgnitePdsSingleNodeWithIndexingPutGetPersistenceTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgnitePdsSingleNodeWithIndexingPutGetPersistenceTest.java
@@ -23,6 +23,8 @@ import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.internal.processors.database.IgniteDbSingleNodeWithIndexingPutGetTest;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -41,7 +43,7 @@ public class IgnitePdsSingleNodeWithIndexingPutGetPersistenceTest extends Ignite
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
 
         super.beforeTestsStarted();
     }
@@ -50,6 +52,6 @@ public class IgnitePdsSingleNodeWithIndexingPutGetPersistenceTest extends Ignite
     @Override protected void afterTestsStopped() throws Exception {
         super.afterTestsStopped();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/LongIndexNameTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/LongIndexNameTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/LongIndexNameTest.java
index 92883a4..544eb74 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/LongIndexNameTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/LongIndexNameTest.java
@@ -35,6 +35,8 @@ import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.List;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  *
  */
@@ -152,7 +154,7 @@ public class LongIndexNameTest extends GridCommonAbstractTest {
      * @throws IgniteCheckedException If failed.
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbSingleNodeWithIndexingWalRestoreTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbSingleNodeWithIndexingWalRestoreTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbSingleNodeWithIndexingWalRestoreTest.java
index 385cf19..8f6afd8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbSingleNodeWithIndexingWalRestoreTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbSingleNodeWithIndexingWalRestoreTest.java
@@ -30,12 +30,13 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
 /**
  * Test verifies that binary metadata of values stored in cache and indexes upon these values
  * is handled correctly on cluster restart when persistent store is enabled and compact footer is turned on.
@@ -185,13 +186,13 @@ public class IgniteDbSingleNodeWithIndexingWalRestoreTest extends GridCommonAbst
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         stopAllGrids();
 
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreQueryWithMultipleClassesPerCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreQueryWithMultipleClassesPerCacheTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreQueryWithMultipleClassesPerCacheTest.java
index 432f41f..5bb1eb1 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreQueryWithMultipleClassesPerCacheTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreQueryWithMultipleClassesPerCacheTest.java
@@ -36,6 +36,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 
 /**
  *
@@ -126,7 +127,7 @@ public class IgnitePersistentStoreQueryWithMultipleClassesPerCacheTest extends G
      * @throws IgniteCheckedException On error.
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/afaa5e03/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java
index b4c08b2..85ce4d8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java
@@ -42,6 +42,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 
 /**
  *
@@ -247,7 +248,7 @@ public class IgnitePersistentStoreSchemaLoadTest extends GridCommonAbstractTest
      *
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**


[21/37] ignite git commit: IGNITE-6120 Fixed missing "lazy query" flag.

Posted by vo...@apache.org.
IGNITE-6120 Fixed missing "lazy query" flag.


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

Branch: refs/heads/ignite-2.3
Commit: 55886b14cc6e7171acf05991f7f265dbadcb8db2
Parents: 2c39351
Author: vsisko <vs...@gridgain.com>
Authored: Sun Oct 1 19:00:22 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Mon Oct 2 13:56:51 2017 +0700

----------------------------------------------------------------------
 modules/web-console/frontend/app/filters/duration.filter.js   | 2 +-
 .../frontend/app/modules/agent/AgentManager.service.js        | 7 ++++---
 .../web-console/frontend/app/modules/sql/sql.controller.js    | 2 +-
 modules/web-console/frontend/package.json                     | 2 +-
 4 files changed, 7 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/55886b14/modules/web-console/frontend/app/filters/duration.filter.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/filters/duration.filter.js b/modules/web-console/frontend/app/filters/duration.filter.js
index 46d50d8..703b6a4 100644
--- a/modules/web-console/frontend/app/filters/duration.filter.js
+++ b/modules/web-console/frontend/app/filters/duration.filter.js
@@ -36,6 +36,6 @@ export default [() => {
         const s = Math.floor((t - d * cd - h * ch - m * cm) / cs);
         const ms = Math.round(t % 1000);
 
-        return a(d, 'd') + a(h, 'h') + a(m, 'm') + a(s, 's') + (t < cm ? ms + 'ms' : '');
+        return a(d, 'd') + a(h, 'h') + a(m, 'm') + a(s, 's') + (t === 0 || (t < cm && ms !== 0) ? ms + 'ms' : '');
     };
 }];

http://git-wip-us.apache.org/repos/asf/ignite/blob/55886b14/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
index 9a736b6..e4bde6f 100644
--- a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
+++ b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
@@ -310,7 +310,7 @@ export default class IgniteAgentManager {
             this.socket.removeListener('disconnect', onDisconnect);
 
             if (err)
-                latch.reject(err);
+                return latch.reject(err);
 
             latch.resolve(res);
         });
@@ -552,9 +552,10 @@ export default class IgniteAgentManager {
      * @param {Boolean} enforceJoinOrder Flag whether enforce join order is enabled.
      * @param {Boolean} replicatedOnly Flag whether query contains only replicated tables.
      * @param {Boolean} local Flag whether to execute query locally.
+     * @param {Boolean} lazy query flag.
      * @returns {Promise}
      */
-    querySqlGetAll(nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local) {
+    querySqlGetAll(nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, lazy) {
         // Page size for query.
         const pageSz = 1024;
 
@@ -572,7 +573,7 @@ export default class IgniteAgentManager {
                 });
         };
 
-        return this.querySql(nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, pageSz)
+        return this.querySql(nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, pageSz, lazy)
             .then(fetchResult);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/55886b14/modules/web-console/frontend/app/modules/sql/sql.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/sql.controller.js b/modules/web-console/frontend/app/modules/sql/sql.controller.js
index c43dc5c..420c4b9 100644
--- a/modules/web-console/frontend/app/modules/sql/sql.controller.js
+++ b/modules/web-console/frontend/app/modules/sql/sql.controller.js
@@ -1630,7 +1630,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
             return Promise.resolve(args.localNid || _chooseNode(args.cacheName, false))
                 .then((nid) => args.type === 'SCAN'
                     ? agentMgr.queryScanGetAll(nid, args.cacheName, args.query, !!args.regEx, !!args.caseSensitive, !!args.near, !!args.localNid)
-                    : agentMgr.querySqlGetAll(nid, args.cacheName, args.query, !!args.nonCollocatedJoins, !!args.enforceJoinOrder, false, !!args.localNid))
+                    : agentMgr.querySqlGetAll(nid, args.cacheName, args.query, !!args.nonCollocatedJoins, !!args.enforceJoinOrder, false, !!args.localNid, !!args.lazy))
                 .then((res) => _export(paragraph.name + '-all.csv', paragraph.gridOptions.columnDefs, res.columns, res.rows))
                 .catch(Messages.showError)
                 .then(() => paragraph.ace && paragraph.ace.focus());

http://git-wip-us.apache.org/repos/asf/ignite/blob/55886b14/modules/web-console/frontend/package.json
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/package.json b/modules/web-console/frontend/package.json
index f96c322..d828e17 100644
--- a/modules/web-console/frontend/package.json
+++ b/modules/web-console/frontend/package.json
@@ -76,7 +76,7 @@
     "html-webpack-plugin": "2.29.0",
     "jquery": "3.2.1",
     "json-loader": "0.5.7",
-    "jszip": "3.1.3",
+    "jszip": "3.1.4",
     "lodash": "4.17.4",
     "node-sass": "4.5.3",
     "nvd3": "1.8.4",


[20/37] ignite git commit: IGNITE-5863 Fixed select ALL logic.

Posted by vo...@apache.org.
IGNITE-5863 Fixed select ALL logic.


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

Branch: refs/heads/ignite-2.3
Commit: 2c393516f618820d27a2cb8ad58a22c046b4fe68
Parents: 4cb0fc73
Author: Dmitriy Shabalin <dm...@gmail.com>
Authored: Sun Oct 1 18:57:18 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Mon Oct 2 13:56:37 2017 +0700

----------------------------------------------------------------------
 .../frontend/app/components/grid-item-selected/controller.js      | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2c393516/modules/web-console/frontend/app/components/grid-item-selected/controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/grid-item-selected/controller.js b/modules/web-console/frontend/app/components/grid-item-selected/controller.js
index 4923924..0e8dadc 100644
--- a/modules/web-console/frontend/app/components/grid-item-selected/controller.js
+++ b/modules/web-console/frontend/app/components/grid-item-selected/controller.js
@@ -27,7 +27,10 @@ export default class {
             this.applyValues();
 
             this.gridApi.grid.registerDataChangeCallback(() => this.applyValues(), [this.uiGridConstants.dataChange.ROW]);
+            // Used to toggle selected of one row.
             this.gridApi.selection.on.rowSelectionChanged(this.$scope, () => this.applyValues());
+            // Used to toggle all selected of rows.
+            this.gridApi.selection.on.rowSelectionChangedBatch(this.$scope, () => this.applyValues());
         }
     }
 


[27/37] ignite git commit: IGNITE-5915 - Added warning for WAL NONE mode

Posted by vo...@apache.org.
IGNITE-5915 - Added warning for WAL NONE mode


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

Branch: refs/heads/ignite-2.3
Commit: f57b6776775308753a1fa579c0e7599df241be61
Parents: 252eedc
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Oct 2 15:05:42 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Oct 2 15:06:12 2017 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/configuration/WALMode.java | 14 ++++++++++----
 .../persistence/wal/FileWriteAheadLogManager.java     |  5 ++++-
 2 files changed, 14 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f57b6776/modules/core/src/main/java/org/apache/ignite/configuration/WALMode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/WALMode.java b/modules/core/src/main/java/org/apache/ignite/configuration/WALMode.java
index 9d4520e..bea0cba 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/WALMode.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/WALMode.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.configuration;
 
+import org.apache.ignite.Ignite;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -24,22 +25,27 @@ import org.jetbrains.annotations.Nullable;
  */
 public enum WALMode {
     /**
-     * Default mode: full-sync disk writes. These writes survive power loss scenarios.
+     * Default mode: full-sync disk writes. These writes survive power loss scenarios. When a control is returned
+     * from the trnasaction commit operation, the changes are guaranteed to be forced on disk according to the
+     * transaction write synchronization mode.
      */
     DEFAULT,
 
     /**
-     * Log only mode: flushes application buffers. These writes survive process crash.
+     * Log only mode: flushes application buffers. These writes survive process crash. When a control is returned
+     * from the transaction commit operation, the changes are guaranteed to be forced to the OS buffer cache.
      */
     LOG_ONLY,
 
     /**
-     * Background mode. Does not force application buffer flush. Data may be lost in case of process crash.
+     * Background mode. Does not force application buffer flush. Last updates may be lost in case of a process crash.
      */
     BACKGROUND,
 
     /**
-     * WAL disabled.
+     * WAL is disabled. Data is guaranteed to be persisted on disk only in case of graceful cluster shutdown using
+     * {@link Ignite#active(boolean)} method. If an Ignite node is forefully terminated in NONE mode, it is likely
+     * that data stored on disk is corrupted and work directory will need to be cleared for a successfull node restart.
      */
     NONE;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f57b6776/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
index 76edcea..c8715aa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
@@ -294,10 +294,13 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
             archiver = new FileArchiver(tup == null ? -1 : tup.get2());
 
-            if (mode != WALMode.DEFAULT) {
+            if (mode != WALMode.NONE) {
                 if (log.isInfoEnabled())
                     log.info("Started write-ahead log manager [mode=" + mode + ']');
             }
+            else
+                U.quietAndWarn(log, "Started write-ahead log manager in NONE mode, persisted data may be lost in " +
+                    "a case of unexpected node failure. Make sure to deactivate the cluster before shutdown.");
         }
     }
 


[03/37] ignite git commit: .NET: Fix ServicesTest flakiness

Posted by vo...@apache.org.
.NET: Fix ServicesTest flakiness


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

Branch: refs/heads/ignite-2.3
Commit: cc6c5e7e41fdb9c0af9d75add9177a658aecc6dd
Parents: 0077076
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Sep 28 13:13:39 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Sep 28 13:13:39 2017 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests/Services/ServicesTest.cs       | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cc6c5e7e/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 ae36bcd..04f968c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Tests.Services
 {
     using System;
     using System.Collections;
+    using System.Collections.Generic;
     using System.Diagnostics.CodeAnalysis;
     using System.Linq;
     using System.Threading;
@@ -545,6 +546,7 @@ namespace Apache.Ignite.Core.Tests.Services
             var ex = Assert.Throws<ServiceDeploymentException>(() =>
                 Services.DeployMultiple(SvcName, svc, Grids.Length, 1));
             
+            Assert.IsNotNull(ex.InnerException);
             Assert.AreEqual("Expected exception", ex.InnerException.Message);
 
             var svc0 = Services.GetService<TestIgniteServiceSerializable>(SvcName);
@@ -705,11 +707,12 @@ namespace Apache.Ignite.Core.Tests.Services
         /// </summary>
         private static void CheckServiceStarted(IIgnite grid, int count = 1)
         {
-            var services = grid.GetServices().GetServices<TestIgniteServiceSerializable>(SvcName);
+            Func<ICollection<TestIgniteServiceSerializable>> getServices = () =>
+                grid.GetServices().GetServices<TestIgniteServiceSerializable>(SvcName);
 
-            Assert.AreEqual(count, services.Count);
+            Assert.IsTrue(TestUtils.WaitForCondition(() => count == getServices().Count, 5000));
 
-            var svc = services.First();
+            var svc = getServices().First();
 
             Assert.IsNotNull(svc);
 


[15/37] ignite git commit: File header fixed

Posted by vo...@apache.org.
File header fixed


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

Branch: refs/heads/ignite-2.3
Commit: c4047b4ef02b4f884b303000c6903e37e5c51069
Parents: 8f599c5
Author: Andrey Gura <ag...@apache.org>
Authored: Fri Sep 29 16:01:29 2017 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Fri Sep 29 16:01:29 2017 +0300

----------------------------------------------------------------------
 .../persistence/wal/RecordDataSerializer.java      | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c4047b4e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordDataSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordDataSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordDataSerializer.java
index 242641d..5a14095 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordDataSerializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordDataSerializer.java
@@ -1,3 +1,20 @@
+/*
+ * 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.persistence.wal;
 
 import java.io.IOException;


[10/37] ignite git commit: IGNITE-6516 .NET: PersistentStoreConfiguration.WriteThrottlingEnabled

Posted by vo...@apache.org.
IGNITE-6516 .NET: PersistentStoreConfiguration.WriteThrottlingEnabled


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

Branch: refs/heads/ignite-2.3
Commit: 5d6ff970d733adec934825556856d319cfba196d
Parents: 2108d0c
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Sep 28 19:17:18 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Sep 28 19:17:18 2017 +0300

----------------------------------------------------------------------
 .../platform/utils/PlatformConfigurationUtils.java   |  4 +++-
 .../IgniteConfigurationSerializerTest.cs             |  6 ++++--
 .../IgniteConfigurationTest.cs                       |  6 +++++-
 .../IgniteConfigurationSection.xsd                   |  5 +++++
 .../PersistentStore/PersistentStoreConfiguration.cs  | 15 +++++++++++++++
 5 files changed, 32 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5d6ff970/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 fe214da..aa4f168 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
@@ -1578,7 +1578,8 @@ public class PlatformConfigurationUtils {
                 .setMetricsEnabled(in.readBoolean())
                 .setSubIntervals(in.readInt())
                 .setRateTimeInterval(in.readLong())
-                .setCheckpointWriteOrder(CheckpointWriteOrder.fromOrdinal(in.readInt()));
+                .setCheckpointWriteOrder(CheckpointWriteOrder.fromOrdinal(in.readInt()))
+                .setWriteThrottlingEnabled(in.readBoolean());
     }
 
     /**
@@ -1612,6 +1613,7 @@ public class PlatformConfigurationUtils {
             w.writeInt(cfg.getSubIntervals());
             w.writeLong(cfg.getRateTimeInterval());
             w.writeInt(cfg.getCheckpointWriteOrder().ordinal());
+            w.writeBoolean(cfg.isWriteThrottlingEnabled());
 
         } else {
             w.writeBoolean(false);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5d6ff970/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 2530243..83700cb 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -143,7 +143,7 @@ namespace Apache.Ignite.Core.Tests
                             </memoryConfiguration>
                             <sqlConnectorConfiguration host='bar' port='10' portRange='11' socketSendBufferSize='12' socketReceiveBufferSize='13' tcpNoDelay='true' maxOpenCursorsPerConnection='14' threadPoolSize='15' />
                             <clientConnectorConfiguration host='bar' port='10' portRange='11' socketSendBufferSize='12' socketReceiveBufferSize='13' tcpNoDelay='true' maxOpenCursorsPerConnection='14' threadPoolSize='15' />
-                            <persistentStoreConfiguration alwaysWriteFullPages='true' checkpointingFrequency='00:00:1' checkpointingPageBufferSize='2' checkpointingThreads='3' lockWaitTime='00:00:04' persistentStorePath='foo' tlbSize='5' walArchivePath='bar' walFlushFrequency='00:00:06' walFsyncDelayNanos='7' walHistorySize='8' walMode='None' walRecordIteratorBufferSize='9' walSegments='10' walSegmentSize='11' walStorePath='baz' metricsEnabled='true' rateTimeInterval='0:0:6' subIntervals='3' checkpointWriteOrder='Random' />
+                            <persistentStoreConfiguration alwaysWriteFullPages='true' checkpointingFrequency='00:00:1' checkpointingPageBufferSize='2' checkpointingThreads='3' lockWaitTime='00:00:04' persistentStorePath='foo' tlbSize='5' walArchivePath='bar' walFlushFrequency='00:00:06' walFsyncDelayNanos='7' walHistorySize='8' walMode='None' walRecordIteratorBufferSize='9' walSegments='10' walSegmentSize='11' walStorePath='baz' metricsEnabled='true' rateTimeInterval='0:0:6' subIntervals='3' checkpointWriteOrder='Random' writeThrottlingEnabled='true' />
                             <consistentId type='System.String'>someId012</consistentId>
                             <localEventListeners>
                               <localEventListener type='Apache.Ignite.Core.Events.LocalEventListener`1[[Apache.Ignite.Core.Events.CacheRebalancingEvent]]'>
@@ -349,6 +349,7 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(3, pers.SubIntervals);
             Assert.AreEqual(TimeSpan.FromSeconds(6), pers.RateTimeInterval);
             Assert.AreEqual(CheckpointWriteOrder.Random, pers.CheckpointWriteOrder);
+            Assert.IsTrue(pers.WriteThrottlingEnabled);
 
             var listeners = cfg.LocalEventListeners;
             Assert.AreEqual(2, listeners.Count);
@@ -906,7 +907,8 @@ namespace Apache.Ignite.Core.Tests
                     SubIntervals = 25,
                     MetricsEnabled = true,
                     RateTimeInterval = TimeSpan.FromDays(1),
-                    CheckpointWriteOrder = CheckpointWriteOrder.Random
+                    CheckpointWriteOrder = CheckpointWriteOrder.Random,
+                    WriteThrottlingEnabled = true
                 },
                 IsActiveOnStart = false,
                 ConsistentId = "myId123",

http://git-wip-us.apache.org/repos/asf/ignite/blob/5d6ff970/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 995924a..efa404d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -277,6 +277,8 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual(pers.MetricsEnabled, resPers.MetricsEnabled);
                 Assert.AreEqual(pers.RateTimeInterval, resPers.RateTimeInterval);
                 Assert.AreEqual(pers.SubIntervals, resPers.SubIntervals);
+                Assert.AreEqual(pers.CheckpointWriteOrder, resPers.CheckpointWriteOrder);
+                Assert.AreEqual(pers.WriteThrottlingEnabled, resPers.WriteThrottlingEnabled);
             }
         }
 
@@ -588,6 +590,7 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(PersistentStoreConfiguration.DefaultWalStorePath, cfg.WalStorePath);
             Assert.AreEqual(PersistentStoreConfiguration.DefaultWalArchivePath, cfg.WalArchivePath);
             Assert.AreEqual(PersistentStoreConfiguration.DefaultCheckpointWriteOrder, cfg.CheckpointWriteOrder);
+            Assert.AreEqual(PersistentStoreConfiguration.DefaultWriteThrottlingEnabled, cfg.WriteThrottlingEnabled);
         }
 
         /// <summary>
@@ -826,7 +829,8 @@ namespace Apache.Ignite.Core.Tests
                     MetricsEnabled = true,
                     SubIntervals = 7,
                     RateTimeInterval = TimeSpan.FromSeconds(9),
-                    CheckpointWriteOrder = CheckpointWriteOrder.Random
+                    CheckpointWriteOrder = CheckpointWriteOrder.Random,
+                    WriteThrottlingEnabled = true
                 },
                 ConsistentId = new MyConsistentId {Data = "abc"}
             };

http://git-wip-us.apache.org/repos/asf/ignite/blob/5d6ff970/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 80bacf4..6788ef0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -1468,6 +1468,11 @@
                                 <xs:documentation>Checkpoint page write order on disk.</xs:documentation>
                             </xs:annotation>
                         </xs:attribute>
+                        <xs:attribute name="writeThrottlingEnabled" type="xs:boolean">
+                            <xs:annotation>
+                                <xs:documentation>Threads that generate dirty pages too fast during ongoing checkpoint will be throttled.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
                     </xs:complexType>
                 </xs:element>
                 <xs:element name="pluginConfigurations" minOccurs="0">

http://git-wip-us.apache.org/repos/asf/ignite/blob/5d6ff970/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
index f71f50e..7a2248a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
@@ -107,6 +107,11 @@ namespace Apache.Ignite.Core.PersistentStore
         public const CheckpointWriteOrder DefaultCheckpointWriteOrder = CheckpointWriteOrder.Sequential;
 
         /// <summary>
+        /// Default value for <see cref="WriteThrottlingEnabled"/>.
+        /// </summary>
+        public const bool DefaultWriteThrottlingEnabled = false;
+
+        /// <summary>
         /// Initializes a new instance of the <see cref="PersistentStoreConfiguration"/> class.
         /// </summary>
         public PersistentStoreConfiguration()
@@ -126,6 +131,7 @@ namespace Apache.Ignite.Core.PersistentStore
             WalArchivePath = DefaultWalArchivePath;
             WalStorePath = DefaultWalStorePath;
             CheckpointWriteOrder = DefaultCheckpointWriteOrder;
+            WriteThrottlingEnabled = DefaultWriteThrottlingEnabled;
         }
 
         /// <summary>
@@ -156,6 +162,7 @@ namespace Apache.Ignite.Core.PersistentStore
             SubIntervals = reader.ReadInt();
             RateTimeInterval = reader.ReadLongAsTimespan();
             CheckpointWriteOrder = (CheckpointWriteOrder) reader.ReadInt();
+            WriteThrottlingEnabled = reader.ReadBoolean();
         }
 
         /// <summary>
@@ -186,6 +193,7 @@ namespace Apache.Ignite.Core.PersistentStore
             writer.WriteInt(SubIntervals);
             writer.WriteTimeSpanAsLong(RateTimeInterval);
             writer.WriteInt((int) CheckpointWriteOrder);
+            writer.WriteBoolean(WriteThrottlingEnabled);
         }
 
         /// <summary>
@@ -312,5 +320,12 @@ namespace Apache.Ignite.Core.PersistentStore
         /// </summary>
         [DefaultValue(DefaultCheckpointWriteOrder)]
         public CheckpointWriteOrder CheckpointWriteOrder { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether threads that generate dirty
+        /// pages too fast during ongoing checkpoint will be throttled.
+        /// </summary>
+        [DefaultValue(DefaultWriteThrottlingEnabled)]
+        public bool WriteThrottlingEnabled { get; set; }
     }
 }


[05/37] ignite git commit: IGNITE-5732 Provide API to test compatibility with old releases (HotFix - TeamCity)

Posted by vo...@apache.org.
IGNITE-5732 Provide API to test compatibility with old releases (HotFix - TeamCity)


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

Branch: refs/heads/ignite-2.3
Commit: 507ac67568743cd0838d0822979022a698a79957
Parents: f2dedca
Author: Vyacheslav Daradur <da...@gmail.com>
Authored: Thu Sep 28 15:42:11 2017 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Sep 28 15:42:11 2017 +0300

----------------------------------------------------------------------
 ...nitePersistenceCompatibilityAbstractTest.java |  2 +-
 .../junits/IgniteCompatibilityAbstractTest.java  | 19 +++++++++++--------
 2 files changed, 12 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/507ac675/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java
index 6eb49e4..87c24cb 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java
@@ -28,7 +28,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 /**
  * Super class for all persistence compatibility tests.
  */
-public class IgnitePersistenceCompatibilityAbstractTest extends IgniteCompatibilityAbstractTest {
+public abstract class IgnitePersistenceCompatibilityAbstractTest extends IgniteCompatibilityAbstractTest {
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();

http://git-wip-us.apache.org/repos/asf/ignite/blob/507ac675/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityAbstractTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityAbstractTest.java
index 728227a..07d63fd 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityAbstractTest.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityAbstractTest.java
@@ -18,6 +18,8 @@
 package org.apache.ignite.compatibility.testframework.junits;
 
 import java.io.File;
+import java.net.URL;
+import java.net.URLClassLoader;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.UUID;
@@ -30,10 +32,8 @@ import org.apache.ignite.compatibility.testframework.util.MavenUtils;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.resource.GridSpringResourceContext;
-import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteInClosure;
-import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.testframework.junits.multijvm.IgniteProcessProxy;
 import org.jetbrains.annotations.Nullable;
@@ -42,6 +42,9 @@ import org.jetbrains.annotations.Nullable;
  * Super class for all compatibility tests.
  */
 public abstract class IgniteCompatibilityAbstractTest extends GridCommonAbstractTest {
+    /** */
+    private static final ClassLoader CLASS_LOADER = IgniteCompatibilityAbstractTest.class.getClassLoader();
+
     /** Using for synchronization of nodes startup in case of starting remote nodes first. */
     public static final String SYNCHRONIZATION_LOG_MESSAGE_PREPARED = "[Compatibility] Node has been started, id=";
 
@@ -160,16 +163,16 @@ public abstract class IgniteCompatibilityAbstractTest extends GridCommonAbstract
                         filteredJvmArgs.add(arg);
                 }
 
-                String classPath = System.getProperty("java.class.path");
-
-                String[] paths = classPath.split(File.pathSeparator);
+                URLClassLoader ldr = (URLClassLoader)CLASS_LOADER;
 
                 StringBuilder pathBuilder = new StringBuilder();
 
-                String corePathTemplate = "ignite.modules.core.target.classes".replace(".", File.separator);
-                String coreTestsPathTemplate = "ignite.modules.core.target.test-classes".replace(".", File.separator);
+                String corePathTemplate = "modules/core/target/classes";
+                String coreTestsPathTemplate = "modules/core/target/test-classes";
+
+                for (URL url : ldr.getURLs()) {
+                    String path = url.getPath();
 
-                for (String path : paths) {
                     if (!path.contains(corePathTemplate) && !path.contains(coreTestsPathTemplate))
                         pathBuilder.append(path).append(File.pathSeparator);
                 }


[04/37] ignite git commit: Fixed exception type in GridIoManager.

Posted by vo...@apache.org.
Fixed exception type in GridIoManager.


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

Branch: refs/heads/ignite-2.3
Commit: f2dedcab2c2e5e278574ebd7237f4cf593d7d773
Parents: cc6c5e7
Author: sboikov <sb...@gridgain.com>
Authored: Thu Sep 28 14:54:50 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Sep 28 14:54:50 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/managers/communication/GridIoManager.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f2dedcab/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index f63f5f0..9c6271a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -1667,7 +1667,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
         ClusterNode node = ctx.discovery().node(nodeId);
 
         if (node == null)
-            throw new IgniteCheckedException("Failed to send message to node (has node left grid?): " + nodeId);
+            throw new ClusterTopologyCheckedException("Failed to send message to node (has node left grid?): " + nodeId);
 
         sendToCustomTopic(node, topic, msg, plc);
     }
@@ -1685,7 +1685,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
         ClusterNode node = ctx.discovery().node(nodeId);
 
         if (node == null)
-            throw new IgniteCheckedException("Failed to send message to node (has node left grid?): " + nodeId);
+            throw new ClusterTopologyCheckedException("Failed to send message to node (has node left grid?): " + nodeId);
 
         send(node, topic, topic.ordinal(), msg, plc, false, 0, false, null, false);
     }


[36/37] ignite git commit: IGNITE-6382 .NET: Set up NDepend project

Posted by vo...@apache.org.
IGNITE-6382 .NET: Set up NDepend project

This closes #2786


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

Branch: refs/heads/ignite-2.3
Commit: b21f750f74d0b6598693184d5925addfe828b7ca
Parents: f42f34b
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Oct 3 13:27:51 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Oct 3 13:27:51 2017 +0300

----------------------------------------------------------------------
 modules/platforms/.gitignore                    |     3 +-
 .../Impl/Binary/BinaryProcessorClient.cs        |     4 +-
 .../Client/Cache/Query/ClientQueryCursor.cs     |     1 -
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |     2 +-
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |     2 +-
 modules/platforms/dotnet/Apache.Ignite.ndproj   | 11139 +++++++++++++++++
 parent/pom.xml                                  |     1 +
 7 files changed, 11146 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b21f750f/modules/platforms/.gitignore
----------------------------------------------------------------------
diff --git a/modules/platforms/.gitignore b/modules/platforms/.gitignore
index 6c05af0..4b415cb 100644
--- a/modules/platforms/.gitignore
+++ b/modules/platforms/.gitignore
@@ -28,4 +28,5 @@ ipch/
 [Rr]elease*/
 packages
 *.classname
-*.nupkg
\ No newline at end of file
+*.nupkg
+NDependOut
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b21f750f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessorClient.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessorClient.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessorClient.cs
index 26a8e9b..816e24a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessorClient.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessorClient.cs
@@ -59,7 +59,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /** <inheritdoc /> */
         public List<IBinaryType> GetBinaryTypes()
         {
-            throw new NotImplementedException();
+            throw new NotSupportedException();
         }
 
         /** <inheritdoc /> */
@@ -96,7 +96,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /** <inheritdoc /> */
         public BinaryType RegisterEnum(string typeName, IEnumerable<KeyValuePair<string, int>> values)
         {
-            throw new NotImplementedException();
+            throw new NotSupportedException();
         }
 
         /** <inheritdoc /> */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b21f750f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs
index 75aa5df..ff891db 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs
@@ -18,7 +18,6 @@
 namespace Apache.Ignite.Core.Impl.Client.Cache.Query
 {
     using System.Collections.Generic;
-    using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
     using System.Linq;
     using Apache.Ignite.Core.Cache;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b21f750f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
index 55f5be8..a81523a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
@@ -284,7 +284,7 @@ namespace Apache.Ignite.Core.Impl.Events
         /** <inheritDoc /> */
         public void RecordLocal(IEvent evt)
         {
-            throw new NotImplementedException("IGNITE-1410");
+            throw new NotSupportedException("IGNITE-1410");
         }
 
         /** <inheritDoc /> */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b21f750f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
index 37a24b1..2cc3659 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
@@ -1060,7 +1060,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
             return 0;
         }
 
-        private long MemoryReallocate(long memPtr, long cap, long unused, void* arg)
+        private static long MemoryReallocate(long memPtr, long cap, long unused, void* arg)
         {
             IgniteManager.Memory.Get(memPtr).Reallocate((int)cap);
 


[07/37] ignite git commit: IGNITE-6046: JDBC thin driver: allowed multiple statements execution (e.g. "SELECT something; SELECT something_else". This closes #2618.

Posted by vo...@apache.org.
IGNITE-6046: JDBC thin driver: allowed multiple statements execution (e.g. "SELECT something; SELECT something_else". This closes #2618.


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

Branch: refs/heads/ignite-2.3
Commit: 405749a736c85c054e2c4ac1dc197ea903e8d75f
Parents: 507ac67
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Thu Sep 28 15:59:12 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Sep 28 15:59:12 2017 +0300

----------------------------------------------------------------------
 .../jdbc/thin/JdbcThinStatementSelfTest.java    | 138 +++++++++++--
 .../ignite/cache/query/SqlFieldsQuery.java      |  18 ++
 .../jdbc/thin/JdbcThinDatabaseMetadata.java     |   2 +-
 .../jdbc/thin/JdbcThinPreparedStatement.java    |   2 +-
 .../internal/jdbc/thin/JdbcThinResultSet.java   |   8 +-
 .../internal/jdbc/thin/JdbcThinStatement.java   | 196 +++++++++++++------
 .../internal/jdbc/thin/JdbcThinTcpIo.java       |   9 +-
 .../odbc/jdbc/JdbcConnectionContext.java        |  10 +-
 ...dbcQueryExecuteMultipleStatementsResult.java | 134 +++++++++++++
 .../odbc/jdbc/JdbcRequestHandler.java           |  82 ++++++--
 .../processors/odbc/jdbc/JdbcResult.java        |   8 +
 .../processors/odbc/jdbc/JdbcResultInfo.java    |  95 +++++++++
 .../processors/query/GridQueryIndexing.java     |   7 +-
 .../processors/query/GridQueryProcessor.java    |  28 ++-
 ...IgniteClientCacheInitializationFailTest.java |   6 +-
 .../query/h2/DmlStatementsProcessor.java        |  35 ++--
 .../processors/query/h2/IgniteH2Indexing.java   | 181 +++++++++++++----
 .../query/h2/ddl/DdlStatementsProcessor.java    |  11 +-
 .../query/h2/sql/GridSqlQueryParser.java        |  81 ++++++++
 .../query/h2/sql/GridSqlQuerySplitter.java      |  17 +-
 .../MultipleStatementsSqlQuerySelfTest.java     | 154 +++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 22 files changed, 1037 insertions(+), 187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStatementSelfTest.java
index 16f118c..5309465 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStatementSelfTest.java
@@ -92,7 +92,6 @@ public class JdbcThinStatementSelfTest extends JdbcThinAbstractSelfTest {
 
         startGridsMultiThreaded(3);
 
-
         fillCache();
     }
 
@@ -414,22 +413,120 @@ public class JdbcThinStatementSelfTest extends JdbcThinAbstractSelfTest {
     /**
      * @throws Exception If failed.
      */
-    public void testExecuteQueryMultipleResultSets() throws Exception {
-        assert !conn.getMetaData().supportsMultipleResultSets();
+    public void testExecuteQueryMultipleOnlyResultSets() throws Exception {
+        assert conn.getMetaData().supportsMultipleResultSets();
 
-        fail("https://issues.apache.org/jira/browse/IGNITE-6046");
+        int stmtCnt = 10;
 
-        final String sqlText = "select 1; select 1";
+        StringBuilder sql = new StringBuilder();
 
-        GridTestUtils.assertThrows(log,
-            new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    return stmt.executeQuery(sqlText);
-                }
-            },
-            SQLException.class,
-            "Multiple result sets"
-        );
+        for (int i = 0; i < stmtCnt; ++i)
+            sql.append("select ").append(i).append("; ");
+
+        assert stmt.execute(sql.toString());
+
+        for (int i = 0; i < stmtCnt; ++i) {
+            assert stmt.getMoreResults();
+
+            ResultSet rs = stmt.getResultSet();
+
+            assert rs.next();
+            assert rs.getInt(1) == i;
+            assert !rs.next();
+        }
+
+        assert !stmt.getMoreResults();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExecuteQueryMultipleOnlyDml() throws Exception {
+        conn.setSchema(null);
+
+        int stmtCnt = 10;
+
+        StringBuilder sql = new StringBuilder("drop table if exists test; create table test(ID int primary key, NAME varchar(20)); ");
+
+        for (int i = 0; i < stmtCnt; ++i)
+            sql.append("insert into test (ID, NAME) values (" + i + ", 'name_" + i +"'); ");
+
+        assert !stmt.execute(sql.toString());
+
+        // DROP TABLE statement
+        assert stmt.getResultSet() == null;
+        assert stmt.getUpdateCount() == 0;
+
+        // CREATE TABLE statement
+        assert stmt.getResultSet() == null;
+        assert stmt.getUpdateCount() == 0;
+
+        for (int i = 0; i < stmtCnt; ++i) {
+            assert stmt.getMoreResults();
+
+            assert stmt.getResultSet() == null;
+            assert stmt.getUpdateCount() == 1;
+        }
+
+        assert !stmt.getMoreResults();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExecuteQueryMultipleMixed() throws Exception {
+        conn.setSchema(null);
+
+        int stmtCnt = 10;
+
+        StringBuilder sql = new StringBuilder("drop table if exists test; create table test(ID int primary key, NAME varchar(20)); ");
+
+        for (int i = 0; i < stmtCnt; ++i) {
+            if (i % 2 == 0)
+                sql.append(" insert into test (ID, NAME) values (" + i + ", 'name_" + i + "'); ");
+            else
+                sql.append(" select * from test where id < " + i + "; ");
+        }
+
+        assert !stmt.execute(sql.toString());
+
+        // DROP TABLE statement
+        assert stmt.getResultSet() == null;
+        assert stmt.getUpdateCount() == 0;
+
+        // CREATE TABLE statement
+        assert stmt.getResultSet() == null;
+        assert stmt.getUpdateCount() == 0;
+
+        boolean notEmptyResult = false;
+
+        for (int i = 0; i < stmtCnt; ++i) {
+            assert stmt.getMoreResults();
+
+            if (i % 2 == 0) {
+                assert stmt.getResultSet() == null;
+                assert stmt.getUpdateCount() == 1;
+            }
+            else {
+                assert stmt.getUpdateCount() == -1;
+
+                ResultSet rs = stmt.getResultSet();
+
+                int rowsCnt = 0;
+
+                while(rs.next())
+                    rowsCnt++;
+
+                assert rowsCnt <= (i + 1) / 2;
+
+                if (rowsCnt == (i + 1) / 2)
+                    notEmptyResult = true;
+            }
+        }
+
+        assert notEmptyResult;
+
+        assert !stmt.getMoreResults();
     }
 
     /**
@@ -462,7 +559,7 @@ public class JdbcThinStatementSelfTest extends JdbcThinAbstractSelfTest {
                 }
             },
             SQLException.class,
-            "The query is not DML"
+            "Given statement type does not match that declared by JDBC driver"
         );
     }
 
@@ -776,12 +873,14 @@ public class JdbcThinStatementSelfTest extends JdbcThinAbstractSelfTest {
     public void testGetMoreResults() throws Exception {
         assert !stmt.getMoreResults();
 
-        stmt.execute("select 1");
+        stmt.execute("select 1; ");
 
         ResultSet rs = stmt.getResultSet();
 
         assert !stmt.getMoreResults();
 
+        assert stmt.getResultSet() == null;
+
         assert rs.isClosed();
 
         stmt.close();
@@ -801,7 +900,7 @@ public class JdbcThinStatementSelfTest extends JdbcThinAbstractSelfTest {
         assert !stmt.getMoreResults(Statement.KEEP_CURRENT_RESULT);
         assert !stmt.getMoreResults(Statement.CLOSE_ALL_RESULTS);
 
-        stmt.execute("select 1");
+        stmt.execute("select 1; ");
 
         ResultSet rs = stmt.getResultSet();
 
@@ -997,7 +1096,10 @@ public class JdbcThinStatementSelfTest extends JdbcThinAbstractSelfTest {
     /**
      * @throws Exception If failed.
      */
-    public void testStatementTypeMismatchSelect() throws Exception {
+    public void testStatementTypeMismatchSelectForCachedQuery() throws Exception {
+        // Put query to cache.
+        stmt.executeQuery("select 1;");
+
         GridTestUtils.assertThrows(log,
             new Callable<Object>() {
                 @Override public Object call() throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
index 54f8396..2d128d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
@@ -81,6 +81,24 @@ public class SqlFieldsQuery extends Query<List<?>> {
     private String schema;
 
     /**
+     * Copy constructs SQL fields query.
+     *
+     * @param qry SQL query.
+     */
+    public SqlFieldsQuery(SqlFieldsQuery qry) {
+        sql = qry.sql;
+        args = qry.args;
+        collocated = qry.collocated;
+        timeout = qry.timeout;
+        enforceJoinOrder = qry.enforceJoinOrder;
+        distributedJoins = qry.distributedJoins;
+        replicatedOnly = qry.replicatedOnly;
+        lazy = qry.lazy;
+        parts = qry.parts;
+        schema = qry.schema;
+    }
+
+    /**
      * Constructs SQL fields query.
      *
      * @param sql SQL query.

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
index d13ef68..2ce7983 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
@@ -311,7 +311,7 @@ public class JdbcThinDatabaseMetadata implements DatabaseMetaData {
 
     /** {@inheritDoc} */
     @Override public boolean supportsMultipleResultSets() throws SQLException {
-        return false;
+        return true;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
index ce1b65c..fb2810d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
@@ -238,7 +238,7 @@ public class JdbcThinPreparedStatement extends JdbcThinStatement implements Prep
     @Override public boolean execute() throws SQLException {
         executeWithArguments(JdbcStatementType.ANY_STATEMENT_TYPE);
 
-        return rs.isQuery();
+        return resultSets.get(0).isQuery();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinResultSet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinResultSet.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinResultSet.java
index 189175b..ff93274 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinResultSet.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinResultSet.java
@@ -187,7 +187,7 @@ public class JdbcThinResultSet implements ResultSet {
             this.fetchSize = fetchSize;
             this.rows = rows;
 
-            rowsIter = rows.iterator();
+            rowsIter = rows != null ? rows.iterator() : null;
         }
         else
             this.updCnt = updCnt;
@@ -228,10 +228,10 @@ public class JdbcThinResultSet implements ResultSet {
 
     /** {@inheritDoc} */
     @Override public void close() throws SQLException {
-        if (closeStmt)
-            stmt.close();
-
         close0();
+
+        if (closeStmt)
+            stmt.closeIfAllResultsClosed();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
index 8e096c8..603545b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
@@ -25,6 +25,7 @@ import java.sql.SQLFeatureNotSupportedException;
 import java.sql.SQLWarning;
 import java.sql.Statement;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import org.apache.ignite.cache.query.SqlQuery;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
@@ -33,9 +34,12 @@ import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteRequest;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteResult;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQuery;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryExecuteMultipleStatementsResult;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryExecuteRequest;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryExecuteResult;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcStatementType;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResultInfo;
 
 import static java.sql.ResultSet.CONCUR_READ_ONLY;
 import static java.sql.ResultSet.FETCH_FORWARD;
@@ -60,15 +64,9 @@ public class JdbcThinStatement implements Statement {
     /** Query timeout. */
     private int timeout;
 
-    /** Current result set. */
-    protected JdbcThinResultSet rs;
-
     /** Fetch size. */
     private int pageSize = DFLT_PAGE_SIZE;
 
-    /** Result set or update count has been already read. */
-    private boolean alreadyRead;
-
     /** Result set  holdability*/
     private final int resHoldability;
 
@@ -78,6 +76,12 @@ public class JdbcThinStatement implements Statement {
     /** Close this statement on result set close. */
     private boolean closeOnCompletion;
 
+    /** Result sets. */
+    protected List<JdbcThinResultSet> resultSets;
+
+    /** Current result index. */
+    protected int curRes;
+
     /**
      * Creates new statement.
      *
@@ -113,24 +117,58 @@ public class JdbcThinStatement implements Statement {
     protected void execute0(JdbcStatementType stmtType, String sql, List<Object> args) throws SQLException {
         ensureNotClosed();
 
-        if (rs != null) {
-            rs.close0();
-
-            rs = null;
-        }
-
-        alreadyRead = false;
+        closeResults();
 
         if (sql == null || sql.isEmpty())
             throw new SQLException("SQL query is empty.");
 
-        JdbcQueryExecuteResult res = conn.sendRequest(new JdbcQueryExecuteRequest(stmtType, conn.getSchema(), pageSize,
+        JdbcResult res0 = conn.sendRequest(new JdbcQueryExecuteRequest(stmtType, conn.getSchema(), pageSize,
             maxRows, sql, args == null ? null : args.toArray(new Object[args.size()])));
 
-        assert res != null;
+        assert res0 != null;
 
-        rs = new JdbcThinResultSet(this, res.getQueryId(), pageSize, res.last(), res.items(),
-            res.isQuery(), conn.autoCloseServerCursor(), res.updateCount(), closeOnCompletion);
+        if (res0 instanceof JdbcQueryExecuteResult) {
+            JdbcQueryExecuteResult res = (JdbcQueryExecuteResult)res0;
+
+            resultSets = Collections.singletonList(new JdbcThinResultSet(this, res.getQueryId(), pageSize,
+                res.last(), res.items(), res.isQuery(), conn.autoCloseServerCursor(), res.updateCount(),
+                closeOnCompletion));
+        }
+        else if (res0 instanceof JdbcQueryExecuteMultipleStatementsResult) {
+            JdbcQueryExecuteMultipleStatementsResult res = (JdbcQueryExecuteMultipleStatementsResult)res0;
+
+            List<JdbcResultInfo> resInfos = res.results();
+
+            resultSets = new ArrayList<>(resInfos.size());
+
+            boolean firstRes = true;
+
+            for(JdbcResultInfo rsInfo : resInfos) {
+                if (!rsInfo.isQuery()) {
+                    resultSets.add(new JdbcThinResultSet(this, -1, pageSize,
+                        true, Collections.<List<Object>>emptyList(), false,
+                        conn.autoCloseServerCursor(), rsInfo.updateCount(), closeOnCompletion));
+                }
+                else {
+                    if (firstRes) {
+                        firstRes = false;
+
+                        resultSets.add(new JdbcThinResultSet(this, rsInfo.queryId(), pageSize,
+                            res.isLast(), res.items(), true,
+                            conn.autoCloseServerCursor(), -1, closeOnCompletion));
+                    }
+                    else {
+                        resultSets.add(new JdbcThinResultSet(this, rsInfo.queryId(), pageSize,
+                            false, null, true,
+                            conn.autoCloseServerCursor(), -1, closeOnCompletion));
+                    }
+                }
+            }
+        }
+        else
+            throw new SQLException("Unexpected result [res=" + res0 + ']');
+
+        assert resultSets.size() > 0 : "At least one results set is expected";
     }
 
     /** {@inheritDoc} */
@@ -140,7 +178,7 @@ public class JdbcThinStatement implements Statement {
         int res = getUpdateCount();
 
         if (res == -1)
-            throw new SQLException("The query is not DML statement: " + sql, SqlStateCode.PARSING_EXCEPTION);
+            throw new SQLException("The query is not DML statememt: " + sql);
 
         return res;
     }
@@ -150,12 +188,25 @@ public class JdbcThinStatement implements Statement {
         if (isClosed())
             return;
 
-        if (rs != null)
-            rs.close0();
+        closeResults();
 
         closed = true;
     }
 
+    /**
+     * Close results.
+     * @throws SQLException On error.
+     */
+    private void closeResults() throws SQLException {
+        if (resultSets != null) {
+            for (JdbcThinResultSet rs : resultSets)
+                rs.close0();
+
+            resultSets = null;
+            curRes = 0;
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public int getMaxFieldSize() throws SQLException {
         ensureNotClosed();
@@ -242,31 +293,39 @@ public class JdbcThinStatement implements Statement {
 
         execute0(JdbcStatementType.ANY_STATEMENT_TYPE, sql, null);
 
-        return rs.isQuery();
+        return resultSets.get(0).isQuery();
     }
 
     /** {@inheritDoc} */
     @Override public ResultSet getResultSet() throws SQLException {
-        JdbcThinResultSet rs = lastResultSet();
+        JdbcThinResultSet rs = nextResultSet();
 
-        ResultSet res = rs == null || !rs.isQuery() ? null : rs;
+        if (rs == null)
+            return null;
 
-        if (res != null)
-            alreadyRead = true;
+        if (!rs.isQuery()) {
+            curRes--;
 
-        return res;
+            return null;
+        }
+
+        return rs;
     }
 
     /** {@inheritDoc} */
     @Override public int getUpdateCount() throws SQLException {
-        JdbcThinResultSet rs = lastResultSet();
+        JdbcThinResultSet rs = nextResultSet();
+
+        if (rs == null)
+            return -1;
 
-        int res = rs == null || rs.isQuery() ? -1 : (int)rs.updatedCount();
+        if (rs.isQuery()) {
+            curRes--;
 
-        if (res != -1)
-            alreadyRead = true;
+            return -1;
+        }
 
-        return res;
+        return (int)rs.updatedCount();
     }
 
     /**
@@ -275,13 +334,13 @@ public class JdbcThinStatement implements Statement {
      * @return Result set or null.
      * @throws SQLException If failed.
      */
-    private JdbcThinResultSet lastResultSet() throws SQLException {
+    private JdbcThinResultSet nextResultSet() throws SQLException {
         ensureNotClosed();
 
-        if (rs == null || alreadyRead)
+        if (resultSets == null || curRes >= resultSets.size())
             return null;
-
-        return rs;
+        else
+            return resultSets.get(curRes++);
     }
 
     /** {@inheritDoc} */
@@ -358,13 +417,7 @@ public class JdbcThinStatement implements Statement {
     @Override public int[] executeBatch() throws SQLException {
         ensureNotClosed();
 
-        if (rs != null) {
-            rs.close0();
-
-            rs = null;
-        }
-
-        alreadyRead = false;
+        closeResults();
 
         if (batch == null || batch.isEmpty())
             throw new SQLException("Batch is empty.");
@@ -395,22 +448,32 @@ public class JdbcThinStatement implements Statement {
     @Override public boolean getMoreResults(int curr) throws SQLException {
         ensureNotClosed();
 
-        switch (curr) {
-            case CLOSE_CURRENT_RESULT:
-            case CLOSE_ALL_RESULTS:
-                if (rs != null)
-                    rs.close();
+        if (resultSets != null) {
+            assert curRes <= resultSets.size() : "Invalid results state: [resultsCount=" + resultSets.size() +
+                ", curRes=" + curRes + ']';
 
-                break;
+            switch (curr) {
+                case CLOSE_CURRENT_RESULT:
+                    if (curRes > 0)
+                        resultSets.get(curRes - 1).close0();
 
-            case KEEP_CURRENT_RESULT:
-                break;
+                    break;
 
-            default:
-                throw new SQLException("Invalid 'current' parameter.");
+                case CLOSE_ALL_RESULTS:
+                    for (int i = 0; i < curRes; ++i)
+                        resultSets.get(i).close0();
+
+                    break;
+
+                case KEEP_CURRENT_RESULT:
+                    break;
+
+                default:
+                    throw new SQLException("Invalid 'current' parameter.");
+            }
         }
 
-        return false;
+        return (resultSets != null && curRes < resultSets.size());
     }
 
     /** {@inheritDoc} */
@@ -532,8 +595,10 @@ public class JdbcThinStatement implements Statement {
 
         closeOnCompletion = true;
 
-        if (rs != null)
-            rs.closeStatement(true);
+        if (resultSets != null) {
+            for (JdbcThinResultSet rs : resultSets)
+                rs.closeStatement(true);
+        }
     }
 
     /** {@inheritDoc} */
@@ -568,4 +633,25 @@ public class JdbcThinStatement implements Statement {
         if (isClosed())
             throw new SQLException("Statement is closed.");
     }
+
+    /**
+     * Used by statement on closeOnCompletion mode.
+     * @throws SQLException On error.
+     */
+    void closeIfAllResultsClosed() throws SQLException {
+        if (isClosed())
+            return;
+
+        boolean allRsClosed = true;
+
+        if (resultSets != null) {
+            for (JdbcThinResultSet rs : resultSets) {
+                if (!rs.isClosed())
+                    allRsClosed = false;
+            }
+        }
+
+        if (allRsClosed)
+            close();
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
index d7fa9d0..7ac9c2c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
@@ -46,12 +46,15 @@ import org.apache.ignite.lang.IgniteProductVersion;
  * JDBC IO layer implementation based on blocking IPC streams.
  */
 public class JdbcThinTcpIo {
-    /** Current version. */
-    private static final ClientListenerProtocolVersion CURRENT_VER = ClientListenerProtocolVersion.create(2, 1, 5);
-
     /** Version 2.1.0. */
     private static final ClientListenerProtocolVersion VER_2_1_0 = ClientListenerProtocolVersion.create(2, 1, 0);
 
+    /** Version 2.3.1. */
+    private static final ClientListenerProtocolVersion VER_2_3_1 = ClientListenerProtocolVersion.create(2, 3, 1);
+
+    /** Current version. */
+    private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_3_1;
+
     /** Initial output stream capacity for handshake. */
     private static final int HANDSHAKE_MSG_SIZE = 13;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
index 12be361..38d1972 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
@@ -37,8 +37,11 @@ public class JdbcConnectionContext implements ClientListenerConnectionContext {
     /** Version 2.1.5: added "lazy" flag. */
     private static final ClientListenerProtocolVersion VER_2_1_5 = ClientListenerProtocolVersion.create(2, 1, 5);
 
+    /** Version 2.3.1: added "multiple statements query" feature. */
+    public static final ClientListenerProtocolVersion VER_2_3_1 = ClientListenerProtocolVersion.create(2, 3, 1);
+
     /** Current version. */
-    private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_1_5;
+    private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_3_1;
 
     /** Supported versions. */
     private static final Set<ClientListenerProtocolVersion> SUPPORTED_VERS = new HashSet<>();
@@ -60,6 +63,7 @@ public class JdbcConnectionContext implements ClientListenerConnectionContext {
 
     static {
         SUPPORTED_VERS.add(CURRENT_VER);
+        SUPPORTED_VERS.add(VER_2_1_5);
         SUPPORTED_VERS.add(VER_2_1_0);
     }
 
@@ -100,8 +104,8 @@ public class JdbcConnectionContext implements ClientListenerConnectionContext {
         if (ver.compareTo(VER_2_1_5) >= 0)
             lazyExec = reader.readBoolean();
 
-        handler = new JdbcRequestHandler(ctx, busyLock, maxCursors, distributedJoins,
-                enforceJoinOrder, collocated, replicatedOnly, autoCloseCursors, lazyExec);
+        handler = new JdbcRequestHandler(ctx, busyLock, maxCursors, distributedJoins, enforceJoinOrder,
+            collocated, replicatedOnly, autoCloseCursors, lazyExec, ver);
 
         parser = new JdbcMessageParser(ctx);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteMultipleStatementsResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteMultipleStatementsResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteMultipleStatementsResult.java
new file mode 100644
index 0000000..9bbdd59
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteMultipleStatementsResult.java
@@ -0,0 +1,134 @@
+/*
+ * 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.odbc.jdbc;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC query execute result for query with multiple SQL statements.
+ */
+public class JdbcQueryExecuteMultipleStatementsResult extends JdbcResult {
+    /** Statements results. */
+    private List<JdbcResultInfo> results;
+
+    /** Query result rows for the first query. */
+    private List<List<Object>> items;
+
+    /** Flag indicating the query has no unfetched results for the first query. */
+    private boolean last;
+
+    /**
+     * Default constructor.
+     */
+    JdbcQueryExecuteMultipleStatementsResult() {
+        super(QRY_EXEC_MULT);
+    }
+
+    /**
+     * @param results Statements results.
+     * @param items Query result rows for the first query.
+     * @param last Flag indicating the query has no unfetched results for the first query.
+     */
+    public JdbcQueryExecuteMultipleStatementsResult(List<JdbcResultInfo> results,
+        List<List<Object>> items, boolean last) {
+        super(QRY_EXEC_MULT);
+        this.results = results;
+        this.items = items;
+        this.last = last;
+    }
+
+    /**
+     * @return Update counts of query IDs.
+     */
+    public List<JdbcResultInfo> results() {
+        return results;
+    }
+
+    /**
+     * @return Query result rows for the first query.
+     */
+    public List<List<Object>> items() {
+        return items;
+    }
+
+    /**
+     * @return Flag indicating the query has no unfetched results for the first query.
+     */
+    public boolean isLast() {
+        return last;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        if (results != null && results.size() > 0) {
+            writer.writeInt(results.size());
+
+            for (JdbcResultInfo r : results)
+                r.writeBinary(writer);
+
+            if (results.get(0).isQuery()) {
+                writer.writeBoolean(last);
+
+                JdbcUtils.writeItems(writer, items);
+            }
+        }
+        else
+            writer.writeInt(0);
+    }
+
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        int cnt = reader.readInt();
+
+        if (cnt == 0)
+            results = Collections.emptyList();
+        else {
+            results = new ArrayList<>(cnt);
+
+            for (int i = 0; i < cnt; ++i) {
+                JdbcResultInfo r = new JdbcResultInfo();
+
+                r.readBinary(reader);
+
+                results.add(r);
+            }
+
+            if (results.get(0).isQuery()) {
+                last = reader.readBoolean();
+
+                items = JdbcUtils.readItems(reader);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcQueryExecuteMultipleStatementsResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
index ea25b11..202f813 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
@@ -37,6 +37,7 @@ import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.jdbc2.JdbcSqlFieldsQuery;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequest;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler;
 import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
@@ -102,6 +103,9 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
     /** Automatic close of cursors. */
     private final boolean autoCloseCursors;
 
+    /** Protocol version. */
+    private ClientListenerProtocolVersion protocolVer;
+
     /**
      * Constructor.
      *
@@ -114,10 +118,11 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
      * @param replicatedOnly Replicated only flag.
      * @param autoCloseCursors Flag to automatically close server cursors.
      * @param lazy Lazy query execution flag.
+     * @param protocolVer Protocol version.
      */
     public JdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors,
         boolean distributedJoins, boolean enforceJoinOrder, boolean collocated, boolean replicatedOnly,
-        boolean autoCloseCursors, boolean lazy) {
+        boolean autoCloseCursors, boolean lazy, ClientListenerProtocolVersion protocolVer) {
         this.ctx = ctx;
         this.busyLock = busyLock;
         this.maxCursors = maxCursors;
@@ -127,6 +132,7 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
         this.replicatedOnly = replicatedOnly;
         this.autoCloseCursors = autoCloseCursors;
         this.lazy = lazy;
+        this.protocolVer = protocolVer;
 
         log = ctx.log(getClass());
     }
@@ -284,31 +290,71 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
 
             qry.setSchema(schemaName);
 
-            FieldsQueryCursor<List<?>> qryCur = ctx.query().querySqlFieldsNoCache(qry, true);
+            List<FieldsQueryCursor<List<?>>> results = ctx.query().querySqlFieldsNoCache(qry, true,
+                protocolVer.compareTo(JdbcConnectionContext.VER_2_3_1) < 0);
+
+            if (results.size() == 1) {
+                FieldsQueryCursor<List<?>> qryCur = results.get(0);
+
+                JdbcQueryCursor cur = new JdbcQueryCursor(qryId, req.pageSize(), req.maxRows(), (QueryCursorImpl)qryCur);
+
+                JdbcQueryExecuteResult res;
+
+                if (cur.isQuery())
+                    res = new JdbcQueryExecuteResult(qryId, cur.fetchRows(), !cur.hasNext());
+                else {
+                    List<List<Object>> items = cur.fetchRows();
 
-            JdbcQueryCursor cur = new JdbcQueryCursor(qryId, req.pageSize(), req.maxRows(), (QueryCursorImpl)qryCur);
+                    assert items != null && items.size() == 1 && items.get(0).size() == 1
+                        && items.get(0).get(0) instanceof Long :
+                        "Invalid result set for not-SELECT query. [qry=" + sql +
+                            ", res=" + S.toString(List.class, items) + ']';
 
-            JdbcQueryExecuteResult res;
+                    res = new JdbcQueryExecuteResult(qryId, (Long)items.get(0).get(0));
+                }
+
+                if (res.last() && (!res.isQuery() || autoCloseCursors))
+                    cur.close();
+                else
+                    qryCursors.put(qryId, cur);
 
-            if (cur.isQuery())
-                res = new JdbcQueryExecuteResult(qryId, cur.fetchRows(), !cur.hasNext());
+                return new JdbcResponse(res);
+            }
             else {
-                List<List<Object>> items = cur.fetchRows();
+                List<JdbcResultInfo> jdbcResults = new ArrayList<>(results.size());
+                List<List<Object>> items = null;
+                boolean last = true;
+
+                for (FieldsQueryCursor<List<?>> c : results) {
+                    QueryCursorImpl qryCur = (QueryCursorImpl)c;
+
+                    JdbcResultInfo jdbcRes;
 
-                assert items != null && items.size() == 1 && items.get(0).size() == 1
-                    && items.get(0).get(0) instanceof Long :
-                    "Invalid result set for not-SELECT query. [qry=" + sql +
-                        ", res=" + S.toString(List.class, items) + ']';
+                    if (qryCur.isQuery()) {
+                        jdbcRes = new JdbcResultInfo(true, -1, qryId);
 
-                res = new JdbcQueryExecuteResult(qryId, (Long)items.get(0).get(0));
+                        JdbcQueryCursor cur = new JdbcQueryCursor(qryId, req.pageSize(), req.maxRows(),
+                            (QueryCursorImpl)qryCur);
+
+                        qryCursors.put(qryId, cur);
+
+                        qryId = QRY_ID_GEN.getAndIncrement();
+
+                        if (items == null) {
+                            items = cur.fetchRows();
+                            last = cur.hasNext();
+                        }
+                    }
+                    else
+                        jdbcRes = new JdbcResultInfo(false, (Long)((List<?>)qryCur.getAll().get(0)).get(0), -1);
+
+                    jdbcResults.add(jdbcRes);
+                }
+
+                return new JdbcResponse(new JdbcQueryExecuteMultipleStatementsResult(jdbcResults, items, last));
             }
 
-            if (res.last() && (!res.isQuery() || autoCloseCursors))
-                cur.close();
-            else
-                qryCursors.put(qryId, cur);
 
-            return new JdbcResponse(res);
         }
         catch (Exception e) {
             qryCursors.remove(qryId);
@@ -440,7 +486,7 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
                 qry.setSchema(schemaName);
 
                 QueryCursorImpl<List<?>> qryCur = (QueryCursorImpl<List<?>>)ctx.query()
-                    .querySqlFieldsNoCache(qry, true);
+                    .querySqlFieldsNoCache(qry, true, true).get(0);
 
                 assert !qryCur.isQuery();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
index 202905b..c6c7438 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
@@ -56,6 +56,9 @@ public class JdbcResult implements JdbcRawBinarylizable {
     /** Database schemas metadata result. */
     static final byte META_SCHEMAS = 12;
 
+    /** Multiple statements query results. */
+    static final byte QRY_EXEC_MULT = 13;
+
     /** Success status. */
     private byte type;
 
@@ -139,6 +142,11 @@ public class JdbcResult implements JdbcRawBinarylizable {
 
                 break;
 
+            case QRY_EXEC_MULT:
+                res = new JdbcQueryExecuteMultipleStatementsResult();
+
+                break;
+
             default:
                 throw new IgniteException("Unknown SQL listener request ID: [request ID=" + resId + ']');
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResultInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResultInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResultInfo.java
new file mode 100644
index 0000000..f0706e4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResultInfo.java
@@ -0,0 +1,95 @@
+/*
+ * 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.odbc.jdbc;
+
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC statement result information. Keeps statement type (SELECT or UPDATE) and
+ * queryId or update count (depends on statement type).
+ */
+public class JdbcResultInfo implements JdbcRawBinarylizable {
+    /** Query flag. */
+    private boolean isQuery;
+
+    /** Update count. */
+    private long updCnt;
+
+    /** Query ID. */
+    private long qryId;
+
+    /**
+     * Default constructor is used for serialization.
+     */
+    JdbcResultInfo() {
+        // No-op.
+    }
+
+    /**
+     * @param isQuery Query flag.
+     * @param updCnt Update count.
+     * @param qryId  Query ID.
+     */
+    public JdbcResultInfo(boolean isQuery, long updCnt, long qryId) {
+        this.isQuery = isQuery;
+        this.updCnt = updCnt;
+        this.qryId = qryId;
+    }
+
+    /**
+     * @return Query flag.
+     */
+    public boolean isQuery() {
+        return isQuery;
+    }
+
+    /**
+     * @return Query ID.
+     */
+    public long queryId() {
+        return qryId;
+    }
+
+    /**
+     * @return Update count.
+     */
+    public long updateCount() {
+        return updCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) {
+        writer.writeBoolean(isQuery);
+        writer.writeLong(updCnt);
+        writer.writeLong(qryId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) {
+        isQuery = reader.readBoolean();
+        updCnt = reader.readLong();
+        qryId = reader.readLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcResultInfo.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index cecc5dd..b8445ca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -83,11 +83,14 @@ public interface GridQueryIndexing {
      * @param keepBinary Keep binary flag.
      * @param cancel Query cancel.
      * @param mainCacheId Main cache ID.
+     * @param failOnMultipleStmts If {@code true} the method must throws exception when query contains
+     *      more then one SQL statement.
      * @return Cursor.
      * @throws IgniteCheckedException If failed.
      */
-    public FieldsQueryCursor<List<?>> queryDistributedSqlFields(String schemaName, SqlFieldsQuery qry,
-        boolean keepBinary, GridQueryCancel cancel, @Nullable Integer mainCacheId) throws IgniteCheckedException;
+    public List<FieldsQueryCursor<List<?>>> queryDistributedSqlFields(String schemaName, SqlFieldsQuery qry,
+        boolean keepBinary, GridQueryCancel cancel, @Nullable Integer mainCacheId, boolean failOnMultipleStmts)
+        throws IgniteCheckedException;
 
     /**
      * Perform a MERGE statement using data streamer as receiver.

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index e8cc852..58c3ce9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -1867,7 +1867,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         if (cctx.config().getQueryParallelism() > 1) {
                             qry.setDistributedJoins(true);
 
-                            cur = idx.queryDistributedSqlFields(schemaName, qry, keepBinary, cancel, mainCacheId);
+                            cur = idx.queryDistributedSqlFields(schemaName, qry,
+                                keepBinary, cancel, mainCacheId, true).get(0);
                         }
                         else {
                             IndexingQueryFilter filter = idx.backupFilter(requestTopVer.get(), qry.getPartitions());
@@ -1884,7 +1885,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             else {
                 clo = new IgniteOutClosureX<FieldsQueryCursor<List<?>>>() {
                     @Override public FieldsQueryCursor<List<?>> applyx() throws IgniteCheckedException {
-                        return idx.queryDistributedSqlFields(schemaName, qry, keepBinary, null, mainCacheId);
+                        return idx.queryDistributedSqlFields(schemaName, qry, keepBinary, null, mainCacheId, true).get(0);
                     }
                 };
             }
@@ -1906,7 +1907,20 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param keepBinary Keep binary flag.
      * @return Cursor.
      */
-    public FieldsQueryCursor<List<?>> querySqlFieldsNoCache(final SqlFieldsQuery qry, final boolean keepBinary) {
+    public FieldsQueryCursor<List<?>> querySqlFieldsNoCache(final SqlFieldsQuery qry,
+        final boolean keepBinary) {
+        return querySqlFieldsNoCache(qry, keepBinary, true).get(0);
+    }
+
+    /**
+     * Query SQL fields without strict dependency on concrete cache.
+     *
+     * @param qry Query.
+     * @param keepBinary Keep binary flag.
+     * @return Cursor.
+     */
+    public List<FieldsQueryCursor<List<?>>> querySqlFieldsNoCache(final SqlFieldsQuery qry,
+        final boolean keepBinary, final boolean failOnMultipleStmts) {
         checkxEnabled();
 
         validateSqlFieldsQuery(qry);
@@ -1921,11 +1935,13 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            IgniteOutClosureX<FieldsQueryCursor<List<?>>> clo = new IgniteOutClosureX<FieldsQueryCursor<List<?>>>() {
-                @Override public FieldsQueryCursor<List<?>> applyx() throws IgniteCheckedException {
+            IgniteOutClosureX<List<FieldsQueryCursor<List<?>>>> clo =
+                new IgniteOutClosureX<List<FieldsQueryCursor<List<?>>>>() {
+                @Override public List<FieldsQueryCursor<List<?>>> applyx() throws IgniteCheckedException {
                     GridQueryCancel cancel = new GridQueryCancel();
 
-                    return idx.queryDistributedSqlFields(qry.getSchema(), qry, keepBinary, cancel, null);
+                    return idx.queryDistributedSqlFields(qry.getSchema(), qry, keepBinary, cancel, null,
+                        failOnMultipleStmts);
                 }
             };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
index c745d8a..1ebf556 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
@@ -242,12 +242,12 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT
         }
 
         /** {@inheritDoc} */
-        @Override public FieldsQueryCursor<List<?>> queryDistributedSqlFields(String schemaName, SqlFieldsQuery qry,
-            boolean keepBinary, GridQueryCancel cancel, @Nullable Integer mainCacheId) throws IgniteCheckedException {
+        @Override public List<FieldsQueryCursor<List<?>>> queryDistributedSqlFields(String schemaName, SqlFieldsQuery qry,
+            boolean keepBinary, GridQueryCancel cancel,
+            @Nullable Integer mainCacheId, boolean failOnMultipleStmts) throws IgniteCheckedException {
             return null;
         }
 
-
         /** {@inheritDoc} */
         @Override public long streamUpdateQuery(String spaceName, String qry, @Nullable Object[] params,
             IgniteDataStreamer<?, ?> streamer) throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index f2f2fd4..ee1875f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -150,7 +150,7 @@ public class DmlStatementsProcessor {
      * Execute DML statement, possibly with few re-attempts in case of concurrent data modifications.
      *
      * @param schemaName Schema.
-     * @param stmt JDBC statement.
+     * @param prepared Prepared JDBC statement.
      * @param fieldsQry Original query.
      * @param loc Query locality flag.
      * @param filters Cache name and key filter.
@@ -158,13 +158,13 @@ public class DmlStatementsProcessor {
      * @return Update result (modified items count and failed keys).
      * @throws IgniteCheckedException if failed.
      */
-    private UpdateResult updateSqlFields(String schemaName, PreparedStatement stmt, SqlFieldsQuery fieldsQry,
+    private UpdateResult updateSqlFields(String schemaName, Prepared prepared, SqlFieldsQuery fieldsQry,
         boolean loc, IndexingQueryFilter filters, GridQueryCancel cancel) throws IgniteCheckedException {
         Object[] errKeys = null;
 
         long items = 0;
 
-        UpdatePlan plan = getPlanForStatement(schemaName, stmt, null);
+        UpdatePlan plan = getPlanForStatement(schemaName, prepared, null);
 
         GridCacheContext<?, ?> cctx = plan.tbl.rowDescriptor().context();
 
@@ -188,7 +188,7 @@ public class DmlStatementsProcessor {
             UpdateResult r;
 
             try {
-                r = executeUpdateStatement(schemaName, cctx, stmt, fieldsQry, loc, filters, cancel, errKeys);
+                r = executeUpdateStatement(schemaName, cctx, prepared, fieldsQry, loc, filters, cancel, errKeys);
             }
             finally {
                 cctx.operationContextPerCall(opCtx);
@@ -213,16 +213,16 @@ public class DmlStatementsProcessor {
 
     /**
      * @param schemaName Schema.
-     * @param stmt Prepared statement.
+     * @param p Prepared.
      * @param fieldsQry Initial query
      * @param cancel Query cancel.
      * @return Update result wrapped into {@link GridQueryFieldsResult}
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings("unchecked")
-    QueryCursorImpl<List<?>> updateSqlFieldsDistributed(String schemaName, PreparedStatement stmt,
+    QueryCursorImpl<List<?>> updateSqlFieldsDistributed(String schemaName, Prepared p,
         SqlFieldsQuery fieldsQry, GridQueryCancel cancel) throws IgniteCheckedException {
-        UpdateResult res = updateSqlFields(schemaName, stmt, fieldsQry, false, null, cancel);
+        UpdateResult res = updateSqlFields(schemaName, p, fieldsQry, false, null, cancel);
 
         QueryCursorImpl<List<?>> resCur = (QueryCursorImpl<List<?>>)new QueryCursorImpl(Collections.singletonList
             (Collections.singletonList(res.cnt)), cancel, false);
@@ -247,7 +247,8 @@ public class DmlStatementsProcessor {
     GridQueryFieldsResult updateSqlFieldsLocal(String schemaName, PreparedStatement stmt,
         SqlFieldsQuery fieldsQry, IndexingQueryFilter filters, GridQueryCancel cancel)
         throws IgniteCheckedException {
-        UpdateResult res = updateSqlFields(schemaName, stmt, fieldsQry, true, filters, cancel);
+        UpdateResult res = updateSqlFields(schemaName,  GridSqlQueryParser.prepared(stmt), fieldsQry, true,
+            filters, cancel);
 
         return new GridQueryFieldsResultAdapter(UPDATE_RESULT_META,
             new IgniteSingletonIterator(Collections.singletonList(res.cnt)));
@@ -340,22 +341,24 @@ public class DmlStatementsProcessor {
      *
      * @param schemaName Schema name.
      * @param cctx Cache context.
-     * @param prepStmt Prepared statement for DML query.
+     * @param prepared Prepared statement for DML query.
      * @param fieldsQry Fields query.
+     * @param loc Local query flag.
      * @param filters Cache name and key filter.
+     * @param cancel Query cancel state holder.
      * @param failedKeys Keys to restrict UPDATE and DELETE operations with. Null or empty array means no restriction.
      * @return Pair [number of successfully processed items; keys that have failed to be processed]
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings({"ConstantConditions", "unchecked"})
     private UpdateResult executeUpdateStatement(String schemaName, final GridCacheContext cctx,
-        PreparedStatement prepStmt, SqlFieldsQuery fieldsQry, boolean loc, IndexingQueryFilter filters,
+        Prepared prepared, SqlFieldsQuery fieldsQry, boolean loc, IndexingQueryFilter filters,
         GridQueryCancel cancel, Object[] failedKeys) throws IgniteCheckedException {
         int mainCacheId = CU.cacheId(cctx.name());
 
         Integer errKeysPos = null;
 
-        UpdatePlan plan = getPlanForStatement(schemaName, prepStmt, errKeysPos);
+        UpdatePlan plan = getPlanForStatement(schemaName, prepared, errKeysPos);
 
         if (plan.fastUpdateArgs != null) {
             assert F.isEmpty(failedKeys) && errKeysPos == null;
@@ -378,8 +381,8 @@ public class DmlStatementsProcessor {
                 .setPageSize(fieldsQry.getPageSize())
                 .setTimeout(fieldsQry.getTimeout(), TimeUnit.MILLISECONDS);
 
-            cur = (QueryCursorImpl<List<?>>) idx.queryDistributedSqlFields(schemaName, newFieldsQry, true, cancel,
-                mainCacheId);
+            cur = (QueryCursorImpl<List<?>>)idx.queryDistributedSqlFields(schemaName, newFieldsQry, true,
+                cancel, mainCacheId, true).get(0);
         }
         else {
             final GridQueryFieldsResult res = idx.queryLocalSqlFields(schemaName, plan.selectQry,
@@ -423,14 +426,12 @@ public class DmlStatementsProcessor {
      * if available.
      *
      * @param schema Schema.
-     * @param prepStmt JDBC statement.
+     * @param p Prepared JDBC statement.
      * @return Update plan.
      */
     @SuppressWarnings({"unchecked", "ConstantConditions"})
-    private UpdatePlan getPlanForStatement(String schema, PreparedStatement prepStmt, @Nullable Integer errKeysPos)
+    private UpdatePlan getPlanForStatement(String schema, Prepared p, @Nullable Integer errKeysPos)
         throws IgniteCheckedException {
-        Prepared p = GridSqlQueryParser.prepared(prepStmt);
-
         H2DmlPlanKey planKey = new H2DmlPlanKey(schema, p.getSQL());
 
         UpdatePlan res = (errKeysPos == null ? planCache.get(planKey) : null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 00a37ce..9e6a1fa 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -145,7 +145,6 @@ import org.h2.command.dml.Insert;
 import org.h2.engine.Session;
 import org.h2.engine.SysProperties;
 import org.h2.index.Index;
-import org.h2.jdbc.JdbcPreparedStatement;
 import org.h2.jdbc.JdbcStatement;
 import org.h2.server.web.WebServer;
 import org.h2.table.IndexColumn;
@@ -1244,7 +1243,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (qry.getTimeout() > 0)
             fqry.setTimeout(qry.getTimeout(), TimeUnit.MILLISECONDS);
 
-        final QueryCursor<List<?>> res = queryDistributedSqlFields(schemaName, fqry, keepBinary, null, mainCacheId);
+        final QueryCursor<List<?>> res =
+            queryDistributedSqlFields(schemaName, fqry, keepBinary, null, mainCacheId, true).get(0);
 
         final Iterable<Cache.Entry<K, V>> converted = new Iterable<Cache.Entry<K, V>>() {
             @Override public Iterator<Cache.Entry<K, V>> iterator() {
@@ -1277,10 +1277,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public FieldsQueryCursor<List<?>> queryDistributedSqlFields(String schemaName,
-        SqlFieldsQuery qry, boolean keepBinary, GridQueryCancel cancel, @Nullable Integer mainCacheId) {
-        final String sqlQry = qry.getSql();
-
+    @Override public List<FieldsQueryCursor<List<?>>> queryDistributedSqlFields(String schemaName, SqlFieldsQuery qry,
+        boolean keepBinary, GridQueryCancel cancel, @Nullable Integer mainCacheId, boolean failOnMultipleStmts) {
         Connection c = connectionForSchema(schemaName);
 
         final boolean enforceJoinOrder = qry.isEnforceJoinOrder();
@@ -1289,18 +1287,38 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         final DistributedJoinMode distributedJoinMode = distributedJoinMode(qry.isLocal(), distributedJoins);
 
-        GridCacheTwoStepQuery twoStepQry = null;
-        List<GridQueryFieldMetadata> meta;
+        String sqlQry = qry.getSql();
 
-        final H2TwoStepCachedQueryKey cachedQryKey = new H2TwoStepCachedQueryKey(schemaName, sqlQry, grpByCollocated,
+        H2TwoStepCachedQueryKey cachedQryKey = new H2TwoStepCachedQueryKey(schemaName, sqlQry, grpByCollocated,
             distributedJoins, enforceJoinOrder, qry.isLocal());
         H2TwoStepCachedQuery cachedQry = twoStepCache.get(cachedQryKey);
 
         if (cachedQry != null) {
-            twoStepQry = cachedQry.query().copy();
-            meta = cachedQry.meta();
+            checkQueryType(qry, true);
+
+            GridCacheTwoStepQuery twoStepQry = cachedQry.query().copy();
+
+            List<GridQueryFieldMetadata> meta = cachedQry.meta();
+
+            List<FieldsQueryCursor<List<?>>> res = Collections.singletonList(executeTwoStepsQuery(schemaName, qry.getPageSize(), qry.getPartitions(),
+                qry.getArgs(), keepBinary, qry.isLazy(), qry.getTimeout(), cancel, sqlQry, enforceJoinOrder,
+                twoStepQry, meta));
+
+            return res;
         }
-        else {
+
+        List<FieldsQueryCursor<List<?>>> res = new ArrayList<>(1);
+
+        Object[] argsOrig = qry.getArgs();
+        int firstArg = 0;
+        Object[] args;
+        String remainingSql = sqlQry;
+
+        while (remainingSql != null) {
+            args = null;
+            GridCacheTwoStepQuery twoStepQry = null;
+            List<GridQueryFieldMetadata> meta;
+
             final UUID locNodeId = ctx.localNodeId();
 
             // Here we will just parse the statement, no need to optimize it at all.
@@ -1319,16 +1337,16 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                     while (true) {
                         try {
                             // Do not cache this statement because the whole query object will be cached later on.
-                            stmt = prepareStatement(c, sqlQry, false);
+                            stmt = prepareStatement(c, remainingSql, false);
 
                             break;
                         }
                         catch (SQLException e) {
                             if (!cachesCreated && (
                                 e.getErrorCode() == ErrorCode.SCHEMA_NOT_FOUND_1 ||
-                                e.getErrorCode() == ErrorCode.TABLE_OR_VIEW_NOT_FOUND_1 ||
-                                e.getErrorCode() == ErrorCode.INDEX_NOT_FOUND_1)
-                            ) {
+                                    e.getErrorCode() == ErrorCode.TABLE_OR_VIEW_NOT_FOUND_1 ||
+                                    e.getErrorCode() == ErrorCode.INDEX_NOT_FOUND_1)
+                                ) {
                                 try {
                                     ctx.cache().createMissingQueryCaches();
                                 }
@@ -1344,19 +1362,59 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                         }
                     }
 
-                    prepared = GridSqlQueryParser.prepared(stmt);
+                    GridSqlQueryParser.PreparedWithRemaining prep = GridSqlQueryParser.preparedWithRemaining(stmt);
+
+                    // remaining == null if the query string contains single SQL statement.
+                    remainingSql = prep.remainingSql();
+
+                    if (remainingSql != null && failOnMultipleStmts)
+                        throw new IgniteSQLException("Multiple statements queries are not supported");
+
+                    sqlQry = prep.prepared().getSQL();
+
+                    prepared = prep.prepared();
+
+                    int paramsCnt = prepared.getParameters().size();
+
+                    if (paramsCnt > 0) {
+                        if (argsOrig == null || argsOrig.length < firstArg + paramsCnt) {
+                            throw new IgniteException("Invalid number of query parameters. " +
+                                "Cannot find " + (argsOrig.length + 1 - firstArg) + " parameter.");
+                        }
+
+                        args = Arrays.copyOfRange(argsOrig, firstArg, firstArg + paramsCnt);
+
+                        firstArg += paramsCnt;
+                    }
+
+                    cachedQryKey = new H2TwoStepCachedQueryKey(schemaName, sqlQry, grpByCollocated,
+                        distributedJoins, enforceJoinOrder, qry.isLocal());
+
+                    cachedQry = twoStepCache.get(cachedQryKey);
+
+                    if (cachedQry != null) {
+                        checkQueryType(qry, true);
+
+                        twoStepQry = cachedQry.query().copy();
+                        meta = cachedQry.meta();
 
-                    if (qry instanceof JdbcSqlFieldsQuery && ((JdbcSqlFieldsQuery) qry).isQuery() != prepared.isQuery())
-                        throw new IgniteSQLException("Given statement type does not match that declared by JDBC driver",
-                            IgniteQueryErrorCode.STMT_TYPE_MISMATCH);
+                        res.add(executeTwoStepsQuery(schemaName, qry.getPageSize(), qry.getPartitions(), args, keepBinary,
+                            qry.isLazy(), qry.getTimeout(), cancel, sqlQry, enforceJoinOrder,
+                            twoStepQry, meta));
 
-                    if (prepared.isQuery()) {
-                        bindParameters(stmt, F.asList(qry.getArgs()));
+                        continue;
+                    }
+                    else {
+                        checkQueryType(qry, prepared.isQuery());
+
+                        if (prepared.isQuery()) {
+                            bindParameters(stmt, F.asList(args));
 
-                        twoStepQry = GridSqlQuerySplitter.split((JdbcPreparedStatement)stmt, qry.getArgs(),
-                            grpByCollocated, distributedJoins, enforceJoinOrder, this);
+                            twoStepQry = GridSqlQuerySplitter.split(c, prepared, args,
+                                grpByCollocated, distributedJoins, enforceJoinOrder, this);
 
-                        assert twoStepQry != null;
+                            assert twoStepQry != null;
+                        }
                     }
                 }
                 finally {
@@ -1367,17 +1425,22 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 if (twoStepQry == null) {
                     if (DmlStatementsProcessor.isDmlStatement(prepared)) {
                         try {
-                            return dmlProc.updateSqlFieldsDistributed(schemaName, stmt, qry, cancel);
+                            res.add(dmlProc.updateSqlFieldsDistributed(schemaName, prepared,
+                                new SqlFieldsQuery(qry).setSql(sqlQry).setArgs(args), cancel));
+
+                            continue;
                         }
                         catch (IgniteCheckedException e) {
                             throw new IgniteSQLException("Failed to execute DML statement [stmt=" + sqlQry +
-                                ", params=" + Arrays.deepToString(qry.getArgs()) + "]", e);
+                                ", params=" + Arrays.deepToString(args) + "]", e);
                         }
                     }
 
                     if (DdlStatementsProcessor.isDdlStatement(prepared)) {
                         try {
-                            return ddlProc.runDdlStatement(sqlQry, stmt);
+                            res.add(ddlProc.runDdlStatement(sqlQry, prepared));
+
+                            continue;
                         }
                         catch (IgniteCheckedException e) {
                             throw new IgniteSQLException("Failed to execute DDL statement [stmt=" + sqlQry + ']', e);
@@ -1428,39 +1491,75 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             finally {
                 U.close(stmt, log);
             }
+
+            res.add(executeTwoStepsQuery(schemaName, qry.getPageSize(), qry.getPartitions(), args, keepBinary,
+                qry.isLazy(), qry.getTimeout(), cancel, sqlQry, enforceJoinOrder,
+                twoStepQry, meta));
+
+            if (cachedQry == null && !twoStepQry.explain()) {
+                cachedQry = new H2TwoStepCachedQuery(meta, twoStepQry.copy());
+
+                twoStepCache.putIfAbsent(cachedQryKey, cachedQry);
+            }
         }
 
+        return res;
+    }
+
+    /**
+     * Check expected statement type (when it is set by JDBC) and given statement type.
+     *
+     * @param qry Query.
+     * @param isQry {@code true} for select queries, otherwise (DML/DDL queries) {@code false}.
+     */
+    private void checkQueryType(SqlFieldsQuery qry, boolean isQry) {
+        if (qry instanceof JdbcSqlFieldsQuery && ((JdbcSqlFieldsQuery)qry).isQuery() != isQry)
+            throw new IgniteSQLException("Given statement type does not match that declared by JDBC driver",
+                IgniteQueryErrorCode.STMT_TYPE_MISMATCH);
+    }
+
+    /**
+     * @param schemaName Schema name.
+     * @param pageSize Page size.
+     * @param partitions Partitions.
+     * @param args Arguments.
+     * @param keepBinary Keep binary flag.
+     * @param lazy Lazy flag.
+     * @param timeout Timeout.
+     * @param cancel Cancel.
+     * @param sqlQry SQL query string.
+     * @param enforceJoinOrder Enforce join orded flag.
+     * @param twoStepQry Two-steps query.
+     * @param meta Metadata.
+     * @return Cursor.
+     */
+    private FieldsQueryCursor<List<?>> executeTwoStepsQuery(String schemaName, int pageSize, int partitions[],
+        Object[] args, boolean keepBinary, boolean lazy, int timeout,
+        GridQueryCancel cancel, String sqlQry, boolean enforceJoinOrder, GridCacheTwoStepQuery twoStepQry,
+        List<GridQueryFieldMetadata> meta) {
         if (log.isDebugEnabled())
             log.debug("Parsed query: `" + sqlQry + "` into two step query: " + twoStepQry);
 
-        twoStepQry.pageSize(qry.getPageSize());
+        twoStepQry.pageSize(pageSize);
 
         if (cancel == null)
             cancel = new GridQueryCancel();
 
-        int partitions[] = qry.getPartitions();
-
         if (partitions == null && twoStepQry.derivedPartitions() != null) {
             try {
-                partitions = calculateQueryPartitions(twoStepQry.derivedPartitions(), qry.getArgs());
+                partitions = calculateQueryPartitions(twoStepQry.derivedPartitions(), args);
             } catch (IgniteCheckedException e) {
                 throw new CacheException("Failed to calculate derived partitions: [qry=" + sqlQry + ", params=" +
-                    Arrays.deepToString(qry.getArgs()) + "]", e);
+                    Arrays.deepToString(args) + "]", e);
             }
         }
 
         QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(
-            runQueryTwoStep(schemaName, twoStepQry, keepBinary, enforceJoinOrder, qry.getTimeout(), cancel,
-                qry.getArgs(), partitions, qry.isLazy()), cancel);
+            runQueryTwoStep(schemaName, twoStepQry, keepBinary, enforceJoinOrder, timeout, cancel,
+                args, partitions, lazy), cancel);
 
         cursor.fieldsMeta(meta);
 
-        if (cachedQry == null && !twoStepQry.explain()) {
-            cachedQry = new H2TwoStepCachedQuery(meta, twoStepQry.copy());
-
-            twoStepCache.putIfAbsent(cachedQryKey, cachedQry);
-        }
-
         return cursor;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
index 4c3264c..8a901dc 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.query.h2.ddl;
 
-import java.sql.PreparedStatement;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
@@ -59,7 +58,6 @@ import org.h2.command.ddl.CreateIndex;
 import org.h2.command.ddl.CreateTable;
 import org.h2.command.ddl.DropIndex;
 import org.h2.command.ddl.DropTable;
-import org.h2.jdbc.JdbcPreparedStatement;
 import org.h2.table.Column;
 import org.h2.value.DataType;
 
@@ -91,17 +89,18 @@ public class DdlStatementsProcessor {
      * Execute DDL statement.
      *
      * @param sql SQL.
-     * @param stmt H2 statement to parse and execute.
+     * @param prepared Prepared.
+     * @return Cursor on query results.
+     * @throws IgniteCheckedException On error.
      */
     @SuppressWarnings({"unchecked", "ThrowableResultOfMethodCallIgnored"})
-    public FieldsQueryCursor<List<?>> runDdlStatement(String sql, PreparedStatement stmt)
+    public FieldsQueryCursor<List<?>> runDdlStatement(String sql, Prepared prepared)
         throws IgniteCheckedException {
-        assert stmt instanceof JdbcPreparedStatement;
 
         IgniteInternalFuture fut = null;
 
         try {
-            GridSqlStatement stmt0 = new GridSqlQueryParser(false).parse(GridSqlQueryParser.prepared(stmt));
+            GridSqlStatement stmt0 = new GridSqlQueryParser(false).parse(prepared);
 
             if (stmt0 instanceof GridSqlCreateIndex) {
                 GridSqlCreateIndex cmd = (GridSqlCreateIndex)stmt0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
index a379a91..3d7a1a0 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
@@ -397,6 +397,29 @@ public class GridSqlQueryParser {
     /** */
     private static final Getter<Column, Expression> COLUMN_CHECK_CONSTRAINT = getter(Column.class, "checkConstraint");
 
+    /** Class for private class: 'org.h2.command.CommandList'. */
+    private static final Class<? extends Command> CLS_COMMAND_LIST;
+
+    /** */
+    private static final Getter<Command, Command> LIST_COMMAND;
+
+    /** */
+    private static final Getter<Command, String> REMAINING;
+
+    static {
+        try {
+            CLS_COMMAND_LIST = (Class<? extends Command>)CommandContainer.class.getClassLoader()
+                .loadClass("org.h2.command.CommandList");
+
+            LIST_COMMAND = getter(CLS_COMMAND_LIST, "command");
+
+            REMAINING = getter(CLS_COMMAND_LIST, "remaining");
+        }
+        catch (ClassNotFoundException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
     /** */
     private static final Getter<AlterTableAlterColumn, String> ALTER_COLUMN_TBL_NAME =
         getter(AlterTableAlterColumn.class, "tableName");
@@ -485,6 +508,26 @@ public class GridSqlQueryParser {
     }
 
     /**
+     * @param stmt Prepared statement.
+     * @return Parsed select.
+     */
+    public static PreparedWithRemaining preparedWithRemaining(PreparedStatement stmt) {
+        Command cmd = COMMAND.get((JdbcPreparedStatement)stmt);
+
+        if (cmd instanceof CommandContainer)
+            return new PreparedWithRemaining(PREPARED.get(cmd), null);
+        else {
+            Class<?> cmdCls = cmd.getClass();
+
+            if (cmdCls.getName().equals("org.h2.command.CommandList")) {
+                return new PreparedWithRemaining(PREPARED.get(LIST_COMMAND.get(cmd)), REMAINING.get(cmd));
+            }
+            else
+                throw new IgniteSQLException("Unexpected statement command");
+        }
+    }
+
+    /**
      * @param qry Query expression to parse.
      * @return Subquery AST.
      */
@@ -1798,4 +1841,42 @@ public class GridSqlQueryParser {
             }
         }
     }
+
+    /**
+     *
+     */
+    public static class PreparedWithRemaining {
+        /** Prepared. */
+        private Prepared prepared;
+
+        /** Remaining sql. */
+        private String remainingSql;
+
+        /**
+         * @param prepared Prepared.
+         * @param sql Remaining SQL.
+         */
+        public PreparedWithRemaining(Prepared prepared, String sql) {
+            this.prepared = prepared;
+
+            if (sql != null)
+                sql = sql.trim();
+
+            remainingSql = !F.isEmpty(sql) ? sql : null;
+        }
+
+        /**
+         * @return Prepared.
+         */
+        public Prepared prepared() {
+            return prepared;
+        }
+
+        /**
+         * @return Remaining SQL.
+         */
+        public String remainingSql() {
+            return remainingSql;
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index b20cbd5..7f28203 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -51,7 +51,6 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.h2.command.Prepared;
 import org.h2.command.dml.Query;
 import org.h2.command.dml.SelectUnion;
-import org.h2.jdbc.JdbcPreparedStatement;
 import org.h2.table.IndexColumn;
 import org.h2.value.Value;
 import org.jetbrains.annotations.Nullable;
@@ -172,7 +171,8 @@ public class GridSqlQuerySplitter {
     }
 
     /**
-     * @param stmt Prepared statement.
+     * @param conn Connection.
+     * @param prepared Prepared.
      * @param params Parameters.
      * @param collocatedGrpBy Whether the query has collocated GROUP BY keys.
      * @param distributedJoins If distributed joins enabled.
@@ -183,7 +183,8 @@ public class GridSqlQuerySplitter {
      * @throws IgniteCheckedException If failed.
      */
     public static GridCacheTwoStepQuery split(
-        JdbcPreparedStatement stmt,
+        Connection conn,
+        Prepared prepared,
         Object[] params,
         boolean collocatedGrpBy,
         boolean distributedJoins,
@@ -195,7 +196,7 @@ public class GridSqlQuerySplitter {
 
         // Here we will just do initial query parsing. Do not use optimized
         // subqueries because we do not have unique FROM aliases yet.
-        GridSqlQuery qry = parse(prepared(stmt), false);
+        GridSqlQuery qry = parse(prepared, false);
 
         String originalSql = qry.getSQL();
 
@@ -213,8 +214,6 @@ public class GridSqlQuerySplitter {
 
 //        debug("NORMALIZED", qry.getSQL());
 
-        Connection conn = stmt.getConnection();
-
         // Here we will have correct normalized AST with optimized join order.
         // The distributedJoins parameter is ignored because it is not relevant for
         // the REDUCE query optimization.
@@ -234,12 +233,12 @@ public class GridSqlQuerySplitter {
             boolean allCollocated = true;
 
             for (GridCacheSqlQuery mapSqlQry : splitter.mapSqlQrys) {
-                Prepared prepared = optimize(h2, conn, mapSqlQry.query(), mapSqlQry.parameters(params),
+                Prepared prepared0 = optimize(h2, conn, mapSqlQry.query(), mapSqlQry.parameters(params),
                     true, enforceJoinOrder);
 
-                allCollocated &= isCollocated((Query)prepared);
+                allCollocated &= isCollocated((Query)prepared0);
 
-                mapSqlQry.query(parse(prepared, true).getSQL());
+                mapSqlQry.query(parse(prepared0, true).getSQL());
             }
 
             // We do not need distributed joins if all MAP queries are collocated.

http://git-wip-us.apache.org/repos/asf/ignite/blob/405749a7/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/MultipleStatementsSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/MultipleStatementsSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/MultipleStatementsSqlQuerySelfTest.java
new file mode 100644
index 0000000..8b9bf40
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/MultipleStatementsSqlQuerySelfTest.java
@@ -0,0 +1,154 @@
+/*
+ * 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.query;
+
+import java.util.List;
+import java.util.concurrent.Callable;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Tests for schemas.
+ */
+public class MultipleStatementsSqlQuerySelfTest extends GridCommonAbstractTest {
+    /** Node. */
+    private IgniteEx node;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        node = (IgniteEx)startGrid();
+
+        startGrid(2);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * Test query without caches.
+     *
+     * @throws Exception If failed.
+     */
+    public void testQuery() throws Exception {
+        GridQueryProcessor qryProc = node.context().query();
+
+        SqlFieldsQuery qry = new SqlFieldsQuery(
+            "create table test(ID int primary key, NAME varchar(20)); " +
+                "insert into test (ID, NAME) values (1, 'name_1');" +
+                "insert into test (ID, NAME) values (2, 'name_2'), (3, 'name_3');" +
+                "select * from test;")
+            .setSchema("PUBLIC");
+
+        List<FieldsQueryCursor<List<?>>> res = qryProc.querySqlFieldsNoCache(qry, true, false);
+
+        assert res.size() == 4 : "Unexpected cursors count: " + res.size();
+
+        assert !((QueryCursorImpl)res.get(0)).isQuery() : "Results of DDL statement is expected ";
+
+        List<List<?>> rows = res.get(1).getAll();
+
+        assert !((QueryCursorImpl)res.get(1)).isQuery() : "Results of DDL statement is expected ";
+        assert Long.valueOf(1).equals(rows.get(0).get(0)) : "1 row must be updated. [actual=" + rows.get(0).get(0) + ']';
+
+        rows = res.get(2).getAll();
+
+        assert !((QueryCursorImpl)res.get(2)).isQuery() : "Results of DML statement is expected ";
+        assert Long.valueOf(2).equals(rows.get(0).get(0)) : "2 row must be updated";
+
+        rows = res.get(3).getAll();
+
+        assert ((QueryCursorImpl)res.get(3)).isQuery() : "Results of SELECT statement is expected ";
+
+        assert rows.size() == 3 : "Invalid rows count: " + rows.size();
+
+        for (int i = 0; i < rows.size(); ++i) {
+            assert Integer.valueOf(1).equals(rows.get(i).get(0))
+                || Integer.valueOf(2).equals(rows.get(i).get(0))
+                || Integer.valueOf(3).equals(rows.get(i).get(0))
+                : "Invalid ID: " + rows.get(i).get(0);
+        }
+    }
+
+    /**
+     * Test query without caches.
+     *
+     * @throws Exception If failed.
+     */
+    public void testQueryWithParameters() throws Exception {
+        GridQueryProcessor qryProc = node.context().query();
+
+        SqlFieldsQuery qry = new SqlFieldsQuery(
+            "create table test(ID int primary key, NAME varchar(20)); " +
+                "insert into test (ID, NAME) values (?, ?);" +
+                "insert into test (ID, NAME) values (?, ?), (?, ?);" +
+                "select * from test;")
+            .setSchema("PUBLIC")
+            .setArgs(1, "name_1", 2, "name2", 3, "name_3");
+
+        List<FieldsQueryCursor<List<?>>> res = qryProc.querySqlFieldsNoCache(qry, true, false);
+
+        assert res.size() == 4 : "Unexpected cursors count: " + res.size();
+
+        assert !((QueryCursorImpl)res.get(0)).isQuery() : "Results of DDL statement is expected ";
+
+        List<List<?>> rows = res.get(1).getAll();
+
+        assert !((QueryCursorImpl)res.get(1)).isQuery() : "Results of DDL statement is expected ";
+        assert Long.valueOf(1).equals(rows.get(0).get(0)) : "1 row must be updated. [actual=" + rows.get(0).get(0) + ']';
+
+        rows = res.get(2).getAll();
+
+        assert !((QueryCursorImpl)res.get(2)).isQuery() : "Results of DML statement is expected ";
+        assert Long.valueOf(2).equals(rows.get(0).get(0)) : "2 row must be updated";
+
+        rows = res.get(3).getAll();
+
+        assert ((QueryCursorImpl)res.get(3)).isQuery() : "Results of SELECT statement is expected ";
+
+        assert rows.size() == 3 : "Invalid rows count: " + rows.size();
+
+        for (int i = 0; i < rows.size(); ++i) {
+            assert Integer.valueOf(1).equals(rows.get(i).get(0))
+                || Integer.valueOf(2).equals(rows.get(i).get(0))
+                || Integer.valueOf(3).equals(rows.get(i).get(0))
+                : "Invalid ID: " + rows.get(i).get(0);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueryMultipleStatementsFailed() throws Exception {
+        final SqlFieldsQuery qry = new SqlFieldsQuery("select 1; select 1;").setSchema("PUBLIC");
+
+        GridTestUtils.assertThrows(log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    node.context().query().querySqlFieldsNoCache(qry, true);
+
+                    return null;
+                }
+            }, IgniteSQLException.class, "Multiple statements queries are not supported");
+    }
+}
\ No newline at end of file


[14/37] ignite git commit: IGNITE-6523 .NET: QueryField.NotNull

Posted by vo...@apache.org.
IGNITE-6523 .NET: QueryField.NotNull


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

Branch: refs/heads/ignite-2.3
Commit: 8f599c57104513aa8ba940cca10f680467e12cb3
Parents: b8c4863
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Sep 29 15:22:36 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Sep 29 15:22:36 2017 +0300

----------------------------------------------------------------------
 .../utils/PlatformConfigurationUtils.java       |  9 +++++
 .../Cache/CacheConfigurationTest.cs             |  3 +-
 .../Cache/Query/CacheDmlQueriesTest.cs          | 40 +++++++++++++++++++-
 .../Query/CacheQueriesCodeConfigurationTest.cs  |  5 ++-
 .../Cache/Query/CacheQueriesTest.cs             | 11 +++++-
 .../Config/cache-query.xml                      |  5 +++
 .../Config/full-config.xml                      |  2 +-
 .../IgniteConfigurationSerializerTest.cs        |  7 +++-
 .../Cache/Configuration/QueryEntity.cs          | 17 +++++----
 .../Cache/Configuration/QueryField.cs           | 32 ++++++++++++++++
 .../Configuration/QuerySqlFieldAttribute.cs     |  5 +++
 .../IgniteConfigurationSection.xsd              |  5 +++
 12 files changed, 126 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8f599c57/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 e223193..66160fb 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
@@ -459,6 +459,7 @@ public class PlatformConfigurationUtils {
         // Fields
         int cnt = in.readInt();
         Set<String> keyFields = new HashSet<>(cnt);
+        Set<String> notNullFields = new HashSet<>(cnt);
 
         if (cnt > 0) {
             LinkedHashMap<String, String> fields = new LinkedHashMap<>(cnt);
@@ -471,12 +472,18 @@ public class PlatformConfigurationUtils {
 
                 if (in.readBoolean())
                     keyFields.add(fieldName);
+
+                if (in.readBoolean())
+                    notNullFields.add(fieldName);
             }
 
             res.setFields(fields);
 
             if (!keyFields.isEmpty())
                 res.setKeyFields(keyFields);
+
+            if (!notNullFields.isEmpty())
+                res.setNotNullFields(notNullFields);
         }
 
         // Aliases
@@ -937,6 +944,7 @@ public class PlatformConfigurationUtils {
 
         if (fields != null) {
             Set<String> keyFields = queryEntity.getKeyFields();
+            Set<String> notNullFields = queryEntity.getNotNullFields();
 
             writer.writeInt(fields.size());
 
@@ -944,6 +952,7 @@ public class PlatformConfigurationUtils {
                 writer.writeString(field.getKey());
                 writer.writeString(field.getValue());
                 writer.writeBoolean(keyFields != null && keyFields.contains(field.getKey()));
+                writer.writeBoolean(notNullFields != null && notNullFields.contains(field.getKey()));
             }
         }
         else

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f599c57/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
index 4e5d443..abf8af0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
@@ -511,6 +511,7 @@ namespace Apache.Ignite.Core.Tests.Cache
             Assert.AreEqual(x.Name, y.Name);
             Assert.AreEqual(x.FieldTypeName, y.FieldTypeName);
             Assert.AreEqual(x.IsKeyField, y.IsKeyField);
+            Assert.AreEqual(x.NotNull, y.NotNull);
         }
 
         /// <summary>
@@ -587,7 +588,7 @@ namespace Apache.Ignite.Core.Tests.Cache
                         {
                             new QueryField("length", typeof(int)), 
                             new QueryField("name", typeof(string)) {IsKeyField = true},
-                            new QueryField("location", typeof(string)),
+                            new QueryField("location", typeof(string)) {NotNull = true},
                         },
                         Aliases = new [] {new QueryAlias("length", "len") },
                         Indexes = new[]

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f599c57/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs
index 9d16799..a6ddc8c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs
@@ -103,6 +103,44 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         }
 
         /// <summary>
+        /// Tests the NotNull constraint.
+        /// </summary>
+        [Test]
+        public void TestNotNull()
+        {
+            var cfg = new CacheConfiguration("not_null", new QueryEntity(typeof(int), typeof(Foo))
+            {
+                Fields = new[]
+                {
+                    new QueryField("id", typeof(int)) {NotNull = true}, 
+                    new QueryField("name", typeof(string)) 
+                }
+            });
+
+            var cache = Ignition.GetIgnite().CreateCache<int, Foo>(cfg);
+
+            var ex = Assert.Throws<IgniteException>(() => cache.QueryFields(new SqlFieldsQuery(
+                "insert into foo(_key, name) values (?, ?)", 1, "bar")).GetAll());
+
+            Assert.AreEqual("Null value is not allowed for field 'ID'", ex.Message);
+        }
+
+        /// <summary>
+        /// Tests the NotNull constraint.
+        /// </summary>
+        [Test]
+        public void TestNotNullAttribute()
+        {
+            var cfg = new CacheConfiguration("not_null_attr", new QueryEntity(typeof(int), typeof(Foo)));
+            var cache = Ignition.GetIgnite().CreateCache<int, Foo>(cfg);
+
+            var ex = Assert.Throws<IgniteException>(() => cache.QueryFields(new SqlFieldsQuery(
+                "insert into foo(_key, id) values (?, ?)", 1, 2)).GetAll());
+
+            Assert.AreEqual("Null value is not allowed for field 'NAME'", ex.Message);
+        }
+
+        /// <summary>
         /// Tests all primitive key types.
         /// </summary>
         [Test]
@@ -388,7 +426,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         private class Foo
         {
             [QuerySqlField] public int Id { get; set; }
-            [QuerySqlField] public string Name { get; set; }
+            [QuerySqlField(NotNull = true)] public string Name { get; set; }
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f599c57/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
index 7421b95..2e24ff6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
@@ -125,6 +125,9 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
                 "GroupIndex1", "GroupIndex2", "GroupIndex3"
             }, fields.Select(x => x.Name));
 
+            Assert.IsTrue(fields.Single(x => x.Name == "SqlField").NotNull);
+            Assert.IsFalse(fields.Single(x => x.Name == "IndexedField1").NotNull);
+
             var idx = qe.Indexes.ToArray();
 
             Assert.AreEqual(QueryIndexType.Sorted, idx[0].IndexType);
@@ -314,7 +317,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         /// </summary>
         private class AttributeTest
         {
-            [QuerySqlField]
+            [QuerySqlField(NotNull = true)]
             public double SqlField { get; set; }
 
             [QuerySqlField(IsIndexed = true, Name = "IndexedField1", IsDescending = true, IndexInlineSize = 513)]

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f599c57/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
index 6361850..9c63c73 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
@@ -687,8 +687,15 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         {
             var entity = Cache().GetConfiguration().QueryEntities.Single();
 
-            Assert.AreEqual(typeof(int), entity.Fields.Single(x => x.Name == "age").FieldType);
-            Assert.AreEqual(typeof(string), entity.Fields.Single(x => x.Name == "name").FieldType);
+            var ageField = entity.Fields.Single(x => x.Name == "age");
+            Assert.AreEqual(typeof(int), ageField.FieldType);
+            Assert.IsFalse(ageField.NotNull);
+            Assert.IsFalse(ageField.IsKeyField);
+
+            var nameField = entity.Fields.Single(x => x.Name == "name");
+            Assert.AreEqual(typeof(string), nameField.FieldType);
+            Assert.IsTrue(nameField.NotNull);
+            Assert.IsFalse(nameField.IsKeyField);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f599c57/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/cache-query.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/cache-query.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/cache-query.xml
index 3b61d8d..87e65a2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/cache-query.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/cache-query.xml
@@ -66,6 +66,11 @@
                                         <entry key="name" value="java.lang.String" />
                                     </util:map>
                                 </property>
+                                <property name="notNullFields">
+                                    <list>
+                                        <value>name</value>
+                                    </list>
+                                </property>
                                 <property name="indexes">
                                     <list>
                                         <bean class="org.apache.ignite.cache.QueryIndex">

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f599c57/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
index f6854e1..229d42e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
@@ -55,7 +55,7 @@
             <queryEntities>
                 <queryEntity keyType='System.Int32' valueType='System.String' tableName='myTable'>
                     <fields>
-                        <queryField name='length' fieldType='System.Int32' isKeyField='true' />
+                        <queryField name='length' fieldType='System.Int32' isKeyField='true' notNull='true' />
                     </fields>
                     <aliases>
                         <queryAlias fullName='somefield.field' alias='shortField' />

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f599c57/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 b04f466..edecccc 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -115,6 +115,7 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual("length", queryEntity.Fields.Single().Name);
             Assert.AreEqual(typeof(int), queryEntity.Fields.Single().FieldType);
             Assert.IsTrue(queryEntity.Fields.Single().IsKeyField);
+            Assert.IsTrue(queryEntity.Fields.Single().NotNull);
             Assert.AreEqual("somefield.field", queryEntity.Aliases.Single().FullName);
             Assert.AreEqual("shortField", queryEntity.Aliases.Single().Alias);
 
@@ -600,7 +601,11 @@ namespace Apache.Ignite.Core.Tests
                             {
                                 Fields = new[]
                                 {
-                                    new QueryField("field", typeof (int)) { IsKeyField = true }
+                                    new QueryField("field", typeof (int))
+                                    {
+                                        IsKeyField = true,
+                                        NotNull = true
+                                    }
                                 },
                                 Indexes = new[]
                                 {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f599c57/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
index b40231c..e8d0c91 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
@@ -51,6 +51,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /** */
         private Dictionary<string, string> _aliasMap;
 
+        /** */
         private ICollection<QueryAlias> _aliases;
 
         /// <summary>
@@ -241,9 +242,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
             var count = reader.ReadInt();
             Fields = count == 0
                 ? null
-                : Enumerable.Range(0, count).Select(x =>
-                    new QueryField(reader.ReadString(), reader.ReadString()) {IsKeyField = reader.ReadBoolean()})
-                    .ToList();
+                : Enumerable.Range(0, count).Select(x => new QueryField(reader)).ToList();
 
             count = reader.ReadInt();
             Aliases = count == 0 ? null : Enumerable.Range(0, count)
@@ -271,9 +270,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
 
                 foreach (var field in Fields)
                 {
-                    writer.WriteString(field.Name);
-                    writer.WriteString(field.FieldTypeName);
-                    writer.WriteBoolean(field.IsKeyField);
+                    field.Write(writer);
                 }
             }
             else
@@ -456,11 +453,15 @@ namespace Apache.Ignite.Core.Cache.Configuration
                         indexes.Add(new QueryIndexEx(columnName, attr.IsDescending, QueryIndexType.Sorted,
                             attr.IndexGroups)
                         {
-                            InlineSize = attr.IndexInlineSize
+                            InlineSize = attr.IndexInlineSize,
                         });
                     }
 
-                    fields.Add(new QueryField(columnName, memberInfo.Value) {IsKeyField = isKey});
+                    fields.Add(new QueryField(columnName, memberInfo.Value)
+                    {
+                        IsKeyField = isKey,
+                        NotNull = attr.NotNull
+                    });
 
                     ScanAttributes(memberInfo.Value, fields, indexes, columnName, visitedTypes, isKey);
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f599c57/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs
index 596837a..b8142fd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs
@@ -21,6 +21,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
 {
     using System;
     using System.Diagnostics;
+    using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Log;
@@ -73,6 +74,32 @@ namespace Apache.Ignite.Core.Cache.Configuration
         }
 
         /// <summary>
+        /// Initializes a new instance of the <see cref="QueryField"/> class.
+        /// </summary>
+        internal QueryField(IBinaryRawReader reader)
+        {
+            Debug.Assert(reader != null);
+
+            Name = reader.ReadString();
+            FieldTypeName = reader.ReadString();
+            IsKeyField = reader.ReadBoolean();
+            NotNull = reader.ReadBoolean();
+        }
+
+        /// <summary>
+        /// Writes this instance to the specified writer.
+        /// </summary>
+        internal void Write(IBinaryRawWriter writer)
+        {
+            Debug.Assert(writer != null);
+
+            writer.WriteString(Name);
+            writer.WriteString(FieldTypeName);
+            writer.WriteBoolean(IsKeyField);
+            writer.WriteBoolean(NotNull);
+        }
+
+        /// <summary>
         /// Gets or sets the field name.
         /// </summary>
         public string Name { get; set; }
@@ -115,6 +142,11 @@ namespace Apache.Ignite.Core.Cache.Configuration
         public bool IsKeyField { get; set; }
 
         /// <summary>
+        /// Gets or sets a value indicating whether null value is allowed for the field.
+        /// </summary>
+        public bool NotNull { get; set; }
+
+        /// <summary>
         /// Validates this instance and outputs information to the log, if necessary.
         /// </summary>
         internal void Validate(ILogger log, string logInfo)

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f599c57/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QuerySqlFieldAttribute.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QuerySqlFieldAttribute.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QuerySqlFieldAttribute.cs
index b920304..d15cc1a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QuerySqlFieldAttribute.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QuerySqlFieldAttribute.cs
@@ -71,5 +71,10 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// </summary>
         [DefaultValue(QueryIndex.DefaultInlineSize)]
         public int IndexInlineSize { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether null values are allowed for this field.
+        /// </summary>
+        public bool NotNull { get; set; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f599c57/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 730cb9f..988fa1f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -328,6 +328,11 @@
                                                                                             <xs:documentation>Indicates whether this field belongs to the cache key.</xs:documentation>
                                                                                         </xs:annotation>
                                                                                     </xs:attribute>
+                                                                                    <xs:attribute name="notNull" type="xs:boolean">
+                                                                                        <xs:annotation>
+                                                                                            <xs:documentation>Indicates whether null value is allowed for the field.</xs:documentation>
+                                                                                        </xs:annotation>
+                                                                                    </xs:attribute>
                                                                                 </xs:complexType>
                                                                             </xs:element>
                                                                         </xs:sequence>


[32/37] ignite git commit: IGNITE-6350: SQL: forbid configurations with NOT NULL fields and read-through cache store. This closes #2656.

Posted by vo...@apache.org.
IGNITE-6350: SQL: forbid configurations with NOT NULL fields and read-through cache store. This closes #2656.


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

Branch: refs/heads/ignite-2.3
Commit: 3256ee27e7cd929246f2e5813fcc35adda58cc3f
Parents: 16d2370
Author: Sergey Kalashnikov <sk...@gridgain.com>
Authored: Tue Oct 3 10:13:06 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Oct 3 10:13:06 2017 +0300

----------------------------------------------------------------------
 .../ignite/jdbc/JdbcErrorsAbstractSelfTest.java |  88 ++++++++-
 .../processors/cache/GridCacheUtils.java        |   6 +-
 .../processors/query/GridQueryProcessor.java    |   6 +-
 .../internal/processors/query/QueryUtils.java   |  17 ++
 .../query/h2/ddl/DdlStatementsProcessor.java    |  15 +-
 .../query/IgniteSqlNotNullConstraintTest.java   | 179 ++++++++++++++++++-
 .../Cache/CacheConfigurationTest.cs             |   2 +-
 7 files changed, 304 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3256ee27/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
index 78020cf..952baa5 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.jdbc;
 
+import java.io.Serializable;
 import java.net.URL;
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
@@ -29,8 +30,14 @@ import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.Collections;
 import java.util.List;
+import javax.cache.Cache;
+import javax.cache.integration.CacheLoaderException;
+import javax.cache.integration.CacheWriterException;
+import org.apache.ignite.cache.CacheInterceptorAdapter;
 import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -39,13 +46,27 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
  * Test SQLSTATE codes propagation with (any) Ignite JDBC driver.
  */
 public abstract class JdbcErrorsAbstractSelfTest extends GridCommonAbstractTest {
+    /** */
+    protected static final String CACHE_STORE_TEMPLATE = "cache_store";
+
+    /** */
+    protected static final String CACHE_INTERCEPTOR_TEMPLATE = "cache_interceptor";
+
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
-        startGrid(getConfiguration(getTestIgniteInstanceName(0))
+        IgniteEx grid = startGrid(getConfiguration(getTestIgniteInstanceName(0))
             .setCacheConfiguration(new CacheConfiguration("test")
                 .setQueryEntities(Collections.singletonList(new QueryEntity(Integer.class, Integer.class)))));
+
+        // add cache template for cache with enabled read-through cache store
+        grid.addCacheConfiguration(new CacheConfiguration<>(CACHE_STORE_TEMPLATE)
+            .setCacheStoreFactory(singletonFactory(new TestCacheStore())).setReadThrough(true));
+
+        // add cache template for cache with enabled cache interceptor
+        grid.addCacheConfiguration(new CacheConfiguration<>(CACHE_INTERCEPTOR_TEMPLATE)
+            .setInterceptor(new TestCacheInterceptor()));
     }
 
     /** {@inheritDoc} */
@@ -505,6 +526,44 @@ public abstract class JdbcErrorsAbstractSelfTest extends GridCommonAbstractTest
     }
 
     /**
+     * Check error code for the case not null field is configured for table belonging to cache
+     * with enabled read-through cache store.
+     *
+     * @throws SQLException if failed.
+     */
+    public void testNotNullRestrictionReadThroughCacheStore() throws SQLException {
+        checkErrorState(new ConnClosure() {
+            @Override public void run(Connection conn) throws Exception {
+                conn.setSchema("PUBLIC");
+
+                try (Statement stmt = conn.createStatement()) {
+                    stmt.execute("CREATE TABLE cache_store_nulltest(id INT PRIMARY KEY, age INT NOT NULL) " +
+                        "WITH \"template=" + CACHE_STORE_TEMPLATE + "\"");
+                }
+            }
+        }, "0A000");
+    }
+
+    /**
+     * Check error code for the case not null field is configured for table belonging to cache
+     * with configured cache interceptor.
+     *
+     * @throws SQLException if failed.
+     */
+    public void testNotNullRestrictionCacheInterceptor() throws SQLException {
+        checkErrorState(new ConnClosure() {
+            @Override public void run(Connection conn) throws Exception {
+                conn.setSchema("PUBLIC");
+
+                try (Statement stmt = conn.createStatement()) {
+                    stmt.execute("CREATE TABLE cache_interceptor_nulltest(id INT PRIMARY KEY, age INT NOT NULL) " +
+                        "WITH \"template=" + CACHE_INTERCEPTOR_TEMPLATE + "\"");
+                }
+            }
+        }, "0A000");
+    }
+
+    /**
      * @return Connection to execute statements on.
      * @throws SQLException if failed.
      */
@@ -570,4 +629,31 @@ public abstract class JdbcErrorsAbstractSelfTest extends GridCommonAbstractTest
          */
         void run(Connection conn) throws Exception;
     }
+
+    /**
+     * Cache store stub.
+     */
+    protected class TestCacheStore extends CacheStoreAdapter<Object,Object> implements Serializable {
+        /** {@inheritDoc} */
+        @Override public Object load(Object key) throws CacheLoaderException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<?, ?> entry) throws CacheWriterException {
+            // No-op
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            // No-op
+        }
+    }
+
+    /**
+     * Cache interceptor stub.
+     */
+    private static class TestCacheInterceptor extends CacheInterceptorAdapter<Object, Object> implements Serializable {
+        // No-op
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3256ee27/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 2018a64..4f76875 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -1663,8 +1663,12 @@ public class GridCacheUtils {
         if (!F.isEmpty(entities)) {
             Collection<QueryEntity> normalEntities = new ArrayList<>(entities.size());
 
-            for (QueryEntity entity : entities)
+            for (QueryEntity entity : entities) {
+                if (!F.isEmpty(entity.getNotNullFields()))
+                    QueryUtils.checkNotNullAllowed(cfg);
+
                 normalEntities.add(QueryUtils.normalizeQueryEntity(entity, cfg.isSqlEscapeAll()));
+            }
 
             cfg.clearQueryEntities().setQueryEntities(normalEntities);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3256ee27/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 58c3ce9..56e8a42 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -39,7 +39,6 @@ import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheKeyConfiguration;
@@ -1355,7 +1354,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param schemaName Schema name to create table in.
      * @param entity Entity to create table from.
      * @param templateName Template name.
-     * @param cacheName
+     * @param cacheName Cache name.
      * @param cacheGroup Cache group name.
      * @param affinityKey Affinity key column name.
      * @param atomicityMode Atomicity mode.
@@ -1389,6 +1388,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new SchemaOperationException("Template cache already contains query entities which it should not: " +
                 templateName);
 
+        if (!F.isEmpty(entity.getNotNullFields()))
+            QueryUtils.checkNotNullAllowed(ccfg);
+
         if (F.isEmpty(cacheName))
             cacheName = QueryUtils.createTableCacheName(schemaName, entity.getTableName());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3256ee27/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index e76a6ca..1b61ce9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -46,6 +46,7 @@ import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheDefaultAffinityKeyMapper;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.property.QueryBinaryProperty;
 import org.apache.ignite.internal.processors.query.property.QueryClassProperty;
 import org.apache.ignite.internal.processors.query.property.QueryFieldAccessor;
@@ -1204,6 +1205,22 @@ public class QueryUtils {
     }
 
     /**
+     * Performs checks to forbid cache configurations that are not compatible with NOT NULL query fields.
+     * See {@link QueryEntity#setNotNullFields(Set)}.
+     *
+     * @param cfg Cache configuration.
+     */
+    public static void checkNotNullAllowed(CacheConfiguration cfg) {
+        if (cfg.isReadThrough())
+            throw new IgniteSQLException("NOT NULL constraint is not supported when CacheConfiguration.readThrough " +
+                "is enabled.", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+        if (cfg.getInterceptor() != null)
+            throw new IgniteSQLException("NOT NULL constraint is not supported when CacheConfiguration.interceptor " +
+                "is set.", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+    }
+
+    /**
      * Private constructor.
      */
     private QueryUtils() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/3256ee27/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
index 8a901dc..affd903 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
@@ -229,6 +229,8 @@ public class DdlStatementsProcessor {
                 else {
                     List<QueryField> cols = new ArrayList<>(cmd.columns().length);
 
+                    boolean allFieldsNullable = true;
+
                     for (GridSqlColumn col : cmd.columns()) {
                         if (tbl.doesColumnExist(col.columnName())) {
                             if ((!cmd.ifNotExists() || cmd.columns().length != 1)) {
@@ -242,14 +244,21 @@ public class DdlStatementsProcessor {
                             }
                         }
 
-                        cols.add(new QueryField(col.columnName(),
+                        QueryField field = new QueryField(col.columnName(),
                             DataType.getTypeClassName(col.column().getType()),
-                            col.column().isNullable()));
+                            col.column().isNullable());
+
+                        cols.add(field);
+
+                        allFieldsNullable &= field.isNullable();
                     }
 
                     if (cols != null) {
                         assert tbl.rowDescriptor() != null;
 
+                        if (!allFieldsNullable)
+                            QueryUtils.checkNotNullAllowed(tbl.cache().config());
+
                         fut = ctx.query().dynamicColumnAdd(tbl.cacheName(), cmd.schemaName(),
                             tbl.rowDescriptor().type().tableName(), cols, cmd.ifTableExists(), cmd.ifNotExists());
                     }
@@ -276,7 +285,7 @@ public class DdlStatementsProcessor {
             throw e;
         }
         catch (Exception e) {
-            throw new IgniteSQLException("Unexpected DLL operation failure: " + e.getMessage(), e);
+            throw new IgniteSQLException("Unexpected DDL operation failure: " + e.getMessage(), e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3256ee27/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java
index b724f02..b372eb3 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
+import javax.cache.Cache;
 import javax.cache.CacheException;
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorException;
@@ -32,6 +33,7 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheInterceptor;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.QueryEntity;
@@ -47,6 +49,7 @@ import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.lang.IgniteBiInClosure;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
@@ -76,9 +79,29 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
     /** Name of SQL table. */
     private static String TABLE_PERSON = "\"" + CACHE_PERSON +  "\".\"PERSON\"";
 
+    /** Template of cache with read-through setting. */
+    private static String CACHE_READ_THROUGH = "cacheReadThrough";
+
+    /** Template of cache with interceptor setting. */
+    private static String CACHE_INTERCEPTOR = "cacheInterceptor";
+
     /** Expected error message. */
     private static String ERR_MSG = "Null value is not allowed for field 'NAME'";
 
+    /** Expected error message for read-through restriction. */
+    private static String READ_THROUGH_ERR_MSG = "NOT NULL constraint is not supported when " +
+        "CacheConfiguration.readThrough is enabled.";
+
+    /** Expected error message for cache interceptor restriction. */
+    private static String INTERCEPTOR_ERR_MSG = "NOT NULL constraint is not supported when " +
+        "CacheConfiguration.interceptor is set.";
+
+    /** Name of the node which configuration includes restricted cache config. */
+    private static String READ_THROUGH_CFG_NODE_NAME = "nodeCacheReadThrough";
+
+    /** Name of the node which configuration includes restricted cache config. */
+    private static String INTERCEPTOR_CFG_NODE_NAME = "nodeCacheInterceptor";
+
     /** OK value. */
     private final Person okValue = new Person("Name", 18);
 
@@ -100,6 +123,18 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
 
         ccfgs.addAll(cacheConfigurations());
 
+        if (gridName.equals(READ_THROUGH_CFG_NODE_NAME)) {
+            ccfgs.add(buildCacheConfigurationRestricted("BadCfgTestCacheRT", true, false, true));
+
+            c.setClientMode(true);
+        }
+
+        if (gridName.equals(INTERCEPTOR_CFG_NODE_NAME)) {
+            ccfgs.add(buildCacheConfigurationRestricted("BadCfgTestCacheINT", false, true, true));
+
+            c.setClientMode(true);
+        }
+
         c.setCacheConfiguration(ccfgs.toArray(new CacheConfiguration[ccfgs.size()]));
 
         if (gridName.equals(NODE_CLIENT)) {
@@ -161,6 +196,30 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
         return cfg;
     }
 
+    /** */
+    private CacheConfiguration buildCacheConfigurationRestricted(String cacheName, boolean readThrough,
+        boolean interceptor, boolean hasQueryEntity) {
+        CacheConfiguration cfg = new CacheConfiguration<Integer, Person>()
+            .setName(cacheName)
+            .setCacheMode(CacheMode.PARTITIONED)
+            .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+
+        if (readThrough) {
+            cfg.setCacheStoreFactory(singletonFactory(new TestStore()));
+            cfg.setReadThrough(true);
+        }
+
+        if (interceptor)
+            cfg.setInterceptor(new TestInterceptor());
+
+        if (hasQueryEntity) {
+            cfg.setQueryEntities(F.asList(new QueryEntity(Integer.class, Person.class)
+                .setNotNullFields(Collections.singleton("name"))));
+        }
+
+        return cfg;
+    }
+
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
@@ -169,6 +228,14 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
 
         startGrid(NODE_CLIENT);
 
+        // Add cache template with read-through cache store.
+        grid(NODE_CLIENT).addCacheConfiguration(
+            buildCacheConfigurationRestricted(CACHE_READ_THROUGH, true, false, false));
+
+        // Add cache template with cache interceptor.
+        grid(NODE_CLIENT).addCacheConfiguration(
+            buildCacheConfigurationRestricted(CACHE_INTERCEPTOR, false, true, false));
+
         awaitPartitionMapExchange();
     }
 
@@ -806,6 +873,86 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
         assertEquals("Bob", result.get(2).get(1));
     }
 
+    /** Check QueryEntity configuration fails with NOT NULL field and read-through. */
+    public void testReadThroughRestrictionQueryEntity() throws Exception {
+        // Node start-up failure (read-through cache store).
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return startGrid(READ_THROUGH_CFG_NODE_NAME);
+            }
+        }, IgniteCheckedException.class, READ_THROUGH_ERR_MSG);
+
+        // Dynamic cache start-up failure (read-through cache store)
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return grid(NODE_CLIENT).createCache(
+                    buildCacheConfigurationRestricted("dynBadCfgCacheRT", true, false, true));
+            }
+        }, IgniteCheckedException.class, READ_THROUGH_ERR_MSG);
+    }
+
+    /** Check QueryEntity configuration fails with NOT NULL field and cache interceptor. */
+    public void testInterceptorRestrictionQueryEntity() throws Exception {
+        // Node start-up failure (interceptor).
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return startGrid(INTERCEPTOR_CFG_NODE_NAME);
+            }
+        }, IgniteCheckedException.class, INTERCEPTOR_ERR_MSG);
+
+        // Dynamic cache start-up failure (interceptor)
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return grid(NODE_CLIENT).createCache(
+                    buildCacheConfigurationRestricted("dynBadCfgCacheINT", false, true, true));
+            }
+        }, IgniteCheckedException.class, INTERCEPTOR_ERR_MSG);
+    }
+
+    /** Check create table fails with NOT NULL field and read-through. */
+    public void testReadThroughRestrictionCreateTable() throws Exception {
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return executeSql("CREATE TABLE test(id INT PRIMARY KEY, name char NOT NULL) " +
+                    "WITH \"template=" + CACHE_READ_THROUGH+ "\"");
+            }
+        }, IgniteSQLException.class, READ_THROUGH_ERR_MSG);
+    }
+
+    /** Check create table fails with NOT NULL field and cache interceptor. */
+    public void testInterceptorRestrictionCreateTable() throws Exception {
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return executeSql("CREATE TABLE test(id INT PRIMARY KEY, name char NOT NULL) " +
+                    "WITH \"template=" + CACHE_INTERCEPTOR + "\"");
+            }
+        }, IgniteSQLException.class, INTERCEPTOR_ERR_MSG);
+    }
+
+    /** Check alter table fails with NOT NULL field and read-through. */
+    public void testReadThroughRestrictionAlterTable() throws Exception {
+        executeSql("CREATE TABLE test(id INT PRIMARY KEY, age INT) " +
+            "WITH \"template=" + CACHE_READ_THROUGH + "\"");
+
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return executeSql("ALTER TABLE test ADD COLUMN name char NOT NULL");
+            }
+        }, IgniteSQLException.class, READ_THROUGH_ERR_MSG);
+    }
+
+    /** Check alter table fails with NOT NULL field and cache interceptor. */
+    public void testInterceptorRestrictionAlterTable() throws Exception {
+        executeSql("CREATE TABLE test(id INT PRIMARY KEY, age INT) " +
+            "WITH \"template=" + CACHE_INTERCEPTOR + "\"");
+
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return executeSql("ALTER TABLE test ADD COLUMN name char NOT NULL");
+            }
+        }, IgniteSQLException.class, INTERCEPTOR_ERR_MSG);
+    }
+
     /** */
     private void executeWithAllCaches(TestClosure clo) throws Exception {
         for (CacheConfiguration ccfg : cacheConfigurations())
@@ -1034,7 +1181,7 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Test store.
+     * Test cache store stub.
      */
     private static class TestStore extends CacheStoreAdapter<Integer, Person> {
         /** {@inheritDoc} */
@@ -1057,4 +1204,34 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
             // No-op
         }
     }
+
+    /**
+     * Test interceptor stub.
+     */
+    private static class TestInterceptor implements CacheInterceptor<Integer, Person> {
+        /** {@inheritDoc} */
+        @Nullable @Override public Person onGet(Integer key, @Nullable Person val) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Person onBeforePut(Cache.Entry<Integer, Person> entry, Person newVal) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onAfterPut(Cache.Entry<Integer, Person> entry) {
+            // No-op
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public IgniteBiTuple<Boolean, Person> onBeforeRemove(Cache.Entry<Integer, Person> entry) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onAfterRemove(Cache.Entry<Integer, Person> entry) {
+            // No-op
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3256ee27/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
index abf8af0..ddf669d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
@@ -573,7 +573,7 @@ namespace Apache.Ignite.Core.Tests.Cache
                 WriteBehindEnabled = false,
                 WriteSynchronizationMode = CacheWriteSynchronizationMode.PrimarySync,
                 CacheStoreFactory = new CacheStoreFactoryTest(),
-                ReadThrough = true,
+                ReadThrough = false,
                 WriteThrough = true,
                 WriteBehindCoalescing = false,
                 GroupName = "someGroup",


[16/37] ignite git commit: IGNITE-6513 introduce system property for manage wal serializer version (IGNITE_WAL_SERIALIZER_VERSION)

Posted by vo...@apache.org.
IGNITE-6513 introduce system property for manage wal serializer version (IGNITE_WAL_SERIALIZER_VERSION)

Signed-off-by: Andrey Gura <ag...@apache.org>


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

Branch: refs/heads/ignite-2.3
Commit: 0749ec45f0af2dbd18a08e5e25b3773a01a16a0e
Parents: c4047b4
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Fri Sep 29 16:07:58 2017 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Fri Sep 29 16:07:58 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |  5 +
 .../wal/FileWriteAheadLogManager.java           |  5 +-
 .../db/wal/IgniteWalSerializerVersionTest.java  | 96 ++++++++++++++++++++
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |  3 +
 4 files changed, 108 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0749ec45/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 39e65e1..e1e72f7 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -732,6 +732,11 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_OVERRIDE_WRITE_THROTTLING_ENABLED = "IGNITE_OVERRIDE_WRITE_THROTTLING_ENABLED";
 
     /**
+     * Property for setup WAL serializer version.
+     */
+    public static final String IGNITE_WAL_SERIALIZER_VERSION = "IGNITE_WAL_SERIALIZER_VERSION";
+
+    /**
      * Enforces singleton.
      */
     private IgniteSystemProperties() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0749ec45/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
index c4582cf..76edcea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
@@ -42,6 +42,7 @@ import java.util.concurrent.locks.ReentrantLock;
 import java.util.regex.Pattern;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.configuration.WALMode;
@@ -87,6 +88,7 @@ import org.jetbrains.annotations.Nullable;
 import static java.nio.file.StandardOpenOption.CREATE;
 import static java.nio.file.StandardOpenOption.READ;
 import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_SERIALIZER_VERSION;
 
 /**
  * File WAL manager.
@@ -164,7 +166,8 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     private RecordSerializer serializer;
 
     /** Serializer latest version to use. */
-    private int serializerVersion = LATEST_SERIALIZER_VERSION;
+    private final int serializerVersion =
+        IgniteSystemProperties.getInteger(IGNITE_WAL_SERIALIZER_VERSION, LATEST_SERIALIZER_VERSION);
 
     /** */
     private volatile long oldestArchiveSegmentIdx;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0749ec45/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java
new file mode 100644
index 0000000..6d9d80e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java
@@ -0,0 +1,96 @@
+/*
+ * 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.persistence.db.wal;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
+import org.apache.ignite.internal.processors.cache.persistence.wal.RecordSerializer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV2Serializer;
+import org.apache.ignite.internal.util.typedef.internal.GPC;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_SERIALIZER_VERSION;
+
+/**
+ *
+ */
+public class IgniteWalSerializerVersionTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setPersistentStoreConfiguration(new PersistentStoreConfiguration());
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCheckDifferentSerializerVersions() throws Exception {
+        IgniteEx ig0 = (IgniteEx)startGrid();
+
+        IgniteWriteAheadLogManager wal0 = ig0.context().cache().context().wal();
+
+        RecordSerializer ser0 = U.field(wal0, "serializer");
+
+        assertTrue(ser0 instanceof RecordV1Serializer);
+
+        stopGrid();
+
+        System.setProperty(IGNITE_WAL_SERIALIZER_VERSION, "2");
+
+        IgniteEx ig1 = (IgniteEx)startGrid();
+
+        IgniteWriteAheadLogManager wal1 = ig1.context().cache().context().wal();
+
+        RecordSerializer ser1 = U.field(wal1, "serializer");
+
+        assertTrue(ser1 instanceof RecordV2Serializer);
+
+        stopGrid();
+
+        System.setProperty(IGNITE_WAL_SERIALIZER_VERSION, "3");
+
+        GridTestUtils.assertThrowsAnyCause(log, new GPC<Void>() {
+            @Override public Void call() throws Exception {
+                startGrid();
+
+                return null;
+            }
+        }, IgniteCheckedException.class, "Failed to create a serializer with the given version");
+
+        System.setProperty(IGNITE_WAL_SERIALIZER_VERSION, "1");
+
+        IgniteEx ig2 = (IgniteEx)startGrid();
+
+        IgniteWriteAheadLogManager wal2 = ig2.context().cache().context().wal();
+
+        RecordSerializer ser2 = U.field(wal2, "serializer");
+
+        assertTrue(ser2 instanceof RecordV1Serializer);
+
+        stopGrid();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0749ec45/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
index 29ea64c..29f7255 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsTrans
 import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsWholeClusterRestartTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushFailoverTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalHistoryReservationsTest;
+import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalSerializerVersionTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.crc.IgniteDataIntegrityTests;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.reader.IgniteWalReaderTest;
 
@@ -77,6 +78,8 @@ public class IgnitePdsTestSuite2 extends TestSuite {
 
         suite.addTestSuite(IgnitePdsExchangeDuringCheckpointTest.class);
 
+        suite.addTestSuite(IgniteWalSerializerVersionTest.class);
+
         return suite;
     }
 }


[23/37] ignite git commit: IGNITE-6286: SQL: fixed BigDecimal argument handling. This closes #2750.

Posted by vo...@apache.org.
IGNITE-6286: SQL: fixed BigDecimal argument handling. This closes #2750.


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

Branch: refs/heads/ignite-2.3
Commit: 013d7dbf7811a1b8e207d6238aff67e43c28adad
Parents: 59ee8af
Author: Sergey Chernolyas <se...@gmail.com>
Authored: Mon Oct 2 10:52:21 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Oct 2 10:52:21 2017 +0300

----------------------------------------------------------------------
 .../processors/query/h2/IgniteH2Indexing.java   |   3 +
 .../query/IgniteSqlParameterizedQueryTest.java  | 392 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   3 +-
 3 files changed, 397 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/013d7dbf/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 9e6a1fa..fd7b9a8 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.query.h2;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
+import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -511,6 +512,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 stmt.setNull(idx, Types.VARCHAR);
             else if (obj instanceof BigInteger)
                 stmt.setObject(idx, obj, Types.JAVA_OBJECT);
+            else if (obj instanceof BigDecimal)
+                stmt.setObject(idx, obj, Types.DECIMAL);
             else
                 stmt.setObject(idx, obj);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/013d7dbf/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlParameterizedQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlParameterizedQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlParameterizedQueryTest.java
new file mode 100644
index 0000000..b5039cd
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlParameterizedQueryTest.java
@@ -0,0 +1,392 @@
+/*
+ * 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.query;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.Date;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Test sql queries with parameters for all types.
+ * The test is fix  for issue 'IGNITE-6286'
+ *
+ * @author Sergey Chernolyas &amp;sergey_chernolyas@gmail.com&amp;
+ * @see <a href="https://issues.apache.org/jira/browse/IGNITE-6286">IGNITE-6286</a>
+ */
+public class IgniteSqlParameterizedQueryTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final String CACHE_BOOKMARK = "Bookmark";
+
+    /** */
+    private static final String NODE_CLIENT = "client";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        c.setDiscoverySpi(disco);
+
+        c.setCacheConfiguration(buildCacheConfiguration(CACHE_BOOKMARK));
+        if (gridName.equals(NODE_CLIENT))
+            c.setClientMode(true);
+
+        return c;
+    }
+
+    /**
+     * build cache configuration
+     * @param name cache name
+     * @return configuration
+     * @see CacheConfiguration
+     */
+    private CacheConfiguration buildCacheConfiguration(String name) {
+        CacheConfiguration ccfg = new CacheConfiguration(name);
+        ccfg.setIndexedTypes(String.class, Bookmark.class);
+        return ccfg;
+
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        startGrid(0);
+        startGrid(NODE_CLIENT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * method for create parametrized query and get first result
+     * @param field name of field
+     * @param val value
+     * @return fist searched object
+     * @see Bookmark
+     */
+    private Object columnValue(String field, Object val) {
+        IgniteCache<String, Bookmark> cache = grid(NODE_CLIENT).cache(CACHE_BOOKMARK);
+        SqlFieldsQuery qry = new SqlFieldsQuery("SELECT " + field + " from  Bookmark where " + field + " = ?");
+        qry.setArgs(val);
+
+        QueryCursor<List<?>> cursor = cache.query(qry);
+        List<List<?>> results = cursor.getAll();
+        assertEquals("Search by field '" + field +"' returns incorrect row count!",1, results.size());
+        List<?> row0 = results.get(0);
+        return row0.get(0);
+    }
+
+    /**
+     * testing parametrized query by field with supported type
+     * @throws Exception if any error occurs
+     */
+    public void testSupportedTypes() throws Exception {
+        IgniteCache<String, Bookmark> cache = grid(NODE_CLIENT).cache(CACHE_BOOKMARK);
+        Bookmark bookmark = new Bookmark();
+        bookmark.setId(UUID.randomUUID().toString());
+        bookmark.setStockCount(Integer.MAX_VALUE);
+        bookmark.setUrlPort(Short.MAX_VALUE);
+        bookmark.setUserId(Long.MAX_VALUE);
+        bookmark.setVisitRatio(Float.MAX_VALUE);
+        bookmark.setTaxPercentage(Double.MAX_VALUE);
+        bookmark.setFavourite(true);
+        bookmark.setDisplayMask(Byte.MAX_VALUE);
+        bookmark.setSerialNumber(UUID.randomUUID());
+        bookmark.setVisitCount(new BigInteger("1000000000000000"));
+        bookmark.setSiteWeight(new BigDecimal("1000000000000000.001"));
+        bookmark.setCreated(new Date());
+        cache.put(bookmark.id, bookmark);
+
+        assertEquals(bookmark.getId(), columnValue("id", bookmark.getId()));
+        assertEquals(bookmark.getStockCount(), columnValue("stockcount", bookmark.getStockCount()));
+        assertEquals(bookmark.getUrlPort(), columnValue("urlport", bookmark.getUrlPort()));
+        assertEquals(bookmark.getUserId(), columnValue("userid", bookmark.getUserId()));
+        assertEquals(bookmark.getVisitRatio(), columnValue("visitratio", bookmark.getVisitRatio()));
+        assertEquals(bookmark.getTaxPercentage(), columnValue("taxpercentage", bookmark.getTaxPercentage()));
+        assertEquals(bookmark.getFavourite(), columnValue("favourite", bookmark.getFavourite()));
+        assertEquals(bookmark.getDisplayMask(), columnValue("displaymask", bookmark.getDisplayMask()));
+        assertEquals(bookmark.getSerialNumber(), columnValue("serialnumber", bookmark.getSerialNumber()));
+        assertEquals(bookmark.getVisitCount(), columnValue("visitcount", bookmark.getVisitCount()));
+        assertEquals(bookmark.getSiteWeight(), columnValue("siteweight", bookmark.getSiteWeight()));
+        assertEquals(bookmark.getCreated(), columnValue("created", bookmark.getCreated()));
+    }
+
+    /**
+     * Object with all predefined SQL Data Types
+     * @see <a href="https://apacheignite.readme.io/docs/dml#section-advanced-configuration">SQL Data Types</a>
+     */
+    private static class Bookmark implements Serializable {
+        /** */
+        @QuerySqlField
+        private String id;
+
+        /** */
+        @QuerySqlField
+        private Integer stockCount;
+
+        /** */
+        @QuerySqlField
+        private Short urlPort;
+
+        /** */
+        @QuerySqlField
+        private Long userId;
+
+        /** */
+        @QuerySqlField
+        private Float visitRatio;
+
+        /** */
+        @QuerySqlField
+        private Double taxPercentage;
+
+        /** */
+        @QuerySqlField
+        private Boolean favourite;
+
+        /** */
+        @QuerySqlField
+        private Byte displayMask;
+
+        /** */
+        @QuerySqlField
+        private UUID serialNumber;
+
+        /** */
+        @QuerySqlField
+        private BigDecimal siteWeight;
+
+        /** */
+        @QuerySqlField
+        private BigInteger visitCount;
+
+        /** */
+        @QuerySqlField
+        private Date created;
+
+        /**
+         *
+         */
+        public String getId() {
+            return id;
+        }
+
+        /**
+         *
+         */
+        public void setId(String id) {
+            this.id = id;
+        }
+
+        /**
+         *
+         */
+        public Integer getStockCount() {
+            return stockCount;
+        }
+
+        /**
+         *
+         */
+        public void setStockCount(Integer stockCount) {
+            this.stockCount = stockCount;
+        }
+
+        /**
+         *
+         */
+        public Short getUrlPort() {
+            return urlPort;
+        }
+
+        /**
+         *
+         */
+        public void setUrlPort(Short urlPort) {
+            this.urlPort = urlPort;
+        }
+
+        /**
+         *
+         */
+        public Long getUserId() {
+            return userId;
+        }
+
+        /**
+         *
+         */
+        public void setUserId(Long userId) {
+            this.userId = userId;
+        }
+
+        /**
+         *
+         */
+        public Float getVisitRatio() {
+            return visitRatio;
+        }
+
+        /**
+         *
+         */
+        public void setVisitRatio(Float visitRatio) {
+            this.visitRatio = visitRatio;
+        }
+
+        /**
+         *
+         */
+        public Double getTaxPercentage() {
+            return taxPercentage;
+        }
+
+        /**
+         *
+         */
+        public void setTaxPercentage(Double taxPercentage) {
+            this.taxPercentage = taxPercentage;
+        }
+
+        /**
+         *
+         */
+        public Boolean getFavourite() {
+            return favourite;
+        }
+
+        /**
+         *
+         */
+        public void setFavourite(Boolean favourite) {
+            this.favourite = favourite;
+        }
+
+        /**
+         *
+         */
+        public Byte getDisplayMask() {
+            return displayMask;
+        }
+
+        /**
+         *
+         */
+        public void setDisplayMask(Byte displayMask) {
+            this.displayMask = displayMask;
+        }
+
+        /**
+         *
+         */
+        public UUID getSerialNumber() {
+            return serialNumber;
+        }
+
+        /**
+         *
+         */
+        public void setSerialNumber(UUID serialNumber) {
+            this.serialNumber = serialNumber;
+        }
+
+        /**
+         *
+         */
+        public BigDecimal getSiteWeight() {
+            return siteWeight;
+        }
+
+        /**
+         *
+         */
+        public void setSiteWeight(BigDecimal siteWeight) {
+            this.siteWeight = siteWeight;
+        }
+
+        /**
+         *
+         */
+        public BigInteger getVisitCount() {
+            return visitCount;
+        }
+
+        /**
+         *
+         */
+        public void setVisitCount(BigInteger visitCount) {
+            this.visitCount = visitCount;
+        }
+
+        /**
+         *
+         */
+        public Date getCreated() {
+            return created;
+        }
+
+        /**
+         *
+         */
+        public void setCreated(Date created) {
+            this.created = created;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+            Bookmark bookmark = (Bookmark)o;
+            return Objects.equals(id, bookmark.id);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return Objects.hash(id);
+        }
+    }
+
+}
+
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/013d7dbf/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index aaa8e57..c49649b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -123,6 +123,7 @@ import org.apache.ignite.internal.processors.cache.query.IgniteCacheQueryCacheDe
 import org.apache.ignite.internal.processors.cache.query.IndexingSpiQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.query.IndexingSpiQueryTxSelfTest;
 import org.apache.ignite.internal.processors.client.ClientConnectorConfigurationValidationSelfTest;
+import org.apache.ignite.internal.processors.query.IgniteSqlParameterizedQueryTest;
 import org.apache.ignite.internal.processors.query.h2.IgniteSqlBigIntegerKeyTest;
 import org.apache.ignite.internal.processors.query.IgniteQueryDedicatedPoolTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlEntryCacheModeAgnosticTest;
@@ -336,8 +337,8 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteSqlRoutingTest.class);
         suite.addTestSuite(IgniteSqlNotNullConstraintTest.class);
         suite.addTestSuite(LongIndexNameTest.class);
-
         suite.addTestSuite(GridCacheQuerySqlFieldInlineSizeSelfTest.class);
+        suite.addTestSuite(IgniteSqlParameterizedQueryTest.class);
 
         return suite;
     }


[26/37] ignite git commit: ignite-5733 Add ability to activate/deactivate/checkstate through HTTP REST API

Posted by vo...@apache.org.
ignite-5733 Add ability to activate/deactivate/checkstate through HTTP REST API

Signed-off-by: Andrey Gura <ag...@apache.org>


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

Branch: refs/heads/ignite-2.3
Commit: 252eedcccf33c81f1a4007627eb1a29b48b72f7f
Parents: d7fbbd5
Author: Alexander Belyak <al...@xored.com>
Authored: Mon Oct 2 12:58:18 2017 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Mon Oct 2 12:58:18 2017 +0300

----------------------------------------------------------------------
 .../JettyRestProcessorAbstractSelfTest.java     | 46 ++++++++++++++++++++
 .../processors/rest/GridRestCommand.java        |  2 +-
 .../processors/rest/GridRestProcessor.java      |  5 ++-
 .../http/jetty/GridJettyRestHandler.java        | 18 ++++++++
 4 files changed, 68 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/252eedcc/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
index 5f83f6f..b2725b8 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
@@ -710,6 +710,18 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
     /**
      * @throws Exception If failed.
      */
+    public void testDeactivateActivate() throws Exception {
+
+        assertClusterState(true);
+
+        changeClusterState(false);
+
+        changeClusterState(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testPut() throws Exception {
         String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_PUT.key(),
             "key", "putKey", "val", "putVal"));
@@ -2375,4 +2387,38 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
 
         return cfg;
     }
+
+    /**
+     * Test if current cluster state equals expected.
+     *
+     * @param exp Expected state.
+     * @throws Exception If failed.
+     */
+    private void assertClusterState(boolean exp) throws Exception {
+        String ret = content(F.asMap("cmd", GridRestCommand.CLUSTER_CURRENT_STATE.key()));
+
+        info("Cluster state: " + ret);
+        JsonNode res = jsonResponse(ret);
+
+        assertEquals(exp, res.asBoolean());
+        assertEquals(exp, grid(0).active());
+    }
+
+    /**
+     * Change cluster state and test new state.
+     *
+     * @param state Desired state.
+     * @throws Exception If failed.
+     */
+    private void changeClusterState(boolean state) throws Exception {
+        String cmd = (state ? GridRestCommand.CLUSTER_ACTIVE : GridRestCommand.CLUSTER_INACTIVE).key();
+
+        String ret = content(F.asMap("cmd", cmd));
+
+        JsonNode res = jsonResponse(ret);
+
+        assertTrue(res.isNull());
+
+        assertClusterState(state);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/252eedcc/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java
index 24b4bda..2b53b54 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java
@@ -166,7 +166,7 @@ public enum GridRestCommand {
     CLUSTER_INACTIVE("inactive"),
 
     /** */
-    CLUSTER_CURRENT_STATE("currentState");
+    CLUSTER_CURRENT_STATE("currentstate");
 
     /** Enum values. */
     private static final GridRestCommand[] VALS = values();

http://git-wip-us.apache.org/repos/asf/ignite/blob/252eedcc/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
index 9842883..64ab173 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
@@ -67,8 +67,6 @@ import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.internal.util.worker.GridWorkerFuture;
-import org.apache.ignite.internal.visor.compute.VisorGatewayTask;
-import org.apache.ignite.internal.visor.misc.VisorChangeGridActiveStateTask;
 import org.apache.ignite.internal.visor.util.VisorClusterGroupEmptyException;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteInClosure;
@@ -811,6 +809,9 @@ public class GridRestProcessor extends GridProcessorAdapter {
             case ATOMIC_DECREMENT:
             case NAME:
             case LOG:
+            case CLUSTER_CURRENT_STATE:
+            case CLUSTER_ACTIVE:
+            case CLUSTER_INACTIVE:
                 break;
 
             default:

http://git-wip-us.apache.org/repos/asf/ignite/blob/252eedcc/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
----------------------------------------------------------------------
diff --git a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
index 327c13a..b8a42f3 100644
--- a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
+++ b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
@@ -45,6 +45,7 @@ import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
 import org.apache.ignite.internal.processors.rest.GridRestResponse;
 import org.apache.ignite.internal.processors.rest.request.DataStructuresRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
+import org.apache.ignite.internal.processors.rest.request.GridRestChangeStateRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestLogRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestTaskRequest;
@@ -63,6 +64,8 @@ import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_G
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_PUT_ALL;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_REMOVE_ALL;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.EXECUTE_SQL_QUERY;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_ACTIVE;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_CURRENT_STATE;
 import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS_FAILED;
 
 /**
@@ -540,6 +543,21 @@ public class GridJettyRestHandler extends AbstractHandler {
                 break;
             }
 
+            case CLUSTER_ACTIVE:
+            case CLUSTER_INACTIVE:
+            case CLUSTER_CURRENT_STATE: {
+                GridRestChangeStateRequest restReq0 = new GridRestChangeStateRequest();
+
+                if (cmd == CLUSTER_CURRENT_STATE)
+                    restReq0.reqCurrentState();
+                else
+                    restReq0.active(cmd == CLUSTER_ACTIVE);
+
+                restReq = restReq0;
+
+                break;
+            }
+
             case EXECUTE_SQL_QUERY:
             case EXECUTE_SQL_FIELDS_QUERY: {
                 RestQueryRequest restReq0 = new RestQueryRequest();


[09/37] ignite git commit: IGNITE-6509: SQL: Added QueryEntity.notNullFields property. This closes #2769.

Posted by vo...@apache.org.
IGNITE-6509: SQL: Added QueryEntity.notNullFields property. This closes #2769.


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

Branch: refs/heads/ignite-2.3
Commit: 2108d0c5ebc6dbf8fabfa6a81c588356d99fcab7
Parents: 7868843
Author: devozerov <vo...@gridgain.com>
Authored: Thu Sep 28 16:57:22 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Sep 28 16:57:22 2017 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/cache/QueryEntity.java    | 24 +++++++++++++++++---
 .../processors/query/QueryEntityEx.java         |  7 +-----
 .../internal/processors/query/QueryUtils.java   | 11 ++-------
 .../query/IgniteSqlNotNullConstraintTest.java   |  8 +++----
 4 files changed, 28 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2108d0c5/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
index b824209..0b82d6a 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
@@ -83,6 +83,9 @@ public class QueryEntity implements Serializable {
     /** Table name. */
     private String tableName;
 
+    /** Fields that must have non-null value. NB: DO NOT remove underscore to avoid clashes with QueryEntityEx. */
+    private Set<String> _notNullFields;
+
     /**
      * Creates an empty query entity.
      */
@@ -109,6 +112,8 @@ public class QueryEntity implements Serializable {
         idxs = other.idxs != null ? new ArrayList<>(other.idxs) : null;
 
         tableName = other.tableName;
+
+        _notNullFields = other._notNullFields != null ? new HashSet<>(other._notNullFields) : null;
     }
 
     /**
@@ -360,7 +365,19 @@ public class QueryEntity implements Serializable {
      * @return Set of names of fields that must have non-null values.
      */
     @Nullable public Set<String> getNotNullFields() {
-        return null;
+        return _notNullFields;
+    }
+
+    /**
+     * Sets names of fields that must checked for null.
+     *
+     * @param notNullFields Set of names of fields that must have non-null values.
+     * @return {@code this} for chaining.
+     */
+    public QueryEntity setNotNullFields(@Nullable Set<String> notNullFields) {
+        this._notNullFields = notNullFields;
+
+        return this;
     }
 
     /**
@@ -614,13 +631,14 @@ public class QueryEntity implements Serializable {
             F.eq(keyFields, entity.keyFields) &&
             F.eq(aliases, entity.aliases) &&
             F.eqNotOrdered(idxs, entity.idxs) &&
-            F.eq(tableName, entity.tableName);
+            F.eq(tableName, entity.tableName) &&
+            F.eq(_notNullFields, entity._notNullFields);
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
         return Objects.hash(keyType, valType, keyFieldName, valueFieldName, fields, keyFields, aliases, idxs,
-            tableName);
+            tableName, _notNullFields);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/2108d0c5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEntityEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEntityEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEntityEx.java
index ec0d5fa..a697882 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEntityEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEntityEx.java
@@ -62,12 +62,7 @@ public class QueryEntityEx extends QueryEntity {
         return notNullFields;
     }
 
-    /**
-     * Sets names of fields that must checked for null.
-     *
-     * @param notNullFields Set of names of fields that must have non-null values.
-     * @return {@code this} for chaining.
-     */
+    /** {@inheritDoc} */
     public QueryEntity setNotNullFields(@Nullable Set<String> notNullFields) {
         this.notNullFields = notNullFields;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2108d0c5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index 3e96db2..e76a6ca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -213,7 +213,7 @@ public class QueryUtils {
             return entity;
         }
 
-        QueryEntity normalEntity = new QueryEntity();
+        QueryEntity normalEntity = entity instanceof QueryEntityEx ? new QueryEntityEx() : new QueryEntity();
 
         // Propagate plain properties.
         normalEntity.setKeyType(entity.getKeyType());
@@ -222,14 +222,7 @@ public class QueryUtils {
         normalEntity.setKeyFields(entity.getKeyFields());
         normalEntity.setKeyFieldName(entity.getKeyFieldName());
         normalEntity.setValueFieldName(entity.getValueFieldName());
-
-        if (!F.isEmpty(entity.getNotNullFields())) {
-            QueryEntityEx normalEntity0 = new QueryEntityEx(normalEntity);
-
-            normalEntity0.setNotNullFields(entity.getNotNullFields());
-
-            normalEntity = normalEntity0;
-        }
+        normalEntity.setNotNullFields(entity.getNotNullFields());
 
         // Normalize table name.
         String normalTblName = entity.getTableName();

http://git-wip-us.apache.org/repos/asf/ignite/blob/2108d0c5/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java
index 607dea8..b724f02 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java
@@ -144,7 +144,7 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
         cfg.setAtomicityMode(atomicityMode);
         cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
 
-        QueryEntityEx qe = new QueryEntityEx(new QueryEntity(Integer.class, Person.class));
+        QueryEntity qe = new QueryEntity(new QueryEntity(Integer.class, Person.class));
 
         qe.setNotNullFields(Collections.singleton("name"));
 
@@ -188,7 +188,7 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
 
     /** */
     public void testQueryEntityGetSetNotNullFields() throws Exception {
-        QueryEntityEx qe = new QueryEntityEx();
+        QueryEntity qe = new QueryEntity();
 
         assertNull(qe.getNotNullFields());
 
@@ -205,9 +205,9 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
 
     /** */
     public void testQueryEntityEquals() throws Exception {
-        QueryEntityEx a = new QueryEntityEx();
+        QueryEntity a = new QueryEntity();
 
-        QueryEntityEx b = new QueryEntityEx();
+        QueryEntity b = new QueryEntity();
 
         assertEquals(a, b);
 


[34/37] ignite git commit: IGNITE-6382 .NET: Set up NDepend project

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/b21f750f/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 4ee4310..b133b1e 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -882,6 +882,7 @@
                                         <exclude>**/*.FxCop</exclude>
                                         <exclude>**/*.ruleset</exclude>
                                         <exclude>**/*.csproj</exclude>
+                                        <exclude>**/*.ndproj</exclude>
                                         <exclude>**/*.csprojrel</exclude>
                                         <exclude>**/*.vcxproj</exclude>
                                         <exclude>**/*.vcxprojrel</exclude>


[37/37] ignite git commit: Fixed affinityReadyFuture to use actual for result.

Posted by vo...@apache.org.
Fixed affinityReadyFuture to use actual for result.


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

Branch: refs/heads/ignite-2.3
Commit: db69e90279d05ea980cf33ab4230c2c9d4dd2c01
Parents: b21f750
Author: sboikov <sb...@gridgain.com>
Authored: Tue Oct 3 16:42:01 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Oct 3 16:42:01 2017 +0300

----------------------------------------------------------------------
 .../internal/processors/affinity/GridAffinityAssignmentCache.java  | 2 +-
 .../ignite/internal/processors/cache/GridCacheAffinityManager.java | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/db69e902/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index f921251..cbc4eae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -392,7 +392,7 @@ public class GridAffinityAssignmentCache {
                 log.debug("Completing topology ready future right away [head=" + aff.topologyVersion() +
                     ", topVer=" + topVer + ']');
 
-            fut.onDone(topVer);
+            fut.onDone(aff.topologyVersion());
         }
         else if (stopErr != null)
             fut.onDone(stopErr);

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69e902/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
index 702b848..fbe1a95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
@@ -99,7 +99,7 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
 
         IgniteInternalFuture<AffinityTopologyVersion> fut = aff.readyFuture(topVer);
 
-        return fut != null ? fut : new GridFinishedFuture<>(topVer);
+        return fut != null ? fut : cctx.shared().exchange().lastFinishedFuture();
     }
 
     /**


[11/37] ignite git commit: IGNITE-5784 .NET: QueryIndex.InlineSize

Posted by vo...@apache.org.
IGNITE-5784 .NET: QueryIndex.InlineSize

This closes #2771


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

Branch: refs/heads/ignite-2.3
Commit: c2cdbd986d06de07a40ef7f768dc884cb78f1484
Parents: 5d6ff97
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Sep 29 11:35:38 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Sep 29 11:35:38 2017 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/cache/QueryIndex.java     |  31 ++++-
 .../query/annotations/QueryGroupIndex.java      |   2 +-
 .../utils/PlatformConfigurationUtils.java       |   6 +
 .../Apache.Ignite.Core.Tests.csproj             |   3 +
 .../Cache/CacheConfigurationTest.cs             |  10 +-
 .../Query/CacheQueriesCodeConfigurationTest.cs  |  31 ++++-
 .../Cache/Query/CacheQueriesTest.cs             |  31 +++++
 .../Config/cache-query.xml                      |   1 +
 .../Config/full-config.xml                      | 134 +++++++++++++++++++
 .../IgniteConfigurationSerializerTest.cs        | 114 ++--------------
 .../IgniteConfigurationTest.cs                  |   1 +
 .../Cache/Configuration/CacheConfiguration.cs   |  13 ++
 .../Cache/Configuration/QueryEntity.cs          |   5 +-
 .../Cache/Configuration/QueryIndex.cs           |  50 ++++++-
 .../Configuration/QuerySqlFieldAttribute.cs     |  15 +++
 .../IgniteConfigurationSection.xsd              |  10 ++
 16 files changed, 338 insertions(+), 119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cdbd98/modules/core/src/main/java/org/apache/ignite/cache/QueryIndex.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryIndex.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryIndex.java
index f83f85c..8dc9c79 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/QueryIndex.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryIndex.java
@@ -21,6 +21,9 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.LinkedHashMap;
 import java.util.Objects;
+
+import org.apache.ignite.cache.query.annotations.QueryGroupIndex;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -252,16 +255,38 @@ public class QueryIndex implements Serializable {
     }
 
     /**
-     * Gets inline size.
+     * Gets index inline size in bytes. When enabled part of indexed value will be placed directly to index pages,
+     * thus minimizing data page accesses, thus increasing query performance.
+     * <p>
+     * Allowed values:
+     * <ul>
+     *     <li>{@code -1} (default) - determine inline size automatically (see below)</li>
+     *     <li>{@code 0} - index inline is disabled (not recommended)</li>
+     *     <li>positive value - fixed index inline</li>
+     * </ul>
+     * When set to {@code -1}, Ignite will try to detect inline size automatically. It will be no more than
+     * {@link CacheConfiguration#getSqlIndexMaxInlineSize()}. Index inline will be enabled for all fixed-length types,
+     * but <b>will not be enabled</b> for {@code String}.
      *
-     * @return inline size.
+     * @return Index inline size in bytes.
      */
     public int getInlineSize() {
         return inlineSize;
     }
 
     /**
-     * Sets inline size.
+     * Sets index inline size in bytes. When enabled part of indexed value will be placed directly to index pages,
+     * thus minimizing data page accesses, thus increasing query performance.
+     * <p>
+     * Allowed values:
+     * <ul>
+     *     <li>{@code -1} (default) - determine inline size automatically (see below)</li>
+     *     <li>{@code 0} - index inline is disabled (not recommended)</li>
+     *     <li>positive value - fixed index inline</li>
+     * </ul>
+     * When set to {@code -1}, Ignite will try to detect inline size automatically. It will be no more than
+     * {@link CacheConfiguration#getSqlIndexMaxInlineSize()}. Index inline will be enabled for all fixed-length types,
+     * but <b>will not be enabled</b> for {@code String}.
      *
      * @param inlineSize Inline size.
      * @return {@code this} for chaining.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cdbd98/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryGroupIndex.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryGroupIndex.java b/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryGroupIndex.java
index a947d0f..9670c32 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryGroupIndex.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryGroupIndex.java
@@ -39,7 +39,7 @@ public @interface QueryGroupIndex {
 
     /**
      * Index inline size in bytes. When enabled part of indexed value will be placed directly to index pages,
-     * thus minimizing data page accesses, thus incraesing query performance.
+     * thus minimizing data page accesses, thus increasing query performance.
      * <p>
      * Allowed values:
      * <ul>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cdbd98/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 aa4f168..e223193 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
@@ -194,6 +194,8 @@ public class PlatformConfigurationUtils {
         if (storeFactory != null)
             ccfg.setCacheStoreFactory(new PlatformDotNetCacheStoreFactoryNative(storeFactory));
 
+        ccfg.setSqlIndexMaxInlineSize(in.readInt());
+
         int qryEntCnt = in.readInt();
 
         if (qryEntCnt > 0) {
@@ -518,6 +520,7 @@ public class PlatformConfigurationUtils {
 
         res.setName(in.readString());
         res.setIndexType(QueryIndexType.values()[in.readByte()]);
+        res.setInlineSize(in.readInt());
 
         int cnt = in.readInt();
 
@@ -869,6 +872,8 @@ public class PlatformConfigurationUtils {
         else
             writer.writeObject(null);
 
+        writer.writeInt(ccfg.getSqlIndexMaxInlineSize());
+
         Collection<QueryEntity> qryEntities = ccfg.getQueryEntities();
 
         if (qryEntities != null) {
@@ -985,6 +990,7 @@ public class PlatformConfigurationUtils {
 
         writer.writeString(index.getName());
         writeEnumByte(writer, index.getIndexType());
+        writer.writeInt(index.getInlineSize());
 
         LinkedHashMap<String, Boolean> fields = index.getFields();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cdbd98/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 7f5f4b8..ec85ca2 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
@@ -357,6 +357,9 @@
     <Content Include="Config\Dynamic\dynamic-data.xml">
       <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
     </Content>
+    <Content Include="Config\full-config.xml">
+      <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
+    </Content>
     <Content Include="Config\Lifecycle\lifecycle-beans.xml">
       <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
     </Content>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cdbd98/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
index b400ef6..4e5d443 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
@@ -302,6 +302,7 @@ namespace Apache.Ignite.Core.Tests.Cache
             Assert.AreEqual(x.WriteBehindCoalescing, y.WriteBehindCoalescing);
             Assert.AreEqual(x.GroupName, y.GroupName);
             Assert.AreEqual(x.WriteSynchronizationMode, y.WriteSynchronizationMode);
+            Assert.AreEqual(x.SqlIndexMaxInlineSize, y.SqlIndexMaxInlineSize);
 
             if (x.ExpiryPolicyFactory != null)
             {
@@ -494,6 +495,7 @@ namespace Apache.Ignite.Core.Tests.Cache
 
             Assert.AreEqual(x.Name, y.Name);
             Assert.AreEqual(x.IndexType, y.IndexType);
+            Assert.AreEqual(x.InlineSize, y.InlineSize);
 
             AssertConfigsAreEqual(x.Fields, y.Fields);
         }
@@ -594,7 +596,8 @@ namespace Apache.Ignite.Core.Tests.Cache
                             new QueryIndex(new QueryIndexField("location", true))
                             {
                                 Name= "index2",
-                                IndexType = QueryIndexType.FullText
+                                IndexType = QueryIndexType.FullText,
+                                InlineSize = 1024
                             }
                         }
                     }
@@ -624,7 +627,8 @@ namespace Apache.Ignite.Core.Tests.Cache
                 EnableStatistics = true,
                 MemoryPolicyName = "myMemPolicy",
                 PartitionLossPolicy = PartitionLossPolicy.ReadOnlySafe,
-                PluginConfigurations = new[] { new MyPluginConfiguration() }
+                PluginConfigurations = new[] { new MyPluginConfiguration() },
+                SqlIndexMaxInlineSize = 10000
             };
         }
         /// <summary>
@@ -795,7 +799,7 @@ namespace Apache.Ignite.Core.Tests.Cache
 
             public void WriteBinary(IBinaryRawWriter writer)
             {
-                throw new NotImplementedException();
+                throw new NotSupportedException();
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cdbd98/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
index e25ddc0..7421b95 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
@@ -42,21 +42,35 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         {
             var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
             {
-                BinaryConfiguration = new BinaryConfiguration(typeof (QueryPerson)),
+                BinaryConfiguration = new BinaryConfiguration(typeof(QueryPerson)),
                 CacheConfiguration = new[]
                 {
-                    new CacheConfiguration(CacheName, new QueryEntity(typeof (int), typeof (QueryPerson))
+                    new CacheConfiguration(CacheName, new QueryEntity(typeof(int), typeof(QueryPerson))
                     {
                         TableName = "CustomTableName",
                         Fields = new[]
                         {
-                            new QueryField("Name", typeof (string)),
-                            new QueryField("Age", typeof (int)),
+                            new QueryField("Name", typeof(string)),
+                            new QueryField("Age", typeof(int)),
                             new QueryField("Birthday", typeof(DateTime)),
                         },
                         Indexes = new[]
                         {
-                            new QueryIndex(false, QueryIndexType.FullText, "Name"), new QueryIndex("Age")
+                            new QueryIndex
+                            {
+                                InlineSize = 2048,
+                                IndexType = QueryIndexType.FullText,
+                                Fields = new[]
+                                {
+                                    new QueryIndexField
+                                    {
+                                        IsDescending = false,
+                                        Name = "Name"
+                                    }
+                                }
+
+                            },
+                            new QueryIndex("Age")
                         }
                     })
                 }
@@ -122,6 +136,11 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             CollectionAssert.AreEquivalent(new[] {"GroupIndex1", "GroupIndex3"}, idx[1].Fields.Select(f => f.Name));
             CollectionAssert.AreEquivalent(new[] {"IndexedField1"}, idx[2].Fields.Select(f => f.Name));
             CollectionAssert.AreEquivalent(new[] {"FullTextField"}, idx[3].Fields.Select(f => f.Name));
+
+            Assert.AreEqual(-1, idx[0].InlineSize);
+            Assert.AreEqual(-1, idx[1].InlineSize);
+            Assert.AreEqual(513, idx[2].InlineSize);
+            Assert.AreEqual(-1, idx[3].InlineSize);
         }
 
         /// <summary>
@@ -298,7 +317,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             [QuerySqlField]
             public double SqlField { get; set; }
 
-            [QuerySqlField(IsIndexed = true, Name = "IndexedField1", IsDescending = true)]
+            [QuerySqlField(IsIndexed = true, Name = "IndexedField1", IsDescending = true, IndexInlineSize = 513)]
             public int IndexedField { get; set; }
 
             [QueryTextField]

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cdbd98/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
index 65ab18c..6361850 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
@@ -421,6 +421,37 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         }
 
         /// <summary>
+        /// Tests that query configuration propagates from Spring XML correctly.
+        /// </summary>
+        [Test]
+        public void TestQueryConfiguration()
+        {
+            var qe = Cache().GetConfiguration().QueryEntities.Single();
+
+            Assert.AreEqual(typeof(QueryPerson).FullName, qe.ValueTypeName);
+
+            var age = qe.Fields.First();
+            Assert.AreEqual("age", age.Name);
+            Assert.AreEqual(typeof(int), age.FieldType);
+            Assert.IsFalse(age.IsKeyField);
+
+            var name = qe.Fields.Last();
+            Assert.AreEqual("name", name.Name);
+            Assert.AreEqual(typeof(string), name.FieldType);
+            Assert.IsFalse(name.IsKeyField);
+
+            var textIdx = qe.Indexes.First();
+            Assert.AreEqual(QueryIndexType.FullText, textIdx.IndexType);
+            Assert.AreEqual("name", textIdx.Fields.Single().Name);
+            Assert.AreEqual(QueryIndex.DefaultInlineSize, textIdx.InlineSize);
+
+            var sqlIdx = qe.Indexes.Last();
+            Assert.AreEqual(QueryIndexType.Sorted, sqlIdx.IndexType);
+            Assert.AreEqual("age", sqlIdx.Fields.Single().Name);
+            Assert.AreEqual(2345, sqlIdx.InlineSize);
+        }
+
+        /// <summary>
         /// Check text query.
         /// </summary>
         [Test]

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cdbd98/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/cache-query.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/cache-query.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/cache-query.xml
index 2b41efa..3b61d8d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/cache-query.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/cache-query.xml
@@ -83,6 +83,7 @@
                                                 </map>
                                             </property>
                                             <property name="indexType" value="SORTED"/>
+                                            <property name="inlineSize" value="2345"/>
                                         </bean>
                                     </list>
                                 </property>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cdbd98/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
new file mode 100644
index 0000000..f6854e1
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
@@ -0,0 +1,134 @@
+<?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.
+-->
+
+<!-- Full XML config with all properties having custom values. -->
+
+<igniteConfiguration
+              workDirectory='c:' JvmMaxMemoryMb='1024' MetricsLogFrequency='0:0:10' isDaemon='true' 
+              isLateAffinityAssignment='false' springConfigUrl='c:\myconfig.xml' autoGenerateIgniteInstanceName='true' 
+              peerAssemblyLoadingMode='CurrentAppDomain' longQueryWarningTimeout='1:2:3' isActiveOnStart='false' 
+              consistentId='someId012'>
+    <localhost>127.1.1.1</localhost>
+    <binaryConfiguration compactFooter='false' keepDeserialized='true'>
+        <nameMapper type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+NameMapper' bar='testBar' />
+        <idMapper type='Apache.Ignite.Core.Tests.Binary.IdMapper' />
+        <types>
+            <string>Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+FooClass, Apache.Ignite.Core.Tests</string>
+        </types>
+        <typeConfigurations>
+            <binaryTypeConfiguration affinityKeyFieldName='affKeyFieldName' isEnum='true' keepDeserialized='True' typeName='typeName'>
+                <idMapper type='Apache.Ignite.Core.Tests.Binary.IdMapper, Apache.Ignite.Core.Tests' />
+                <nameMapper type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+NameMapper, Apache.Ignite.Core.Tests' />
+                <serializer type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+TestSerializer, Apache.Ignite.Core.Tests' />
+            </binaryTypeConfiguration>
+        </typeConfigurations>
+    </binaryConfiguration>
+    <discoverySpi type='TcpDiscoverySpi' joinTimeout='0:1:0' localAddress='192.168.1.1' localPort='6655'>
+        <ipFinder type='TcpDiscoveryMulticastIpFinder' addressRequestAttempts='7' />
+    </discoverySpi>
+    <communicationSpi type='TcpCommunicationSpi' ackSendThreshold='33' idleConnectionTimeout='0:1:2' />
+    <jvmOptions>
+        <string>-Xms1g</string>
+        <string>-Xmx4g</string>
+    </jvmOptions>
+    <lifecycleHandlers>
+        <iLifecycleHandler type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+LifecycleBean' foo='15' />
+    </lifecycleHandlers>
+    <cacheConfiguration>
+        <cacheConfiguration cacheMode='Replicated' readThrough='true' writeThrough='true' enableStatistics='true' writeBehindCoalescing='false' partitionLossPolicy='ReadWriteAll' groupName='fooGroup'>
+            <queryEntities>
+                <queryEntity keyType='System.Int32' valueType='System.String' tableName='myTable'>
+                    <fields>
+                        <queryField name='length' fieldType='System.Int32' isKeyField='true' />
+                    </fields>
+                    <aliases>
+                        <queryAlias fullName='somefield.field' alias='shortField' />
+                    </aliases>
+                    <indexes>
+                        <queryIndex name='idx' indexType='Geospatial' inlineSize='123'>
+                            <fields>
+                                <queryIndexField name='indexFld' isDescending='true' />
+                            </fields>
+                        </queryIndex>
+                    </indexes>
+                </queryEntity>
+            </queryEntities>
+            <evictionPolicy type='LruEvictionPolicy' batchSize='1' maxSize='2' maxMemorySize='3' />
+            <nearConfiguration nearStartSize='7'>
+                <evictionPolicy type='FifoEvictionPolicy' batchSize='10' maxSize='20' maxMemorySize='30' />
+            </nearConfiguration>
+            <affinityFunction type='RendezvousAffinityFunction' partitions='99' excludeNeighbors='true' />
+            <expiryPolicyFactory type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+MyPolicyFactory, Apache.Ignite.Core.Tests' />
+            <pluginConfigurations>
+                <iCachePluginConfiguration type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+MyPluginConfiguration, Apache.Ignite.Core.Tests' />
+            </pluginConfigurations>
+        </cacheConfiguration>
+        <cacheConfiguration name='secondCache' />
+    </cacheConfiguration>
+    <includedEventTypes>
+        <int>42</int>
+        <int>TaskFailed</int>
+        <int>JobFinished</int>
+    </includedEventTypes>
+    <userAttributes>
+        <pair key='myNode' value='true' />
+        <pair key='foo'>
+            <value type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+FooClass, Apache.Ignite.Core.Tests'>
+                <bar>Baz</bar>
+            </value>
+        </pair>
+    </userAttributes>
+    <atomicConfiguration backups='2' cacheMode='Local' atomicSequenceReserveSize='250' />
+    <transactionConfiguration defaultTransactionConcurrency='Optimistic' defaultTransactionIsolation='RepeatableRead' defaultTimeout='0:1:2' pessimisticTransactionLogSize='15' pessimisticTransactionLogLinger='0:0:33' />
+    <logger type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+TestLogger, Apache.Ignite.Core.Tests' />
+    <pluginConfigurations>
+        <iPluginConfiguration type='Apache.Ignite.Core.Tests.Plugin.TestIgnitePluginConfiguration, Apache.Ignite.Core.Tests' />
+    </pluginConfigurations>
+    <eventStorageSpi type='MemoryEventStorageSpi' expirationTimeout='00:00:23.45' maxEventCount='129' />
+    <memoryConfiguration concurrencyLevel='3' defaultMemoryPolicyName='dfPlc' pageSize='45' systemCacheInitialSize='67' systemCacheMaxSize='68'>
+        <memoryPolicies>
+            <memoryPolicyConfiguration emptyPagesPoolSize='1' evictionThreshold='0.2' name='dfPlc' pageEvictionMode='RandomLru' initialSize='89' maxSize='98' swapFilePath='abc' metricsEnabled='true' rateTimeInterval='0:1:2' subIntervals='9' />
+        </memoryPolicies>
+    </memoryConfiguration>
+    <sqlConnectorConfiguration host='bar' port='10' portRange='11' socketSendBufferSize='12' socketReceiveBufferSize='13' tcpNoDelay='true' maxOpenCursorsPerConnection='14' threadPoolSize='15' />
+    <clientConnectorConfiguration host='bar' port='10' portRange='11' socketSendBufferSize='12' socketReceiveBufferSize='13' tcpNoDelay='true' maxOpenCursorsPerConnection='14' threadPoolSize='15' />
+    <persistentStoreConfiguration alwaysWriteFullPages='true' checkpointingFrequency='00:00:1' checkpointingPageBufferSize='2' 
+                                  checkpointingThreads='3' lockWaitTime='00:00:04' persistentStorePath='foo' tlbSize='5' 
+                                  walArchivePath='bar' walFlushFrequency='00:00:06' walFsyncDelayNanos='7' walHistorySize='8' 
+                                  walMode='None' walRecordIteratorBufferSize='9' walSegments='10' walSegmentSize='11' 
+                                  walStorePath='baz' metricsEnabled='true' rateTimeInterval='0:0:6' subIntervals='3' 
+                                  checkpointWriteOrder='Random' writeThrottlingEnabled='true' />
+    <consistentId type='System.String'>someId012</consistentId>
+    <localEventListeners>
+        <localEventListener type='Apache.Ignite.Core.Events.LocalEventListener`1[[Apache.Ignite.Core.Events.CacheRebalancingEvent]]'>
+            <eventTypes>
+                <int>CacheObjectPut</int>
+                <int>81</int>
+            </eventTypes>
+            <listener type='Apache.Ignite.Core.Tests.EventsTestLocalListeners+Listener`1[[Apache.Ignite.Core.Events.CacheRebalancingEvent]]' />
+        </localEventListener>
+        <localEventListener type='Apache.Ignite.Core.Events.LocalEventListener`1[[Apache.Ignite.Core.Events.IEvent]]'>
+            <eventTypes>
+                <int>CacheObjectPut</int>
+                <int>81</int>
+            </eventTypes>
+            <listener type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+MyEventListener' />
+        </localEventListener>
+    </localEventListeners>
+</igniteConfiguration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cdbd98/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 83700cb..b04f466 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -67,101 +67,7 @@ namespace Apache.Ignite.Core.Tests
         [Test]
         public void TestPredefinedXml()
         {
-            var xml = @"<igniteConfig workDirectory='c:' JvmMaxMemoryMb='1024' MetricsLogFrequency='0:0:10' isDaemon='true' isLateAffinityAssignment='false' springConfigUrl='c:\myconfig.xml' autoGenerateIgniteInstanceName='true' peerAssemblyLoadingMode='CurrentAppDomain' longQueryWarningTimeout='1:2:3' isActiveOnStart='false' consistentId='someId012'>
-                            <localhost>127.1.1.1</localhost>
-                            <binaryConfiguration compactFooter='false' keepDeserialized='true'>
-                                <nameMapper type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+NameMapper' bar='testBar' />
-                                <idMapper type='Apache.Ignite.Core.Tests.Binary.IdMapper' />
-                                <types>
-                                    <string>Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+FooClass, Apache.Ignite.Core.Tests</string>
-                                </types>
-                                <typeConfigurations>
-                                    <binaryTypeConfiguration affinityKeyFieldName='affKeyFieldName' isEnum='true' keepDeserialized='True' typeName='typeName'>
-                                        <idMapper type='Apache.Ignite.Core.Tests.Binary.IdMapper, Apache.Ignite.Core.Tests' />
-                                        <nameMapper type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+NameMapper, Apache.Ignite.Core.Tests' />
-                                        <serializer type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+TestSerializer, Apache.Ignite.Core.Tests' />
-                                    </binaryTypeConfiguration>
-                                </typeConfigurations>
-                            </binaryConfiguration>
-                            <discoverySpi type='TcpDiscoverySpi' joinTimeout='0:1:0' localAddress='192.168.1.1' localPort='6655'>
-                                <ipFinder type='TcpDiscoveryMulticastIpFinder' addressRequestAttempts='7' />
-                            </discoverySpi>
-                            <communicationSpi type='TcpCommunicationSpi' ackSendThreshold='33' idleConnectionTimeout='0:1:2' />
-                            <jvmOptions><string>-Xms1g</string><string>-Xmx4g</string></jvmOptions>
-                            <lifecycleHandlers>
-                                <iLifecycleHandler type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+LifecycleBean' foo='15' />
-                            </lifecycleHandlers>
-                            <cacheConfiguration>
-                                <cacheConfiguration cacheMode='Replicated' readThrough='true' writeThrough='true' enableStatistics='true' writeBehindCoalescing='false' partitionLossPolicy='ReadWriteAll' groupName='fooGroup'>
-                                    <queryEntities>    
-                                        <queryEntity keyType='System.Int32' valueType='System.String' tableName='myTable'>
-                                            <fields>
-                                                <queryField name='length' fieldType='System.Int32' isKeyField='true' />
-                                            </fields>
-                                            <aliases>
-                                                <queryAlias fullName='somefield.field' alias='shortField' />
-                                            </aliases>
-                                            <indexes>
-                                                <queryIndex name='idx' indexType='Geospatial'>
-                                                    <fields>
-                                                        <queryIndexField name='indexFld' isDescending='true' />
-                                                    </fields>
-                                                </queryIndex>
-                                            </indexes>
-                                        </queryEntity>
-                                    </queryEntities>
-                                    <evictionPolicy type='LruEvictionPolicy' batchSize='1' maxSize='2' maxMemorySize='3' />
-                                    <nearConfiguration nearStartSize='7'>
-                                        <evictionPolicy type='FifoEvictionPolicy' batchSize='10' maxSize='20' maxMemorySize='30' />
-                                    </nearConfiguration>
-                                    <affinityFunction type='RendezvousAffinityFunction' partitions='99' excludeNeighbors='true' />
-                                    <expiryPolicyFactory type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+MyPolicyFactory, Apache.Ignite.Core.Tests' />
-                                    <pluginConfigurations><iCachePluginConfiguration type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+MyPluginConfiguration, Apache.Ignite.Core.Tests' /></pluginConfigurations>
-                                </cacheConfiguration>
-                                <cacheConfiguration name='secondCache' />
-                            </cacheConfiguration>
-                            <includedEventTypes>
-                                <int>42</int>
-                                <int>TaskFailed</int>
-                                <int>JobFinished</int>
-                            </includedEventTypes>
-                            <userAttributes>
-                                <pair key='myNode' value='true' />
-                                <pair key='foo'><value type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+FooClass, Apache.Ignite.Core.Tests'><bar>Baz</bar></value></pair>
-                            </userAttributes>
-                            <atomicConfiguration backups='2' cacheMode='Local' atomicSequenceReserveSize='250' />
-                            <transactionConfiguration defaultTransactionConcurrency='Optimistic' defaultTransactionIsolation='RepeatableRead' defaultTimeout='0:1:2' pessimisticTransactionLogSize='15' pessimisticTransactionLogLinger='0:0:33' />
-                            <logger type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+TestLogger, Apache.Ignite.Core.Tests' />
-                            <pluginConfigurations>
-                                <iPluginConfiguration type='Apache.Ignite.Core.Tests.Plugin.TestIgnitePluginConfiguration, Apache.Ignite.Core.Tests' />
-                            </pluginConfigurations>
-                            <eventStorageSpi type='MemoryEventStorageSpi' expirationTimeout='00:00:23.45' maxEventCount='129' />
-                            <memoryConfiguration concurrencyLevel='3' defaultMemoryPolicyName='dfPlc' pageSize='45' systemCacheInitialSize='67' systemCacheMaxSize='68'>
-                                <memoryPolicies>
-                                    <memoryPolicyConfiguration emptyPagesPoolSize='1' evictionThreshold='0.2' name='dfPlc' pageEvictionMode='RandomLru' initialSize='89' maxSize='98' swapFilePath='abc' metricsEnabled='true' rateTimeInterval='0:1:2' subIntervals='9' />
-                                </memoryPolicies>
-                            </memoryConfiguration>
-                            <sqlConnectorConfiguration host='bar' port='10' portRange='11' socketSendBufferSize='12' socketReceiveBufferSize='13' tcpNoDelay='true' maxOpenCursorsPerConnection='14' threadPoolSize='15' />
-                            <clientConnectorConfiguration host='bar' port='10' portRange='11' socketSendBufferSize='12' socketReceiveBufferSize='13' tcpNoDelay='true' maxOpenCursorsPerConnection='14' threadPoolSize='15' />
-                            <persistentStoreConfiguration alwaysWriteFullPages='true' checkpointingFrequency='00:00:1' checkpointingPageBufferSize='2' checkpointingThreads='3' lockWaitTime='00:00:04' persistentStorePath='foo' tlbSize='5' walArchivePath='bar' walFlushFrequency='00:00:06' walFsyncDelayNanos='7' walHistorySize='8' walMode='None' walRecordIteratorBufferSize='9' walSegments='10' walSegmentSize='11' walStorePath='baz' metricsEnabled='true' rateTimeInterval='0:0:6' subIntervals='3' checkpointWriteOrder='Random' writeThrottlingEnabled='true' />
-                            <consistentId type='System.String'>someId012</consistentId>
-                            <localEventListeners>
-                              <localEventListener type='Apache.Ignite.Core.Events.LocalEventListener`1[[Apache.Ignite.Core.Events.CacheRebalancingEvent]]'>
-                                <eventTypes>
-                                  <int>CacheObjectPut</int>
-                                  <int>81</int>
-                                </eventTypes>
-                                <listener type='Apache.Ignite.Core.Tests.EventsTestLocalListeners+Listener`1[[Apache.Ignite.Core.Events.CacheRebalancingEvent]]' />
-                            </localEventListener>
-                              <localEventListener type='Apache.Ignite.Core.Events.LocalEventListener`1[[Apache.Ignite.Core.Events.IEvent]]'>
-                                <eventTypes>
-                                  <int>CacheObjectPut</int>
-                                  <int>81</int>
-                                </eventTypes>
-                                <listener type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+MyEventListener' />
-                            </localEventListener>
-                          </localEventListeners>
-                        </igniteConfig>";
+            var xml = File.ReadAllText("Config\\full-config.xml");
 
             var cfg = IgniteConfiguration.FromXml(xml);
 
@@ -211,9 +117,12 @@ namespace Apache.Ignite.Core.Tests
             Assert.IsTrue(queryEntity.Fields.Single().IsKeyField);
             Assert.AreEqual("somefield.field", queryEntity.Aliases.Single().FullName);
             Assert.AreEqual("shortField", queryEntity.Aliases.Single().Alias);
-            Assert.AreEqual(QueryIndexType.Geospatial, queryEntity.Indexes.Single().IndexType);
-            Assert.AreEqual("indexFld", queryEntity.Indexes.Single().Fields.Single().Name);
-            Assert.AreEqual(true, queryEntity.Indexes.Single().Fields.Single().IsDescending);
+
+            var queryIndex = queryEntity.Indexes.Single();
+            Assert.AreEqual(QueryIndexType.Geospatial, queryIndex.IndexType);
+            Assert.AreEqual("indexFld", queryIndex.Fields.Single().Name);
+            Assert.AreEqual(true, queryIndex.Fields.Single().IsDescending);
+            Assert.AreEqual(123, queryIndex.InlineSize);
 
             var nearCfg = cacheCfg.NearConfiguration;
             Assert.IsNotNull(nearCfg);
@@ -695,7 +604,11 @@ namespace Apache.Ignite.Core.Tests
                                 },
                                 Indexes = new[]
                                 {
-                                    new QueryIndex("field") {IndexType = QueryIndexType.FullText}
+                                    new QueryIndex("field")
+                                    {
+                                        IndexType = QueryIndexType.FullText,
+                                        InlineSize = 32
+                                    }
                                 },
                                 Aliases = new[]
                                 {
@@ -747,7 +660,8 @@ namespace Apache.Ignite.Core.Tests
                         },
                         MemoryPolicyName = "somePolicy",
                         PartitionLossPolicy = PartitionLossPolicy.ReadOnlyAll,
-                        GroupName = "abc"
+                        GroupName = "abc",
+                        SqlIndexMaxInlineSize = 24
                     }
                 },
                 ClientMode = true,

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cdbd98/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 efa404d..3fd4772 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -92,6 +92,7 @@ namespace Apache.Ignite.Core.Tests
             CheckDefaultValueAttributes(new ClientConnectorConfiguration());
             CheckDefaultValueAttributes(new PersistentStoreConfiguration());
             CheckDefaultValueAttributes(new IgniteClientConfiguration());
+            CheckDefaultValueAttributes(new QueryIndex());
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cdbd98/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
index 9413ed5..c6b81f0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
@@ -137,6 +137,9 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// <summary> Default value for <see cref="PartitionLossPolicy"/>. </summary>
         public const PartitionLossPolicy DefaultPartitionLossPolicy = PartitionLossPolicy.Ignore;
 
+        /// <summary> Default value for <see cref="SqlIndexMaxInlineSize"/>. </summary>
+        public const int DefaultSqlIndexMaxInlineSize = -1;
+
         /// <summary>
         /// Gets or sets the cache name.
         /// </summary>
@@ -184,6 +187,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
             WriteBehindFlushThreadCount= DefaultWriteBehindFlushThreadCount;
             WriteBehindCoalescing = DefaultWriteBehindCoalescing;
             PartitionLossPolicy = DefaultPartitionLossPolicy;
+            SqlIndexMaxInlineSize = DefaultSqlIndexMaxInlineSize;
         }
 
         /// <summary>
@@ -286,6 +290,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
             PartitionLossPolicy = (PartitionLossPolicy) reader.ReadInt();
             GroupName = reader.ReadString();
             CacheStoreFactory = reader.ReadObject<IFactory<ICacheStore>>();
+            SqlIndexMaxInlineSize = reader.ReadInt();
 
             var count = reader.ReadInt();
             QueryEntities = count == 0
@@ -365,6 +370,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
             writer.WriteInt((int) PartitionLossPolicy);
             writer.WriteString(GroupName);
             writer.WriteObject(CacheStoreFactory);
+            writer.WriteInt(SqlIndexMaxInlineSize);
 
             if (QueryEntities != null)
             {
@@ -769,5 +775,12 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// Grouping caches reduces overall overhead, since internal data structures are shared.
         /// </summary>
         public string GroupName { get;set; }
+
+        /// <summary>
+        /// Gets or sets maximum inline size in bytes for sql indexes. See also <see cref="QueryIndex.InlineSize"/>.
+        /// -1 for automatic.
+        /// </summary>
+        [DefaultValue(DefaultSqlIndexMaxInlineSize)]
+        public int SqlIndexMaxInlineSize { get; set; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cdbd98/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
index 1273f08..b40231c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
@@ -454,7 +454,10 @@ namespace Apache.Ignite.Core.Cache.Configuration
                     if (attr.IsIndexed)
                     {
                         indexes.Add(new QueryIndexEx(columnName, attr.IsDescending, QueryIndexType.Sorted,
-                            attr.IndexGroups));
+                            attr.IndexGroups)
+                        {
+                            InlineSize = attr.IndexInlineSize
+                        });
                     }
 
                     fields.Add(new QueryField(columnName, memberInfo.Value) {IsKeyField = isKey});

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cdbd98/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryIndex.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryIndex.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryIndex.cs
index 7079606..d109111 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryIndex.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryIndex.cs
@@ -20,6 +20,8 @@ namespace Apache.Ignite.Core.Cache.Configuration
 {
     using System;
     using System.Collections.Generic;
+    using System.ComponentModel;
+    using System.Diagnostics.CodeAnalysis;
     using System.Linq;
     using Apache.Ignite.Core.Binary;
 
@@ -29,9 +31,17 @@ namespace Apache.Ignite.Core.Cache.Configuration
     public class QueryIndex
     {
         /// <summary>
+        /// Default value for <see cref="InlineSize"/>.
+        /// </summary>
+        public const int DefaultInlineSize = -1;
+
+        /** Inline size. */
+        private int _inlineSize = DefaultInlineSize;
+
+        /// <summary>
         /// Initializes a new instance of the <see cref="QueryIndex"/> class.
         /// </summary>
-        public QueryIndex()
+        public QueryIndex() : this((string[]) null)
         {
             // No-op.
         }
@@ -50,9 +60,10 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// </summary>
         /// <param name="isDescending">Sort direction.</param>
         /// <param name="fieldNames">Names of the fields to index.</param>
-        public QueryIndex(bool isDescending, params string[] fieldNames)
+        public QueryIndex(bool isDescending, params string[] fieldNames) 
+            : this(isDescending, QueryIndexType.Sorted, fieldNames)
         {
-            Fields = fieldNames.Select(f => new QueryIndexField(f, isDescending)).ToArray();
+            // No-op.
         }
 
         /// <summary>
@@ -62,8 +73,12 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// <param name="indexType">Type of the index.</param>
         /// <param name="fieldNames">Names of the fields to index.</param>
         public QueryIndex(bool isDescending, QueryIndexType indexType, params string[] fieldNames) 
-            : this(isDescending, fieldNames)
         {
+            if (fieldNames != null)
+            {
+                Fields = fieldNames.Select(f => new QueryIndexField(f, isDescending)).ToArray();
+            }
+
             IndexType = indexType;
         }
 
@@ -96,7 +111,30 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// <summary>
         /// Gets or sets a collection of fields to be indexed.
         /// </summary>
-        public ICollection<QueryIndexField> Fields { get; private set; }
+        [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")]
+        public ICollection<QueryIndexField> Fields { get; set; }
+
+        /// <summary>
+        /// Gets index inline size in bytes. When enabled part of indexed value will be placed directly to index pages,
+        /// thus minimizing data page accesses and increasing query performance.
+        /// <para />
+        /// Allowed values:
+        /// <ul>
+        /// <li><c>-1</c> (default) - determine inline size automatically(see below)</li>
+        /// <li><c>0</c> - index inline is disabled(not recommended)</li>
+        /// <li>positive value - fixed index inline</li >
+        /// </ul>
+        /// When set to <c>-1</c>, Ignite will try to detect inline size automatically. It will be no more than
+        /// <see cref="CacheConfiguration.SqlIndexMaxInlineSize"/>.
+        /// Index inline will be enabled for all fixed-length types,
+        ///  but <b>will not be enabled</b> for <see cref="string"/>.
+        /// </summary>
+        [DefaultValue(DefaultInlineSize)]
+        public int InlineSize
+        {
+            get { return _inlineSize; }
+            set { _inlineSize = value; }
+        }
 
         /// <summary>
         /// Initializes a new instance of the <see cref="QueryIndex"/> class.
@@ -106,6 +144,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
         {
             Name = reader.ReadString();
             IndexType = (QueryIndexType) reader.ReadByte();
+            InlineSize = reader.ReadInt();
 
             var count = reader.ReadInt();
             Fields = count == 0 ? null : Enumerable.Range(0, count).Select(x =>
@@ -119,6 +158,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
         {
             writer.WriteString(Name);
             writer.WriteByte((byte) IndexType);
+            writer.WriteInt(InlineSize);
 
             if (Fields != null)
             {

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cdbd98/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QuerySqlFieldAttribute.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QuerySqlFieldAttribute.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QuerySqlFieldAttribute.cs
index a522115..b920304 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QuerySqlFieldAttribute.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QuerySqlFieldAttribute.cs
@@ -18,6 +18,7 @@
 namespace Apache.Ignite.Core.Cache.Configuration
 {
     using System;
+    using System.ComponentModel;
     using System.Diagnostics.CodeAnalysis;
 
     /// <summary>
@@ -29,6 +30,14 @@ namespace Apache.Ignite.Core.Cache.Configuration
     public sealed class QuerySqlFieldAttribute : Attribute
     {
         /// <summary>
+        /// Initializes a new instance of the <see cref="QuerySqlFieldAttribute"/> class.
+        /// </summary>
+        public QuerySqlFieldAttribute()
+        {
+            IndexInlineSize = QueryIndex.DefaultInlineSize;
+        }
+
+        /// <summary>
         /// Gets or sets the sql field name.
         /// If not provided, property or field name will be used.
         /// </summary>
@@ -56,5 +65,11 @@ namespace Apache.Ignite.Core.Cache.Configuration
         [SuppressMessage("Microsoft.Performance", "CA1819:PropertiesShouldNotReturnArrays", 
             Justification = "Attribute initializers do not allow collections")]
         public string[] IndexGroups { get; set; }
+
+        /// <summary>
+        /// Gets or sets the index inline size, see <see cref="QueryIndex.InlineSize"/>.
+        /// </summary>
+        [DefaultValue(QueryIndex.DefaultInlineSize)]
+        public int IndexInlineSize { get; set; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cdbd98/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 6788ef0..730cb9f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -399,6 +399,11 @@
                                                                                             <xs:documentation>Index type.</xs:documentation>
                                                                                         </xs:annotation>
                                                                                     </xs:attribute>
+                                                                                    <xs:attribute name="inlineSize" type="xs:int">
+                                                                                        <xs:annotation>
+                                                                                            <xs:documentation>Index inline size in bytes. When enabled part of indexed value will be placed directly to index pages, thus minimizing data page accesses and increasing query performance.</xs:documentation>
+                                                                                        </xs:annotation>
+                                                                                    </xs:attribute>
                                                                                 </xs:complexType>
                                                                             </xs:element>
                                                                         </xs:sequence>
@@ -764,6 +769,11 @@
                                             <xs:documentation>Cache group name. Caches with the same group name share single underlying 'physical' cache (partition set), but are logically isolated.</xs:documentation>
                                         </xs:annotation>
                                     </xs:attribute>
+                                    <xs:attribute name="sqlIndexMaxInlineSize" type="xs:int">
+                                        <xs:annotation>
+                                            <xs:documentation>Maximum size of SQL inline index. Part of indexed value will be placed directly to index pages, thus minimizing data page accesses and increasing query performance.</xs:documentation>
+                                        </xs:annotation>
+                                    </xs:attribute>
                                 </xs:complexType>
                             </xs:element>
                         </xs:sequence>


[25/37] ignite git commit: IGNITE-6485: Binary marshalling with writeReplace/readResolve fixed. This closes #2778.

Posted by vo...@apache.org.
IGNITE-6485: Binary marshalling with writeReplace/readResolve fixed. This closes #2778.


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

Branch: refs/heads/ignite-2.3
Commit: d7fbbd546126252af45cb48a5d173a7a66ad9838
Parents: 9160d5e
Author: Andrey Gura <ag...@apache.org>
Authored: Mon Oct 2 12:38:34 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Oct 2 12:38:34 2017 +0300

----------------------------------------------------------------------
 .../internal/binary/BinaryClassDescriptor.java  |   4 +-
 .../ignite/internal/util/IgniteUtils.java       |  43 ++++--
 .../binary/BinaryMarshallerSelfTest.java        | 136 +++++++++++++------
 3 files changed, 130 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d7fbbd54/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
index 4950a53..935211e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
@@ -344,9 +344,9 @@ public class BinaryClassDescriptor {
         Method writeReplaceMthd;
 
         if (mode == BinaryWriteMode.BINARY || mode == BinaryWriteMode.OBJECT) {
-            readResolveMtd = U.findNonPublicMethod(cls, "readResolve");
+            readResolveMtd = U.getNonPublicMethod(cls, "readResolve");
 
-            writeReplaceMthd = U.findNonPublicMethod(cls, "writeReplace");
+            writeReplaceMthd = U.getNonPublicMethod(cls, "writeReplace");
         }
         else {
             readResolveMtd = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7fbbd54/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 31b556d..bdcf87e 100755
--- 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
@@ -9407,22 +9407,14 @@ public abstract class IgniteUtils {
      * @param cls The class to search,
      * @param name Name of the method.
      * @param paramTypes Method parameters.
-     * @return Method or {@code null}
+     * @return Method or {@code null}.
      */
     @Nullable public static Method findNonPublicMethod(Class<?> cls, String name, Class<?>... paramTypes) {
         while (cls != null) {
-            try {
-                Method mtd = cls.getDeclaredMethod(name, paramTypes);
+            Method mtd = getNonPublicMethod(cls, name, paramTypes);
 
-                if (mtd.getReturnType() != void.class) {
-                    mtd.setAccessible(true);
-
-                    return mtd;
-                }
-            }
-            catch (NoSuchMethodException ignored) {
-                // No-op.
-            }
+            if (mtd != null)
+                return mtd;
 
             cls = cls.getSuperclass();
         }
@@ -9431,6 +9423,33 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * Gets a method from the class.
+     *
+     * Method.getMethod() does not return non-public method.
+     *
+     * @param cls Target class.
+     * @param name Name of the method.
+     * @param paramTypes Method parameters.
+     * @return Method or {@code null}.
+     */
+    @Nullable public static Method getNonPublicMethod(Class<?> cls, String name, Class<?>... paramTypes) {
+        try {
+            Method mtd = cls.getDeclaredMethod(name, paramTypes);
+
+            if (mtd.getReturnType() != void.class) {
+                mtd.setAccessible(true);
+
+                return mtd;
+            }
+        }
+        catch (NoSuchMethodException ignored) {
+            // No-op.
+        }
+
+        return null;
+    }
+
+    /**
      * @param cls The class to search.
      * @param name Name of a field to get.
      * @return Field or {@code null}.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7fbbd54/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 926b3c0..ef68cd1 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
@@ -938,6 +938,23 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testWriteReplace() throws Exception {
+        BinaryMarshaller marsh = binaryMarshaller(Collections.singleton(
+            new BinaryTypeConfiguration(TestObject.class.getName())
+        ));
+
+        TestObject obj = new TestObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        assertEquals(obj, po.deserialize());
+
+        assertEquals(obj.val, ((BinaryObject)po.field("val")).deserialize());
+    }
+
+    /**
      *
      */
     private static class EnclosingObj implements Serializable {
@@ -2626,19 +2643,6 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @throws Exception If ecxeption thrown.
-     */
-    public void testDeclareReadResolveInParent() throws Exception {
-        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(new BinaryTypeConfiguration(ChildBinary.class.getName())));
-
-        BinaryObjectImpl binaryObj = marshal(new ChildBinary(), marsh);
-
-        ChildBinary singleton = binaryObj.deserialize();
-
-        assertNotNull(singleton.s);
-    }
-
-    /**
      *
      */
     public void testDecimalFields() throws Exception {
@@ -5117,13 +5121,6 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     /**
      *
      */
-    public static class ChildBinary extends ParentBinary {
-
-    }
-
-    /**
-     *
-     */
     public static class SimpleEnclosingObject {
         /** */
         private Object simpl;
@@ -5180,25 +5177,6 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     *
-     */
-    private static class ParentBinary {
-        /** */
-        public String s;
-
-        /**
-         * Package only visibility!!!!
-         *
-         * @return Object.
-         */
-        Object readResolve() {
-            s = "readResolve";
-
-            return this;
-        }
-    }
-
-    /**
      * Class B for duplicate fields test.
      */
     private static class DuplicateFieldsA {
@@ -5517,4 +5495,84 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
         abstract boolean isSupported();
     }
+
+    /** */
+    interface Intf {
+        /** */
+        long value();
+    }
+
+    /** */
+    static class TestObject {
+        /** Value. */
+        Intf val = new IntfImpl();
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestObject obj = (TestObject)o;
+
+            return val.equals(obj.val);
+        }
+    }
+
+    /** */
+    static class IntfImpl extends Cls implements Intf {
+        /** {@inheritDoc} */
+        @Override public long value() {
+            return longValue();
+        }
+    }
+
+    /** */
+    static class Cls implements Serializable {
+        /** Value. */
+        long val;
+
+        /** */
+        public long longValue() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            Cls cls = (Cls)o;
+
+            return val == cls.val;
+        }
+
+        /** */
+        private Object writeReplace() {
+            return new SerializationProxy(this);
+        }
+
+        /** */
+        private static class SerializationProxy implements Serializable {
+            /** Value. */
+            private final long val;
+
+            /** */
+            SerializationProxy(Cls a) {
+                val = a.longValue();
+            }
+
+            /** */
+            private Object readResolve() {
+                Cls a = new Cls();
+
+                a.val = val;
+
+                return a;
+            }
+        }
+    }
 }
\ No newline at end of file


[31/37] ignite git commit: IGNITE-6422 Visor CMD: Fixed cache statistics output.

Posted by vo...@apache.org.
IGNITE-6422 Visor CMD: Fixed cache statistics output.


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

Branch: refs/heads/ignite-2.3
Commit: 16d23700d25961aae8d75e11c55349fab78f2a79
Parents: 5764960
Author: vsisko <vs...@gridgain.com>
Authored: Mon Oct 2 23:08:40 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Mon Oct 2 23:08:40 2017 +0700

----------------------------------------------------------------------
 .../cache/VisorCacheAggregatedMetrics.java      | 24 ++++++-------
 .../commands/cache/VisorCacheCommand.scala      | 38 ++++++++++----------
 2 files changed, 31 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/16d23700/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java
index ccefa67..a0258cc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java
@@ -208,35 +208,35 @@ public class VisorCacheAggregatedMetrics extends VisorDataTransferObject {
 
     /**
      * @param metric Metrics to process.
-     * @return Off heap entries count.
+     * @return Off heap primary entries count.
      */
-    private long getOffHeapEntriesCount(VisorCacheMetrics metric) {
-        return metric.getOffHeapEntriesCount();
+    private long getOffHeapPrimaryEntriesCount(VisorCacheMetrics metric) {
+        return metric.getOffHeapPrimaryEntriesCount();
     }
 
     /**
-     * @return Minimum number of elements in off heap.
+     * @return Minimum number of primary elements in off heap.
      */
-    public long getMinimumOffHeapSize() {
+    public long getMinimumOffHeapPrimarySize() {
         if (minOffHeapSize == null) {
             minOffHeapSize = Long.MAX_VALUE;
 
             for (VisorCacheMetrics metric : metrics.values())
-                minOffHeapSize = Math.min(minOffHeapSize, getOffHeapEntriesCount(metric));
+                minOffHeapSize = Math.min(minOffHeapSize, getOffHeapPrimaryEntriesCount(metric));
         }
 
         return minOffHeapSize;
     }
 
     /**
-     * @return Average number of elements in off heap.
+     * @return Average number of primary elements in off heap.
      */
-    public double getAverageOffHeapSize() {
+    public double getAverageOffHeapPrimarySize() {
         if (avgOffHeapSize == null) {
             avgOffHeapSize = 0.0d;
 
             for (VisorCacheMetrics metric : metrics.values())
-                avgOffHeapSize += getOffHeapEntriesCount(metric);
+                avgOffHeapSize += getOffHeapPrimaryEntriesCount(metric);
 
             avgOffHeapSize /= metrics.size();
         }
@@ -245,14 +245,14 @@ public class VisorCacheAggregatedMetrics extends VisorDataTransferObject {
     }
 
     /**
-     * @return Maximum number of elements in off heap in the cache.
+     * @return Maximum number of primary elements in off heap.
      */
-    public long getMaximumOffHeapSize() {
+    public long getMaximumOffHeapPrimarySize() {
         if (maxOffHeapSize == null) {
             maxOffHeapSize = Long.MIN_VALUE;
 
             for (VisorCacheMetrics metric : metrics.values())
-                maxOffHeapSize = Math.max(maxOffHeapSize, getOffHeapEntriesCount(metric));
+                maxOffHeapSize = Math.max(maxOffHeapSize, getOffHeapPrimaryEntriesCount(metric));
         }
 
         return maxOffHeapSize;

http://git-wip-us.apache.org/repos/asf/ignite/blob/16d23700/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
index f754275..dde3289 100755
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
@@ -328,12 +328,12 @@ class VisorCacheCommand {
                         ad.getMode,
                         ad.getNodes.size(),
                         (
-                            "min: " + (ad.getMinimumHeapSize + ad.getMinimumOffHeapSize) +
-                                " (" + ad.getMinimumHeapSize + " / " + ad.getMinimumOffHeapSize + ")",
-                            "avg: " + formatDouble(ad.getAverageHeapSize + ad.getAverageOffHeapSize) +
-                                " (" + formatDouble(ad.getAverageHeapSize) + " / " + formatDouble(ad.getAverageOffHeapSize) + ")",
-                            "max: " + (ad.getMaximumHeapSize + ad.getMaximumOffHeapSize) +
-                                " (" + ad.getMaximumHeapSize + " / " + ad.getMaximumOffHeapSize + ")"
+                            "min: " + (ad.getMinimumHeapSize + ad.getMinimumOffHeapPrimarySize) +
+                                " (" + ad.getMinimumHeapSize + " / " + ad.getMinimumOffHeapPrimarySize + ")",
+                            "avg: " + formatDouble(ad.getAverageHeapSize + ad.getAverageOffHeapPrimarySize) +
+                                " (" + formatDouble(ad.getAverageHeapSize) + " / " + formatDouble(ad.getAverageOffHeapPrimarySize) + ")",
+                            "max: " + (ad.getMaximumHeapSize + ad.getMaximumOffHeapPrimarySize) +
+                                " (" + ad.getMaximumHeapSize + " / " + ad.getMaximumOffHeapPrimarySize + ")"
                             ),
                         (
                             "min: " + ad.getMinimumHits,
@@ -385,13 +385,13 @@ class VisorCacheCommand {
 
                     csT += ("Name(@)", cacheNameVar)
                     csT += ("Nodes", m.size())
-                    csT += ("Total size Min/Avg/Max", (ad.getMinimumHeapSize + ad.getMinimumOffHeapSize) + " / " +
-                        formatDouble(ad.getAverageHeapSize + ad.getAverageOffHeapSize) + " / " +
-                        (ad.getMaximumHeapSize + ad.getMaximumOffHeapSize))
+                    csT += ("Total size Min/Avg/Max", (ad.getMinimumHeapSize + ad.getMinimumOffHeapPrimarySize) + " / " +
+                        formatDouble(ad.getAverageHeapSize + ad.getAverageOffHeapPrimarySize) + " / " +
+                        (ad.getMaximumHeapSize + ad.getMaximumOffHeapPrimarySize))
                     csT += ("  Heap size Min/Avg/Max", ad.getMinimumHeapSize + " / " +
                         formatDouble(ad.getAverageHeapSize) + " / " + ad.getMaximumHeapSize)
-                    csT += ("  Off-heap size Min/Avg/Max", ad.getMinimumOffHeapSize + " / " +
-                        formatDouble(ad.getAverageOffHeapSize) + " / " + ad.getMaximumOffHeapSize)
+                    csT += ("  Off-heap size Min/Avg/Max", ad.getMinimumOffHeapPrimarySize + " / " +
+                        formatDouble(ad.getAverageOffHeapPrimarySize) + " / " + ad.getMaximumOffHeapPrimarySize)
 
                     val ciT = VisorTextTable()
 
@@ -408,9 +408,9 @@ class VisorCacheCommand {
                             formatDouble(nm.getCurrentCpuLoad * 100d) + " %",
                             X.timeSpan2HMSM(nm.getUpTime),
                             (
-                                "Total: " + (cm.getHeapEntriesCount + cm.getOffHeapEntriesCount()),
+                                "Total: " + (cm.getHeapEntriesCount + cm.getOffHeapPrimaryEntriesCount),
                                 "  Heap: " + cm.getHeapEntriesCount,
-                                "  Off-Heap: " + cm.getOffHeapEntriesCount(),
+                                "  Off-Heap: " + cm.getOffHeapPrimaryEntriesCount,
                                 "  Off-Heap Memory: " + formatMemory(cm.getOffHeapAllocatedSize)
                             ),
                             (
@@ -644,12 +644,12 @@ class VisorCacheCommand {
                 mkCacheName(ad.getName),
                 ad.getMode,
                 (
-                    "min: " + (ad.getMinimumHeapSize + ad.getMinimumOffHeapSize) +
-                        " (" + ad.getMinimumHeapSize + " / " + ad.getMinimumOffHeapSize + ")",
-                    "avg: " + formatDouble(ad.getAverageHeapSize + ad.getAverageOffHeapSize) +
-                        " (" + formatDouble(ad.getAverageHeapSize) + " / " + formatDouble(ad.getAverageOffHeapSize) + ")",
-                    "max: " + (ad.getMaximumHeapSize + ad.getMaximumOffHeapSize) +
-                        " (" + ad.getMaximumHeapSize + " / " + ad.getMaximumOffHeapSize + ")"
+                    "min: " + (ad.getMinimumHeapSize + ad.getMinimumOffHeapPrimarySize) +
+                        " (" + ad.getMinimumHeapSize + " / " + ad.getMinimumOffHeapPrimarySize + ")",
+                    "avg: " + formatDouble(ad.getAverageHeapSize + ad.getAverageOffHeapPrimarySize) +
+                        " (" + formatDouble(ad.getAverageHeapSize) + " / " + formatDouble(ad.getAverageOffHeapPrimarySize) + ")",
+                    "max: " + (ad.getMaximumHeapSize + ad.getMaximumOffHeapPrimarySize) +
+                        " (" + ad.getMaximumHeapSize + " / " + ad.getMaximumOffHeapPrimarySize + ")"
                 ))
         })
 


[22/37] ignite git commit: ignite-2.2.0 Update version.

Posted by vo...@apache.org.
ignite-2.2.0 Update version.


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

Branch: refs/heads/ignite-2.3
Commit: 59ee8af5ce8076a3c97ef69765075cbe9e5694d0
Parents: 55886b1
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Mon Oct 2 14:07:40 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Mon Oct 2 14:07:40 2017 +0700

----------------------------------------------------------------------
 .../frontend/app/modules/agent/AgentManager.service.js       | 4 +++-
 .../web-console/frontend/app/modules/sql/sql.controller.js   | 8 ++++----
 modules/web-console/frontend/app/services/Version.service.js | 6 +++---
 3 files changed, 10 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/59ee8af5/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
index e4bde6f..20d2976 100644
--- a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
+++ b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
@@ -26,6 +26,8 @@ const State = {
     CONNECTED: 'CONNECTED'
 };
 
+const LAZY_QUERY_SINCE = [['2.1.4-p1', '2.2.0'], '2.2.1'];
+
 class ConnectionState {
     constructor(cluster) {
         this.agents = [];
@@ -499,7 +501,7 @@ export default class IgniteAgentManager {
      */
     querySql(nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, pageSz, lazy) {
         if (this.available('2.0.0')) {
-            const task = this.available('2.1.4-p1') ?
+            const task = this.available(...LAZY_QUERY_SINCE) ?
                 this.visorTask('querySqlX2', nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, pageSz, lazy) :
                 this.visorTask('querySqlX2', nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, pageSz);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/59ee8af5/modules/web-console/frontend/app/modules/sql/sql.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/sql.controller.js b/modules/web-console/frontend/app/modules/sql/sql.controller.js
index 420c4b9..5f06c1e 100644
--- a/modules/web-console/frontend/app/modules/sql/sql.controller.js
+++ b/modules/web-console/frontend/app/modules/sql/sql.controller.js
@@ -28,9 +28,9 @@ const ROW_IDX = {value: -2, type: 'java.lang.Integer', label: 'ROW_IDX'};
 
 const NON_COLLOCATED_JOINS_SINCE = '1.7.0';
 
-const ENFORCE_JOIN_VERS = [['1.7.9', '1.8.0'], ['1.8.4', '1.9.0'], ['1.9.1']];
+const ENFORCE_JOIN_SINCE = [['1.7.9', '1.8.0'], ['1.8.4', '1.9.0'], '1.9.1'];
 
-const LAZY_QUERY_VERS = ['2.1.4-p1'];
+const LAZY_QUERY_SINCE = [['2.1.4-p1', '2.2.0'], '2.2.1'];
 
 const _fullColName = (col) => {
     const res = [];
@@ -1367,7 +1367,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
             const cache = _.find($scope.caches, {name: paragraph.cacheName});
 
             if (cache)
-                return !!_.find(cache.nodes, (node) => Version.since(node.version, ...ENFORCE_JOIN_VERS));
+                return !!_.find(cache.nodes, (node) => Version.since(node.version, ...ENFORCE_JOIN_SINCE));
 
             return false;
         };
@@ -1376,7 +1376,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
             const cache = _.find($scope.caches, {name: paragraph.cacheName});
 
             if (cache)
-                return !!_.find(cache.nodes, (node) => Version.since(node.version, ...LAZY_QUERY_VERS));
+                return !!_.find(cache.nodes, (node) => Version.since(node.version, ...LAZY_QUERY_SINCE));
 
             return false;
         };

http://git-wip-us.apache.org/repos/asf/ignite/blob/59ee8af5/modules/web-console/frontend/app/services/Version.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/services/Version.service.js b/modules/web-console/frontend/app/services/Version.service.js
index 8ae23d3..beb27fa 100644
--- a/modules/web-console/frontend/app/services/Version.service.js
+++ b/modules/web-console/frontend/app/services/Version.service.js
@@ -73,12 +73,12 @@ const compare = (a, b) => {
 
 export default class IgniteVersion {
     constructor() {
-        this.webConsole = '2.1.0';
+        this.webConsole = '2.2.0';
 
         this.supportedVersions = [
             {
-                label: 'Ignite 2.1',
-                ignite: '2.1.0'
+                label: 'Ignite 2.x',
+                ignite: '2.2.0'
             },
             {
                 label: 'Ignite 2.0',


[17/37] ignite git commit: Static IP finder added to IgniteWalSerializerVersionTest test case

Posted by vo...@apache.org.
Static IP finder added to IgniteWalSerializerVersionTest test case


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

Branch: refs/heads/ignite-2.3
Commit: 0ab17ab7bad025543591bef2efff176174696bfa
Parents: 0749ec4
Author: Andrey Gura <ag...@apache.org>
Authored: Fri Sep 29 16:33:03 2017 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Fri Sep 29 16:33:03 2017 +0300

----------------------------------------------------------------------
 .../persistence/db/wal/IgniteWalSerializerVersionTest.java   | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0ab17ab7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java
index 6d9d80e..f31d0f9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java
@@ -27,6 +27,9 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.Re
 import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV2Serializer;
 import org.apache.ignite.internal.util.typedef.internal.GPC;
 import org.apache.ignite.internal.util.typedef.internal.U;
+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.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
@@ -36,10 +39,15 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_SERIALIZER_VER
  *
  */
 public class IgniteWalSerializerVersionTest extends GridCommonAbstractTest {
+    /** Ip finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(name);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
         cfg.setPersistentStoreConfiguration(new PersistentStoreConfiguration());
 
         return cfg;


[33/37] ignite git commit: IGNITE-6546 Fixed faveicon.

Posted by vo...@apache.org.
IGNITE-6546 Fixed faveicon.


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

Branch: refs/heads/ignite-2.3
Commit: f42f34b8dc293e2077ea99a49136d3e2d439e832
Parents: 3256ee2
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Tue Oct 3 14:33:10 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Tue Oct 3 14:33:10 2017 +0700

----------------------------------------------------------------------
 .../rest/protocols/http/jetty/favicon.ico        | Bin 1406 -> 1150 bytes
 1 file changed, 0 insertions(+), 0 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f42f34b8/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/favicon.ico
----------------------------------------------------------------------
diff --git a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/favicon.ico b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/favicon.ico
index 6f6ed44..b36f8d7 100644
Binary files a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/favicon.ico and b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/favicon.ico differ


[29/37] ignite git commit: .NET: Suppress some false VS2017 warnings

Posted by vo...@apache.org.
.NET: Suppress some false VS2017 warnings


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

Branch: refs/heads/ignite-2.3
Commit: 5ca7909fdfb129eed1a58650f57d7dd968b1e287
Parents: 7aaacd8
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Oct 2 16:34:12 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Oct 2 16:34:12 2017 +0300

----------------------------------------------------------------------
 modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs        | 2 ++
 .../Apache.Ignite.Core/Impl/Cache/Query/QueryCursorBase.cs     | 3 +++
 .../Impl/Client/Cache/Query/ClientQueryCursor.cs               | 3 +++
 .../dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs      | 2 ++
 .../dotnet/Apache.Ignite.Core/Impl/PlatformJniTarget.cs        | 6 ++++++
 .../Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs    | 1 +
 6 files changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5ca7909f/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
index 886dee9..4e5eae5 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
@@ -454,6 +454,8 @@ namespace Apache.Ignite.Core
         /// </summary>
         /// <param name="interopProc">Interop processor.</param>
         /// <param name="stream">Stream.</param>
+        [SuppressMessage("Microsoft.Reliability", "CA2000:Dispose objects before losing scope",
+            Justification = "PlatformJniTarget is passed further")]
         internal static void OnStart(IUnmanagedTarget interopProc, IBinaryStream stream)
         {
             try

http://git-wip-us.apache.org/repos/asf/ignite/blob/5ca7909f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/QueryCursorBase.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/QueryCursorBase.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/QueryCursorBase.cs
index 8f00a2e..216d7ea 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/QueryCursorBase.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/QueryCursorBase.cs
@@ -21,6 +21,7 @@ namespace Apache.Ignite.Core.Impl.Cache.Query
     using System.Collections;
     using System.Collections.Generic;
     using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
     using Apache.Ignite.Core.Cache.Query;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Binary.IO;
@@ -63,6 +64,8 @@ namespace Apache.Ignite.Core.Impl.Cache.Query
         /// <param name="marsh">Marshaller.</param>
         /// <param name="keepBinary">Keep binary flag.</param>
         /// <param name="initialBatchStream">Optional stream with initial batch.</param>
+        [SuppressMessage("Microsoft.Usage", "CA2214:DoNotCallOverridableMethodsInConstructors",
+            Justification = "ConvertGetBatch calls Read, which does not rely on constructor being run.")]
         protected QueryCursorBase(Marshaller marsh, bool keepBinary, IBinaryStream initialBatchStream = null)
         {
             Debug.Assert(marsh != null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5ca7909f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs
index 898bbbe..75aa5df 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs
@@ -18,6 +18,8 @@
 namespace Apache.Ignite.Core.Impl.Client.Cache.Query
 {
     using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
     using System.Linq;
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Impl.Binary;
@@ -65,6 +67,7 @@ namespace Apache.Ignite.Core.Impl.Client.Cache.Query
         }
 
         /** <inheritdoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         protected override ICacheEntry<TK, TV> Read(BinaryReader reader)
         {
             return new CacheEntry<TK, TV>(reader.ReadObject<TK>(), reader.ReadObject<TV>());

http://git-wip-us.apache.org/repos/asf/ignite/blob/5ca7909f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
index 4eb53dd..e565f31 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
@@ -201,6 +201,8 @@ namespace Apache.Ignite.Core.Impl.Client
         /// <summary>
         /// Connects the socket.
         /// </summary>
+        [SuppressMessage("Microsoft.Reliability", "CA2000:Dispose objects before losing scope", 
+            Justification = "Socket is returned from this method.")]
         private static Socket Connect(IgniteClientConfiguration cfg)
         {
             List<Exception> errors = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/5ca7909f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformJniTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformJniTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformJniTarget.cs
index 725c112..c7d9ab4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformJniTarget.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformJniTarget.cs
@@ -174,6 +174,7 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
+        [SuppressMessage("Microsoft.Reliability", "CA2000:Dispose objects before losing scope")]
         public unsafe TR InObjectStreamOutObjectStream<TR>(int type, Action<IBinaryStream> writeAction, 
             Func<IBinaryStream, IPlatformTargetInternal, TR> readAction, IPlatformTargetInternal arg)
         {
@@ -244,6 +245,7 @@ namespace Apache.Ignite.Core.Impl
         /// <param name="keepBinary">Keep binary flag, only applicable to object futures. False by default.</param>
         /// <param name="convertFunc">The function to read future result from stream.</param>
         /// <returns>Created future.</returns>
+        [SuppressMessage("Microsoft.Reliability", "CA2000:Dispose objects before losing scope")]
         private Future<T> GetFuture<T>(Func<long, int, IUnmanagedTarget> listenAction, bool keepBinary = false,
             Func<BinaryReader, T> convertFunc = null)
         {
@@ -325,6 +327,7 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         public long InStreamOutLong(int type, Action<IBinaryRawWriter> writeAction)
         {
             using (var stream = IgniteManager.Memory.Allocate().GetStream())
@@ -340,6 +343,7 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         public T InStreamOutStream<T>(int type, Action<IBinaryRawWriter> writeAction, 
             Func<IBinaryRawReader, T> readAction)
         {
@@ -361,6 +365,7 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         public IPlatformTarget InStreamOutObject(int type, Action<IBinaryRawWriter> writeAction)
         {
             using (var stream = IgniteManager.Memory.Allocate().GetStream())
@@ -429,6 +434,7 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         public T OutStream<T>(int type, Func<IBinaryRawReader, T> readAction)
         {
             using (var stream = IgniteManager.Memory.Allocate().GetStream())

http://git-wip-us.apache.org/repos/asf/ignite/blob/5ca7909f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
index b291b3d..37a24b1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
@@ -1210,6 +1210,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         #region AffinityFunction
 
+        [SuppressMessage("Microsoft.Reliability", "CA2000:Dispose objects before losing scope")]
         private long AffinityFunctionInit(long memPtr, long unused, long unused1, void* baseFunc)
         {
             using (var stream = IgniteManager.Memory.Get(memPtr).GetStream())


[18/37] ignite git commit: Fixed "IGNITE-2092 Switched to the up-to-date java image which has build-in functionality to detect container CPU limitation".

Posted by vo...@apache.org.
Fixed "IGNITE-2092 Switched to the up-to-date java image which has build-in functionality to detect container CPU limitation".

Signed-off-by: nikolay_tikhonov <nt...@gridgain.com>


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

Branch: refs/heads/ignite-2.3
Commit: 0919d9f468e73b141783279267db6612664e6eb8
Parents: 0ab17ab
Author: Maxim Neverov <ne...@gmail.com>
Authored: Fri Sep 29 16:37:27 2017 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Fri Sep 29 16:38:08 2017 +0300

----------------------------------------------------------------------
 modules/docker/2.2.0/Dockerfile | 46 ++++++++++++++++++++++++++++++++
 modules/docker/2.2.0/run.sh     | 51 ++++++++++++++++++++++++++++++++++++
 modules/docker/Dockerfile       |  4 +--
 3 files changed, 99 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0919d9f4/modules/docker/2.2.0/Dockerfile
----------------------------------------------------------------------
diff --git a/modules/docker/2.2.0/Dockerfile b/modules/docker/2.2.0/Dockerfile
new file mode 100644
index 0000000..4c09653
--- /dev/null
+++ b/modules/docker/2.2.0/Dockerfile
@@ -0,0 +1,46 @@
+#
+# 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.
+#
+
+# Start from a Java image.
+FROM openjdk:8
+
+# Ignite version
+ENV IGNITE_VERSION 2.2.0
+
+# Ignite home
+ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-${IGNITE_VERSION}-bin
+
+# Do not rely on anything provided by base image(s), but be explicit, if they are installed already it is noop then
+RUN apt-get update && apt-get install -y --no-install-recommends \
+        unzip \
+        curl \
+    && rm -rf /var/lib/apt/lists/*
+
+WORKDIR /opt/ignite
+
+RUN curl https://dist.apache.org/repos/dist/release/ignite/${IGNITE_VERSION}/apache-ignite-fabric-${IGNITE_VERSION}-bin.zip -o ignite.zip \
+    && unzip ignite.zip \
+    && rm ignite.zip
+
+# Copy sh files and set permission
+COPY ./run.sh $IGNITE_HOME/
+
+RUN chmod +x $IGNITE_HOME/run.sh
+
+CMD $IGNITE_HOME/run.sh
+
+EXPOSE 11211 47100 47500 49112
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0919d9f4/modules/docker/2.2.0/run.sh
----------------------------------------------------------------------
diff --git a/modules/docker/2.2.0/run.sh b/modules/docker/2.2.0/run.sh
new file mode 100644
index 0000000..3aafc30
--- /dev/null
+++ b/modules/docker/2.2.0/run.sh
@@ -0,0 +1,51 @@
+#!/bin/bash
+#
+# 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.
+#
+
+if [ ! -z "$OPTION_LIBS" ]; then
+  IFS=, LIBS_LIST=("$OPTION_LIBS")
+
+  for lib in ${LIBS_LIST[@]}; do
+    cp -r $IGNITE_HOME/libs/optional/"$lib"/* \
+        $IGNITE_HOME/libs/
+  done
+fi
+
+if [ ! -z "$EXTERNAL_LIBS" ]; then
+  IFS=, LIBS_LIST=("$EXTERNAL_LIBS")
+
+  for lib in ${LIBS_LIST[@]}; do
+    echo $lib >> temp
+  done
+
+  wget -i temp -P $IGNITE_HOME/libs
+
+  rm temp
+fi
+
+QUIET=""
+
+if [ "$IGNITE_QUIET" = "false" ]; then
+  QUIET="-v"
+fi
+
+if [ -z $CONFIG_URI ]; then
+  $IGNITE_HOME/bin/ignite.sh $QUIET
+else
+  $IGNITE_HOME/bin/ignite.sh $QUIET $CONFIG_URI
+fi
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/0919d9f4/modules/docker/Dockerfile
----------------------------------------------------------------------
diff --git a/modules/docker/Dockerfile b/modules/docker/Dockerfile
index 6a0eecd..4c09653 100644
--- a/modules/docker/Dockerfile
+++ b/modules/docker/Dockerfile
@@ -16,10 +16,10 @@
 #
 
 # Start from a Java image.
-FROM java:8
+FROM openjdk:8
 
 # Ignite version
-ENV IGNITE_VERSION 2.1.0
+ENV IGNITE_VERSION 2.2.0
 
 # Ignite home
 ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-${IGNITE_VERSION}-bin


[02/37] ignite git commit: IGNITE-6029 Record serializer refactoring and initial stuff for Record V2 serialization.

Posted by vo...@apache.org.
IGNITE-6029 Record serializer refactoring and initial stuff for Record V2 serialization.

Signed-off-by: Andrey Gura <ag...@apache.org>


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

Branch: refs/heads/ignite-2.3
Commit: 007707674e32b4123708e267feec04075a1b4663
Parents: b7bb792
Author: Pavel Kovalenko <jo...@gmail.com>
Authored: Thu Sep 28 12:15:19 2017 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Thu Sep 28 12:16:15 2017 +0300

----------------------------------------------------------------------
 .../pagemem/wal/record/SwitchSegmentRecord.java |   28 +
 .../wal/AbstractWalRecordsIterator.java         |   54 +-
 .../wal/FileWriteAheadLogManager.java           |  193 +-
 .../persistence/wal/RecordDataSerializer.java   |   41 +
 .../wal/WalSegmentTailReachedException.java     |   37 +
 .../wal/reader/IgniteWalIteratorFactory.java    |    5 +-
 .../reader/StandaloneWalRecordsIterator.java    |   27 +-
 .../wal/serializer/RecordDataV1Serializer.java  | 1574 ++++++++++++++++
 .../wal/serializer/RecordDataV2Serializer.java  |   64 +
 .../wal/serializer/RecordV1Serializer.java      | 1673 ++----------------
 .../wal/serializer/RecordV2Serializer.java      |  170 ++
 .../persistence/wal/serializer/io/RecordIO.java |   60 +
 12 files changed, 2305 insertions(+), 1621 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/00770767/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/SwitchSegmentRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/SwitchSegmentRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/SwitchSegmentRecord.java
new file mode 100644
index 0000000..948ec7e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/SwitchSegmentRecord.java
@@ -0,0 +1,28 @@
+/*
+ * 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.pagemem.wal.record;
+
+/**
+ * Record is needed to mark end of segment.
+ */
+public class SwitchSegmentRecord extends WALRecord {
+    /** {@inheritDoc} */
+    @Override public RecordType type() {
+        return RecordType.SWITCH_SEGMENT_RECORD;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/00770767/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java
index d5a2555..5be6e55 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java
@@ -30,7 +30,6 @@ import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
-import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord;
 import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.NotNull;
@@ -154,21 +153,31 @@ public abstract class AbstractWalRecordsIterator
      */
     protected void advance() throws IgniteCheckedException {
         while (true) {
-            curRec = advanceRecord(currWalSegment);
-
-            if (curRec != null)
-                return;
-            else {
-                currWalSegment = advanceSegment(currWalSegment);
+            try {
+                curRec = advanceRecord(currWalSegment);
 
-                if (currWalSegment == null)
+                if (curRec != null)
                     return;
+                else {
+                    currWalSegment = advanceSegment(currWalSegment);
+
+                    if (currWalSegment == null)
+                        return;
+                }
+            }
+            catch (WalSegmentTailReachedException e) {
+                log.warning(e.getMessage());
+
+                curRec = null;
+
+                return;
             }
         }
     }
 
     /**
      * Closes and returns WAL segment (if any)
+     *
      * @return closed handle
      * @throws IgniteCheckedException if IO failed
      */
@@ -199,7 +208,8 @@ public abstract class AbstractWalRecordsIterator
      * @return next advanced record
      */
     private IgniteBiTuple<WALPointer, WALRecord> advanceRecord(
-        @Nullable final FileWriteAheadLogManager.ReadFileHandle hnd) {
+        @Nullable final FileWriteAheadLogManager.ReadFileHandle hnd
+    ) throws IgniteCheckedException {
         if (hnd == null)
             return null;
 
@@ -217,8 +227,12 @@ public abstract class AbstractWalRecordsIterator
             return new IgniteBiTuple<>((WALPointer)ptr, postProcessRecord(rec));
         }
         catch (IOException | IgniteCheckedException e) {
+            if (e instanceof WalSegmentTailReachedException)
+                throw (WalSegmentTailReachedException)e;
+
             if (!(e instanceof SegmentEofException))
                 handleRecordException(e, ptr);
+
             return null;
         }
     }
@@ -261,24 +275,18 @@ public abstract class AbstractWalRecordsIterator
             FileIO fileIO = ioFactory.create(desc.file);
 
             try {
-                FileInput in = new FileInput(fileIO, buf);
+                int serVer = FileWriteAheadLogManager.readSerializerVersion(fileIO);
 
-                // Header record must be agnostic to the serializer version.
-                WALRecord rec = serializer.readRecord(in,
-                    new FileWALPointer(desc.idx, (int)fileIO.position(), 0));
+                RecordSerializer ser = FileWriteAheadLogManager.forVersion(sharedCtx, serVer);
 
-                if (rec == null)
-                    return null;
-
-                if (rec.type() != WALRecord.RecordType.HEADER_RECORD)
-                    throw new IOException("Missing file header record: " + desc.file.getAbsoluteFile());
-
-                int ver = ((HeaderRecord)rec).version();
+                FileInput in = new FileInput(fileIO, buf);
 
-                RecordSerializer ser = FileWriteAheadLogManager.forVersion(sharedCtx, ver, serializer.writePointer());
+                if (start != null && desc.idx == start.index()) {
+                    // Make sure we skip header with serializer version.
+                    long startOffset = Math.max(start.fileOffset(), fileIO.position());
 
-                if (start != null && desc.idx == start.index())
-                    in.seek(start.fileOffset());
+                    in.seek(startOffset);
+                }
 
                 return new FileWriteAheadLogManager.ReadFileHandle(fileIO, desc.idx, sharedCtx.igniteInstanceName(), ser, in);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/00770767/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
index 9b2d948..c4582cf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.pagemem.wal.StorageException;
 import org.apache.ignite.internal.pagemem.wal.WALIterator;
 import org.apache.ignite.internal.pagemem.wal.WALPointer;
 import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord;
+import org.apache.ignite.internal.pagemem.wal.record.SwitchSegmentRecord;
 import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
@@ -62,8 +63,12 @@ import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabase
 import org.apache.ignite.internal.processors.cache.persistence.PersistenceMetricsImpl;
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32;
 import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord;
+import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordDataV1Serializer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordDataV2Serializer;
 import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV2Serializer;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.GridUnsafe;
@@ -116,6 +121,9 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         }
     };
 
+    /** Latest serializer version to use. */
+    public static final int LATEST_SERIALIZER_VERSION = 1;
+
     /** */
     private final boolean alwaysWriteFullPages;
 
@@ -152,9 +160,12 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     /** WAL archive directory (including consistent ID as subfolder) */
     private File walArchiveDir;
 
-    /** Serializer of current version, used to read header record and for write records */
+    /** Serializer of latest version. */
     private RecordSerializer serializer;
 
+    /** Serializer latest version to use. */
+    private int serializerVersion = LATEST_SERIALIZER_VERSION;
+
     /** */
     private volatile long oldestArchiveSegmentIdx;
 
@@ -266,7 +277,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                 "write ahead log archive directory"
             );
 
-            serializer = new RecordV1Serializer(cctx);
+            serializer = forVersion(cctx, serializerVersion);
 
             GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager)cctx.database();
 
@@ -818,10 +829,17 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             FileIO fileIO = ioFactory.create(curFile);
 
             try {
-                // readSerializerVersion will change the channel position.
-                // This is fine because the FileWriteHandle consitructor will move it
-                // to offset + len anyways.
-                int serVer = readSerializerVersion(fileIO, curFile, absIdx);
+                int serVer = serializerVersion;
+
+                // If we have existing segment, try to read version from it.
+                if (lastReadPtr != null) {
+                    try {
+                        serVer = readSerializerVersion(fileIO);
+                    }
+                    catch (SegmentEofException | EOFException ignore) {
+                        serVer = serializerVersion;
+                    }
+                }
 
                 RecordSerializer ser = forVersion(cctx, serVer);
 
@@ -837,13 +855,9 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                     maxWalSegmentSize,
                     ser);
 
-                if (lastReadPtr == null) {
-                    HeaderRecord header = new HeaderRecord(serializer.version());
-
-                    header.size(serializer.size(header));
-
-                    hnd.addRecord(header);
-                }
+                // For new handle write serializer version to it.
+                if (lastReadPtr == null)
+                    hnd.writeSerializerVersion();
 
                 archiver.currentWalIndex(absIdx);
 
@@ -887,11 +901,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                 maxWalSegmentSize,
                 serializer);
 
-            HeaderRecord header = new HeaderRecord(serializer.version());
-
-            header.size(serializer.size(header));
-
-            hnd.addRecord(header);
+            hnd.writeSerializerVersion();
 
             return hnd;
         }
@@ -1010,10 +1020,11 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     }
 
     /**
+     * @param cctx Shared context.
      * @param ver Serializer version.
      * @return Entry serializer.
      */
-    static RecordSerializer forVersion(GridCacheSharedContext cctx, int ver) throws IgniteCheckedException {
+    public static RecordSerializer forVersion(GridCacheSharedContext cctx, int ver) throws IgniteCheckedException {
         return forVersion(cctx, ver, false);
     }
 
@@ -1027,7 +1038,12 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
         switch (ver) {
             case 1:
-                return new RecordV1Serializer(cctx, writePointer);
+                return new RecordV1Serializer(new RecordDataV1Serializer(cctx), writePointer);
+
+            case 2:
+                RecordDataV2Serializer dataV2Serializer = new RecordDataV2Serializer(new RecordDataV1Serializer(cctx));
+
+                return new RecordV2Serializer(dataV2Serializer, writePointer);
 
             default:
                 throw new IgniteCheckedException("Failed to create a serializer with the given version " +
@@ -1434,29 +1450,103 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     }
 
     /**
+     * Reads record serializer version from provided {@code io}.
+     * NOTE: Method mutates position of {@code io}.
+     *
      * @param io I/O interface for file.
-     * @param file File object.
-     * @param idx File index to read.
      * @return Serializer version stored in the file.
-     * @throws IOException If failed to read serializer version.
      * @throws IgniteCheckedException If failed to read serializer version.
      */
-    private int readSerializerVersion(FileIO io, File file, long idx)
-        throws IOException, IgniteCheckedException {
-        try (ByteBufferExpander buf = new ByteBufferExpander(RecordV1Serializer.HEADER_RECORD_SIZE, ByteOrder.nativeOrder())){
+    public static int readSerializerVersion(FileIO io)
+            throws IgniteCheckedException, IOException {
+        try (ByteBufferExpander buf = new ByteBufferExpander(RecordV1Serializer.HEADER_RECORD_SIZE, ByteOrder.nativeOrder())) {
             FileInput in = new FileInput(io, buf);
 
-            // Header record must be agnostic to the serializer version.
-            WALRecord rec = serializer.readRecord(in, new FileWALPointer(idx, 0, 0));
+            in.ensure(RecordV1Serializer.HEADER_RECORD_SIZE);
+
+            int recordType = in.readUnsignedByte();
+
+            if (recordType == WALRecord.RecordType.STOP_ITERATION_RECORD_TYPE)
+                throw new SegmentEofException("Reached logical end of the segment", null);
+
+            WALRecord.RecordType type = WALRecord.RecordType.fromOrdinal(recordType - 1);
+
+            if (type != WALRecord.RecordType.HEADER_RECORD)
+                throw new IOException("Can't read serializer version", null);
+
+            // Read file pointer.
+            FileWALPointer ptr = RecordV1Serializer.readPosition(in);
+
+            assert ptr.fileOffset() == 0 : "Header record should be placed at the beginning of file " + ptr;
+
+            long headerMagicNumber = in.readLong();
+
+            if (headerMagicNumber != HeaderRecord.MAGIC)
+                throw new IOException("Magic is corrupted [exp=" + U.hexLong(HeaderRecord.MAGIC) +
+                        ", actual=" + U.hexLong(headerMagicNumber) + ']');
+
+            // Read serializer version.
+            int version = in.readInt();
+
+            // Read and skip CRC.
+            in.readInt();
+
+            return version;
+        }
+    }
+
+    /**
+     * Writes record serializer version to provided {@code io}.
+     * NOTE: Method mutates position of {@code io}.
+     *
+     * @param io I/O interface for file.
+     * @param idx Segment index.
+     * @param version Serializer version.
+     * @return I/O position after write version.
+     * @throws IOException If failed to write serializer version.
+     */
+    public static long writeSerializerVersion(FileIO io, long idx, int version) throws IOException {
+        ByteBuffer buffer = ByteBuffer.allocate(RecordV1Serializer.HEADER_RECORD_SIZE);
+        buffer.order(ByteOrder.nativeOrder());
+
+        // Write record type.
+        buffer.put((byte) (WALRecord.RecordType.HEADER_RECORD.ordinal() + 1));
+
+        // Write position.
+        RecordV1Serializer.putPosition(buffer, new FileWALPointer(idx, 0, 0));
 
-            if (rec.type() != WALRecord.RecordType.HEADER_RECORD)
-                throw new IOException("Missing file header record: " + file.getAbsoluteFile());
+        // Place magic number.
+        buffer.putLong(HeaderRecord.MAGIC);
 
-            return ((HeaderRecord)rec).version();
+        // Place serializer version.
+        buffer.putInt(version);
+
+        // Place CRC if needed.
+        if (!RecordV1Serializer.SKIP_CRC) {
+            int curPos = buffer.position();
+
+            buffer.position(0);
+
+            // This call will move buffer position to the end of the record again.
+            int crcVal = PureJavaCrc32.calcCrc32(buffer, curPos);
+
+            buffer.putInt(crcVal);
         }
-        catch (SegmentEofException | EOFException ignore) {
-            return serializer.version();
+        else
+            buffer.putInt(0);
+
+        // Write header record through io.
+        buffer.position(0);
+
+        do {
+            io.write(buffer);
         }
+        while (buffer.hasRemaining());
+
+        // Flush
+        io.force();
+
+        return io.position();
     }
 
     /**
@@ -1715,6 +1805,27 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         }
 
         /**
+         * Write serializer version to current handle.
+         * NOTE: Method mutates {@code fileIO} position, written and lastFsyncPos fields.
+         *
+         * @throws IgniteCheckedException If fail to write serializer version.
+         */
+        public void writeSerializerVersion() throws IgniteCheckedException {
+            try {
+                assert fileIO.position() == 0 : "Serializer version can be written only at the begin of file " + fileIO.position();
+
+                long updatedPosition = FileWriteAheadLogManager.writeSerializerVersion(fileIO, idx, serializer.version());
+
+                written = updatedPosition;
+                lastFsyncPos = updatedPosition;
+                head.set(new FakeRecord(new FileWALPointer(idx, (int)updatedPosition, 0), false));
+            }
+            catch (IOException e) {
+                throw new IgniteCheckedException("Unable to write serializer version for segment " + idx, e);
+            }
+        }
+
+        /**
          * Checks if current head is a close fake record and returns {@code true} if so.
          *
          * @return {@code true} if current head is close record.
@@ -2080,15 +2191,17 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                     assert stopped() : "Segment is not closed after close flush: " + head.get();
 
                     try {
-                        int switchSegmentRecSize = RecordV1Serializer.REC_TYPE_SIZE + RecordV1Serializer.FILE_WAL_POINTER_SIZE;
+                        int switchSegmentRecSize = RecordV1Serializer.REC_TYPE_SIZE + RecordV1Serializer.FILE_WAL_POINTER_SIZE + RecordV1Serializer.CRC_SIZE;
+
+                    if (rollOver && written < (maxSegmentSize - switchSegmentRecSize)) {
+                        RecordV1Serializer backwardSerializer =
+                            new RecordV1Serializer(new RecordDataV1Serializer(cctx), true);
 
-                        if (rollOver && written < (maxSegmentSize - switchSegmentRecSize)) {
-                            //it is expected there is sufficient space for this record because rollover should run early
-                            final ByteBuffer buf = ByteBuffer.allocate(switchSegmentRecSize);
-                            buf.put((byte)(WALRecord.RecordType.SWITCH_SEGMENT_RECORD.ordinal() + 1));
+                        final ByteBuffer buf = ByteBuffer.allocate(switchSegmentRecSize);
 
-                            final FileWALPointer pointer = new FileWALPointer(idx, (int)fileIO.position(), -1);
-                            RecordV1Serializer.putPosition(buf, pointer);
+                        SwitchSegmentRecord segmentRecord = new SwitchSegmentRecord();
+                        segmentRecord.position( new FileWALPointer(idx, (int)written, -1));
+                        backwardSerializer.writeRecord(segmentRecord,buf);
 
                             buf.rewind();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/00770767/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordDataSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordDataSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordDataSerializer.java
new file mode 100644
index 0000000..242641d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordDataSerializer.java
@@ -0,0 +1,41 @@
+package org.apache.ignite.internal.processors.cache.persistence.wal;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
+
+/**
+ * Interface to provide size, read and write operations with WAL records
+ * <b>without any headers and meta information</b>.
+ */
+public interface RecordDataSerializer {
+    /**
+     * Calculates size of record data.
+     *
+     * @param record WAL record.
+     * @return Size of record in bytes.
+     * @throws IgniteCheckedException If it's unable to calculate record data size.
+     */
+    int size(WALRecord record) throws IgniteCheckedException;
+
+    /**
+     * Reads record data of {@code type} from buffer {@code in}.
+     *
+     * @param type Record type.
+     * @param in Buffer to read.
+     * @return WAL record.
+     * @throws IOException In case of I/O problems.
+     * @throws IgniteCheckedException If it's unable to read record.
+     */
+    WALRecord readRecord(WALRecord.RecordType type, ByteBufferBackedDataInput in) throws IOException, IgniteCheckedException;
+
+    /**
+     * Writes record data to buffer {@code buf}.
+     *
+     * @param record WAL record.
+     * @param buf Buffer to write.
+     * @throws IgniteCheckedException If it's unable to write record.
+     */
+    void writeRecord(WALRecord record, ByteBuffer buf) throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/00770767/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/WalSegmentTailReachedException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/WalSegmentTailReachedException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/WalSegmentTailReachedException.java
new file mode 100644
index 0000000..36298dc
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/WalSegmentTailReachedException.java
@@ -0,0 +1,37 @@
+/*
+ * 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.persistence.wal;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * An exception is thrown when we reached tail of WAL segment cyclic buffer
+ * during reading from WAL.
+ */
+public class WalSegmentTailReachedException extends IgniteCheckedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     *
+     */
+    public WalSegmentTailReachedException(String msg, @Nullable Throwable cause) {
+        super(msg, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/00770767/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
index 3a34e28..0fb8adf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
@@ -133,7 +133,7 @@ public class IgniteWalIteratorFactory {
      * @param pageSize Page size which was used in Ignite Persistent Data store to read WAL from, size is validated
      * according its boundaries.
      */
-    public IgniteWalIteratorFactory(@NotNull final IgniteLogger log, final int pageSize) {
+    public IgniteWalIteratorFactory(@NotNull final IgniteLogger log, int pageSize) {
         this(log, new PersistentStoreConfiguration().getFileIOFactory(), pageSize);
     }
 
@@ -148,8 +148,7 @@ public class IgniteWalIteratorFactory {
      * @return closable WAL records iterator, should be closed when non needed
      * @throws IgniteCheckedException if failed to read folder
      */
-    public WALIterator iteratorArchiveDirectory(
-        @NotNull final File walDirWithConsistentId) throws IgniteCheckedException {
+    public WALIterator iteratorArchiveDirectory(@NotNull final File walDirWithConsistentId) throws IgniteCheckedException {
         return new StandaloneWalRecordsIterator(walDirWithConsistentId, log, prepareSharedCtx(), ioFactory, keepBinary);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/00770767/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
index c92d572..f1258a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
@@ -100,14 +100,15 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator {
      * (BinaryObjects will be used instead)
      */
     StandaloneWalRecordsIterator(
-        @NotNull final File walFilesDir,
-        @NotNull final IgniteLogger log,
-        @NotNull final GridCacheSharedContext sharedCtx,
-        @NotNull final FileIOFactory ioFactory,
-        final boolean keepBinary) throws IgniteCheckedException {
+        @NotNull File walFilesDir,
+        @NotNull IgniteLogger log,
+        @NotNull GridCacheSharedContext sharedCtx,
+        @NotNull FileIOFactory ioFactory,
+        boolean keepBinary
+    ) throws IgniteCheckedException {
         super(log,
             sharedCtx,
-            new RecordV1Serializer(sharedCtx, true),
+            FileWriteAheadLogManager.forVersion(sharedCtx, FileWriteAheadLogManager.LATEST_SERIALIZER_VERSION),
             ioFactory,
             BUF_SIZE);
         this.keepBinary = keepBinary;
@@ -127,15 +128,15 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator {
      * @param walFiles Wal files.
      */
     StandaloneWalRecordsIterator(
-        @NotNull final IgniteLogger log,
-        @NotNull final GridCacheSharedContext sharedCtx,
-        @NotNull final FileIOFactory ioFactory,
-        final boolean workDir,
-        final boolean keepBinary,
-        @NotNull final File... walFiles) throws IgniteCheckedException {
+            @NotNull IgniteLogger log,
+            @NotNull GridCacheSharedContext sharedCtx,
+            @NotNull FileIOFactory ioFactory,
+            boolean workDir,
+            boolean keepBinary,
+            @NotNull File... walFiles) throws IgniteCheckedException {
         super(log,
             sharedCtx,
-            new RecordV1Serializer(sharedCtx, true),
+            FileWriteAheadLogManager.forVersion(sharedCtx, FileWriteAheadLogManager.LATEST_SERIALIZER_VERSION),
             ioFactory,
             BUF_SIZE);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/00770767/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
new file mode 100644
index 0000000..8b5e6ba
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
@@ -0,0 +1,1574 @@
+/*
+ * 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.persistence.wal.serializer;
+
+import java.io.DataInput;
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.wal.record.CacheState;
+import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord;
+import org.apache.ignite.internal.pagemem.wal.record.DataEntry;
+import org.apache.ignite.internal.pagemem.wal.record.DataRecord;
+import org.apache.ignite.internal.pagemem.wal.record.LazyDataEntry;
+import org.apache.ignite.internal.pagemem.wal.record.MemoryRecoveryRecord;
+import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
+import org.apache.ignite.internal.pagemem.wal.record.TxRecord;
+import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageInsertFragmentRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageInsertRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageRemoveRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageSetFreeListPageRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageUpdateRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.FixCountRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.FixLeftmostChildRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.FixRemoveId;
+import org.apache.ignite.internal.pagemem.wal.record.delta.InitNewPageRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.InnerReplaceRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.InsertRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.MergeRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageAddRootRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageCutRootRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRootInlineRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRootRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdateLastAllocatedIndex;
+import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdateLastSuccessfulFullSnapshotId;
+import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdateLastSuccessfulSnapshotId;
+import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdateNextSnapshotId;
+import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdatePartitionDataRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.NewRootInitRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.PageListMetaResetCountRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.PagesListAddPageRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.PagesListInitNewPageRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.PagesListRemovePageRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.PagesListSetNextRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.PagesListSetPreviousRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.PartitionDestroyRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.PartitionMetaStateRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.RecycleRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.RemoveRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.ReplaceRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.SplitExistingPageRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.SplitForwardPageRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.TrackingPageDeltaRecord;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheOperation;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO;
+import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput;
+import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.RecordDataSerializer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.CRC_SIZE;
+
+/**
+ * Record data V1 serializer.
+ */
+public class RecordDataV1Serializer implements RecordDataSerializer {
+    /** Length of HEADER record data. */
+    static final int HEADER_RECORD_DATA_SIZE = /*Magic*/8 + /*Version*/4;
+
+    /** Cache shared context */
+    private final GridCacheSharedContext cctx;
+
+    /** Size of page used for PageMemory regions */
+    private final int pageSize;
+
+    /** Cache object processor to reading {@link DataEntry DataEntries} */
+    private final IgniteCacheObjectProcessor co;
+
+    /** Serializer of {@link TxRecord} records. */
+    private TxRecordSerializer txRecordSerializer;
+
+    /**
+     * @param cctx Cctx.
+     */
+    public RecordDataV1Serializer(GridCacheSharedContext cctx) {
+        this.cctx = cctx;
+        this.txRecordSerializer = new TxRecordSerializer(cctx);
+        this.co = cctx.kernalContext().cacheObjects();
+        this.pageSize = cctx.database().pageSize();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int size(WALRecord record) throws IgniteCheckedException {
+        switch (record.type()) {
+            case PAGE_RECORD:
+                assert record instanceof PageSnapshot;
+
+                PageSnapshot pageRec = (PageSnapshot)record;
+
+                return pageRec.pageData().length + 12;
+
+            case CHECKPOINT_RECORD:
+                CheckpointRecord cpRec = (CheckpointRecord)record;
+
+                assert cpRec.checkpointMark() == null || cpRec.checkpointMark() instanceof FileWALPointer :
+                        "Invalid WAL record: " + cpRec;
+
+                int cacheStatesSize = cacheStatesSize(cpRec.cacheGroupStates());
+
+                FileWALPointer walPtr = (FileWALPointer)cpRec.checkpointMark();
+
+                return 18 + cacheStatesSize + (walPtr == null ? 0 : 16);
+
+            case META_PAGE_INIT:
+                return /*cache ID*/4 + /*page ID*/8 + /*ioType*/2  + /*ioVer*/2 +  /*tree root*/8 + /*reuse root*/8;
+
+            case PARTITION_META_PAGE_UPDATE_COUNTERS:
+                return /*cache ID*/4 + /*page ID*/8 + /*upd cntr*/8 + /*rmv id*/8 + /*part size*/4 + /*counters page id*/8 + /*state*/ 1
+                        + /*allocatedIdxCandidate*/ 4;
+
+            case MEMORY_RECOVERY:
+                return 8;
+
+            case PARTITION_DESTROY:
+                return /*cacheId*/4 + /*partId*/4;
+
+            case DATA_RECORD:
+                DataRecord dataRec = (DataRecord)record;
+
+                return 4 + dataSize(dataRec);
+
+            case HEADER_RECORD:
+                return HEADER_RECORD_DATA_SIZE;
+
+            case DATA_PAGE_INSERT_RECORD:
+                DataPageInsertRecord diRec = (DataPageInsertRecord)record;
+
+                return 4 + 8 + 2 + diRec.payload().length;
+
+            case DATA_PAGE_UPDATE_RECORD:
+                DataPageUpdateRecord uRec = (DataPageUpdateRecord)record;
+
+                return 4 + 8 + 2 + 4 +
+                        uRec.payload().length;
+
+            case DATA_PAGE_INSERT_FRAGMENT_RECORD:
+                final DataPageInsertFragmentRecord difRec = (DataPageInsertFragmentRecord)record;
+
+                return 4 + 8 + 8 + 4 + difRec.payloadSize();
+
+            case DATA_PAGE_REMOVE_RECORD:
+                return 4 + 8 + 1;
+
+            case DATA_PAGE_SET_FREE_LIST_PAGE:
+                return 4 + 8 + 8;
+
+            case INIT_NEW_PAGE_RECORD:
+                return 4 + 8 + 2 + 2 + 8;
+
+            case BTREE_META_PAGE_INIT_ROOT:
+                return 4 + 8 + 8;
+
+            case BTREE_META_PAGE_INIT_ROOT2:
+                return 4 + 8 + 8 + 2;
+
+            case BTREE_META_PAGE_ADD_ROOT:
+                return 4 + 8 + 8;
+
+            case BTREE_META_PAGE_CUT_ROOT:
+                return 4 + 8;
+
+            case BTREE_INIT_NEW_ROOT:
+                NewRootInitRecord<?> riRec = (NewRootInitRecord<?>)record;
+
+                return 4 + 8 + 8 + 2 + 2 + 8 + 8 + riRec.io().getItemSize();
+
+            case BTREE_PAGE_RECYCLE:
+                return 4 + 8 + 8;
+
+            case BTREE_PAGE_INSERT:
+                InsertRecord<?> inRec = (InsertRecord<?>)record;
+
+                return 4 + 8 + 2 + 2 + 2 + 8 + inRec.io().getItemSize();
+
+            case BTREE_FIX_LEFTMOST_CHILD:
+                return 4 + 8 + 8;
+
+            case BTREE_FIX_COUNT:
+                return 4 + 8 + 2;
+
+            case BTREE_PAGE_REPLACE:
+                ReplaceRecord<?> rRec = (ReplaceRecord<?>)record;
+
+                return 4 + 8 + 2 + 2 + 2 + rRec.io().getItemSize();
+
+            case BTREE_PAGE_REMOVE:
+                return 4 + 8 + 2 + 2;
+
+            case BTREE_PAGE_INNER_REPLACE:
+                return 4 + 8 + 2 + 8 + 2 + 8;
+
+            case BTREE_FORWARD_PAGE_SPLIT:
+                return 4 + 8 + 8 + 2 + 2 + 8 + 2 + 2;
+
+            case BTREE_EXISTING_PAGE_SPLIT:
+                return 4 + 8 + 2 + 8;
+
+            case BTREE_PAGE_MERGE:
+                return 4 + 8 + 8 + 2 + 8 + 1;
+
+            case BTREE_FIX_REMOVE_ID:
+                return 4 + 8 + 8;
+
+            case PAGES_LIST_SET_NEXT:
+                return 4 + 8 + 8;
+
+            case PAGES_LIST_SET_PREVIOUS:
+                return 4 + 8 + 8;
+
+            case PAGES_LIST_INIT_NEW_PAGE:
+                return 4 + 8 + 4 + 4 + 8 + 8 + 8;
+
+            case PAGES_LIST_ADD_PAGE:
+                return 4 + 8 + 8;
+
+            case PAGES_LIST_REMOVE_PAGE:
+                return 4 + 8 + 8;
+
+            case TRACKING_PAGE_DELTA:
+                return 4 + 8 + 8 + 8 + 8;
+
+            case META_PAGE_UPDATE_LAST_SUCCESSFUL_SNAPSHOT_ID:
+                return 4 + 8 + 8 + 8;
+
+            case META_PAGE_UPDATE_LAST_SUCCESSFUL_FULL_SNAPSHOT_ID:
+                return 4 + 8 + 8;
+
+            case META_PAGE_UPDATE_NEXT_SNAPSHOT_ID:
+                return 4 + 8 + 8;
+
+            case META_PAGE_UPDATE_LAST_ALLOCATED_INDEX:
+                return 4 + 8 + 4;
+
+            case PART_META_UPDATE_STATE:
+                return /*cacheId*/ 4 + /*partId*/ 4 + /*State*/1 + /*Update Counter*/ 8;
+
+            case PAGE_LIST_META_RESET_COUNT_RECORD:
+                return /*cacheId*/ 4 + /*pageId*/ 8;
+
+            case SWITCH_SEGMENT_RECORD:
+                // CRC is not loaded for switch segment.
+                return -CRC_SIZE;
+
+            case TX_RECORD:
+                return txRecordSerializer.sizeOfTxRecord((TxRecord)record);
+
+            default:
+                throw new UnsupportedOperationException("Type: " + record.type());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public WALRecord readRecord(WALRecord.RecordType type, ByteBufferBackedDataInput in) throws IOException, IgniteCheckedException {
+        WALRecord res;
+
+        switch (type) {
+            case PAGE_RECORD:
+                byte[] arr = new byte[pageSize];
+
+                int cacheId = in.readInt();
+                long pageId = in.readLong();
+
+                in.readFully(arr);
+
+                res = new PageSnapshot(new FullPageId(pageId, cacheId), arr);
+
+                break;
+
+            case CHECKPOINT_RECORD:
+                long msb = in.readLong();
+                long lsb = in.readLong();
+                boolean hasPtr = in.readByte() != 0;
+                int idx = hasPtr ? in.readInt() : 0;
+                int offset = hasPtr ? in.readInt() : 0;
+                int len = hasPtr ? in.readInt() : 0;
+
+                Map<Integer, CacheState> states = readPartitionStates(in);
+
+                boolean end = in.readByte() != 0;
+
+                FileWALPointer walPtr = hasPtr ? new FileWALPointer(idx, offset, len) : null;
+
+                CheckpointRecord cpRec = new CheckpointRecord(new UUID(msb, lsb), walPtr, end);
+
+                cpRec.cacheGroupStates(states);
+
+                res = cpRec;
+
+                break;
+
+            case META_PAGE_INIT:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                int ioType = in.readUnsignedShort();
+                int ioVer = in.readUnsignedShort();
+                long treeRoot = in.readLong();
+                long reuseListRoot = in.readLong();
+
+                res = new MetaPageInitRecord(cacheId, pageId, ioType, ioVer, treeRoot, reuseListRoot);
+
+                break;
+
+            case PARTITION_META_PAGE_UPDATE_COUNTERS:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                long updCntr = in.readLong();
+                long rmvId = in.readLong();
+                int partSize = in.readInt();
+                long countersPageId = in.readLong();
+                byte state = in.readByte();
+                int allocatedIdxCandidate = in.readInt();
+
+                res = new MetaPageUpdatePartitionDataRecord(cacheId, pageId, updCntr, rmvId, partSize, countersPageId, state, allocatedIdxCandidate);
+
+                break;
+
+            case MEMORY_RECOVERY:
+                long ts = in.readLong();
+
+                res = new MemoryRecoveryRecord(ts);
+
+                break;
+
+            case PARTITION_DESTROY:
+                cacheId = in.readInt();
+                int partId = in.readInt();
+
+                res = new PartitionDestroyRecord(cacheId, partId);
+
+                break;
+
+            case DATA_RECORD:
+                int entryCnt = in.readInt();
+
+                List<DataEntry> entries = new ArrayList<>(entryCnt);
+
+                for (int i = 0; i < entryCnt; i++)
+                    entries.add(readDataEntry(in));
+
+                res = new DataRecord(entries);
+
+                break;
+
+            case HEADER_RECORD:
+                long magic = in.readLong();
+
+                if (magic != HeaderRecord.MAGIC)
+                    throw new EOFException("Magic is corrupted [exp=" + U.hexLong(HeaderRecord.MAGIC) +
+                            ", actual=" + U.hexLong(magic) + ']');
+
+                int ver = in.readInt();
+
+                res = new HeaderRecord(ver);
+
+                break;
+
+            case DATA_PAGE_INSERT_RECORD: {
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                int size = in.readUnsignedShort();
+
+                in.ensure(size);
+
+                byte[] payload = new byte[size];
+
+                in.readFully(payload);
+
+                res = new DataPageInsertRecord(cacheId, pageId, payload);
+
+                break;
+            }
+
+            case DATA_PAGE_UPDATE_RECORD: {
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                int itemId = in.readInt();
+
+                int size = in.readUnsignedShort();
+
+                in.ensure(size);
+
+                byte[] payload = new byte[size];
+
+                in.readFully(payload);
+
+                res = new DataPageUpdateRecord(cacheId, pageId, itemId, payload);
+
+                break;
+            }
+
+            case DATA_PAGE_INSERT_FRAGMENT_RECORD: {
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                final long lastLink = in.readLong();
+                final int payloadSize = in.readInt();
+
+                final byte[] payload = new byte[payloadSize];
+
+                in.readFully(payload);
+
+                res = new DataPageInsertFragmentRecord(cacheId, pageId, payload, lastLink);
+
+                break;
+            }
+
+            case DATA_PAGE_REMOVE_RECORD:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                int itemId = in.readUnsignedByte();
+
+                res = new DataPageRemoveRecord(cacheId, pageId, itemId);
+
+                break;
+
+            case DATA_PAGE_SET_FREE_LIST_PAGE:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                long freeListPage = in.readLong();
+
+                res = new DataPageSetFreeListPageRecord(cacheId, pageId, freeListPage);
+
+                break;
+
+            case INIT_NEW_PAGE_RECORD:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                ioType = in.readUnsignedShort();
+                ioVer = in.readUnsignedShort();
+                long virtualPageId = in.readLong();
+
+                res = new InitNewPageRecord(cacheId, pageId, ioType, ioVer, virtualPageId);
+
+                break;
+
+            case BTREE_META_PAGE_INIT_ROOT:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                long rootId = in.readLong();
+
+                res = new MetaPageInitRootRecord(cacheId, pageId, rootId);
+
+                break;
+
+            case BTREE_META_PAGE_INIT_ROOT2:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                long rootId2 = in.readLong();
+                int inlineSize = in.readShort();
+
+                res = new MetaPageInitRootInlineRecord(cacheId, pageId, rootId2, inlineSize);
+
+                break;
+
+            case BTREE_META_PAGE_ADD_ROOT:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                rootId = in.readLong();
+
+                res = new MetaPageAddRootRecord(cacheId, pageId, rootId);
+
+                break;
+
+            case BTREE_META_PAGE_CUT_ROOT:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                res = new MetaPageCutRootRecord(cacheId, pageId);
+
+                break;
+
+            case BTREE_INIT_NEW_ROOT:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                rootId = in.readLong();
+                ioType = in.readUnsignedShort();
+                ioVer = in.readUnsignedShort();
+                long leftId = in.readLong();
+                long rightId = in.readLong();
+
+                BPlusIO<?> io = BPlusIO.getBPlusIO(ioType, ioVer);
+
+                byte[] rowBytes = new byte[io.getItemSize()];
+
+                in.readFully(rowBytes);
+
+                res = new NewRootInitRecord<>(cacheId, pageId, rootId, (BPlusInnerIO<?>)io, leftId, rowBytes, rightId);
+
+                break;
+
+            case BTREE_PAGE_RECYCLE:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                long newPageId = in.readLong();
+
+                res = new RecycleRecord(cacheId, pageId, newPageId);
+
+                break;
+
+            case BTREE_PAGE_INSERT:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                ioType = in.readUnsignedShort();
+                ioVer = in.readUnsignedShort();
+                int itemIdx = in.readUnsignedShort();
+                rightId = in.readLong();
+
+                io = BPlusIO.getBPlusIO(ioType, ioVer);
+
+                rowBytes = new byte[io.getItemSize()];
+
+                in.readFully(rowBytes);
+
+                res = new InsertRecord<>(cacheId, pageId, io, itemIdx, rowBytes, rightId);
+
+                break;
+
+            case BTREE_FIX_LEFTMOST_CHILD:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                rightId = in.readLong();
+
+                res = new FixLeftmostChildRecord(cacheId, pageId, rightId);
+
+                break;
+
+            case BTREE_FIX_COUNT:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                int cnt = in.readUnsignedShort();
+
+                res = new FixCountRecord(cacheId, pageId, cnt);
+
+                break;
+
+            case BTREE_PAGE_REPLACE:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                ioType = in.readUnsignedShort();
+                ioVer = in.readUnsignedShort();
+                itemIdx = in.readUnsignedShort();
+
+                io = BPlusIO.getBPlusIO(ioType, ioVer);
+
+                rowBytes = new byte[io.getItemSize()];
+
+                in.readFully(rowBytes);
+
+                res = new ReplaceRecord<>(cacheId, pageId, io, rowBytes, itemIdx);
+
+                break;
+
+            case BTREE_PAGE_REMOVE:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                itemIdx = in.readUnsignedShort();
+                cnt = in.readUnsignedShort();
+
+                res = new RemoveRecord(cacheId, pageId, itemIdx, cnt);
+
+                break;
+
+            case BTREE_PAGE_INNER_REPLACE:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                int dstIdx = in.readUnsignedShort();
+                long srcPageId = in.readLong();
+                int srcIdx = in.readUnsignedShort();
+                rmvId = in.readLong();
+
+                res = new InnerReplaceRecord<>(cacheId, pageId, dstIdx, srcPageId, srcIdx, rmvId);
+
+                break;
+
+            case BTREE_FORWARD_PAGE_SPLIT:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                long fwdId = in.readLong();
+                ioType = in.readUnsignedShort();
+                ioVer = in.readUnsignedShort();
+                srcPageId = in.readLong();
+                int mid = in.readUnsignedShort();
+                cnt = in.readUnsignedShort();
+
+                res = new SplitForwardPageRecord(cacheId, pageId, fwdId, ioType, ioVer, srcPageId, mid, cnt);
+
+                break;
+
+            case BTREE_EXISTING_PAGE_SPLIT:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                mid = in.readUnsignedShort();
+                fwdId = in.readLong();
+
+                res = new SplitExistingPageRecord(cacheId, pageId, mid, fwdId);
+
+                break;
+
+            case BTREE_PAGE_MERGE:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                long prntId = in.readLong();
+                int prntIdx = in.readUnsignedShort();
+                rightId = in.readLong();
+                boolean emptyBranch = in.readBoolean();
+
+                res = new MergeRecord<>(cacheId, pageId, prntId, prntIdx, rightId, emptyBranch);
+
+                break;
+
+            case BTREE_FIX_REMOVE_ID:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                rmvId = in.readLong();
+
+                res = new FixRemoveId(cacheId, pageId, rmvId);
+
+                break;
+
+            case PAGES_LIST_SET_NEXT:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+                long nextPageId = in.readLong();
+
+                res = new PagesListSetNextRecord(cacheId, pageId, nextPageId);
+
+                break;
+
+            case PAGES_LIST_SET_PREVIOUS:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+                long prevPageId = in.readLong();
+
+                res = new PagesListSetPreviousRecord(cacheId, pageId, prevPageId);
+
+                break;
+
+            case PAGES_LIST_INIT_NEW_PAGE:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+                ioType = in.readInt();
+                ioVer = in.readInt();
+                newPageId = in.readLong();
+                prevPageId = in.readLong();
+                long addDataPageId = in.readLong();
+
+                res = new PagesListInitNewPageRecord(cacheId, pageId, ioType, ioVer, newPageId, prevPageId, addDataPageId);
+
+                break;
+
+            case PAGES_LIST_ADD_PAGE:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+                long dataPageId = in.readLong();
+
+                res = new PagesListAddPageRecord(cacheId, pageId, dataPageId);
+
+                break;
+
+            case PAGES_LIST_REMOVE_PAGE:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+                long rmvdPageId = in.readLong();
+
+                res = new PagesListRemovePageRecord(cacheId, pageId, rmvdPageId);
+
+                break;
+
+            case TRACKING_PAGE_DELTA:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                long pageIdToMark = in.readLong();
+                long nextSnapshotId0 = in.readLong();
+                long lastSuccessfulSnapshotId0 = in.readLong();
+
+                res = new TrackingPageDeltaRecord(cacheId, pageId, pageIdToMark, nextSnapshotId0, lastSuccessfulSnapshotId0);
+
+                break;
+
+            case META_PAGE_UPDATE_NEXT_SNAPSHOT_ID:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                long nextSnapshotId = in.readLong();
+
+                res = new MetaPageUpdateNextSnapshotId(cacheId, pageId, nextSnapshotId);
+
+                break;
+
+            case META_PAGE_UPDATE_LAST_SUCCESSFUL_FULL_SNAPSHOT_ID:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                long lastSuccessfulFullSnapshotId = in.readLong();
+
+                res = new MetaPageUpdateLastSuccessfulFullSnapshotId(cacheId, pageId, lastSuccessfulFullSnapshotId);
+
+                break;
+
+            case META_PAGE_UPDATE_LAST_SUCCESSFUL_SNAPSHOT_ID:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                long lastSuccessfulSnapshotId = in.readLong();
+                long lastSuccessfulSnapshotTag = in.readLong();
+
+                res = new MetaPageUpdateLastSuccessfulSnapshotId(cacheId, pageId, lastSuccessfulSnapshotId, lastSuccessfulSnapshotTag);
+
+                break;
+
+            case META_PAGE_UPDATE_LAST_ALLOCATED_INDEX:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                int lastAllocatedIdx = in.readInt();
+
+                res = new MetaPageUpdateLastAllocatedIndex(cacheId, pageId, lastAllocatedIdx);
+
+                break;
+
+            case PART_META_UPDATE_STATE:
+                cacheId = in.readInt();
+                partId = in.readInt();
+
+                state = in.readByte();
+
+                long updateCounter = in.readLong();
+
+                res = new PartitionMetaStateRecord(cacheId, partId, GridDhtPartitionState.fromOrdinal(state), updateCounter);
+
+                break;
+
+            case PAGE_LIST_META_RESET_COUNT_RECORD:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                res = new PageListMetaResetCountRecord(cacheId, pageId);
+                break;
+
+            case SWITCH_SEGMENT_RECORD:
+                throw new EOFException("END OF SEGMENT");
+
+            case TX_RECORD:
+                res = txRecordSerializer.readTxRecord(in);
+
+                break;
+
+            default:
+                throw new UnsupportedOperationException("Type: " + type);
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeRecord(WALRecord record, ByteBuffer buf) throws IgniteCheckedException {
+        switch (record.type()) {
+            case PAGE_RECORD:
+                PageSnapshot snap = (PageSnapshot)record;
+
+                buf.putInt(snap.fullPageId().groupId());
+                buf.putLong(snap.fullPageId().pageId());
+                buf.put(snap.pageData());
+
+                break;
+
+            case MEMORY_RECOVERY:
+                MemoryRecoveryRecord memoryRecoveryRecord = (MemoryRecoveryRecord)record;
+
+                buf.putLong(memoryRecoveryRecord.time());
+
+                break;
+
+            case PARTITION_DESTROY:
+                PartitionDestroyRecord partDestroy = (PartitionDestroyRecord)record;
+
+                buf.putInt(partDestroy.groupId());
+                buf.putInt(partDestroy.partitionId());
+
+                break;
+
+            case META_PAGE_INIT:
+                MetaPageInitRecord updRootsRec = (MetaPageInitRecord)record;
+
+                buf.putInt(updRootsRec.groupId());
+                buf.putLong(updRootsRec.pageId());
+
+                buf.putShort((short)updRootsRec.ioType());
+                buf.putShort((short)updRootsRec.ioVersion());
+                buf.putLong(updRootsRec.treeRoot());
+                buf.putLong(updRootsRec.reuseListRoot());
+
+                break;
+
+            case PARTITION_META_PAGE_UPDATE_COUNTERS:
+                MetaPageUpdatePartitionDataRecord partDataRec = (MetaPageUpdatePartitionDataRecord)record;
+
+                buf.putInt(partDataRec.groupId());
+                buf.putLong(partDataRec.pageId());
+
+                buf.putLong(partDataRec.updateCounter());
+                buf.putLong(partDataRec.globalRemoveId());
+                buf.putInt(partDataRec.partitionSize());
+                buf.putLong(partDataRec.countersPageId());
+                buf.put(partDataRec.state());
+                buf.putInt(partDataRec.allocatedIndexCandidate());
+
+                break;
+
+            case CHECKPOINT_RECORD:
+                CheckpointRecord cpRec = (CheckpointRecord)record;
+
+                assert cpRec.checkpointMark() == null || cpRec.checkpointMark() instanceof FileWALPointer :
+                        "Invalid WAL record: " + cpRec;
+
+                FileWALPointer walPtr = (FileWALPointer)cpRec.checkpointMark();
+                UUID cpId = cpRec.checkpointId();
+
+                buf.putLong(cpId.getMostSignificantBits());
+                buf.putLong(cpId.getLeastSignificantBits());
+
+                buf.put(walPtr == null ? (byte)0 : 1);
+
+                if (walPtr != null) {
+                    buf.putLong(walPtr.index());
+                    buf.putInt(walPtr.fileOffset());
+                    buf.putInt(walPtr.length());
+                }
+
+                putCacheStates(buf, cpRec.cacheGroupStates());
+
+                buf.put(cpRec.end() ? (byte)1 : 0);
+
+                break;
+
+            case DATA_RECORD:
+                DataRecord dataRec = (DataRecord)record;
+
+                buf.putInt(dataRec.writeEntries().size());
+
+                for (DataEntry dataEntry : dataRec.writeEntries())
+                    putDataEntry(buf, dataEntry);
+
+                break;
+
+            case HEADER_RECORD:
+                buf.putLong(HeaderRecord.MAGIC);
+
+                buf.putInt(((HeaderRecord)record).version());
+
+                break;
+
+            case DATA_PAGE_INSERT_RECORD:
+                DataPageInsertRecord diRec = (DataPageInsertRecord)record;
+
+                buf.putInt(diRec.groupId());
+                buf.putLong(diRec.pageId());
+
+                buf.putShort((short)diRec.payload().length);
+
+                buf.put(diRec.payload());
+
+                break;
+
+            case DATA_PAGE_UPDATE_RECORD:
+                DataPageUpdateRecord uRec = (DataPageUpdateRecord)record;
+
+                buf.putInt(uRec.groupId());
+                buf.putLong(uRec.pageId());
+                buf.putInt(uRec.itemId());
+
+                buf.putShort((short)uRec.payload().length);
+
+                buf.put(uRec.payload());
+
+                break;
+
+            case DATA_PAGE_INSERT_FRAGMENT_RECORD:
+                final DataPageInsertFragmentRecord difRec = (DataPageInsertFragmentRecord)record;
+
+                buf.putInt(difRec.groupId());
+                buf.putLong(difRec.pageId());
+
+                buf.putLong(difRec.lastLink());
+                buf.putInt(difRec.payloadSize());
+                buf.put(difRec.payload());
+
+                break;
+
+            case DATA_PAGE_REMOVE_RECORD:
+                DataPageRemoveRecord drRec = (DataPageRemoveRecord)record;
+
+                buf.putInt(drRec.groupId());
+                buf.putLong(drRec.pageId());
+
+                buf.put((byte)drRec.itemId());
+
+                break;
+
+            case DATA_PAGE_SET_FREE_LIST_PAGE:
+                DataPageSetFreeListPageRecord freeListRec = (DataPageSetFreeListPageRecord)record;
+
+                buf.putInt(freeListRec.groupId());
+                buf.putLong(freeListRec.pageId());
+
+                buf.putLong(freeListRec.freeListPage());
+
+                break;
+
+            case INIT_NEW_PAGE_RECORD:
+                InitNewPageRecord inpRec = (InitNewPageRecord)record;
+
+                buf.putInt(inpRec.groupId());
+                buf.putLong(inpRec.pageId());
+
+                buf.putShort((short)inpRec.ioType());
+                buf.putShort((short)inpRec.ioVersion());
+                buf.putLong(inpRec.newPageId());
+
+                break;
+
+            case BTREE_META_PAGE_INIT_ROOT:
+                MetaPageInitRootRecord imRec = (MetaPageInitRootRecord)record;
+
+                buf.putInt(imRec.groupId());
+                buf.putLong(imRec.pageId());
+
+                buf.putLong(imRec.rootId());
+
+                break;
+
+            case BTREE_META_PAGE_INIT_ROOT2:
+                MetaPageInitRootInlineRecord imRec2 = (MetaPageInitRootInlineRecord)record;
+
+                buf.putInt(imRec2.groupId());
+                buf.putLong(imRec2.pageId());
+
+                buf.putLong(imRec2.rootId());
+
+                buf.putShort((short)imRec2.inlineSize());
+                break;
+
+            case BTREE_META_PAGE_ADD_ROOT:
+                MetaPageAddRootRecord arRec = (MetaPageAddRootRecord)record;
+
+                buf.putInt(arRec.groupId());
+                buf.putLong(arRec.pageId());
+
+                buf.putLong(arRec.rootId());
+
+                break;
+
+            case BTREE_META_PAGE_CUT_ROOT:
+                MetaPageCutRootRecord crRec = (MetaPageCutRootRecord)record;
+
+                buf.putInt(crRec.groupId());
+                buf.putLong(crRec.pageId());
+
+                break;
+
+            case BTREE_INIT_NEW_ROOT:
+                NewRootInitRecord<?> riRec = (NewRootInitRecord<?>)record;
+
+                buf.putInt(riRec.groupId());
+                buf.putLong(riRec.pageId());
+
+                buf.putLong(riRec.rootId());
+                buf.putShort((short)riRec.io().getType());
+                buf.putShort((short)riRec.io().getVersion());
+                buf.putLong(riRec.leftId());
+                buf.putLong(riRec.rightId());
+
+                putRow(buf, riRec.rowBytes());
+
+                break;
+
+            case BTREE_PAGE_RECYCLE:
+                RecycleRecord recRec = (RecycleRecord)record;
+
+                buf.putInt(recRec.groupId());
+                buf.putLong(recRec.pageId());
+
+                buf.putLong(recRec.newPageId());
+
+                break;
+
+            case BTREE_PAGE_INSERT:
+                InsertRecord<?> inRec = (InsertRecord<?>)record;
+
+                buf.putInt(inRec.groupId());
+                buf.putLong(inRec.pageId());
+
+                buf.putShort((short)inRec.io().getType());
+                buf.putShort((short)inRec.io().getVersion());
+                buf.putShort((short)inRec.index());
+                buf.putLong(inRec.rightId());
+
+                putRow(buf, inRec.rowBytes());
+
+                break;
+
+            case BTREE_FIX_LEFTMOST_CHILD:
+                FixLeftmostChildRecord flRec = (FixLeftmostChildRecord)record;
+
+                buf.putInt(flRec.groupId());
+                buf.putLong(flRec.pageId());
+
+                buf.putLong(flRec.rightId());
+
+                break;
+
+            case BTREE_FIX_COUNT:
+                FixCountRecord fcRec = (FixCountRecord)record;
+
+                buf.putInt(fcRec.groupId());
+                buf.putLong(fcRec.pageId());
+
+                buf.putShort((short)fcRec.count());
+
+                break;
+
+            case BTREE_PAGE_REPLACE:
+                ReplaceRecord<?> rRec = (ReplaceRecord<?>)record;
+
+                buf.putInt(rRec.groupId());
+                buf.putLong(rRec.pageId());
+
+                buf.putShort((short)rRec.io().getType());
+                buf.putShort((short)rRec.io().getVersion());
+                buf.putShort((short)rRec.index());
+
+                putRow(buf, rRec.rowBytes());
+
+                break;
+
+            case BTREE_PAGE_REMOVE:
+                RemoveRecord rmRec = (RemoveRecord)record;
+
+                buf.putInt(rmRec.groupId());
+                buf.putLong(rmRec.pageId());
+
+                buf.putShort((short)rmRec.index());
+                buf.putShort((short)rmRec.count());
+
+                break;
+
+            case BTREE_PAGE_INNER_REPLACE:
+                InnerReplaceRecord<?> irRec = (InnerReplaceRecord<?>)record;
+
+                buf.putInt(irRec.groupId());
+                buf.putLong(irRec.pageId());
+
+                buf.putShort((short)irRec.destinationIndex());
+                buf.putLong(irRec.sourcePageId());
+                buf.putShort((short)irRec.sourceIndex());
+                buf.putLong(irRec.removeId());
+
+                break;
+
+            case BTREE_FORWARD_PAGE_SPLIT:
+                SplitForwardPageRecord sfRec = (SplitForwardPageRecord)record;
+
+                buf.putInt(sfRec.groupId());
+                buf.putLong(sfRec.pageId());
+
+                buf.putLong(sfRec.forwardId());
+                buf.putShort((short)sfRec.ioType());
+                buf.putShort((short)sfRec.ioVersion());
+                buf.putLong(sfRec.sourcePageId());
+                buf.putShort((short)sfRec.middleIndex());
+                buf.putShort((short)sfRec.count());
+
+                break;
+
+            case BTREE_EXISTING_PAGE_SPLIT:
+                SplitExistingPageRecord seRec = (SplitExistingPageRecord)record;
+
+                buf.putInt(seRec.groupId());
+                buf.putLong(seRec.pageId());
+
+                buf.putShort((short)seRec.middleIndex());
+                buf.putLong(seRec.forwardId());
+
+                break;
+
+            case BTREE_PAGE_MERGE:
+                MergeRecord<?> mRec = (MergeRecord<?>)record;
+
+                buf.putInt(mRec.groupId());
+                buf.putLong(mRec.pageId());
+
+                buf.putLong(mRec.parentId());
+                buf.putShort((short)mRec.parentIndex());
+                buf.putLong(mRec.rightId());
+                buf.put((byte)(mRec.isEmptyBranch() ? 1 : 0));
+
+                break;
+
+            case PAGES_LIST_SET_NEXT:
+                PagesListSetNextRecord plNextRec = (PagesListSetNextRecord)record;
+
+                buf.putInt(plNextRec.groupId());
+                buf.putLong(plNextRec.pageId());
+
+                buf.putLong(plNextRec.nextPageId());
+
+                break;
+
+            case PAGES_LIST_SET_PREVIOUS:
+                PagesListSetPreviousRecord plPrevRec = (PagesListSetPreviousRecord)record;
+
+                buf.putInt(plPrevRec.groupId());
+                buf.putLong(plPrevRec.pageId());
+
+                buf.putLong(plPrevRec.previousPageId());
+
+                break;
+
+            case PAGES_LIST_INIT_NEW_PAGE:
+                PagesListInitNewPageRecord plNewRec = (PagesListInitNewPageRecord)record;
+
+                buf.putInt(plNewRec.groupId());
+                buf.putLong(plNewRec.pageId());
+                buf.putInt(plNewRec.ioType());
+                buf.putInt(plNewRec.ioVersion());
+                buf.putLong(plNewRec.newPageId());
+
+                buf.putLong(plNewRec.previousPageId());
+                buf.putLong(plNewRec.dataPageId());
+
+                break;
+
+            case PAGES_LIST_ADD_PAGE:
+                PagesListAddPageRecord plAddRec = (PagesListAddPageRecord)record;
+
+                buf.putInt(plAddRec.groupId());
+                buf.putLong(plAddRec.pageId());
+
+                buf.putLong(plAddRec.dataPageId());
+
+                break;
+
+            case PAGES_LIST_REMOVE_PAGE:
+                PagesListRemovePageRecord plRmvRec = (PagesListRemovePageRecord)record;
+
+                buf.putInt(plRmvRec.groupId());
+                buf.putLong(plRmvRec.pageId());
+
+                buf.putLong(plRmvRec.removedPageId());
+
+                break;
+
+            case BTREE_FIX_REMOVE_ID:
+                FixRemoveId frRec = (FixRemoveId)record;
+
+                buf.putInt(frRec.groupId());
+                buf.putLong(frRec.pageId());
+
+                buf.putLong(frRec.removeId());
+
+                break;
+
+            case TRACKING_PAGE_DELTA:
+                TrackingPageDeltaRecord tpDelta = (TrackingPageDeltaRecord)record;
+
+                buf.putInt(tpDelta.groupId());
+                buf.putLong(tpDelta.pageId());
+
+                buf.putLong(tpDelta.pageIdToMark());
+                buf.putLong(tpDelta.nextSnapshotId());
+                buf.putLong(tpDelta.lastSuccessfulSnapshotId());
+
+                break;
+
+            case META_PAGE_UPDATE_NEXT_SNAPSHOT_ID:
+                MetaPageUpdateNextSnapshotId mpUpdateNextSnapshotId = (MetaPageUpdateNextSnapshotId)record;
+
+                buf.putInt(mpUpdateNextSnapshotId.groupId());
+                buf.putLong(mpUpdateNextSnapshotId.pageId());
+
+                buf.putLong(mpUpdateNextSnapshotId.nextSnapshotId());
+
+                break;
+
+            case META_PAGE_UPDATE_LAST_SUCCESSFUL_FULL_SNAPSHOT_ID:
+                MetaPageUpdateLastSuccessfulFullSnapshotId mpUpdateLastSuccFullSnapshotId =
+                        (MetaPageUpdateLastSuccessfulFullSnapshotId)record;
+
+                buf.putInt(mpUpdateLastSuccFullSnapshotId.groupId());
+                buf.putLong(mpUpdateLastSuccFullSnapshotId.pageId());
+
+                buf.putLong(mpUpdateLastSuccFullSnapshotId.lastSuccessfulFullSnapshotId());
+
+                break;
+
+            case META_PAGE_UPDATE_LAST_SUCCESSFUL_SNAPSHOT_ID:
+                MetaPageUpdateLastSuccessfulSnapshotId mpUpdateLastSuccSnapshotId =
+                        (MetaPageUpdateLastSuccessfulSnapshotId)record;
+
+                buf.putInt(mpUpdateLastSuccSnapshotId.groupId());
+                buf.putLong(mpUpdateLastSuccSnapshotId.pageId());
+
+                buf.putLong(mpUpdateLastSuccSnapshotId.lastSuccessfulSnapshotId());
+                buf.putLong(mpUpdateLastSuccSnapshotId.lastSuccessfulSnapshotTag());
+
+                break;
+
+            case META_PAGE_UPDATE_LAST_ALLOCATED_INDEX:
+                MetaPageUpdateLastAllocatedIndex mpUpdateLastAllocatedIdx =
+                        (MetaPageUpdateLastAllocatedIndex) record;
+
+                buf.putInt(mpUpdateLastAllocatedIdx.groupId());
+                buf.putLong(mpUpdateLastAllocatedIdx.pageId());
+
+                buf.putInt(mpUpdateLastAllocatedIdx.lastAllocatedIndex());
+
+                break;
+
+            case PART_META_UPDATE_STATE:
+                PartitionMetaStateRecord partMetaStateRecord = (PartitionMetaStateRecord) record;
+
+                buf.putInt(partMetaStateRecord.groupId());
+
+                buf.putInt(partMetaStateRecord.partitionId());
+
+                buf.put(partMetaStateRecord.state());
+
+                buf.putLong(partMetaStateRecord.updateCounter());
+
+                break;
+
+            case PAGE_LIST_META_RESET_COUNT_RECORD:
+                PageListMetaResetCountRecord pageListMetaResetCntRecord = (PageListMetaResetCountRecord) record;
+
+                buf.putInt(pageListMetaResetCntRecord.groupId());
+                buf.putLong(pageListMetaResetCntRecord.pageId());
+
+                break;
+
+            case TX_RECORD:
+                txRecordSerializer.writeTxRecord((TxRecord)record, buf);
+
+                break;
+
+            case SWITCH_SEGMENT_RECORD:
+                break;
+
+            default:
+                throw new UnsupportedOperationException("Type: " + record.type());
+        }
+    }
+
+    /**
+     * @param buf Buffer to write to.
+     * @param entry Data entry.
+     */
+    private static void putDataEntry(ByteBuffer buf, DataEntry entry) throws IgniteCheckedException {
+        buf.putInt(entry.cacheId());
+
+        if (!entry.key().putValue(buf))
+            throw new AssertionError();
+
+        if (entry.value() == null)
+            buf.putInt(-1);
+        else if (!entry.value().putValue(buf))
+            throw new AssertionError();
+
+        buf.put((byte)entry.op().ordinal());
+
+        putVersion(buf, entry.nearXidVersion(), true);
+        putVersion(buf, entry.writeVersion(), false);
+
+        buf.putInt(entry.partitionId());
+        buf.putLong(entry.partitionCounter());
+        buf.putLong(entry.expireTime());
+    }
+
+    /**
+     * @param states Cache states.
+     */
+    private static void putCacheStates(ByteBuffer buf, Map<Integer, CacheState> states) {
+        buf.putShort((short)states.size());
+
+        for (Map.Entry<Integer, CacheState> entry : states.entrySet()) {
+            buf.putInt(entry.getKey());
+
+            CacheState state = entry.getValue();
+
+            // Need 2 bytes for the number of partitions.
+            buf.putShort((short)state.size());
+
+            for (int i = 0; i < state.size(); i++) {
+                buf.putShort((short)state.partitionByIndex(i));
+
+                buf.putLong(state.partitionSizeByIndex(i));
+                buf.putLong(state.partitionCounterByIndex(i));
+            }
+        }
+    }
+
+    /**
+     * @param buf Buffer.
+     * @param ver Version to write.
+     * @param allowNull Is {@code null}version allowed.
+     */
+    private static void putVersion(ByteBuffer buf, GridCacheVersion ver, boolean allowNull) {
+        CacheVersionIO.write(buf, ver, allowNull);
+    }
+
+    /**
+     * @param buf Buffer.
+     * @param rowBytes Row bytes.
+     */
+    @SuppressWarnings("unchecked")
+    private static void putRow(ByteBuffer buf, byte[] rowBytes) {
+        assert rowBytes.length > 0;
+
+        buf.put(rowBytes);
+    }
+
+    /**
+     * @param in Input to read from.
+     * @return Read entry.
+     */
+    private DataEntry readDataEntry(ByteBufferBackedDataInput in) throws IOException, IgniteCheckedException {
+        int cacheId = in.readInt();
+
+        int keySize = in.readInt();
+        byte keyType = in.readByte();
+        byte[] keyBytes = new byte[keySize];
+        in.readFully(keyBytes);
+
+        int valSize = in.readInt();
+
+        byte valType = 0;
+        byte[] valBytes = null;
+
+        if (valSize >= 0) {
+            valType = in.readByte();
+            valBytes = new byte[valSize];
+            in.readFully(valBytes);
+        }
+
+        byte ord = in.readByte();
+
+        GridCacheOperation op = GridCacheOperation.fromOrdinal(ord & 0xFF);
+
+        GridCacheVersion nearXidVer = readVersion(in, true);
+        GridCacheVersion writeVer = readVersion(in, false);
+
+        int partId = in.readInt();
+        long partCntr = in.readLong();
+        long expireTime = in.readLong();
+
+        GridCacheContext cacheCtx = cctx.cacheContext(cacheId);
+
+        if (cacheCtx != null) {
+            CacheObjectContext coCtx = cacheCtx.cacheObjectContext();
+
+            KeyCacheObject key = co.toKeyCacheObject(coCtx, keyType, keyBytes);
+            CacheObject val = valBytes != null ? co.toCacheObject(coCtx, valType, valBytes) : null;
+
+            return new DataEntry(
+                    cacheId,
+                    key,
+                    val,
+                    op,
+                    nearXidVer,
+                    writeVer,
+                    expireTime,
+                    partId,
+                    partCntr
+            );
+        }
+        else
+            return new LazyDataEntry(
+                    cctx,
+                    cacheId,
+                    keyType,
+                    keyBytes,
+                    valType,
+                    valBytes,
+                    op,
+                    nearXidVer,
+                    writeVer,
+                    expireTime,
+                    partId,
+                    partCntr);
+    }
+
+    /**
+     * @param buf Buffer to read from.
+     * @return Read map.
+     */
+    private Map<Integer, CacheState> readPartitionStates(DataInput buf) throws IOException {
+        int caches = buf.readShort() & 0xFFFF;
+
+        if (caches == 0)
+            return Collections.emptyMap();
+
+        Map<Integer, CacheState> states = new HashMap<>(caches, 1.0f);
+
+        for (int i = 0; i < caches; i++) {
+            int cacheId = buf.readInt();
+
+            int parts = buf.readShort() & 0xFFFF;
+
+            CacheState state = new CacheState(parts);
+
+            for (int p = 0; p < parts; p++) {
+                int partId = buf.readShort() & 0xFFFF;
+                long size = buf.readLong();
+                long partCntr = buf.readLong();
+
+                state.addPartitionState(partId, size, partCntr);
+            }
+
+            states.put(cacheId, state);
+        }
+
+        return states;
+    }
+
+    /**
+     * Changes the buffer position by the number of read bytes.
+     *
+     * @param in Data input to read from.
+     * @param allowNull Is {@code null}version allowed.
+     * @return Read cache version.
+     */
+    private GridCacheVersion readVersion(ByteBufferBackedDataInput in, boolean allowNull) throws IOException {
+        // To be able to read serialization protocol version.
+        in.ensure(1);
+
+        try {
+            int size = CacheVersionIO.readSize(in.buffer(), allowNull);
+
+            in.ensure(size);
+
+            return CacheVersionIO.read(in.buffer(), allowNull);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    /**
+     * @param dataRec Data record to serialize.
+     * @return Full data record size.
+     * @throws IgniteCheckedException If failed to obtain the length of one of the entries.
+     */
+    private int dataSize(DataRecord dataRec) throws IgniteCheckedException {
+        int sz = 0;
+
+        for (DataEntry entry : dataRec.writeEntries())
+            sz += entrySize(entry);
+
+        return sz;
+    }
+
+    /**
+     * @param entry Entry to get size for.
+     * @return Entry size.
+     * @throws IgniteCheckedException If failed to get key or value bytes length.
+     */
+    private int entrySize(DataEntry entry) throws IgniteCheckedException {
+        GridCacheContext cctx = this.cctx.cacheContext(entry.cacheId());
+        CacheObjectContext coCtx = cctx.cacheObjectContext();
+
+        return
+            /*cache ID*/4 +
+            /*key*/entry.key().valueBytesLength(coCtx) +
+            /*value*/(entry.value() == null ? 4 : entry.value().valueBytesLength(coCtx)) +
+            /*op*/1 +
+            /*near xid ver*/CacheVersionIO.size(entry.nearXidVersion(), true) +
+            /*write ver*/CacheVersionIO.size(entry.writeVersion(), false) +
+            /*part ID*/4 +
+            /*expire Time*/8 +
+            /*part cnt*/8;
+    }
+
+    /**
+     * @param states Partition states.
+     * @return Size required to write partition states.
+     */
+    private int cacheStatesSize(Map<Integer, CacheState> states) {
+        // Need 4 bytes for the number of caches.
+        int size = 2;
+
+        for (Map.Entry<Integer, CacheState> entry : states.entrySet()) {
+            // Cache ID.
+            size += 4;
+
+            // Need 2 bytes for the number of partitions.
+            size += 2;
+
+            CacheState state = entry.getValue();
+
+            // 2 bytes partition ID, size and counter per partition.
+            size += 18 * state.size();
+        }
+
+        return size;
+    }
+
+}


[19/37] ignite git commit: IGNITE-6535 Web Console: Set IGNITE_HOME for Web Agent.

Posted by vo...@apache.org.
IGNITE-6535 Web Console: Set IGNITE_HOME for Web Agent.


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

Branch: refs/heads/ignite-2.3
Commit: 4cb0fc73f4129895e975c6a2381c5059e3f6e1ce
Parents: 0919d9f
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Sun Oct 1 18:47:08 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Mon Oct 2 13:56:21 2017 +0700

----------------------------------------------------------------------
 .../web-agent/bin/ignite-web-agent.bat           | 19 +++++++++++++++++++
 .../web-agent/bin/ignite-web-agent.sh            |  5 +++++
 2 files changed, 24 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4cb0fc73/modules/web-console/web-agent/bin/ignite-web-agent.bat
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/bin/ignite-web-agent.bat b/modules/web-console/web-agent/bin/ignite-web-agent.bat
index 1f1b52d..387447f 100644
--- a/modules/web-console/web-agent/bin/ignite-web-agent.bat
+++ b/modules/web-console/web-agent/bin/ignite-web-agent.bat
@@ -20,6 +20,25 @@ Setlocal EnableDelayedExpansion
 
 if "%OS%" == "Windows_NT"  setlocal
 
+:: Check IGNITE_HOME.
+pushd "%~dp0"
+set IGNITE_HOME=%CD%
+
+:checkIgniteHome2
+:: Strip double quotes from IGNITE_HOME
+set IGNITE_HOME=%IGNITE_HOME:"=%
+
+:: remove all trailing slashes from IGNITE_HOME.
+if %IGNITE_HOME:~-1,1% == \ goto removeTrailingSlash
+if %IGNITE_HOME:~-1,1% == / goto removeTrailingSlash
+goto checkIgniteHome3
+
+:removeTrailingSlash
+set IGNITE_HOME=%IGNITE_HOME:~0,-1%
+goto checkIgniteHome2
+
+:checkIgniteHome3
+
 :: Check JAVA_HOME.
 if defined JAVA_HOME  goto checkJdk
     echo %0, ERROR:

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cb0fc73/modules/web-console/web-agent/bin/ignite-web-agent.sh
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/bin/ignite-web-agent.sh b/modules/web-console/web-agent/bin/ignite-web-agent.sh
index c2958fc..530bc7f 100644
--- a/modules/web-console/web-agent/bin/ignite-web-agent.sh
+++ b/modules/web-console/web-agent/bin/ignite-web-agent.sh
@@ -60,6 +60,11 @@ fi
 
 SOURCE="${BASH_SOURCE[0]}"
 
+#
+# Set IGNITE_HOME.
+#
+export IGNITE_HOME="$(dirname "$(cd "$(dirname "$0")"; "pwd")")";
+
 DIR="$( dirname "$SOURCE" )"
 
 while [ -h "$SOURCE" ]


[35/37] ignite git commit: IGNITE-6382 .NET: Set up NDepend project

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/b21f750f/modules/platforms/dotnet/Apache.Ignite.ndproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.ndproj b/modules/platforms/dotnet/Apache.Ignite.ndproj
new file mode 100644
index 0000000..9f5287b
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.ndproj
@@ -0,0 +1,11139 @@
+<?xml version="1.0" encoding="utf-8" standalone="yes"?>
+<NDepend AppName="Apache.Ignite" Platform="DotNet" FileWrittenByProductVersion="2017.2.2.8962">
+  <OutputDir KeepXmlFiles="False">c:\w\incubator-ignite\modules\platforms\dotnet\NDependOut</OutputDir>
+  <Assemblies>
+    <Name>Apache.Ignite.Core</Name>
+  </Assemblies>
+  <FrameworkAssemblies>
+    <Name>mscorlib</Name>
+    <Name>System.Core</Name>
+    <Name>System.Xml</Name>
+    <Name>System</Name>
+    <Name>System.Configuration</Name>
+    <Name>System.Transactions</Name>
+  </FrameworkAssemblies>
+  <Dirs>
+    <Dir>C:\WINDOWS\Microsoft.NET\Framework\v4.0.30319</Dir>
+    <Dir>C:\WINDOWS\Microsoft.NET\Framework\v4.0.30319\WPF</Dir>
+    <Dir>$(NdProjectDir)\Apache.Ignite.Core\bin\Debug</Dir>
+  </Dirs>
+  <MergeCodeGeneratedByCompiler>True</MergeCodeGeneratedByCompiler>
+  <Report Kind="0" SectionsEnabled="110591" XslPath="" Flags="261120" />
+  <BuildComparisonSetting ProjectMode="CurrentProject" BuildMode="NDaysAgoAnalysisResult" ProjectFileToCompareWith="" BuildFileToCompareWith="" NDaysAgo="30" />
+  <BaselineInUISetting ProjectMode="CurrentProject" BuildMode="NDaysAgoAnalysisResult" ProjectFileToCompareWith="" BuildFileToCompareWith="" NDaysAgo="30" />
+  <CoverageFiles CoverageDir="" UncoverableAttribute="" />
+  <TrendMetrics UseCustomLog="False" LogRecurrence="3" LogLabel="2" UseCustomDir="False" CustomDir="">
+    <Chart Name="Size" ShowInReport="True">
+      <Serie MetricName="# Lines of Code" MetricUnit="Loc" Color="#FF00BFFF" ChartType="Line" ScaleExp="0" />
+      <Serie MetricName="# Lines of Code Covered" MetricUnit="Loc" Color="#FF32CD32" ChartType="Area" ScaleExp="0" />
+      <Serie MetricName="# Lines of Code (NotMyCode)" MetricUnit="Loc" Color="#FFA9A9A9" ChartType="Area" ScaleExp="0" />
+      <Serie MetricName="# Lines of Comments" MetricUnit="Lines" Color="#FF008000" ChartType="Line" ScaleExp="0" />
+    </Chart>
+    <Chart Name="% Coverage and % Debt" ShowInReport="True">
+      <Serie MetricName="Percentage Code Coverage" MetricUnit="%" Color="#FF32CD32" ChartType="Area" ScaleExp="0" />
+      <Serie MetricName="Percentage Debt (Metric)" MetricUnit="%" Color="#FFFF0000" ChartType="Line" ScaleExp="0" />
+    </Chart>
+    <Chart Name="Issues" ShowInReport="True">
+      <Serie MetricName="# New Issues since Baseline" MetricUnit="issues" Color="#FFFF0000" ChartType="Line" ScaleExp="0" />
+      <Serie MetricName="# Issues Fixed since Baseline" MetricUnit="issues" Color="#FF32CD32" ChartType="Line" ScaleExp="0" />
+      <Serie MetricName="# Blocker/Critical/Major Issues" MetricUnit="issues" Color="#FFFF8C00" ChartType="Line" ScaleExp="0" />
+      <Serie MetricName="# Issues" MetricUnit="issues" Color="#FFFFD700" ChartType="Line" ScaleExp="-2" />
+    </Chart>
+    <Chart Name="Rules" ShowInReport="True">
+      <Serie MetricName="# Rules" MetricUnit="Rules" Color="#FF66CDAA" ChartType="Line" ScaleExp="0" />
+      <Serie MetricName="# Rules Violated" MetricUnit="Rules" Color="#FFFF8C00" ChartType="Area" ScaleExp="0" />
+      <Serie MetricName="# Critical Rules Violated" MetricUnit="Rules" Color="#FFFF0000" ChartType="Area" ScaleExp="0" />
+    </Chart>
+    <Chart Name="Quality Gates" ShowInReport="True">
+      <Serie MetricName="# Quality Gates Fail" MetricUnit="quality gates" Color="#FFFF0000" ChartType="Line" ScaleExp="0" />
+      <Serie MetricName="# Quality Gates Warn" MetricUnit="quality gates" Color="#FFFF8C00" ChartType="Line" ScaleExp="0" />
+      <Serie MetricName="# Quality Gates" MetricUnit="quality gates" Color="#FF32CD32" ChartType="Line" ScaleExp="0" />
+    </Chart>
+    <Chart Name="Debt" ShowInReport="True">
+      <Serie MetricName="Debt (Metric)" MetricUnit="man-days" Color="#FFFF0000" ChartType="Line" ScaleExp="0" />
+      <Serie MetricName="Annual Interest (Metric)" MetricUnit="man-days" Color="#FFFF8C00" ChartType="Line" ScaleExp="0" />
+    </Chart>
+  </TrendMetrics>
+  <HistoricAnalysisResult PersistRecurrence="3" UseCustomDir="False" CustomDir="" />
+  <SourceFileRebasing FromPath="" ToPath="" />
+  <PathVariables />
+  <RuleFiles />
+  <ProjectRules AreActive="True" />
+  <ProjectDebtSettings DebtSettingsStorage="0" SettingsFilePath="">
+    <DebtSettings>
+      <DebtFactor>1</DebtFactor>
+      <AnnualInterestFactor>1</AnnualInterestFactor>
+      <DebtDefault>0</DebtDefault>
+      <AnnualInterestDefault>0</AnnualInterestDefault>
+      <DebtStringFormat>$ManDay$</DebtStringFormat>
+      <MoneyPerManHour>50</MoneyPerManHour>
+      <Currency>USD</Currency>
+      <CurrencyLocation>After</CurrencyLocation>
+      <EstimatedNumberOfManDayToDevelop1000LogicalLinesOfCode>18</EstimatedNumberOfManDayToDevelop1000LogicalLinesOfCode>
+      <NumberOfWorkDayPerYear>240</NumberOfWorkDayPerYear>
+      <NumberOfWorkHourPerDay>8</NumberOfWorkHourPerDay>
+      <A2B_RatingThreshold>5</A2B_RatingThreshold>
+      <B2C_RatingThreshold>10</B2C_RatingThreshold>
+      <C2D_RatingThreshold>20</C2D_RatingThreshold>
+      <D2E_RatingThreshold>50</D2E_RatingThreshold>
+      <Low2Medium_SeverityThreshold>1200000000</Low2Medium_SeverityThreshold>
+      <Medium2High_SeverityThreshold>12000000000</Medium2High_SeverityThreshold>
+      <High2Critical_SeverityThreshold>72000000000</High2Critical_SeverityThreshold>
+      <Critical2Blocker_SeverityThreshold>360000000000</Critical2Blocker_SeverityThreshold>
+    </DebtSettings>
+  </ProjectDebtSettings>
+  <Queries>
+    <Group Name="Quality Gates" Active="True" ShownInReport="True">
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Quality Gates Evolution</Name>
+from qg in QualityGates
+let qgBaseline = qg.OlderVersion()
+let relyOnDiff = qgBaseline == null
+let evolution = relyOnDiff ? (TrendIcon?)null : 
+                // When a quality gate relies on diff between now and baseline
+                // it is not executed against the baseline
+                qg.ValueDiff() == 0d ?
+                TrendIcon.Constant :
+                (qg.ValueDiff() > 0 ? 
+                  ( qg.MoreIsBad ?  TrendIcon.RedUp: TrendIcon.GreenUp) :
+                  (!qg.MoreIsBad ?  TrendIcon.RedDown: TrendIcon.GreenDown))
+select new { qg, 
+   Evolution      =  evolution,
+
+   BaselineStatus =  relyOnDiff? (QualityGateStatus?) null : qgBaseline.Status,
+   Status         =  qg.Status,
+
+   BaselineValue  =  relyOnDiff? (null) : qgBaseline.ValueString,
+   Value          =  qg.ValueString, 
+}
+ 
+// <Description>
+// Show quality gates evolution between baseline and now.
+//
+// When a quality gate relies on diff between now and baseline (like *New Debt since Baseline*)
+// it is not executed against the baseline and as a consequence its evolution is not available.
+//
+// Double-click a quality gate for editing.
+// </Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Percentage Coverage" Unit="%" />
+failif value < 70%
+warnif value < 80%
+codeBase.PercentageCoverage
+
+//<Description>
+// Code coverage is a measure used to describe the degree to which the source code of a program 
+// is tested by a particular test suite. A program with high code coverage, measured as a percentage, 
+// has had more of its source code executed during testing which suggests it has a lower chance of 
+// containing undetected software bugs compared to a program with low code coverage.
+//
+// Code coverage is certainly the most important quality code metric. But coverage is not enough
+// the team needs to ensure that results are checked at test-time. These checks can be done both 
+// in test code, and in application code through assertions. The important part is that a test
+// must fail explicitely when a check gets unvalidated during the test execution.
+//
+// This quality gate define a warn threshold (70%) and a fail threshold (80%). These are 
+// indicative thresholds and in practice the more the better. To achieve high coverage and 
+// low risk, make sure that new and refactored classes gets 100% covered by tests and that
+// the application and test code contains as many checks/assertions as possible.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Percentage Coverage on New Code" Unit="%" />
+failif value < 70%
+warnif value < 80%
+let newMethods = Application.Methods.Where(m => m.WasAdded() && m.NbLinesOfCode > 0)
+let locCovered = newMethods.Sum(m => m.NbLinesOfCodeCovered)
+let loc = newMethods.Sum(m => m.NbLinesOfCode)
+select 100d * locCovered / loc
+
+//<Description>
+// *New Code* is defined as methods added since the baseline.
+//
+// To achieve high code coverage it is essential that new code gets properly
+// tested and covered by tests. It is advised that all non-UI new classes gets
+// 100% covered.
+//
+// Typically 90% of a class is easy to cover by tests and 10% is hard to reach 
+// through tests. It means that this 10% remaining is not easily testable, which 
+// means it is not well designed, which often means that this code is especially 
+// **error-prone**. This is the reason why it is important to reach 100% coverage
+// for a class, to make sure that potentially *error-prone* code gets tested.
+//</Description>
+]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Percentage Coverage on Refactored Code" Unit="%" />
+failif value < 70%
+warnif value < 80%
+let newMethods = Application.Methods.Where(m => m.CodeWasChanged() && m.NbLinesOfCode > 0)
+let locCovered = newMethods.Sum(m => m.NbLinesOfCodeCovered)
+let loc = newMethods.Sum(m => m.NbLinesOfCode)
+select 100d * locCovered / loc
+
+//<Description>
+// *Refactored Code* is defined as methods where *code was changed* since the baseline.
+//
+// Comment changes and formatting changes are not considerd as refactoring.
+//
+// To achieve high code coverage it is essential that refactored code gets properly
+// tested and covered by tests. It is advised that when refactoring a class
+// or a method, it is important to also write tests to make sure it gets 100% covered.
+//
+// Typically 90% of a class is easy to cover by tests and 10% is hard to reach 
+// through tests. It means that this 10% remaining is not easily testable, which 
+// means it is not well designed, which often means that this code is especially 
+// **error-prone**. This is the reason why it is important to reach 100% coverage
+// for a class, to make sure that potentially *error-prone* code gets tested.
+//</Description>
+]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Blocker Issues" Unit="issues" />
+failif count > 0 issues
+from i in Issues
+where i.Severity == Severity.Blocker
+select new { i, i.Severity, i.Debt, i.AnnualInterest }
+
+//<Description>
+// An issue with the severity **Blocker** cannot move to production, it must be fixed.
+//
+// The severity of an issue is either defined explicitely in the rule source code,
+// either inferred from the issue *annual interest* and thresholds defined in the 
+// NDepend Project Properties > Issue and Debt.
+//</Description>
+
+]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Critical Issues" Unit="issues" />
+failif count > 10 issues
+warnif count > 0 issues
+
+from i in Issues
+where i.Severity == Severity.Critical
+select new { i, i.Severity, i.Debt, i.AnnualInterest }
+
+//<Description>
+// An issue with a severity level **Critical** shouldn't move to production. 
+// It still can for business imperative needs purposes, but at worst it must 
+// be fixed during the next iterations. 
+//
+// The severity of an issue is either defined explicitely in the rule source code,
+// either inferred from the issue *annual interest* and thresholds defined in the 
+// NDepend Project Properties > Issue and Debt.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="New Blocker / Critical / High Issues" Unit="issues" />
+failif count > 0 issues
+from i in Issues
+where i.Severity.EqualsAny(Severity.Blocker, Severity.Critical, Severity.High) &&  
+      // Count both the new issues and the issues that became at least Critical
+      (i.WasAdded() || i.OlderVersion().Severity < Severity.High)
+select new { i, i.Severity, i.Debt, i.AnnualInterest }
+
+
+//<Description>
+// An issue with the severity **Blocker** cannot move to production, it must be fixed.
+//
+// An issue with a severity level **Critical** shouldn't move to production. 
+// It still can for business imperative needs purposes, but at worth it must be fixed 
+// during the next iterations. 
+//
+// An issue with a severity level **High** should be fixed quickly, but can wait until 
+// the next scheduled interval.
+//
+// The severity of an issue is either defined explicitely in the rule source code,
+// either inferred from the issue *annual interest* and thresholds defined in the 
+// NDepend Project Properties > Issue and Debt.
+//</Description>
+]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Critical Rules Violated" Unit="rules" />
+failif count > 0 rules
+from r in Rules where r.IsCritical && r.IsViolated()
+select new { r, issues = r.Issues() }
+
+//<Description>
+// The concept of critical rule is useful to pinpoint certain rules that 
+// should not be violated.
+//
+// A rule can be made critical just by checking the *Critical button* in the
+// rule edition control and then saving the rule.
+//
+// This quality gate fails if any critical rule gets any violations.
+//
+// When no baseline is available, rules that rely on diff are not counted.
+// If you observe that this quality gate count slightly decreases with no apparent reason,
+// the reason is certainly that rules that rely on diff are not counted
+// because the baseline is not defined.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Percentage Debt" Unit="%" />
+failif value > 30%
+warnif value > 20%
+let timeToDev = codeBase.EffortToDevelop()
+let debt = Issues.Sum(i => i.Debt)
+select 100d * debt.ToManDay() / timeToDev.ToManDay()
+
+// <Description>
+// % Debt total is defined as a percentage on:
+//
+// • the estimated total effort to develop the code base
+//
+// • and the the estimated total time to fix all issues (the Debt)
+//
+// Estimated total effort to develop the code base is inferred from 
+// # lines of code of the code base and from the 
+// *Estimated number of man-day to develop 1000 logicial lines of code*
+// setting found in NDepend Project Properties > Issue and Debt.
+//
+// Debt documentation: http://www.ndepend.com/docs/technical-debt#Debt
+//
+// This quality gates fails if the estimated debt is more than 30%
+// of the estimated effort to develop the code base, and warns if the 
+// estimated debt is more than 20% of the estimated effort to develop 
+// the code base
+// </Description>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Debt" Unit="man-days" />
+failif value > 50 man-days
+warnif value > 30 man-days
+Issues.Sum(i => i.Debt).ToManDay()
+
+//<Description>
+// This Quality Gate is disabled per default because the fail and warn 
+// thresholds of unacceptable Debt in man-days can only depend on the 
+// project size, number of developers and overall context.
+//
+// However you can refer to the default Quality Gate **Percentage Debt**.
+//
+// The Debt is defined as the sum of estimated effort to fix all issues.
+// Debt documentation: http://www.ndepend.com/docs/technical-debt#Debt
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="New Debt since Baseline" Unit="man-days" />
+failif value > 2 man-days
+warnif value > 0 man-days
+let debt = Issues.Sum(i => i.Debt)
+let debtInBaseline = IssuesInBaseline.Sum(i => i.Debt)
+select (debt - debtInBaseline).ToManDay()
+
+
+//<Description>
+// This Quality Gate fails if the estimated effort to fix new or worsened
+// issues (what is called the *New Debt since Baseline*) is higher
+// than 2 man-days.
+//
+// This Quality Gate warns if this estimated effort is positive.
+//
+// Debt documentation: http://www.ndepend.com/docs/technical-debt#Debt
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Debt Rating per Namespace" Unit="namespaces" />
+failif count > 0 namespaces
+
+from n in Application.Namespaces
+where n.DebtRating() != null &&
+      n.DebtRating().Value.EqualsAny(DebtRating.E, DebtRating.D)
+select new { 
+   n, 
+   debtRating = n.DebtRating(),
+   debtRatio = n.DebtRatio(),  // % of debt from which DebtRating is inferred
+   devTimeInManDay = n.EffortToDevelop().ToDebt(), 
+   debtInManDay = n.AllDebt(),
+   issues = n.AllIssues() 
+}
+
+// <Description>
+// Forbid namespaces with a poor Debt Rating equals to **E** or **D**.
+//
+// The **Debt Rating** for a code element is estimated by the value of the **Debt Ratio**
+// and from the various rating thresholds defined in this project *Debt Settings*. 
+//
+// The **Debt Ratio** of a code element is a percentage of **Debt Amount** (in floating man-days) 
+// compared to the **estimated effort to develop the code element** (also in floating man-days).
+//
+// The **estimated effort to develop the code element** is inferred from the code elements
+// number of lines of code, and from the project *Debt Settings* parameters 
+// *estimated number of man-days to develop 1000* **logical lines of code**.
+//
+// The **logical lines of code** corresponds to the number of debug breakpoints in a method
+// and doesn't depend on code formatting nor comments.
+//
+// The Quality Gate can be modified to match assemblies, types or methods
+// with a poor Debt Rating, instead of matching namespaces.
+// </Description>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Annual Interest" Unit="man-days" />
+failif value > 50 man-days
+warnif value > 30 man-days
+Issues.Sum(i => i.AnnualInterest).ToManDay()
+
+
+//<Description>
+// This Quality Gate is disabled per default because the fail and warn 
+// thresholds of unacceptable Annual-Interest in man-days can only depend
+// on the project size, number of developers and overall context.
+//
+// However you can refer to the default Quality Gate 
+// **New Annual Interest since Baseline**.
+//
+// The Annual-Interest is defined as the sum of estimated annual cost
+// in man-days, to leave all issues unfixed.
+//
+// Each rule can either provide a formula to compute the Annual-Interest 
+// per issue, or assign a **Severity** level for each issue. Some thresholds
+// defined in *Project Properties > Issue and Debt > Annual Interest* are
+// used to infer an Annual-Interest value from a Severity level.
+// Annual Interest documentation: http://www.ndepend.com/docs/technical-debt#AnnualInterest
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="New Annual Interest since Baseline" Unit="man-days" />
+failif value > 2 man-days
+warnif value > 0 man-days
+let ai = Issues.Sum(i => i.AnnualInterest)
+let aiInBaseline = IssuesInBaseline.Sum(i => i.AnnualInterest)
+select (ai - aiInBaseline).ToManDay()
+
+//<Description>
+// This Quality Gate fails if the estimated annual cost to leave all issues
+// unfixed, increased from more than 2 man-days since the baseline.
+//
+// This Quality Gate warns if this estimated annual cost is positive.
+//
+// This estimated annual cost is named the **Annual-Interest**.
+//
+// Each rule can either provide a formula to compute the Annual-Interest 
+// per issue, or assign a **Severity** level for each issue. Some thresholds
+// defined in *Project Properties > Issue and Debt > Annual Interest* are
+// used to infer an Annual-Interest value from a Severity level.
+// Annual Interest documentation: http://www.ndepend.com/docs/technical-debt#AnnualInterest
+//</Description>]]></Query>
+    </Group>
+    <Group Name="Hot Spots" Active="True" ShownInReport="True">
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Types Hot Spots</Name>
+from t in JustMyCode.Types
+where t.AllDebt() > Debt.Zero && 
+      t.AllAnnualInterest() > AnnualInterest.Zero
+orderby t.AllDebt().Value.TotalMinutes descending
+select new { t, 
+   Debt = t.AllDebt(),
+   Issues = t.AllIssues(), // AllIssues = {types issues} union {members issues}
+   AnnualInterest = t.AllAnnualInterest(),
+   BreakingPoint = t.AllBreakingPoint(),
+   t.NbLinesOfCode,
+   // t.PercentageCoverage,  to uncomment if coverage data is imported
+   DebtRating = t.DebtRating(), 
+   DebtRatio = t.DebtRatio() 
+}
+
+//<Description>
+// This query lists **types with most Debt**,
+// or in other words, types with issues that would need 
+// the largest effort to get fixed.
+//
+// Both issues on the type and its members are
+// taken account.
+//
+// Since untested code often generates a lot of 
+// Debt, the type size and percentage coverage is shown
+// (just uncomment *t.PercentageCoverage* in the query 
+// source code once you've imported the coverage data).
+//
+// The *Debt Rating* and *Debt Ratio* are also shown
+// for informational purpose.
+//
+// --
+//
+// The amount of *Debt* is not a measure to prioritize
+// the effort to fix issues, it is an estimation of how far 
+// the team is from clean code that abides by the rules set.
+//
+// For each issue the *Annual Interest* estimates the annual 
+// cost to leave the issues unfixed. The *Severity* of an issue
+// is estimated through thresholds from the *Annual Interest*.
+//
+// The **Debt Breaking Point** represents the duration
+// from now when the estimated cost to leave the issue unfixed
+// costs as much as the estimated effort to fix it.
+//
+// Hence the shorter the **Debt Breaking Point** 
+// the largest the **Return on Investment** for fixing 
+// the issue. The **Breaking Point is the right metric
+// to prioritize issues fix**.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Types to Fix Priority</Name>
+from t in JustMyCode.Types
+where t.AllBreakingPoint() > TimeSpan.Zero &&
+      t.AllDebt().Value > 30.ToMinutes()
+orderby t.AllBreakingPoint().TotalMinutes ascending
+select new { t, 
+   BreakingPoint = t.AllBreakingPoint(),
+   Debt = t.AllDebt(),
+   AnnualInterest = t.AllAnnualInterest(),
+   Issues = t.AllIssues(),
+   t.NbLinesOfCode,
+   // t.PercentageCoverage,  to uncomment if coverage data is imported
+   DebtRating = t.DebtRating(), 
+   DebtRatio = t.DebtRatio() 
+}
+
+//<Description>
+// This query lists types per increasing
+// **Debt Breaking Point**.
+//
+// For each issue the *Debt* estimates the
+// effort to fix the issue, and the *Annual Interest*
+// estimates the annual cost to leave the issue unfixed.
+// The *Severity* of an issue is estimated through 
+// thresholds from the *Annual Interest* of the issue.
+//
+// The **Debt Breaking Point** represents the duration
+// from now when the estimated cost to leave the issue unfixed
+// costs as much as the estimated effort to fix it.
+//
+// Hence the shorter the **Debt Breaking Point** 
+// the largest the **Return on Investment** for fixing 
+// the issues.
+//
+// Often new and refactored types since baseline will be 
+// listed first, because issues on these types get a 
+// higher *Annual Interest* because it is important to
+// focus first on new issues.
+// 
+//
+// --
+//
+// Both issues on the type and its members are
+// taken account.
+//
+// Only types with at least 30 minutes of Debt are listed
+// to avoid parasiting the list with the numerous
+// types with small *Debt*, on which the *Breaking Point*
+// value makes less sense. 
+//
+// The *Annual Interest* estimates the cost per year
+// in man-days to leave these issues unfixed.
+//
+// Since untested code often generates a lot of 
+// Debt, the type size and percentage coverage is shown
+// (just uncomment *t.PercentageCoverage* in the query 
+// source code once you've imported the coverage data).
+//
+// The *Debt Rating* and *Debt Ratio* are also shown
+// for informational purpose.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Issues to Fix Priority</Name>
+from i in Issues
+// Don't show first issues with BreakingPoint equals to zero.
+orderby i.BreakingPoint != TimeSpan.Zero ? i.BreakingPoint : TimeSpan.MaxValue
+select new { i, 
+   Debt = i.Debt,
+   AnnualInterest = i.AnnualInterest,
+   BreakingPoint = i.BreakingPoint,
+   CodeElement = i.CodeElement
+}
+
+//<Description>
+// This query lists issues per increasing
+// **Debt Breaking Point**.
+//
+// Double-click an issue to edit its rule and
+// select the issue in the rule result. This way
+// you can view all information concerning the issue.
+//
+// For each issue the *Debt* estimates the
+// effort to fix the issue, and the *Annual Interest*
+// estimates the annual cost to leave the issue unfixed.
+// The *Severity* of an issue is estimated through 
+// thresholds from the *Annual Interest* of the issue.
+//
+// The **Debt Breaking Point** represents the duration
+// from now when the estimated cost to leave the issue unfixed
+// costs as much as the estimated effort to fix it.
+//
+// Hence the shorter the **Debt Breaking Point** 
+// the largest the **Return on Investment** for fixing 
+// the issue.
+//
+// Often issues on new and refactored code elements since 
+// baseline will be listed first, because such issues get a 
+// higher *Annual Interest* because it is important to
+// focus first on new issues on recent code.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Debt and Issues per Rule</Name>
+from r in Rules
+where r.IsViolated()
+orderby r.Debt().Value descending
+select new { 
+   r, 
+   Issues = r.Issues(), 
+   Debt = r.Debt(), 
+   AnnualInterest = r.AnnualInterest(), 
+   BreakingPoint = r.BreakingPoint(),
+   Category = r.Category
+}
+
+//<Description>
+// This query lists violated rules with most *Debt* first.
+//
+// A rule violated has issues. For each issue the *Debt* 
+// estimates the effort to fix the issue.
+//
+// --
+//
+// The amount of *Debt* is not a measure to prioritize
+// the effort to fix issues, it is an estimation of how far 
+// the team is from clean code that abides by the rules set.
+//
+// For each issue the *Annual Interest* estimates the annual 
+// cost to leave the issues unfixed. The *Severity* of an issue
+// is estimated through thresholds from the *Annual Interest*.
+//
+// The **Debt Breaking Point** represents the duration
+// from now when the estimated cost to leave the issue unfixed
+// costs as much as the estimated effort to fix it.
+//
+// Hence the shorter the **Debt Breaking Point** 
+// the largest the **Return on Investment** for fixing 
+// the issue. The **Breaking Point is the right metric
+// to prioritize issues fix**.
+//
+// --
+//
+// Notice that rules can be grouped in *Rule Category*. This
+// way you'll see categories that generate most *Debt*.
+//
+// Typically the rules that generate most *Debt* are the 
+// ones related to *Code Coverage by Tests*, *Architecture*
+// and *Code Smells*.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>New Debt and Issues per Rule</Name>
+from r in Rules
+where r.IsViolated() && r.IssuesAdded().Count() > 0
+orderby r.DebtDiff().Value descending
+select new { 
+   r, 
+   IssuesAdded = r.IssuesAdded(),
+   IssuesFixed = r.IssuesFixed(),
+   Issues = r.Issues(), 
+   Debt = r.Debt(), 
+   DebtDiff = r.DebtDiff(),
+   Category = r.Category
+}
+
+//<Description>
+// This query lists violated rules that have new issues
+// since baseline, with most **new Debt** first.
+//
+// A rule violated has issues. For each issue the *Debt* 
+// estimates the effort to fix the issue.
+//
+// --
+//
+// New issues since the baseline are consequence of recent code 
+// refactoring sessions. They represent good opportunities
+// of fix because the code recently refactored is fresh in 
+// the developers mind, which means fixing now costs less
+// than fixing later.
+//
+// Fixing issues on recently touched code is also a good way 
+// to foster practices that will lead to higher code quality 
+// and maintainability, including writing unit-tests
+// and avoiding unnecessary complex code.
+//
+// --
+//
+// Notice that rules can be grouped in *Rule Category*. This
+// way you'll see categories that generate most *Debt*.
+//
+// Typically the rules that generate most *Debt* are the 
+// ones related to *Code Coverage by Tests*, *Architecture*
+// and *Code Smells*.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Debt and Issues per Code Element</Name>
+from elem in CodeElements
+where elem.HasIssue()
+orderby elem.Debt().Value descending
+select new { 
+   elem, 
+   Issues = elem.Issues(), 
+   Debt = elem.Debt(), 
+   AnnualInterest = elem.AnnualInterest(), 
+   BreakingPoint = elem.BreakingPoint()
+}
+
+//<Description>
+// This query lists code elements that have issues, 
+// with most *Debt* first.
+//
+// For each code element the *Debt* estimates 
+// the effort to fix the element issues.
+//
+// The amount of *Debt* is not a measure to prioritize
+// the effort to fix issues, it is an estimation of how far 
+// the team is from clean code that abides by the rules set.
+//
+// For each element the *Annual Interest* estimates the annual 
+// cost to leave the elements issues unfixed. The *Severity* of an 
+// issue is estimated through thresholds from the *Annual Interest*
+// of the issue.
+//
+// The **Debt Breaking Point** represents the duration
+// from now when the estimated cost to leave the issues unfixed
+// costs as much as the estimated effort to fix it.
+//
+// Hence the shorter the **Debt Breaking Point** 
+// the largest the **Return on Investment** for fixing 
+// the issue. The **Breaking Point is the right metric
+// to prioritize issues fix**.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>New Debt and Issues per Code Element</Name>
+from elem in CodeElements
+where elem.HasIssue() && elem.IssuesAdded().Count() > 0
+orderby elem.DebtDiff().Value descending
+select new { 
+   elem, 
+   IssuesAdded = elem.IssuesAdded(),
+   IssuesFixed = elem.IssuesFixed(),
+   Issues = elem.Issues(), 
+   Debt = elem.Debt(), 
+   DebtDiff = elem.DebtDiff()
+}
+    //<Description>
+// This query lists code elements that have new issues
+// since baseline, with most **new Debt** first.
+//
+// For each code element the *Debt* estimates 
+// the effort to fix the element issues.
+//
+// New issues since the baseline are consequence of recent code 
+// refactoring sessions. They represent good opportunities
+// of fix because the code recently refactored is fresh in 
+// the developers mind, which means fixing now costs less
+// than fixing later.
+//
+// Fixing issues on recently touched code is also a good way 
+// to foster practices that will lead to higher code quality 
+// and maintainability, including writing unit-tests
+// and avoiding unnecessary complex code.
+//</Description>
+]]></Query>
+    </Group>
+    <Group Name="Code Smells" Active="True" ShownInReport="False">
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="True"><![CDATA[// <Name>Avoid types too big</Name>
+warnif count > 0 from t in JustMyCode.Types where 
+
+   // First filter on type to optimize 
+   t.NbLinesOfCode > 200 
+   // # IL Instructions is commented, because with LINQ syntax, a few lines of code can compile to hundreds of IL instructions.
+   // || t.NbILInstructions > 3000
+
+   // What matters is the # lines of code in JustMyCode
+   let locJustMyCode = t.MethodsAndContructors.Where(m => JustMyCode.Contains(m)).Sum(m => m.NbLinesOfCode)
+   where locJustMyCode > 200
+
+   let isStaticWithNoMutableState = (t.IsStatic && t.Fields.Any(f => !f.IsImmutable))
+   let staticFactor = (isStaticWithNoMutableState ? 0.2 : 1)
+
+   orderby locJustMyCode descending
+select new { 
+   t, 
+   locJustMyCode, 
+   t.NbILInstructions,
+   t.Methods, 
+   t.Fields,
+   
+   Debt = (staticFactor*locJustMyCode.Linear(200, 1, 2000, 10)).ToHours().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity major for 300 loc
+   // to interest for severity critical for 2000 loc
+   AnnualInterest = staticFactor*(locJustMyCode.Linear(
+                          200,  Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes, 
+                          2000, Severity.Critical.AnnualInterestThreshold().Value.TotalMinutes)).ToMinutes().ToAnnualInterest()
+}
+
+//<Description>
+// This rule matches types with more than 200 lines of code.
+// **Only lines of code in JustMyCode methods are taken account.**
+//
+// Types where *NbLinesOfCode > 200* are extremely complex 
+// to develop and maintain.
+// See the definition of the NbLinesOfCode metric here 
+// http://www.ndepend.com/docs/code-metrics#NbLinesOfCode
+//
+// Maybe you are facing the **God Class** phenomenon:
+// A **God Class** is a class that controls way too many other classes 
+// in the system and has grown beyond all logic to become 
+// *The Class That Does Everything*.
+//</Description>
+
+//<HowToFix>
+// Types with many lines of code
+// should be split in a group of smaller types.
+// 
+// To refactor a *God Class* you'll need patience, 
+// and you might even need to recreate everything from scratch.
+// Here are a few refactoring advices:
+//
+// • The logic in the *God Class* must be splitted in smaller classes.
+// These smaller classes can eventually become private classes nested
+// in the original *God Class*, whose instances objects become 
+// composed of instances of smaller nested classes.
+//
+// • Smaller classes partitioning should be driven by the multiple
+// responsibilities handled by the *God Class*. To identify these 
+// responsibilities it often helps to look for subsets of methods
+// strongly coupled with subsets of fields.
+//
+// • If the *God Class* contains way more logic than states, a good 
+// option can be to define one or several static classes that 
+// contains no static field but only pure static methods. A pure static 
+// method is a function that computes a result only from inputs
+// parameters, it doesn't read nor assign any static or instance field.
+// The main advantage of pure static methods is that they are easily
+// testable.
+// 
+// • Try to maintain the interface of the *God Class* at first 
+// and delegate calls to the new extracted classes. 
+// In the end the *God Class* should be a pure facade without its own logic.
+// Then you can keep it for convenience or throw it away and 
+// start to use the new classes only.
+//
+// • Unit Tests can help: write tests for each method before extracting it 
+// to ensure you don't break functionality.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 1 hour for a 200 lines of code type,
+// up to 10 hours for a type with 2.000 or more lines of code.
+//
+// In Debt and Interest computation, this rule takes account of the fact 
+// that static types with no mutable fields are just a collection of 
+// static methods that can be easily splitted and moved from one type 
+// to another.
+//</HowToFix>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid types with too many methods</Name>
+warnif count > 0 from t in JustMyCode.Types 
+
+  // Optimization: Fast discard of non-relevant types 
+  where t.Methods.Count() > 20
+
+  // Don't match these methods
+  let methods = t.Methods.Where(
+       m => !(m.IsGeneratedByCompiler ||
+              m.IsConstructor || m.IsClassConstructor ||
+              m.IsPropertyGetter || m.IsPropertySetter ||
+              m.IsEventAdder || m.IsEventRemover))
+
+  where methods.Count() > 20 
+  orderby methods.Count() descending
+
+  let isStaticWithNoMutableState = (t.IsStatic && t.Fields.Any(f => !f.IsImmutable))
+  let staticFactor = (isStaticWithNoMutableState ? 0.2 : 1)
+
+select new { 
+   t, 
+   nbMethods = methods.Count(),
+   instanceMethods = methods.Where(m => !m.IsStatic), 
+   staticMethods = methods.Where(m => m.IsStatic),
+
+   t.NbLinesOfCode,
+
+   Debt = (staticFactor*methods.Count().Linear(20, 1, 200, 10)).ToHours().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity major for 30 methods
+   // to interest for severity critical for 200 methods
+   AnnualInterest = (staticFactor*methods.Count().Linear(
+                              20,  Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes, 
+                              200, Severity.Critical.AnnualInterestThreshold().Value.TotalMinutes)).ToMinutes().ToAnnualInterest()
+}
+
+//<Description>
+// This rule matches types with more than 20 methods. 
+// Such type might be hard to understand and maintain.
+//
+// Notice that methods like constructors or property 
+// and event accessors are not taken account.
+//
+// Having many methods for a type might be a symptom
+// of too many responsibilities implemented.
+//
+// Maybe you are facing the **God Class** phenomenon:
+// A **God Class** is a class that controls way too many other classes 
+// in the system and has grown beyond all logic to become 
+// *The Class That Does Everything*.
+//</Description>
+
+//<HowToFix>
+// To refactor properly a *God Class* please read *HowToFix advices* 
+// from the default rule **Types to Big**.
+////
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 1 hour for a type with 20 methods,
+// up to 10 hours for a type with 200 or more methods.
+//
+// In Debt and Interest computation, this rule takes account of the fact 
+// that static types with no mutable fields are just a collection of 
+// static methods that can be easily splitted and moved from one type 
+// to another.
+//</HowToFix>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid types with too many fields</Name>
+warnif count > 0 from t in JustMyCode.Types 
+
+  // Optimization: Fast discard of non-relevant types 
+  where !t.IsEnumeration &&
+         t.Fields.Count() > 15
+        
+  // Count instance fields and non-constant static fields
+  let fields = t.Fields.Where(f =>
+          !f.IsGeneratedByCompiler &&
+          !f.IsLiteral &&
+          !(f.IsStatic && f.IsInitOnly) &&
+           JustMyCode.Contains(f) )
+
+  where fields.Count() > 15
+ 
+  let methodsAssigningFields = fields.SelectMany(f => f.MethodsAssigningMe)
+
+  orderby fields.Count() descending
+select new { 
+   t, 
+   instanceFields = fields.Where(f => !f.IsStatic),
+   staticFields = fields.Where(f => f.IsStatic),
+methodsAssigningFields ,   
+
+   // See definition of Size of Instances metric here:
+   // http://www.ndepend.com/docs/code-metrics#SizeOfInst
+   t.SizeOfInst,
+
+   Debt = fields.Count().Linear(15, 1, 200, 10).ToHours().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity major for 30 methods
+   // to interest for severity critical for 200 methods
+   AnnualInterest = fields.Count().Linear(15,  Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes, 
+                                          200, Severity.Critical.AnnualInterestThreshold().Value.TotalMinutes).ToMinutes().ToAnnualInterest()
+}
+
+//<Description>
+// This rule matches types with more than 15 fields. 
+// Such type might be hard to understand and maintain.
+//
+// Notice that constant fields and static-readonly fields are not counted.
+// Enumerations types are not counted also.
+//
+// Having many fields for a type might be a symptom
+// of too many responsibilities implemented.
+//</Description>
+
+//<HowToFix>
+// To refactor such type and increase code quality and maintainability,
+// certainly you'll have to group subsets of fields into smaller types
+// and dispatch the logic implemented into the methods 
+// into these smaller types.
+//
+// More refactoring advices can be found in the default rule 
+// **Types to Big**, *HowToFix* section.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 1 hour for a type with 15 fields,
+// to up to 10 hours for a type with 200 or more fields. 
+//</HowToFix>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="True"><![CDATA[// <Name>Avoid methods too big, too complex</Name>
+warnif count > 0 from m in JustMyCode.Methods where 
+   m.ILNestingDepth > 2 &&
+  (m.NbLinesOfCode > 35 ||
+   m.CyclomaticComplexity > 20 ||
+   m.ILCyclomaticComplexity > 60)
+
+  let complexityScore = m.NbLinesOfCode/2 + m.CyclomaticComplexity + m.ILCyclomaticComplexity/3 + 3*m.ILNestingDepth
+
+  orderby complexityScore descending,
+          m.CyclomaticComplexity descending,
+          m.ILCyclomaticComplexity descending,
+          m.ILNestingDepth descending
+select new { 
+   m, 
+   m.NbLinesOfCode,
+   m.CyclomaticComplexity, 
+   m.ILCyclomaticComplexity,
+   m.ILNestingDepth,
+   complexityScore,
+
+   Debt = complexityScore.Linear(30, 40,    400, 8*60).ToMinutes().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity minor 
+   // to interest for severity major
+   AnnualInterest = complexityScore .Linear(30,     Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes, 
+                                            200, 2*(Severity.High.AnnualInterestThreshold().Value.TotalMinutes)).ToMinutes().ToAnnualInterest()
+   
+}
+
+//<Description>
+// This rule matches methods where *ILNestingDepth* > 2
+// and (*NbLinesOfCode* > 35 
+// or *CyclomaticComplexity* > 20
+// or *ILCyclomaticComplexity* > 60)
+// Such method is typically hard to understand and maintain.
+//
+// Maybe you are facing the **God Method** phenomenon.
+// A "God Method" is a method that does way too many processes in the system 
+// and has grown beyond all logic to become *The Method That Does Everything*.
+// When need for new processes increases suddenly some programmers realize: 
+// why should I create a new method for each processe if I can only add an *if*.
+//
+// See the definition of the *CyclomaticComplexity* metric here:
+// http://www.ndepend.com/docs/code-metrics#CC
+//
+// See the definition of the *ILCyclomaticComplexity* metric here:
+// http://www.ndepend.com/docs/code-metrics#ILCC
+//
+// See the definition of the *ILNestingDepth* metric here:
+// http://www.ndepend.com/docs/code-metrics#ILNestingDepth
+//</Description>
+
+//<HowToFix>
+// A large and complex method should be split in smaller methods, 
+// or even one or several classes can be created for that.
+//
+// During this process it is important to question the scope of each
+// variable local to the method. This can be an indication if
+// such local variable will become an instance field of the newly created class(es).
+//
+// Large *switch…case* structures might be refactored through the help
+// of a set of types that implement a common interface, the interface polymorphism
+// playing the role of the *switch cases tests*.
+//
+// Unit Tests can help: write tests for each method before extracting it 
+// to ensure you don't break functionality.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies from 40 minutes to 8 hours, linearly from a weighted complexity score.
+//</HowToFix>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="True"><![CDATA[// <Name>Avoid methods with too many parameters</Name>
+warnif count > 0 from m in JustMyCode.Methods where 
+  m.NbParameters >= 7
+  orderby m.NbParameters descending
+select new { 
+   m, 
+   m.NbParameters,
+
+   Debt = m.NbParameters.Linear(7, 1,  40, 6).ToHours().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity Medium for 7 parameters
+   // to interest for severity Critical for 40 parameters
+   AnnualInterest = m.NbParameters.Linear(7,  Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes, 
+                                          40, Severity.Critical.AnnualInterestThreshold().Value.TotalMinutes).ToMinutes().ToAnnualInterest()
+}
+
+//<Description>
+// This rule matches methods with more than 8 parameters.
+// Such method is painful to call and might degrade performance.
+// See the definition of the *NbParameters* metric here: 
+// http://www.ndepend.com/docs/code-metrics#NbParameters
+//</Description>
+
+//<HowToFix>
+// More properties/fields can be added to the declaring type to 
+// handle numerous states. An alternative is to provide 
+// a class or a structure dedicated to handle arguments passing.
+// For example see the class *System.Diagnostics.ProcessStartInfo* 
+// and the method *System.Diagnostics.Process.Start(ProcessStartInfo)*.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 1 hour for a method with 7 parameters,
+// up to 6 hours for a methods with 40 or more parameters.
+//</HowToFix>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid methods with too many local variables</Name>
+warnif count > 0 from m in JustMyCode.Methods where 
+  m.NbVariables > 15 
+  orderby m.NbVariables descending
+select new { 
+   m, 
+   m.NbVariables,
+
+   Debt = m.NbVariables.Linear(15, 1,   80, 6).ToHours().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity Medium for 15 variables
+   // to interest for severity Critical for 80 variables
+   AnnualInterest = m.NbVariables.Linear(15,  Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes, 
+                                         80,  Severity.Critical.AnnualInterestThreshold().Value.TotalMinutes).ToMinutes().ToAnnualInterest()
+
+}
+
+//<Description>
+// This rule matches methods with more than 15 variables.
+//
+// Methods where *NbVariables > 8* are hard to understand and maintain.
+// Methods where *NbVariables > 15* are extremely complex and must be refactored. 
+//
+// See the definition of the *Nbvariables* metric here: 
+// http://www.ndepend.com/docs/code-metrics#Nbvariables
+//</Description>
+
+//<HowToFix>
+// To refactor such method and increase code quality and maintainability,
+// certainly you'll have to split the method into several smaller methods
+// or even create one or several classes to implement the logic.
+//
+// During this process it is important to question the scope of each
+// variable local to the method. This can be an indication if
+// such local variable will become an instance field of the newly created class(es).
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 10 minutes for a method with 15 variables,
+// up to 2 hours for a methods with 80 or more variables.
+//</HowToFix>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid methods with too many overloads</Name>
+warnif count > 0 
+
+let lookup = JustMyCode.Methods.Where(m =>
+   m.NbOverloads >= 6 && 
+  !m.IsOperator && // Don't report operator overload
+
+   // Don't match overloads due tu the visitor pattern, based on a naming convention.
+  !m.SimpleName.ToLower().StartsWithAny("visit", "dispatch")
+).ToLookup(m => m.ParentType.FullName + "."+ m.SimpleName)
+
+from @group in lookup
+let overloads = @group.ToArray()
+orderby overloads.Length descending
+
+select new { 
+   m = @group.First(), 
+   overloads,
+   Debt = (3*overloads.Length).ToMinutes().ToDebt(),
+   Severity = Severity.Medium
+}
+
+//<Description>
+// Method overloading is the ability to create multiple methods of the same name 
+// with different implementations, and various set of parameters.
+//
+// This rule matches sets of methods with 6 overloads or more.
+//
+// Such method set might be a problem to maintain 
+// and provokes coupling higher than necessary.
+//
+// See the definition of the *NbOverloads* metric here 
+// http://www.ndepend.com/docs/code-metrics#NbOverloads
+//</Description>
+
+//<HowToFix>
+// Typically the *too many overloads* phenomenon appears when an algorithm
+// takes a various set of in-parameters. Each overload is presented as 
+// a facility to provide a various set of in-parameters.
+// In such situation, the C# and VB.NET language feature named 
+// *Named and Optional arguments* should be used.
+//
+// The *too many overloads* phenomenon can also be a consequence of the usage
+// of the **visitor design pattern** http://en.wikipedia.org/wiki/Visitor_pattern 
+// since a method named *Visit()* must be provided for each sub type.
+// For this reason, the default version of this rule doesn't match overloads whose name
+// start with "visit" or "dispatch" (case-unsensitive) to avoid match 
+// overload visitors, and you can adapt this rule to your own naming convention.
+//
+// Sometime *too many overloads* phenomenon is not the symptom of a problem,
+// for example when a *numeric to something conversion* method applies to 
+// all numeric and nullable numeric types.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// is of 3 minutes per method overload.
+//</HowToFix>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid methods potentially poorly commented</Name>
+warnif count > 0 from m in JustMyCode.Methods where 
+  m.PercentageComment < 10 && 
+  m.NbLinesOfCode > 20  
+
+  let nbLinesOfCodeNotCommented = m.NbLinesOfCode - m.NbLinesOfComment
+
+  orderby nbLinesOfCodeNotCommented descending
+
+select new { 
+   m, 
+   m.PercentageComment, 
+   m.NbLinesOfCode, 
+   m.NbLinesOfComment,
+   nbLinesOfCodeNotCommented,
+
+   Debt = nbLinesOfCodeNotCommented .Linear(20, 2,  200, 20).ToMinutes().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity major for 300 loc
+   // to interest for severity critical for 2000 loc
+   AnnualInterest = m.PercentageComment.Linear(
+                         0,  8 *(Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes), 
+                         20,     Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes).ToMinutes().ToAnnualInterest()  
+}
+
+//<Description>
+// This rule matches methods with less than 10% of comment lines and that have 
+// at least 20 lines of code. Such method might need to be more commented.
+//
+// See the definitions of the *Comments metric* here:
+// http://www.ndepend.com/docs/code-metrics#PercentageComment
+// http://www.ndepend.com/docs/code-metrics#NbLinesOfComment
+// 
+// Notice that only comments about the method implementation
+// (comments in method body) are taken account.
+//</Description>
+
+//<HowToFix>
+// Typically add more comment. But code commenting is subject to controversy.
+// While poorly written and designed code would needs a lot of comment 
+// to be understood, clean code doesn't need that much comment, especially
+// if variables and methods are properly named and convey enough information.
+// Unit-Test code can also play the role of code commenting.
+//
+// However, even when writing clean and well-tested code, one will have
+// to write **hacks** at a point, usually to circumvent some API limitations or bugs.
+// A hack is a non-trivial piece of code, that doesn't make sense at first glance,
+// and that took time and web research to be found.
+// In such situation comments must absolutely be used to express the intention, 
+// the need for the hacks and the source where the solution has been found.
+//
+// The estimated Debt, which means the effort to comment such method,
+// varies linearly from 2 minutes for 10 lines of code not commented,
+// up to 20 minutes for 200 or more, lines of code not commented.
+//</HowToFix>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid types with poor cohesion</Name>
+warnif count > 0 from t in JustMyCode.Types where 
+  t.LCOM > 0.8  && 
+  t.NbFields > 10 && 
+  t.NbMethods >10 
+
+  let poorCohesionScore = 1/(1.01 - t.LCOM)
+  orderby poorCohesionScore descending
+
+  select new { 
+   t, 
+   t.LCOM, 
+   t.NbMethods, 
+   t.NbFields,
+   poorCohesionScore,
+
+   Debt = poorCohesionScore.Linear(5, 5, 50, 4*60).ToMinutes().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity Medium for low poorCohesionScore
+   // to 4 times interest for severity High for high poorCohesionScore
+   AnnualInterest = poorCohesionScore.Linear(5,     Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes, 
+                                             50, 4*(Severity.High.AnnualInterestThreshold().Value.TotalMinutes)).ToMinutes().ToAnnualInterest()
+   
+}
+
+//<Description>
+// This rule is based on the *LCOM code metric*,
+// LCOM stands for **Lack Of Cohesion of Methods**.
+// See the definition of the LCOM metric here 
+// http://www.ndepend.com/docs/code-metrics#LCOM
+//
+// The LCOM metric measures the fact that most methods are using most fields.
+// A class is considered utterly cohesive (which is good)
+// if all its methods use all its instance fields.
+//
+// Only types with enough methods and fields are taken account to avoid bias.
+// The LCOM takes its values in the range [0-1].
+//
+// This rule matches types with LCOM higher than 0.8.
+// Such value generally pinpoints a **poorly cohesive class**.
+//</Description>
+
+//<HowToFix>
+// To refactor a poorly cohesive type and increase code quality and maintainability,
+// certainly you'll have to split the type into several smaller and more cohesive types
+// that together, implement the same logic.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 5 minutes for a type with a low poorCohesionScore,
+// up to 4 hours for a type with high poorCohesionScore.
+//</HowToFix>]]></Query>
+    </Group>
+    <Group Name="Code Smells Regression" Active="True" ShownInReport="False">
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>From now, all types added should respect basic quality principles</Name>
+warnif count > 0 from t in JustMyCode.Types where
+
+// Only match types added since Baseline.
+// Uncomment this line to match also refactored types since Baseline.
+// (t.WasAdded() || t.CodeWasChanged()) &&
+   t.WasAdded() &&
+
+// Eliminate interfaces, enumerations or types only with constant fields
+// by making sure we are matching type with code.
+t.NbLinesOfCode > 10 &&
+
+// Optimization: Fast discard of non-relevant types 
+(t.Fields.Count() > 20 || t.Methods.Count() > 20)
+      
+// Count instance fields and non-constant static fields
+let fields = t.Fields.Where(f => 
+      !f.IsLiteral &&
+      !(f.IsStatic && f.IsInitOnly))
+
+// Don't match these methods
+let methods = t.Methods.Where(
+   m => !(m.IsConstructor || m.IsClassConstructor ||
+          m.IsGeneratedByCompiler ||
+          m.IsPropertyGetter || m.IsPropertySetter ||
+          m.IsEventAdder || m.IsEventRemover))
+  
+where 
+
+// Low Quality types     Metrics' definitions are available here:
+//     http://www.ndepend.com/docs/code-metrics#MetricsOnTypes
+(  // Types with too many methods
+   fields.Count() > 20 ||
+
+   methods.Count() > 20 ||
+               
+   // Complex Types that use more than 50 other types
+   t.NbTypesUsed > 50
+)
+select new { 
+   t, 
+   t.NbLinesOfCode, 
+
+   instanceMethods = methods.Where(m => !m.IsStatic), 
+   staticMethods = methods.Where(m => m.IsStatic),
+  
+   instanceFields = fields.Where(f => !f.IsStatic),
+   staticFields = fields.Where(f => f.IsStatic),
+  
+   t.TypesUsed,
+
+   // Constant Debt estimation, since for such type rules in category "Code Smells" 
+   // accurately estimate the Debt.
+   Debt     = 10.ToMinutes().ToDebt(),
+
+   // The Severity is higher for new types than for refactored types 
+   AnnualInterest= (t.WasAdded() ? 3 : 1) *
+                    Severity.High.AnnualInterestThreshold()
+}
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+// This rule operates only on types added since baseline.
+//
+// This rule can be easily modified to also match types refactored since baseline,
+// that don't satisfy all quality criterions.
+//
+// Types matched by this rule not only have been recently added or refactored,
+// but also somehow violate one or several basic quality principles,
+// whether it has too many methods,
+// it has too many fields,
+// or is using too many types.
+// Any of these criterions is often a symptom of a type with too many responsibilities.
+//
+// Notice that to count methods and fields, methods like constructors 
+// or property and event accessors are not taken account.
+// Notice that constants fields and static-readonly fields are not counted.
+// Enumerations types are not counted also.
+//</Description>
+
+//<HowToFix>
+// To refactor such type and increase code quality and maintainability,
+// certainly you'll have to split the type into several smaller types
+// that together, implement the same logic.
+//
+// Issues of this rule have a constant 10 minutes Debt, because the Debt,
+// which means the effort to fix such issue, is already estimated for issues
+// of rules in the category **Code Smells**.
+//
+// However issues of this rule have a **High** severity, with even more 
+// interests for issues on new types since baseline, because the proper time 
+// to increase the quality of these types is **now**, before they get commited 
+// in the next production release.
+//</HowToFix>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>From now, all types added should be 100% covered by tests</Name>
+warnif count > 0 from t in JustMyCode.Types where
+
+// Only match types added since Baseline.
+// Uncomment this line to match also refactored types since Baseline.
+// (t.WasAdded() || t.CodeWasChanged()) &&
+   t.WasAdded() &&
+
+  // …that are not 100% covered by tests
+  t.PercentageCoverage < 100
+
+  let methodsCulprit = t.Methods.Where(m => m.PercentageCoverage < 100)
+
+select new { 
+   t, 
+   t.PercentageCoverage, 
+   methodsCulprit,
+   t.NbLinesOfCode,
+
+   // Constant Debt estimation, since for such type rules in category "Coverage" 
+   // accurately estimate the untested code Debt.
+   Debt     = 10.ToMinutes().ToDebt(),
+
+   // The Severity is higher for new types than for refactored types 
+   AnnualInterest= (t.WasAdded() ? 3 : 1) *
+                    Severity.High.AnnualInterestThreshold()
+}
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+// This rule operates only on types added since baseline.
+//
+// This rule can be easily modified to also match types refactored since baseline,
+// that are not 100% covered by tests.
+//
+// This rule is executed only if some code coverage data is imported
+// from some code coverage files.
+//
+// Often covering 10% of remaining uncovered code of a class, 
+// requires as much work as covering the first 90%.
+// For this reason, typically teams estimate that 90% coverage is enough.
+// However *untestable code* usually means *poorly written code* 
+// which usually leads to *error prone code*.
+// So it might be worth refactoring and making sure to cover the 10% remaining code
+// because **most tricky bugs might come from this small portion of hard-to-test code**.
+//
+// Not all classes should be 100% covered by tests (like UI code can be hard to test)
+// but you should make sure that most of the logic of your application
+// is defined in some *easy-to-test classes*, 100% covered by tests.
+//
+// In this context, this rule warns when a type added or refactored since the baseline,
+// is not fully covered by tests.
+//</Description>
+
+//<HowToFix>
+// Write more unit-tests dedicated to cover code not covered yet.
+// If you find some *hard-to-test code*, it is certainly a sign that this code
+// is not *well designed* and hence, needs refactoring.
+//
+// You'll find code impossible to cover by unit-tests, like calls to *MessageBox.Show()*.
+// An infrastructure must be defined to be able to *mock* such code at test-time.
+//
+// Issues of this rule have a constant 10 minutes Debt, because the Debt,
+// which means the effort to write tests for the culprit type, is already 
+// estimated for issues in the category **Code Coverage**. 
+//
+// However issues of this rule have a **High** severity, with even more 
+// interests for issues on new types since baseline, because the proper time 
+// to write tests for these types is **now**, before they get commited 
+// in the next production release.
+//</HowToFix>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>From now, all methods added should respect basic quality principles</Name>
+warnif count > 0 from m in JustMyCode.Methods where
+
+// Only match methods added since Baseline.
+// Uncomment this line to match also refactored methods since Baseline.
+// (m.WasAdded() || m.CodeWasChanged()) &&
+   m.WasAdded() &&
+ 
+// Low Quality methods// Metrics' definitions
+(  m.NbLinesOfCode > 30 ||          // http://www.ndepend.com/docs/code-metrics#NbLinesOfCode
+   m.NbILInstructions > 200 ||      // http://www.ndepend.com/docs/code-metrics#NbILInstructions
+   m.CyclomaticComplexity > 20 ||   // http://www.ndepend.com/docs/code-metrics#CC
+   m.ILCyclomaticComplexity > 50 || // http://www.ndepend.com/docs/code-metrics#ILCC
+   m.ILNestingDepth > 4 ||          // http://www.ndepend.com/docs/code-metrics#ILNestingDepth
+   m.NbParameters > 5 ||            // http://www.ndepend.com/docs/code-metrics#NbParameters
+   m.NbVariables > 8 ||             // http://www.ndepend.com/docs/code-metrics#NbVariables
+   m.NbOverloads > 6 )
+select new { 
+   m, 
+   m.NbLinesOfCode,
+   m.NbILInstructions,
+   m.CyclomaticComplexity, 
+   m.ILCyclomaticComplexity,
+   m.ILNestingDepth, 
+   m.NbParameters, 
+   m.NbVariables, 
+   m.NbOverloads, // http://www.ndepend.com/docs/code-metrics#NbOverloads
+
+   // Constant Debt estimation, since for such method rules in category "Code Smells" 
+   // accurately estimate the Debt.
+   Debt     = 5.ToMinutes().ToDebt(),
+
+   // The Severity is higher for new methods than for refactored methods
+   AnnualInterest= (m.WasAdded() ? 3 : 1) *
+                   Severity.High.AnnualInterestThreshold()
+}  
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+// This rule operates only on methods added or refactored since the baseline.
+//
+// This rule can be easily modified to also match methods refactored since baseline,
+// that don't satisfy all quality criterions.
+//
+// Methods matched by this rule not only have been recently added or refactored,
+// but also somehow violate one or several basic quality principles,
+// whether it is too large (too many *lines of code*), 
+// too complex (too many *if*, *switch case*, loops…)
+// has too many variables, too many parameters
+// or has too many overloads.
+//</Description>
+
+//<HowToFix>
+// To refactor such method and increase code quality and maintainability,
+// certainly you'll have to split the method into several smaller methods
+// or even create one or several classes to implement the logic.
+//
+// During this process it is important to question the scope of each
+// variable local to the method. This can be an indication if
+// such local variable will become an instance field of the newly created class(es).
+//
+// Large *switch…case* structures might be refactored through the help
+// of a set of types that implement a common interface, the interface polymorphism
+// playing the role of the *switch cases tests*.
+//
+// Unit Tests can help: write tests for each method before extracting it 
+// to ensure you don't break functionality.
+//
+// Issues of this rule have a constant 5 minutes Debt, because the Debt,
+// which means the effort to fix such issue, is already estimated for issues
+// of rules in the category **Code Smells**.
+// 
+// However issues of this rule have a **High** severity, with even more 
+// interests for issues on new methods since baseline, because the proper time 
+// to increase the quality of these methods is **now**, before they get commited 
+// in the next production release.
+//</HowToFix>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid decreasing code coverage by tests of types</Name>
+warnif count > 0 
+from t in JustMyCode.Types where
+  t.IsPresentInBothBuilds() && t.CoverageDataAvailable && t.OlderVersion().CoverageDataAvailable
+let locDiff = (int)t.NbLinesOfCode.Value - (int)t.OlderVersion().NbLinesOfCode.Value
+where locDiff >= 0
+let uncoveredLoc  = (int)t.NbLinesOfCodeNotCovered.Value - ((int)t.OlderVersion().NbLinesOfCodeNotCovered.Value + locDiff)
+where uncoveredLoc > 0
+
+orderby uncoveredLoc descending
+
+select new { 
+   t,
+   OldCoveragePercent = t.OlderVersion().PercentageCoverage,
+   NewCoveragePercent = t.PercentageCoverage,
+   OldLoc = t.OlderVersion().NbLinesOfCode,
+   NewLoc = t.NbLinesOfCode,
+   uncoveredLoc,
+
+   Debt = uncoveredLoc.Linear(1, 15,  100, 3*60).ToMinutes().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity High for one line of code that is not covered by tests anymore
+   // to interest for severity Critical for 50 lines of code that are not covered by tests anymore
+   AnnualInterest = uncoveredLoc.Linear(1,    Severity.High.AnnualInterestThreshold().Value.TotalMinutes, 
+                                        50, 2*Severity.Critical.AnnualInterestThreshold().Value.TotalMinutes).ToMinutes().ToAnnualInterest()
+   
+
+}
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+//
+// This rule is executed only if some code coverage data is imported
+// from some code coverage files.
+//
+// This rule warns when the number of lines of a type covered by tests
+// decreased since the baseline. In case the type faced some refactoring
+// since the baseline, this loss in coverage is estimated only for types 
+// with more lines of code, where # lines of code covered now is lower
+// than # lines of code covered in baseline + the extra number of
+// lines of code.
+//
+// Such situation can mean that some tests have been removed
+// but more often, this means that the type has been modified,
+// and that changes haven't been covered properly by tests.
+//
+// To visualize changes in code, right-click a matched type and select:
+//
+// • Compare older and newer versions of source file
+//
+// • or Compare older and newer versions disassembled with Reflector
+//</Description>
+
+//<HowToFix>
+// Write more unit-tests dedicated to cover changes in matched types
+// not covered yet.
+// If you find some *hard-to-test code*, it is certainly a sign that this code
+// is not *well designed* and hence, needs refactoring.
+//
+// The estimated Debt, which means the effort to cover by test 
+// code that used to be covered, varies linearly 15 minutes to 3 hours,
+// depending on the number of lines of code that are not covered by tests anymore.
+//
+// Severity of issues of this rule varies from **High** to **Critical** 
+// depending on the number of lines of code that are not covered by tests anymore.
+// Because the loss in code coverage happened since the baseline,
+// the severity is high because it is important to focus on these issues 
+// **now**, before such code gets released in production.
+//</HowToFix>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid making complex methods even more complex</Name>
+warnif count > 0 
+
+let complexityScoreProc = new Func<IMethod, double>(m => 
+    (m.CyclomaticComplexity + m.ILCyclomaticComplexity/3 + 5*m.ILNestingDepth).Value)
+
+from m in JustMyCode.Methods where
+ !m.IsAbstract &&
+  m.IsPresentInBothBuilds() &&
+  m.CodeWasChanged() &&
+  m.OlderVersion().CyclomaticComplexity > 6 
+
+let complexityScore = complexityScoreProc(m)
+let oldComplexityScore = complexityScoreProc(m.OlderVersion())
+where complexityScore > oldComplexityScore 
+
+let complexityScoreDiff = complexityScoreProc(m) - complexityScoreProc(m.OlderVersion())
+orderby complexityScoreDiff descending
+
+select new { 
+   m,
+   oldComplexityScore ,
+   complexityScore ,
+   diff= complexityScoreDiff,
+
+   Debt = complexityScoreDiff.Linear(1, 15,   50, 60).ToMinutes().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity Medium for a tiny complexity increment
+   // to interest for severity critical for 2000 loc
+   AnnualInterest = complexityScoreDiff.Linear(1,     Severity.High.AnnualInterestThreshold().Value.TotalMinutes, 
+                                               50, 4*(Severity.High.AnnualInterestThreshold().Value.TotalMinutes)).ToMinutes().ToAnnualInterest()
+   
+}
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+//
+// The method complexity is measured through the code metric
+// *Cyclomatic Complexity* defined here:
+// http://www.ndepend.com/docs/code-metrics#CC
+//
+// This rule warns when a method already complex
+// (i.e with *Cyclomatic Complexity* higher than 6)
+// become even more complex since the baseline.
+//
+// This rule needs assemblies PDB files and source code 
+// to be available at analysis time, because the *Cyclomatic Complexity*
+// is inferred from the source code and source code location
+// is inferred from PDB files. See:
+// http://www.ndepend.com/docs/ndepend-analysis-inputs-explanation
+//
+// To visualize changes in code, right-click a matched method and select:
+//
+// • Compare older and newer versions of source file
+//
+// • or Compare older and newer versions disassembled with Reflector
+//</Description>
+
+//<HowToFix>
+// A large and complex method should be split in smaller methods, 
+// or even one or several classes can be created for that.
+//
+// During this process it is important to question the scope of each
+// variable local to the method. This can be an indication if
+// such local variable will become an instance field of the newly created class(es).
+//
+// Large *switch…case* structures might be refactored through the help
+// of a set of types that implement a common interface, the interface polymorphism
+// playing the role of the *switch cases tests*.
+//
+// Unit Tests can help: write tests for each method before extracting it 
+// to ensure you don't break functionality.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 15 to 60 minutes depending on the extra complexity added.
+//
+// Issues of this rule have a **High** severity, because it is important to focus 
+// on these issues **now**, before such code gets released in production.
+//</HowToFix>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid making large methods even larger</Name>
+
+warnif count > 0 
+from m in JustMyCode.Methods where
+ !m.IsAbstract &&
+
+ // Eliminate constructors from match, since they get larger
+ // as soons as some fields initialization are added.
+ !m.IsConstructor &&
+ !m.IsClassConstructor &&
+
+  // Filter just here for optimization
+  m.NbLinesOfCode > 15 &&
+
+  m.IsPresentInBothBuilds() &&
+  m.CodeWasChanged() 
+
+let oldLoc = m.OlderVersion().NbLinesOfCode
+where oldLoc > 15 && m.NbLinesOfCode > oldLoc
+
+let diff = m.NbLinesOfCode - oldLoc
+where diff > 0
+orderby diff descending 
+
+select new { 
+   m,
+   oldLoc,
+   newLoc = m.NbLinesOfCode,
+   diff,
+
+   Debt = diff.Linear(1, 10,   100, 60).ToMinutes().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity Medium for a tiny complexity increment
+   // to interest for severity critical for 2000 loc
+   AnnualInterest = diff .Linear(1,      Severity.High.AnnualInterestThreshold().Value.TotalMinutes, 
+                                 100, 4*(Severity.High.AnnualInterestThreshold().Value.TotalMinutes)).ToMinutes().ToAnnualInterest()
+   
+}
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+//
+// This rule warns when a method already large
+// (i.e with more than 15 lines of code)
+// become even larger since the baseline.
+//
+// The method size is measured through the code metric
+// *# Lines of Code* defined here:
+// http://www.ndepend.com/docs/code-metrics#NbLinesOfCode
+//
+// This rule needs assemblies PDB files 
+// to be available at analysis time, because the *# Lines of Code*
+// is inferred from PDB files. See:
+// http://www.ndepend.com/docs/ndepend-analysis-inputs-explanation
+//
+// To visualize changes in code, right-click a matched method and select:
+//
+// • Compare older and newer versions of source file
+//
+// • or Compare older and newer versions disassembled with Reflector
+//</Description>
+
+//<HowToFix>
+// Usually too big methods should be split in smaller methods.
+//
+// But long methods with no branch conditions, that typically initialize some data,
+// are not necessarily a problem to maintain, and might not need refactoring.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 5 to 20 minutes depending 
+// on the number of lines of code added.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 10 to 60 minutes depending on the extra complexity added.
+//
+// Issues of this rule have a **High** severity, because it is important to focus 
+// on these issues **now**, before such code gets released in production.
+//</HowToFix>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid adding methods to a type that already had many methods</Name>
+
+warnif count > 0 
+
+// Don't count constructors and methods generated by the compiler!
+let getMethodsProc = new Func<IType, IList<IMethod>>(
+   t => t.Methods.Where(m =>
+      !m.IsConstructor && !m.IsClassConstructor && 
+      !m.IsGeneratedByCompiler).ToArray()) 
+
+
+from t in JustMyCode.Types where
+  
+  t.NbMethods > 30 &&   // Just here for optimization
+
+  t.IsPresentInBothBuilds()
+
+  // Optimization: fast discard of non-relevant types
+  where t.OlderVersion().NbMethods > 30
+
+  let oldMethods = getMethodsProc(t.OlderVersion())
+  where oldMethods.Count > 30
+
+  let newMethods = getMethodsProc(t)
+  where newMethods.Count > oldMethods.Count
+
+  let addedMethods = newMethods.Where(m => m.WasAdded())
+  let removedMethods = oldMethods.Where(m => m.WasRemoved())
+
+  orderby addedMethods.Count() descending
+
+select new { 
+   t,
+   nbOldMethods = oldMethods.Count,
+   nbNewMethods = newMethods.Count,
+   addedMethods,
+   removedMethods,
+
+   Debt           = (10*addedMethods.Count()).ToMinutes().ToDebt(),
+   AnnualInterest =     addedMethods.Count().Linear(
+          1,       Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes, 
+          100,  4*(Severity.High.AnnualInterestThreshold().Value.TotalMinutes)).ToMinutes().ToAnnualInterest()
+}
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+//
+// Types where number of methods is greater than 15 
+// might be hard to understand and maintain.
+//
+// This rule lists types that already had more than 15 methods
+// at the baseline time, and for which new methods have been added.
+//
+// Having many methods for a type might be a symptom
+// of too many responsibilities implemented.
+//
+// Notice that constructors and methods generated by the compiler 
+// are not taken account.
+//</Description>
+
+//<HowToFix>
+// To refactor such type and increase code quality and maintainability,
+// certainly you'll have to split the type into several smaller types
+// that together, implement the same logic.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// is equal to 10 minutes per method added.
+//
+// Issues of this rule have a **High** severity, because it is important to focus 
+// on these issues **now**, before such code gets released in production.
+//</HowToFix>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid adding instance fields to a type that already had many instance fields</Name>
+
+warnif count > 0 
+
+let getFieldsProc = new Func<IType, IList<IField>>(
+   t => t.Fields.Where(f => 
+          !f.IsLiteral &&
+          !f.IsGeneratedByCompiler &&
+          !f.IsStatic).ToArray()) 
+
+
+from t in JustMyCode.Types where
+  
+ !t.IsEnumeration &&
+  t.IsPresentInBothBuilds()
+
+  // Optimization: fast discard of non-relevant types
+  where t.OlderVersion().NbFields > 15
+
+  let oldFields = getFieldsProc(t.OlderVersion())
+  where oldFields.Count > 15
+
+  let newFields = getFieldsProc(t)
+  where newFields.Count > oldFields.Count
+
+  let addedFields = newFields.Where(f => f.WasAdded())
+  let removedFields = oldFields.Where(f => f.WasRemoved())
+
+  orderby addedFields.Count() descending
+
+select new { 
+   t,
+   nbOldFields = oldFields.Count,
+   nbNewFields = newFields.Count,
+   addedFields,
+   removedFields,
+
+   Debt           = (10*addedFields.Count()).ToMinutes().ToDebt(),
+   AnnualInterest =     addedFields.Count().Linear(
+          1,       Severity.High.AnnualInterestThreshold().Value.TotalMinutes, 
+          100,  4*(Severity.High.AnnualInterestThreshold().Value.TotalMinutes)).ToMinutes().ToAnnualInterest()
+
+}
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+//
+// Types where number of fields is greater than 15 
+// might be hard to understand and maintain.
+//
+// This rule lists types that already had more than 15 fields
+// at the baseline time, and for which new fields have been added.
+//
+// Having many fields for a type might be a symptom
+// of too many responsibilities implemented.
+//
+// Notice that *constants* fields and *static-readonly* fields are not taken account.
+// Enumerations types are not taken account also.
+//</Description>
+
+//<HowToFix>
+// To refactor such type and increase code quality and maintainability,
+// certainly you'll have to group subsets of fields into smaller types
+// and dispatch the logic implemented into the methods 
+// into these smaller types.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// is equal to 10 minutes per field added.
+//
+// Issues of this rule have a **High** severity, because it is important to focus 
+// on these issues **now**, before such code gets released in production.
+//</HowToFix>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[//<Name>Avoid transforming an immutable type into a mutable one</Name>
+
+warnif count > 0
+from t in Application.Types where
+   t.CodeWasChanged() &&
+   t.OlderVersion().IsImmutable &&
+  !t.IsImmutable && 
+  // Don't take account of immutable types transformed into static types (not deemed as immutable)
+  !t.IsStatic
+
+let culpritFields = t.InstanceFields.Where(f => f.IsImmutable)
+select new {
+   t, 
+   culpritFields,
+   Debt = (10 + 10*culpritFields.Count()).ToMinutes().ToDebt(),
+   Severity = Severity.High
+}
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+//
+// A type is considered as *immutable* if its instance fields
+// cannot be modified once an instance has been built by a constructor.
+//
+// Being immutable has several fortunate consequences for a type.
+// For example its instance objects can be used concurrently 
+// from several threads without the need to synchronize accesses.
+//
+// Hence users of such type often rely on the fact that the type is immutable.
+// If an immutable type becomes mutable, there are chances that this will break 
+// users code.
+//
+// This is why this rule warns about such immutable type that become mutable.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// is equal to 2 minutes per instance field that became mutable.
+//</Description>
+
+//<HowToFix>
+// If being immutable is an important property for a matched type,
+// then the code must be refactored to preserve immutability.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// is equal to 10 minutes plus 10 minutes per instance fields of
+// the matched type that is now mutable.
+//
+// Issues of this rule have a **High** severity, because it is important to focus 
+// on these issues **now**, before such code gets released in production.
+//</HowToFix>]]></Query>
+    </Group>
+    <Group Name="Object Oriented Design" Active="True" ShownInReport="True">
+      <Query Active="False" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid interfaces too big</Name>
+warnif count > 0 
+
+from i in JustMyCode.Types
+where i.IsInterface && i.NbMethods >= 10 // Optimization  First threshold
+
+// A get;set; property count as one method
+let properties = i.Methods.Where(m => m.SimpleName.Length > 4 && (m.IsPropertyGetter || m.IsPropertySetter))
+  .Distinct(m => m.SimpleName.Substring(4, m.SimpleName.Length -4))
+
+// An event count as one method
+let events = i.Methods.Where(m => (m.IsEventAdder|| m.IsEventRemover))
+  .Distinct(m => m.SimpleName.Replace("add_","").Replace("remove_",""))
+
+let methods = i.Methods.Where(m => !m.IsPropertyGetter && !m.IsPropertySetter && !m.IsEventAdder && !m.IsEventRemover)
+let methodsCount = methods.Count() + properties.Count() + events.Count()
+where methodsCount >= 10
+let publicFactor = i.IsPubliclyVisible ? 1 : 0.5
+orderby methodsCount descending
+select new {  
+   i, 
+   Methods= methods,
+   Properties = properties,
+   Events = events,
+   Debt = (publicFactor*methodsCount.Linear(10, 20,   100, 7*60)).ToMinutes().ToDebt(),
+   // The annual interest varies linearly from interest for severity Medium for an interface with 10 methods
+   // to interest for severity Critical for an interface with 100 methods and more
+   AnnualInterest = (publicFactor*methodsCount.Linear(
+                       10,  Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes,
+                       100, Severity.Critical.AnnualInterestThreshold().Value.TotalMinutes))
+                     .ToMinutes().ToAnnualInterest()
+}
+
+
+//<Description>
+// This rule matches interfaces with more than 10 methods.
+// Interfaces are abstractions and are meant to simplify the code structure.
+// An interface should represent a single responsibility.
+// Making an interface too large, too complex, necessarily means
+// that the interface has too many responsibilities.
+//
+// A property with getter or setter or both count as one method.
+// An event count as one method.
+//</Description>
+
+//<HowToFix>
+// Typically to fix such issue, the interface must be refactored
+// in a grape of smaller *single-responsibility* interfaces.
+//
+// A classic example is a *ISession* large interface, responsible
+// for holding states, run commands and offer various accesses
+// and facilities.
+//
+// The classic problem for a large public interface is that it has
+// many clients that consume it. As a consequence splitting it in 
+// smaller interfaces has an important impact and it is not always
+// feasible.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 20 minutes for an interface with 10 methods,
+// up to 7 hours for an interface with 100 or more methods.
+// The Debt is divided by two if the interface is not publicly
+// visible, because in such situation only the current project is impacted
+// by the refactoring.
+//</HowToFix>
+]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Base class should not use derivatives</Name>
+warnif count > 0 
+let excludedTypes = new[] {"TcpDiscoveryIpFinderBase", "EvictionPolicyBase", "PlatformTargetAdapter"}
+from baseClass in JustMyCode.Types
+where baseClass.IsClass  && !excludedTypes.Contains(baseClass.Name)
+      && baseClass.NbChildren > 0 // <-- for optimization!
+let derivedClassesUsed = baseClass.DerivedTypes.UsedBy(baseClass)
+   // Don't warn when a base class is using nested private derived class
+   .Where(derivedClass => 
+            !(derivedClass.IsNested && 
+              derivedClass.Visibility == Visibility.Private && 
+              derivedClass.ParentType == baseClass
+              ))
+where derivedClassesUsed.Count() > 0
+
+let derivedClassesMemberUsed = derivedClassesUsed.SelectMany(c => c.Members).UsedBy(baseClass)
+orderby derivedClassesMemberUsed.Count() descending
+
+select new { 
+   baseClass, 
+   derivedClassesUsed,
+   derivedClassesMemberUsed,
+
+   Debt = 3*(derivedClassesUsed.Count()+derivedClassesMemberUsed.Count()).ToMinutes().ToDebt(),
+   Severity = Severity.High
+}
+
+//<Description>
+// In *Object-Oriented Programming*, the **open/closed principle** states:
+// *software entities (components, classes, methods, etc.) should be open 
+// for extension, but closed for modification*. 
+// http://en.wikipedia.org/wiki/Open/closed_principle
+//
+// Hence a base class should be designed properly to make it easy to derive from,
+// this is *extension*. But creating a new derived class, or modifying an
+// existing one, shouldn't provoke any *modification* in the base class.
+// And if a base class is using some derivative classes somehow, there
+// are good chances that such *modification* will be needed.
+//
+// Extending the base class is not anymore a simple operation,
+// this is not good design.
+//
+// Note that this rule doesn't warn when a base class is using a derived class 
+// that is nested in t

<TRUNCATED>