You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/06/09 08:32:54 UTC

[05/24] incubator-ignite git commit: ignite-545: merge from ignite-sprint-6

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Bytes.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Bytes.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Bytes.java
new file mode 100644
index 0000000..31eaaca
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Bytes.java
@@ -0,0 +1,113 @@
+/*
+ * 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.h2.twostep.msg;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.nio.*;
+
+/**
+ * H2 Bytes.
+ */
+public class GridH2Bytes extends GridH2ValueMessage {
+    /** */
+    private byte[] b;
+
+    /**
+     *
+     */
+    public GridH2Bytes() {
+        // No-op.
+    }
+
+    /**
+     * @param val Value.
+     */
+    public GridH2Bytes(Value val) {
+        assert val.getType() == Value.BYTES : val.getType();
+
+        b = val.getBytesNoCopy();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value value(GridKernalContext ctx) {
+        return ValueBytes.getNoCopy(b);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeByteArray("b", b))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                b = reader.readByteArray("b");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -16;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2CacheObject.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2CacheObject.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2CacheObject.java
new file mode 100644
index 0000000..065393d
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2CacheObject.java
@@ -0,0 +1,148 @@
+/*
+ * 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.h2.twostep.msg;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.query.h2.opt.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.nio.*;
+
+/**
+ * H2 Cache object message.
+ */
+public class GridH2CacheObject extends GridH2ValueMessage {
+    /** */
+    private int cacheId;
+
+    /** */
+    private CacheObject obj;
+
+    /**
+     *
+     */
+    public GridH2CacheObject() {
+        // No-op.
+    }
+
+    /**
+     * @param v Value.
+     * @throws IgniteCheckedException If failed.
+     */
+    public GridH2CacheObject(GridH2ValueCacheObject v) throws IgniteCheckedException {
+        this.obj = v.getCacheObject();
+
+        GridCacheContext<?,?> cctx = v.getCacheContext();
+
+        if (cctx != null) {
+            this.cacheId = cctx.cacheId();
+
+            obj.prepareMarshal(cctx.cacheObjectContext());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value value(GridKernalContext ctx) throws IgniteCheckedException {
+        GridCacheContext<?,?> cctx = null;
+
+        if (ctx != null) {
+            cctx = ctx.cache().context().cacheContext(cacheId);
+
+            obj.finishUnmarshal(cctx.cacheObjectContext(), cctx.deploy().globalLoader());
+        }
+
+        return new GridH2ValueCacheObject(cctx, obj);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                cacheId = reader.readInt("cacheId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                obj = reader.readMessage("obj");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeInt("cacheId", cacheId))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeMessage("obj", obj))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -22;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 2;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Date.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Date.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Date.java
new file mode 100644
index 0000000..e5a8372
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Date.java
@@ -0,0 +1,115 @@
+/*
+ * 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.h2.twostep.msg;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.nio.*;
+
+/**
+ * H2 Date.
+ */
+public class GridH2Date extends GridH2ValueMessage {
+    /** */
+    private long date;
+
+    /**
+     *
+     */
+    public GridH2Date() {
+        // No-op.
+    }
+
+    /**
+     * @param val Value.
+     */
+    public GridH2Date(Value val) {
+        assert val.getType() == Value.DATE : val.getType();
+
+        ValueDate d = (ValueDate)val;
+
+        date = d.getDateValue();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value value(GridKernalContext ctx) {
+        return ValueDate.fromDateValue(date);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeLong("date", date))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                date = reader.readLong("date");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -14;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Decimal.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Decimal.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Decimal.java
new file mode 100644
index 0000000..70700a8
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Decimal.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.query.h2.twostep.msg;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.math.*;
+import java.nio.*;
+
+/**
+ * H2 Decimal.
+ */
+public class GridH2Decimal extends GridH2ValueMessage {
+    /** */
+    private int scale;
+
+    /** */
+    private byte[] b;
+
+    /**
+     *
+     */
+    public GridH2Decimal() {
+        // No-op.
+    }
+
+    /**
+     * @param val Value.
+     */
+    public GridH2Decimal(Value val) {
+        assert val.getType() == Value.DECIMAL : val.getType();
+
+        BigDecimal x = val.getBigDecimal();
+
+        scale = x.scale();
+        b = x.unscaledValue().toByteArray();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value value(GridKernalContext ctx) {
+        return ValueDecimal.get(new BigDecimal(new BigInteger(b), scale));
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeByteArray("b", b))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeInt("scale", scale))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                b = reader.readByteArray("b");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                scale = reader.readInt("scale");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -10;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 2;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Double.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Double.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Double.java
new file mode 100644
index 0000000..bdb796f
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Double.java
@@ -0,0 +1,113 @@
+/*
+ * 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.h2.twostep.msg;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.nio.*;
+
+/**
+ * H2 Double.
+ */
+public class GridH2Double extends GridH2ValueMessage {
+    /** */
+    private double x;
+
+    /**
+     *
+     */
+    public GridH2Double() {
+        // No-op.
+    }
+
+    /**
+     * @param val Value.
+     */
+    public GridH2Double(Value val) {
+        assert val.getType() == Value.DOUBLE : val.getType();
+
+        x = val.getDouble();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value value(GridKernalContext ctx) {
+        return ValueDouble.get(x);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeDouble("x", x))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                x = reader.readDouble("x");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -11;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Float.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Float.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Float.java
new file mode 100644
index 0000000..676b79a
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Float.java
@@ -0,0 +1,113 @@
+/*
+ * 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.h2.twostep.msg;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.nio.*;
+
+/**
+ * H2 Float.
+ */
+public class GridH2Float extends GridH2ValueMessage {
+    /** */
+    private float x;
+
+    /**
+     *
+     */
+    public GridH2Float() {
+        // No-op.
+    }
+
+    /**
+     * @param val Value.
+     */
+    public GridH2Float(Value val) {
+        assert val.getType() == Value.FLOAT : val.getType();
+
+        x = val.getFloat();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value value(GridKernalContext ctx) {
+        return ValueFloat.get(x);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeFloat("x", x))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                x = reader.readFloat("x");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -12;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Geometry.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Geometry.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Geometry.java
new file mode 100644
index 0000000..c825a47
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Geometry.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.query.h2.twostep.msg;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.lang.reflect.*;
+import java.nio.*;
+
+/**
+ * H2 Geometry.
+ */
+public class GridH2Geometry extends GridH2ValueMessage {
+    /** */
+    private static final Method GEOMETRY_FROM_BYTES;
+
+    /**
+     * Initialize field.
+     */
+    static {
+        try {
+            GEOMETRY_FROM_BYTES = Class.forName("org.h2.value.ValueGeometry").getMethod("get", byte[].class);
+        }
+        catch (NoSuchMethodException | ClassNotFoundException e) {
+            throw new IllegalStateException("Check H2 version in classpath.");
+        }
+    }
+
+    /** */
+    private byte[] b;
+
+    /**
+     *
+     */
+    public GridH2Geometry() {
+        // No-op.
+    }
+
+    /**
+     * @param val Value.
+     */
+    public GridH2Geometry(Value val) {
+        assert val.getType() == Value.GEOMETRY : val.getType();
+
+        b = val.getBytesNoCopy();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value value(GridKernalContext ctx) {
+        try {
+            return (Value)GEOMETRY_FROM_BYTES.invoke(null, new Object[]{b});
+        }
+        catch (IllegalAccessException | InvocationTargetException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeByteArray("b", b))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                b = reader.readByteArray("b");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -21;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Integer.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Integer.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Integer.java
new file mode 100644
index 0000000..fef4956
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Integer.java
@@ -0,0 +1,113 @@
+/*
+ * 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.h2.twostep.msg;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.nio.*;
+
+/**
+ * H2 Integer.
+ */
+public class GridH2Integer extends GridH2ValueMessage {
+    /** */
+    private int x;
+
+    /**
+     *
+     */
+    public GridH2Integer() {
+        // No-op.
+    }
+
+    /**
+     * @param val Value.
+     */
+    public GridH2Integer(Value val) {
+        assert val.getType() == Value.INT : val.getType();
+
+        x = val.getInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value value(GridKernalContext ctx) {
+        return ValueInt.get(x);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeInt("x", x))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                x = reader.readInt("x");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -8;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2JavaObject.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2JavaObject.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2JavaObject.java
new file mode 100644
index 0000000..17f6c9c
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2JavaObject.java
@@ -0,0 +1,113 @@
+/*
+ * 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.h2.twostep.msg;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.nio.*;
+
+/**
+ * H2 Java Object.
+ */
+public class GridH2JavaObject extends GridH2ValueMessage {
+    /** */
+    private byte[] b;
+
+    /**
+     *
+     */
+    public GridH2JavaObject() {
+        // No-op.
+    }
+
+    /**
+     * @param val Value.
+     */
+    public GridH2JavaObject(Value val) {
+        assert val.getType() == Value.JAVA_OBJECT : val.getType();
+
+        b = val.getBytesNoCopy();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value value(GridKernalContext ctx) {
+        return ValueJavaObject.getNoCopy(null, b, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeByteArray("b", b))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                b = reader.readByteArray("b");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -19;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Long.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Long.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Long.java
new file mode 100644
index 0000000..c798922
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Long.java
@@ -0,0 +1,113 @@
+/*
+ * 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.h2.twostep.msg;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.nio.*;
+
+/**
+ * H2 Long.
+ */
+public class GridH2Long extends GridH2ValueMessage {
+    /** */
+    private long x;
+
+    /**
+     *
+     */
+    public GridH2Long() {
+        // No-op.
+    }
+
+    /**
+     * @param val Value.
+     */
+    public GridH2Long(Value val) {
+        assert val.getType() == Value.LONG : val.getType();
+
+        x = val.getLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value value(GridKernalContext ctx) {
+        return ValueLong.get(x);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeLong("x", x))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                x = reader.readLong("x");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -9;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Null.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Null.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Null.java
new file mode 100644
index 0000000..a2e20d3
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Null.java
@@ -0,0 +1,78 @@
+/*
+ * 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.h2.twostep.msg;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.nio.*;
+
+/**
+ * Message for {@link Value#NULL}.
+ */
+public class GridH2Null extends GridH2ValueMessage {
+    /** */
+    public static GridH2Null INSTANCE = new GridH2Null();
+
+    /**
+     * Disallow new instance creation.
+     */
+    private GridH2Null() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value value(GridKernalContext ctx) {
+        return ValueNull.INSTANCE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        return reader.beforeMessageRead() && super.readFrom(buf, reader);
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -4;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Short.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Short.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Short.java
new file mode 100644
index 0000000..9064559
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Short.java
@@ -0,0 +1,113 @@
+/*
+ * 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.h2.twostep.msg;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.nio.*;
+
+/**
+ * H2 Short.
+ */
+public class GridH2Short extends GridH2ValueMessage {
+    /** */
+    private short x;
+
+    /**
+     *
+     */
+    public GridH2Short() {
+        // No-op.
+    }
+
+    /**
+     * @param val Value.
+     */
+    public GridH2Short(Value val) {
+        assert val.getType() == Value.SHORT : val.getType();
+
+        x = val.getShort();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value value(GridKernalContext ctx) {
+        return ValueShort.get(x);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeShort("x", x))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                x = reader.readShort("x");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -7;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2String.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2String.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2String.java
new file mode 100644
index 0000000..3ea3248
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2String.java
@@ -0,0 +1,115 @@
+/*
+ * 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.h2.twostep.msg;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.nio.*;
+
+/**
+ * H2 String.
+ */
+public class GridH2String extends GridH2ValueMessage {
+    /** */
+    private String x;
+
+    /**
+     *
+     */
+    public GridH2String() {
+        // No-op.
+    }
+
+    /**
+     * @param val Value.
+     */
+    public GridH2String(Value val) {
+        assert val.getType() == Value.STRING ||
+            val.getType() == Value.STRING_FIXED ||
+            val.getType() == Value.STRING_IGNORECASE : val.getType();
+
+        x = val.getString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value value(GridKernalContext ctx) {
+        return ValueString.get(x);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeString("x", x))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                x = reader.readString("x");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -17;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Time.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Time.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Time.java
new file mode 100644
index 0000000..ec34248
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Time.java
@@ -0,0 +1,116 @@
+/*
+ * 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.h2.twostep.msg;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.nio.*;
+
+/**
+ * H2 Time.
+ */
+public class GridH2Time extends GridH2ValueMessage {
+    /** */
+    private long nanos;
+
+    /**
+     *
+     */
+    public GridH2Time() {
+        // No-op.
+    }
+
+    /**
+     * @param val Value.
+     */
+    public GridH2Time(Value val) {
+        assert val.getType() == Value.TIME : val.getType();
+
+        ValueTime t = (ValueTime)val;
+
+        nanos = t.getNanos();
+    }
+
+
+    /** {@inheritDoc} */
+    @Override public Value value(GridKernalContext ctx) {
+        return ValueTime.fromNanos(nanos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeLong("nanos", nanos))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                nanos = reader.readLong("nanos");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -13;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Timestamp.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Timestamp.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Timestamp.java
new file mode 100644
index 0000000..973782b
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Timestamp.java
@@ -0,0 +1,133 @@
+/*
+ * 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.h2.twostep.msg;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.nio.*;
+
+/**
+ * H2 Timestamp.
+ */
+public class GridH2Timestamp extends GridH2ValueMessage {
+    /** */
+    private long date;
+
+    /** */
+    private long nanos;
+
+    /**
+     *
+     */
+    public GridH2Timestamp() {
+        // No-op.
+    }
+
+    /**
+     * @param val Value.
+     */
+    public GridH2Timestamp(Value val) {
+        assert val.getType() == Value.TIMESTAMP : val.getType();
+
+        ValueTimestamp t = (ValueTimestamp)val;
+
+        date = t.getDateValue();
+        nanos = t.getNanos();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value value(GridKernalContext ctx) {
+        return ValueTimestamp.fromDateValueAndNanos(date, nanos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeLong("date", date))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeLong("nanos", nanos))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                date = reader.readLong("date");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                nanos = reader.readLong("nanos");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -15;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 2;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Uuid.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Uuid.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Uuid.java
new file mode 100644
index 0000000..f23b827
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Uuid.java
@@ -0,0 +1,133 @@
+/*
+ * 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.h2.twostep.msg;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.nio.*;
+
+/**
+ * H2 Uuid.
+ */
+public class GridH2Uuid extends GridH2ValueMessage {
+    /** */
+    private long high;
+
+    /** */
+    private long low;
+
+    /**
+     *
+     */
+    public GridH2Uuid() {
+        // No-op.
+    }
+
+    /**
+     * @param val Value.
+     */
+    public GridH2Uuid(Value val) {
+        assert val.getType() == Value.UUID : val.getType();
+
+        ValueUuid uuid = (ValueUuid)val;
+
+        high = uuid.getHigh();
+        low = uuid.getLow();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value value(GridKernalContext ctx) {
+        return ValueUuid.get(high, low);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeLong("high", high))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeLong("low", low))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                high = reader.readLong("high");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                low = reader.readLong("low");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -20;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 2;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java
new file mode 100644
index 0000000..577de40
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2.twostep.msg;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+
+import java.nio.*;
+
+/**
+ * Abstract message wrapper for H2 values.
+ */
+public abstract class GridH2ValueMessage implements Message {
+    /**
+     * Gets H2 value.
+     *
+     * @param ctx Kernal context.
+     * @return Value.
+     * @throws IgniteCheckedException If failed.
+     */
+    public abstract Value value(GridKernalContext ctx) throws IgniteCheckedException;
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessageFactory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessageFactory.java
new file mode 100644
index 0000000..d193910
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessageFactory.java
@@ -0,0 +1,201 @@
+/*
+ * 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.h2.twostep.msg;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.query.h2.opt.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.h2.value.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * H2 Value message factory.
+ */
+public class GridH2ValueMessageFactory implements MessageFactory {
+    /** {@inheritDoc} */
+    @Nullable @Override public Message create(byte type) {
+        switch (type) {
+            case -4:
+                return GridH2Null.INSTANCE;
+
+            case -5:
+                return new GridH2Boolean();
+
+            case -6:
+                return new GridH2Byte();
+
+            case -7:
+                return new GridH2Short();
+
+            case -8:
+                return new GridH2Integer();
+
+            case -9:
+                return new GridH2Long();
+
+            case -10:
+                return new GridH2Decimal();
+
+            case -11:
+                return new GridH2Double();
+
+            case -12:
+                return new GridH2Float();
+
+            case -13:
+                return new GridH2Time();
+
+            case -14:
+                return new GridH2Date();
+
+            case -15:
+                return new GridH2Timestamp();
+
+            case -16:
+                return new GridH2Bytes();
+
+            case -17:
+                return new GridH2String();
+
+            case -18:
+                return new GridH2Array();
+
+            case -19:
+                return new GridH2JavaObject();
+
+            case -20:
+                return new GridH2Uuid();
+
+            case -21:
+                return new GridH2Geometry();
+
+            case -22:
+                return new GridH2CacheObject();
+        }
+
+        return null;
+    }
+
+    /**
+     * @param src Source values.
+     * @param dst Destination collection.
+     * @return Destination collection.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static Collection<Message> toMessages(Collection<Value[]> src, Collection<Message> dst)
+        throws IgniteCheckedException {
+        for (Value[] row : src) {
+            for (Value val : row)
+                dst.add(toMessage(val));
+        }
+
+        return dst;
+    }
+
+    /**
+     * @param src Source iterator.
+     * @param dst Array to fill with values.
+     * @param ctx Kernal context.
+     * @return Filled array.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static Value[] fillArray(Iterator<Message> src, Value[] dst, GridKernalContext ctx)
+        throws IgniteCheckedException {
+        for (int i = 0; i < dst.length; i++) {
+            Message msg = src.next();
+
+            dst[i] = ((GridH2ValueMessage)msg).value(ctx);
+        }
+
+        return dst;
+    }
+
+    /**
+     * @param v Value.
+     * @return Message.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static Message toMessage(Value v) throws IgniteCheckedException {
+        switch (v.getType()) {
+            case Value.NULL:
+                return GridH2Null.INSTANCE;
+
+            case Value.BOOLEAN:
+                return new GridH2Boolean(v);
+
+            case Value.BYTE:
+                return new GridH2Byte(v);
+
+            case Value.SHORT:
+                return new GridH2Short(v);
+
+            case Value.INT:
+                return new GridH2Integer(v);
+
+            case Value.LONG:
+                return new GridH2Long(v);
+
+            case Value.DECIMAL:
+                return new GridH2Decimal(v);
+
+            case Value.DOUBLE:
+                return new GridH2Double(v);
+
+            case Value.FLOAT:
+                return new GridH2Float(v);
+
+            case Value.DATE:
+                return new GridH2Date(v);
+
+            case Value.TIME:
+                return new GridH2Time(v);
+
+            case Value.TIMESTAMP:
+                return new GridH2Timestamp(v);
+
+            case Value.BYTES:
+                return new GridH2Bytes(v);
+
+            case Value.STRING:
+            case Value.STRING_FIXED:
+            case Value.STRING_IGNORECASE:
+                return new GridH2String(v);
+
+            case Value.ARRAY:
+                return new GridH2Array(v);
+
+            case Value.JAVA_OBJECT:
+                if (v instanceof GridH2ValueCacheObject)
+                    return new GridH2CacheObject((GridH2ValueCacheObject)v);
+
+                return new GridH2JavaObject(v);
+
+            case Value.UUID:
+                return new GridH2Uuid(v);
+
+            case Value.GEOMETRY:
+                return new GridH2Geometry(v);
+
+            default:
+                throw new IllegalStateException("Unsupported H2 type: " + v.getType());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
index 226f0fc..0c55551 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.spi.swapspace.*;
 import org.apache.ignite.spi.swapspace.file.*;
 import org.apache.ignite.testframework.junits.common.*;
 
@@ -79,6 +80,14 @@ public class GridCacheOffHeapAndSwapSelfTest extends GridCommonAbstractTest {
     /** */
     private final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
+    /**
+     * Creates a SwapSpaceSpi.
+     * @return the Spi
+     */
+    protected SwapSpaceSpi spi() {
+        return new FileSwapSpaceSpi();
+    }
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -91,7 +100,7 @@ public class GridCacheOffHeapAndSwapSelfTest extends GridCommonAbstractTest {
 
         cfg.setNetworkTimeout(2000);
 
-        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+        cfg.setSwapSpaceSpi(spi());
 
         CacheConfiguration<?,?> cacheCfg = defaultCacheConfiguration();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapSelfTest.java
index 460a64c..2d6855e 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapSelfTest.java
@@ -29,6 +29,7 @@ import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.spi.swapspace.*;
 import org.apache.ignite.spi.swapspace.file.*;
 import org.apache.ignite.testframework.junits.common.*;
 
@@ -65,6 +66,14 @@ public class GridCacheOffHeapSelfTest extends GridCommonAbstractTest {
     /** PeerClassLoadingLocalClassPathExclude enable. */
     private boolean excluded;
 
+    /**
+     * Creates a SwapSpaceSpi.
+     * @return the Spi
+     */
+    protected SwapSpaceSpi spi() {
+        return new FileSwapSpaceSpi();
+    }
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -77,7 +86,7 @@ public class GridCacheOffHeapSelfTest extends GridCommonAbstractTest {
 
         cfg.setNetworkTimeout(2000);
 
-        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+        cfg.setSwapSpaceSpi(spi());
 
         CacheConfiguration<?,?> cacheCfg = defaultCacheConfiguration();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java
new file mode 100644
index 0000000..4e40040
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.spi.swapspace.file.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMemoryMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+import static org.apache.ignite.configuration.DeploymentMode.*;
+
+/**
+ * Tests off heap storage when both offheaped and swapped entries exists.
+ */
+public class GridCacheOffheapIndexGetSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final long OFFHEAP_MEM = 10L * 1024L;
+
+    /** */
+    private final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        cfg.setNetworkTimeout(2000);
+
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+
+        CacheConfiguration cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
+        cacheCfg.setSwapEnabled(true);
+        cacheCfg.setCacheMode(PARTITIONED);
+        cacheCfg.setBackups(1);
+        cacheCfg.setOffHeapMaxMemory(OFFHEAP_MEM);
+        cacheCfg.setEvictSynchronized(true);
+        cacheCfg.setEvictSynchronizedKeyBufferSize(1);
+        cacheCfg.setAtomicityMode(TRANSACTIONAL);
+        cacheCfg.setMemoryMode(OFFHEAP_TIERED);
+        cacheCfg.setEvictionPolicy(null);
+        cacheCfg.setOffHeapMaxMemory(OFFHEAP_MEM);
+        cacheCfg.setIndexedTypes(Long.class, Long.class);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        cfg.setDeploymentMode(SHARED);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(2);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        grid(0).cache(null).clear();
+    }
+
+    /**
+     * Tests behavior on offheaped entries.
+     *
+     * @throws Exception If failed.
+     */
+    public void testGet() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-873");
+
+        IgniteCache<Long, Long> cache = grid(0).cache(null);
+
+        for (long i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        for (long i = 0; i < 100; i++)
+            assertEquals((Long)i, cache.get(i));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
index c2722fa..3159589 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.processors.datastructures.*;
 import org.apache.ignite.internal.processors.query.*;
+import org.apache.ignite.internal.processors.query.h2.sql.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.spi.discovery.*;
@@ -298,6 +299,26 @@ public abstract class IgniteCacheAbstractFieldsQuerySelfTest extends GridCommonA
         }
     }
 
+    /**
+     *
+     */
+    public void testExplain() {
+        List<List<?>> res = grid(0).cache(null).query(new SqlFieldsQuery(
+            "explain select p.age, p.name, o.name " +
+            "from Person p, Organization o where p.orgId = o.id")).getAll();
+
+        for (List<?> row : res)
+            X.println("____ : " + row);
+
+        if (cacheMode() == PARTITIONED) {
+            assertEquals(2, res.size());
+
+            assertTrue(((String)res.get(1).get(0)).contains(GridSqlQuerySplitter.TABLE_FUNC_NAME));
+        }
+        else
+            assertEquals(1, res.size());
+    }
+
     /** @throws Exception If failed. */
     public void testExecuteWithMetaData() throws Exception {
         QueryCursorImpl<List<?>> cursor = (QueryCursorImpl<List<?>>)grid(0).cache(null).query(new SqlFieldsQuery(

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
new file mode 100644
index 0000000..b2095a8
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
@@ -0,0 +1,56 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements.  See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.configuration.*;
+
+import javax.cache.*;
+
+/**
+ * Checks behavior on exception while unmarshalling key.
+ */
+public class IgniteCacheP2pUnmarshallingQueryErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.getCacheConfiguration()[0].setIndexedTypes(TestKey.class, String.class);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testResponseMessageOnUnmarshallingFailed() {
+        readCnt.set(Integer.MAX_VALUE);
+
+        jcache(0).put(new TestKey(String.valueOf(++key)), "");
+
+        //GridCacheQueryRequest unmarshalling failed test
+        readCnt.set(1);
+
+        try {
+            jcache(0).query(new SqlQuery<TestKey, String>(String.class, "field like '" + key + "'")).getAll();
+
+            assert false : "p2p marshalling failed, but error response was not sent";
+        }
+        catch (CacheException e) {
+            // No-op
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f0956d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.java
new file mode 100644
index 0000000..8b09d0f
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.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;
+
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+
+/**
+ * Test queries in off-heap tired mode.
+ */
+public class IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest extends IgniteCacheQueryMultiThreadedSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration() {
+        CacheConfiguration ccfg = super.cacheConfiguration();
+
+        ccfg.setCacheMode(CacheMode.REPLICATED);
+        ccfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
+        ccfg.setOffHeapMaxMemory(0);
+
+        return ccfg;
+    }
+}