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/08/27 08:34:14 UTC

[1/6] ignite git commit: IGNITE-1309: Moved platform affinity to Ignite.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1124 c8604c17f -> 40504fe53


IGNITE-1309: Moved platform affinity to Ignite.


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

Branch: refs/heads/ignite-1124
Commit: 536af49ba0a407fdbc5682f73a48aa07fa3daae0
Parents: d9a1397
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Aug 26 17:03:39 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Aug 26 17:03:39 2015 +0300

----------------------------------------------------------------------
 .../cache/affinity/PlatformAffinity.java        | 293 +++++++++++++++++++
 1 file changed, 293 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/536af49b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
new file mode 100644
index 0000000..d6dfcdb
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
@@ -0,0 +1,293 @@
+/*
+ * 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.platform.cache.affinity;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.managers.discovery.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Native cache wrapper implementation.
+ */
+@SuppressWarnings({"unchecked", "UnusedDeclaration", "TryFinallyCanBeTryWithResources"})
+public class PlatformAffinity extends PlatformAbstractTarget {
+    /** */
+    public static final int OP_AFFINITY_KEY = 1;
+
+    /** */
+    public static final int OP_ALL_PARTITIONS = 2;
+
+    /** */
+    public static final int OP_BACKUP_PARTITIONS = 3;
+
+    /** */
+    public static final int OP_IS_BACKUP = 4;
+
+    /** */
+    public static final int OP_IS_PRIMARY = 5;
+
+    /** */
+    public static final int OP_IS_PRIMARY_OR_BACKUP = 6;
+
+    /** */
+    public static final int OP_MAP_KEY_TO_NODE = 7;
+
+    /** */
+    public static final int OP_MAP_KEY_TO_PRIMARY_AND_BACKUPS = 8;
+
+    /** */
+    public static final int OP_MAP_KEYS_TO_NODES = 9;
+
+    /** */
+    public static final int OP_MAP_PARTITION_TO_NODE = 10;
+
+    /** */
+    public static final int OP_MAP_PARTITION_TO_PRIMARY_AND_BACKUPS = 11;
+
+    /** */
+    public static final int OP_MAP_PARTITIONS_TO_NODES = 12;
+
+    /** */
+    public static final int OP_PARTITION = 13;
+
+    /** */
+    public static final int OP_PRIMARY_PARTITIONS = 14;
+
+    /** */
+    private static final C1<ClusterNode, UUID> TO_NODE_ID = new C1<ClusterNode, UUID>() {
+        @Nullable @Override public UUID apply(ClusterNode node) {
+            return node != null ? node.id() : null;
+        }
+    };
+
+    /** Underlying cache affinity. */
+    private final Affinity<Object> aff;
+
+    /** Discovery manager */
+    private final GridDiscoveryManager discovery;
+
+    /**
+     * Constructor.
+     *
+     * @param platformCtx Context.
+     * @param igniteCtx Ignite context.
+     * @param name Cache name.
+     */
+    public PlatformAffinity(PlatformContext platformCtx, GridKernalContext igniteCtx, @Nullable String name)
+        throws IgniteCheckedException {
+        super(platformCtx);
+
+        this.aff = igniteCtx.grid().affinity(name);
+
+        if (aff == null)
+            throw new IgniteCheckedException("Cache with the given name doesn't exist: " + name);
+
+        discovery = igniteCtx.discovery();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+        switch (type) {
+            case OP_PARTITION:
+                return aff.partition(reader.readObjectDetached());
+
+            case OP_IS_PRIMARY: {
+                UUID nodeId = reader.readUuid();
+
+                Object key = reader.readObjectDetached();
+
+                ClusterNode node = discovery.node(nodeId);
+
+                if (node == null)
+                    return FALSE;
+
+                return aff.isPrimary(node, key) ? TRUE : FALSE;
+            }
+
+            case OP_IS_BACKUP: {
+                UUID nodeId = reader.readUuid();
+
+                Object key = reader.readObjectDetached();
+
+                ClusterNode node = discovery.node(nodeId);
+
+                if (node == null)
+                    return FALSE;
+
+                return aff.isBackup(node, key) ? TRUE : FALSE;
+            }
+
+            case OP_IS_PRIMARY_OR_BACKUP: {
+                UUID nodeId = reader.readUuid();
+
+                Object key = reader.readObjectDetached();
+
+                ClusterNode node = discovery.node(nodeId);
+
+                if (node == null)
+                    return FALSE;
+
+                return aff.isPrimaryOrBackup(node, key) ? TRUE : FALSE;
+            }
+
+            default:
+                return throwUnsupported(type);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"IfMayBeConditional", "ConstantConditions"})
+    @Override protected void processInOutOp(int type, PortableRawReaderEx reader, PortableRawWriterEx writer,
+        Object arg) throws IgniteCheckedException {
+        switch (type) {
+            case OP_PRIMARY_PARTITIONS: {
+                UUID nodeId = reader.readObject();
+
+                ClusterNode node = discovery.node(nodeId);
+
+                int[] parts = node != null ? aff.primaryPartitions(node) : U.EMPTY_INTS;
+
+                writer.writeIntArray(parts);
+
+                break;
+            }
+
+            case OP_BACKUP_PARTITIONS: {
+                UUID nodeId = reader.readObject();
+
+                ClusterNode node = discovery.node(nodeId);
+
+                int[] parts = node != null ? aff.backupPartitions(node) : U.EMPTY_INTS;
+
+                writer.writeIntArray(parts);
+
+                break;
+            }
+
+            case OP_ALL_PARTITIONS: {
+                UUID nodeId = reader.readObject();
+
+                ClusterNode node = discovery.node(nodeId);
+
+                int[] parts = node != null ? aff.allPartitions(node) : U.EMPTY_INTS;
+
+                writer.writeIntArray(parts);
+
+                break;
+            }
+
+            case OP_AFFINITY_KEY: {
+                Object key = reader.readObjectDetached();
+
+                writer.writeObject(aff.affinityKey(key));
+
+                break;
+            }
+
+            case OP_MAP_KEY_TO_NODE: {
+                Object key = reader.readObjectDetached();
+
+                ClusterNode node = aff.mapKeyToNode(key);
+
+                platformCtx.writeNode(writer, node);
+
+                break;
+            }
+
+            case OP_MAP_PARTITION_TO_NODE: {
+                int part = reader.readObject();
+
+                ClusterNode node = aff.mapPartitionToNode(part);
+
+                platformCtx.writeNode(writer, node);
+
+                break;
+            }
+
+            case OP_MAP_KEY_TO_PRIMARY_AND_BACKUPS: {
+                Object key = reader.readObjectDetached();
+
+                platformCtx.writeNodes(writer, aff.mapKeyToPrimaryAndBackups(key));
+
+                break;
+            }
+
+            case OP_MAP_PARTITION_TO_PRIMARY_AND_BACKUPS: {
+                int part = reader.readObject();
+
+                platformCtx.writeNodes(writer, aff.mapPartitionToPrimaryAndBackups(part));
+
+                break;
+            }
+
+            case OP_MAP_KEYS_TO_NODES: {
+                Collection<Object> keys = reader.readCollection();
+
+                Map<ClusterNode, Collection<Object>> map = aff.mapKeysToNodes(keys);
+
+                writer.writeInt(map.size());
+
+                for (Map.Entry<ClusterNode, Collection<Object>> e : map.entrySet()) {
+                    platformCtx.addNode(e.getKey());
+
+                    writer.writeUuid(e.getKey().id());
+                    writer.writeObject(e.getValue());
+                }
+
+                break;
+            }
+
+            case OP_MAP_PARTITIONS_TO_NODES: {
+                Collection<Integer> parts = reader.readCollection();
+
+                Map<Integer, ClusterNode> map = aff.mapPartitionsToNodes(parts);
+
+                writer.writeInt(map.size());
+
+                for (Map.Entry<Integer, ClusterNode> e : map.entrySet()) {
+                    platformCtx.addNode(e.getValue());
+
+                    writer.writeInt(e.getKey());
+
+                    writer.writeUuid(e.getValue().id());
+                }
+
+                break;
+            }
+
+            default:
+                throwUnsupported(type);
+        }
+    }
+
+    /**
+     * @return Gets number of partitions in cache.
+     */
+    public int partitions() {
+        return aff.partitions();
+    }
+}


[3/6] ignite git commit: Restored backward compatibility for cache objects

Posted by sb...@apache.org.
Restored backward compatibility for cache objects


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

Branch: refs/heads/ignite-1124
Commit: 38b3ffdaf2944cb908c45e22e3a1b1b834cf3b6c
Parents: 2938a9b
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Wed Aug 26 15:39:12 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Aug 26 15:39:12 2015 -0700

----------------------------------------------------------------------
 .../processors/cacheobject/IgniteCacheObjectProcessor.java         | 2 +-
 .../processors/cacheobject/IgniteCacheObjectProcessorImpl.java     | 2 +-
 .../ignite/internal/processors/query/GridQueryProcessor.java       | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/38b3ffda/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
index dc0d1e5..b9b6132 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
@@ -78,7 +78,7 @@ public interface IgniteCacheObjectProcessor extends GridProcessor {
      *
      * @return {@code true} If portable objects are enabled.
      */
-    public boolean isPortableEnabled();
+    public boolean isPortableEnabled(CacheConfiguration<?, ?> ccfg);
 
     /**
      * @param obj Portable object to get field from.

http://git-wip-us.apache.org/repos/asf/ignite/blob/38b3ffda/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index 3e59b10..e8f7781 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -233,7 +233,7 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isPortableEnabled() {
+    @Override public boolean isPortableEnabled(CacheConfiguration<?, ?> ccfg) {
         return false;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/38b3ffda/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 f3ad4b2..c8b8166 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
@@ -140,7 +140,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                     TypeId typeId;
 
-                    if (valCls == null || ctx.cacheObjects().isPortableEnabled()) {
+                    if (valCls == null || ctx.cacheObjects().isPortableEnabled(ccfg)) {
                         processPortableMeta(meta, desc);
 
                         typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType()));


[6/6] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-1124

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-1124


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

Branch: refs/heads/ignite-1124
Commit: 40504fe53bc21c3f6815ea47f418f97acb7119b6
Parents: c8604c1 136c099
Author: sboikov <sb...@gridgain.com>
Authored: Thu Aug 27 09:33:46 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Aug 27 09:33:46 2015 +0300

----------------------------------------------------------------------
 .../CacheObjectPortableProcessorImpl.java       |   3 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |   2 +-
 .../IgniteCacheObjectProcessorImpl.java         |   2 +-
 .../processors/query/GridQueryProcessor.java    |   2 +-
 .../platform/cache/PlatformCacheIterator.java   |  72 +++++
 .../cache/affinity/PlatformAffinity.java        | 293 +++++++++++++++++++
 .../cache/store/PlatformCacheStore.java         |  25 ++
 .../cache/store/PlatformCacheStoreCallback.java |  61 ++++
 8 files changed, 456 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[2/6] ignite git commit: Platforms: moved several cache store classes to Ignite.

Posted by sb...@apache.org.
Platforms: moved several cache store classes to Ignite.


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

Branch: refs/heads/ignite-1124
Commit: 2938a9b4a06b0e0a9f19625ce1b168e4a4a2f87d
Parents: 536af49
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Aug 26 17:11:46 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Aug 26 17:11:46 2015 +0300

----------------------------------------------------------------------
 .../cache/store/PlatformCacheStore.java         | 25 ++++++++
 .../cache/store/PlatformCacheStoreCallback.java | 61 ++++++++++++++++++++
 2 files changed, 86 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2938a9b4/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStore.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStore.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStore.java
new file mode 100644
index 0000000..6e0c1d9
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStore.java
@@ -0,0 +1,25 @@
+/*
+ * 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.platform.cache.store;
+
+/**
+ * Marker interface denoting that this instance is platform cache store.
+ */
+public interface PlatformCacheStore {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2938a9b4/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java
new file mode 100644
index 0000000..32e0ab7
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java
@@ -0,0 +1,61 @@
+/*
+ * 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.platform.cache.store;
+
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+
+/**
+ * Platform cache store callback.
+ */
+public abstract class PlatformCacheStoreCallback {
+    /** Context. */
+    protected final PlatformContext ctx;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    protected PlatformCacheStoreCallback(PlatformContext ctx) {
+        this.ctx = ctx;
+    }
+
+    /**
+     * Invoke the callback.
+     *
+     * @param memPtr Memory pointer.
+     */
+    public void invoke(long memPtr) {
+        if (memPtr > 0) {
+            try (PlatformMemory mem = ctx.memory().get(memPtr)) {
+                PortableRawReaderEx reader = ctx.reader(mem);
+
+                invoke0(reader);
+            }
+        }
+    }
+
+    /**
+     * Internal invoke routine.
+     *
+     * @param reader Reader.
+     */
+    protected abstract void invoke0(PortableRawReaderEx reader);
+}


[5/6] ignite git commit: Moved platform cache iterator to Ignite.

Posted by sb...@apache.org.
Moved platform cache iterator to Ignite.


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

Branch: refs/heads/ignite-1124
Commit: 136c099bf4ff1ba400e711dbad34216633b9236a
Parents: e428206
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Aug 27 09:20:44 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 27 09:20:44 2015 +0300

----------------------------------------------------------------------
 .../platform/cache/PlatformCacheIterator.java   | 72 ++++++++++++++++++++
 1 file changed, 72 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/136c099b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
new file mode 100644
index 0000000..45e777d
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
@@ -0,0 +1,72 @@
+/*
+ * 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.platform.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+
+import javax.cache.*;
+import java.util.*;
+
+/**
+ * Interop cache iterator.
+ */
+public class PlatformCacheIterator extends PlatformAbstractTarget {
+    /** Operation: next entry. */
+    private static final int OP_NEXT = 1;
+
+    /** Iterator. */
+    private final Iterator<Cache.Entry> iter;
+
+    /**
+     * Constructor.
+     *
+     * @param platformCtx Context.
+     * @param iter Iterator.
+     */
+    public PlatformCacheIterator(PlatformContext platformCtx, Iterator<Cache.Entry> iter) {
+        super(platformCtx);
+
+        this.iter = iter;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processOutOp(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+        switch (type) {
+            case OP_NEXT:
+                if (iter.hasNext()) {
+                    Cache.Entry e = iter.next();
+
+                    assert e != null;
+
+                    writer.writeBoolean(true);
+
+                    writer.writeObjectDetached(e.getKey());
+                    writer.writeObjectDetached(e.getValue());
+                }
+                else
+                    writer.writeBoolean(false);
+
+                break;
+
+            default:
+                throwUnsupported(type);
+        }
+    }
+}


[4/6] ignite git commit: Restored backward compatibility for cache objects

Posted by sb...@apache.org.
Restored backward compatibility for cache objects


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

Branch: refs/heads/ignite-1124
Commit: e428206d3d2281de72e674fe5abf9f887bf221fc
Parents: 38b3ffd
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Wed Aug 26 15:50:34 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Aug 26 15:50:34 2015 -0700

----------------------------------------------------------------------
 .../cache/portable/CacheObjectPortableProcessorImpl.java          | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e428206d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
index a421129..e70feb9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
@@ -36,6 +36,7 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.portable.*;
+import org.apache.ignite.plugin.*;
 import org.apache.ignite.portable.*;
 
 import org.jetbrains.annotations.*;
@@ -569,7 +570,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isPortableEnabled() {
+    @Override public boolean isPortableEnabled(CacheConfiguration<?, ?> ccfg) {
         return marsh instanceof PortableMarshaller;
     }