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 2016/06/06 11:57:20 UTC

[21/32] ignite git commit: IGFS-3246: IGFS: Infrastructure for client optimizations.

IGFS-3246: IGFS: Infrastructure for client optimizations.


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

Branch: refs/heads/ignite-3212
Commit: 0d623830e0d67be5f0874f704155367d2d9de45b
Parents: 150e0d3
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Sun Jun 5 20:57:15 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Sun Jun 5 21:17:21 2016 +0300

----------------------------------------------------------------------
 .../internal/cluster/ClusterGroupAdapter.java   |   6 +
 .../ignite/internal/cluster/ClusterGroupEx.java |  14 ++-
 .../internal/processors/igfs/IgfsContext.java   |  14 +--
 .../processors/igfs/IgfsMetaManager.java        |  62 +++++++++-
 .../processors/igfs/IgfsNodePredicate.java      |  80 +++++++++++++
 .../internal/processors/igfs/IgfsProcessor.java |  39 +++++--
 .../internal/processors/igfs/IgfsUtils.java     |  21 ++++
 .../igfs/client/IgfsClientAbstractCallable.java | 112 +++++++++++++++++++
 .../multijvm/IgniteClusterProcessProxy.java     |   5 +
 9 files changed, 326 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0d623830/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java
index 75168a1..c664f1e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java
@@ -51,6 +51,7 @@ import org.apache.ignite.internal.IgniteServicesImpl;
 import org.apache.ignite.internal.IgnitionEx;
 import org.apache.ignite.internal.executor.GridExecutorService;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
+import org.apache.ignite.internal.processors.igfs.IgfsNodePredicate;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -602,6 +603,11 @@ public class ClusterGroupAdapter implements ClusterGroupEx, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public ClusterGroup forIgfsMetadataDataNodes(@Nullable String igfsName, @Nullable String metaCacheName) {
+        return forPredicate(new IgfsNodePredicate(igfsName)).forDataNodes(metaCacheName);
+    }
+
+    /** {@inheritDoc} */
     @Override public final ClusterGroup forHost(ClusterNode node) {
         A.notNull(node, "node");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0d623830/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupEx.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupEx.java
index 59da7cf..21533a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupEx.java
@@ -29,7 +29,7 @@ public interface ClusterGroupEx extends ClusterGroup {
      * Creates projection for specified subject ID.
      *
      * @param subjId Subject ID.
-     * @return Internal projection.
+     * @return Cluster group.
      */
     public ClusterGroupEx forSubjectId(UUID subjId);
 
@@ -40,5 +40,15 @@ public interface ClusterGroupEx extends ClusterGroup {
      * @param clientNodes Flag to include client nodes.
      * @return Cluster group.
      */
-    public ClusterGroup forCacheNodes(@Nullable String cacheName, boolean affNodes, boolean nearNodes, boolean clientNodes);
+    public ClusterGroup forCacheNodes(@Nullable String cacheName, boolean affNodes, boolean nearNodes,
+        boolean clientNodes);
+
+    /**
+     * Create projection for IGFS server nodes.
+     *
+     * @param igfsName IGFS name.
+     * @param metaCacheName Metadata cache name.
+     * @return Cluster group.
+     */
+    public ClusterGroup forIgfsMetadataDataNodes(@Nullable String igfsName, @Nullable String metaCacheName);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0d623830/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
index 2b08f28..a638bf3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
@@ -24,11 +24,8 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.util.typedef.F;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGFS;
-
 /**
  * IGFS context holding all required components for IGFS instance.
  */
@@ -178,16 +175,7 @@ public class IgfsContext {
      * @return {@code True} if node has IGFS with this name, {@code false} otherwise.
      */
     public boolean igfsNode(ClusterNode node) {
-        assert node != null;
-
-        IgfsAttributes[] igfs = node.attribute(ATTR_IGFS);
-
-        if (igfs != null)
-            for (IgfsAttributes attrs : igfs)
-                if (F.eq(cfg.getName(), attrs.igfsName()))
-                    return true;
-
-        return false;
+        return IgfsUtils.isIgfsNode(node, cfg.getName());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0d623830/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index 73d0887..8569305 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -18,10 +18,13 @@
 package org.apache.ignite.internal.processors.igfs;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterTopologyException;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.events.EventType;
 import org.apache.ignite.events.IgfsEvent;
@@ -37,6 +40,7 @@ import org.apache.ignite.igfs.IgfsPathIsNotDirectoryException;
 import org.apache.ignite.igfs.IgfsPathNotFoundException;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
@@ -44,6 +48,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheInternal;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
+import org.apache.ignite.internal.processors.igfs.client.IgfsClientAbstractCallable;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryCreateProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileCreateProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileLockProcessor;
@@ -140,13 +145,21 @@ public class IgfsMetaManager extends IgfsManager {
     /** Relaxed flag. */
     private final boolean relaxed;
 
+    /** Client flag. */
+    private final boolean client;
+
+    /** Compute facade for client tasks. */
+    private IgniteCompute cliCompute;
+
     /**
      * Constructor.
      *
      * @param relaxed Relaxed mode flag.
+     * @param client Client flag.
      */
-    public IgfsMetaManager(boolean relaxed) {
+    public IgfsMetaManager(boolean relaxed, boolean client) {
         this.relaxed = relaxed;
+        this.client = client;
     }
 
     /**
@@ -218,6 +231,53 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
+     * @return Client flag.
+     */
+    boolean isClient() {
+        return client;
+    }
+
+    /**
+     * Run client task.
+     *
+     * @param task Task.
+     * @return Result.
+     */
+    <T> T runClientTask(IgfsClientAbstractCallable<T> task) {
+        try {
+            return clientCompute().call(task);
+        }
+        catch (ClusterTopologyException e) {
+            throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes left." , e);
+        }
+    }
+
+    /**
+     * Get compute facade for client tasks.
+     *
+     * @return Compute facade.
+     */
+    private IgniteCompute clientCompute() {
+        assert client;
+
+        IgniteCompute cliCompute0 = cliCompute;
+
+        if (cliCompute0 == null) {
+            IgniteEx ignite = igfsCtx.kernalContext().grid();
+
+            ClusterGroup cluster = ignite.cluster().forIgfsMetadataDataNodes(cfg.getName(), cfg.getMetaCacheName());
+
+            cliCompute0 = ignite.compute(cluster);
+
+            cliCompute = cliCompute0;
+        }
+
+        assert cliCompute0 != null;
+
+        return cliCompute0;
+    }
+
+    /**
      * Return nodes where meta cache is defined.
      *
      * @return Nodes where meta cache is defined.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0d623830/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsNodePredicate.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsNodePredicate.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsNodePredicate.java
new file mode 100644
index 0000000..e3a658a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsNodePredicate.java
@@ -0,0 +1,80 @@
+/*
+ * 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.igfs;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * IGFS node predicate.
+ */
+public class IgfsNodePredicate implements IgnitePredicate<ClusterNode>, Binarylizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** IGFS name. */
+    private String igfsName;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsNodePredicate() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS name.
+     */
+    public IgfsNodePredicate(@Nullable String igfsName) {
+        this.igfsName = igfsName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean apply(ClusterNode node) {
+        return IgfsUtils.isIgfsNode(node, igfsName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter rawWriter = writer.rawWriter();
+
+        rawWriter.writeString(igfsName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader rawReader = reader.rawReader();
+
+        igfsName = rawReader.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsNodePredicate.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0d623830/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
index 82404e7..6df9986 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
@@ -94,10 +94,12 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
 
     /** {@inheritDoc} */
     @Override public void start() throws IgniteCheckedException {
-        if (ctx.config().isDaemon())
+        IgniteConfiguration igniteCfg = ctx.config();
+
+        if (igniteCfg.isDaemon())
             return;
 
-        FileSystemConfiguration[] cfgs = ctx.config().getFileSystemConfiguration();
+        FileSystemConfiguration[] cfgs = igniteCfg.getFileSystemConfiguration();
 
         assert cfgs != null && cfgs.length > 0;
 
@@ -107,10 +109,27 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
         for (FileSystemConfiguration cfg : cfgs) {
             FileSystemConfiguration cfg0 = new FileSystemConfiguration(cfg);
 
+            boolean metaClient = true;
+
+            CacheConfiguration[] cacheCfgs = igniteCfg.getCacheConfiguration();
+
+            if (cacheCfgs != null) {
+                for (CacheConfiguration cacheCfg : cacheCfgs) {
+                    if (F.eq(cacheCfg.getName(), cfg.getMetaCacheName())) {
+                        metaClient = false;
+
+                        break;
+                    }
+                }
+            }
+
+            if (igniteCfg.isClientMode() != null && igniteCfg.isClientMode())
+                metaClient = true;
+
             IgfsContext igfsCtx = new IgfsContext(
                 ctx,
                 cfg0,
-                new IgfsMetaManager(cfg0.isRelaxedConsistency()),
+                new IgfsMetaManager(cfg0.isRelaxedConsistency(), metaClient),
                 new IgfsDataManager(),
                 new IgfsServerManager(),
                 new IgfsFragmentizerManager());
@@ -125,28 +144,26 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
         if (log.isDebugEnabled())
             log.debug("IGFS processor started.");
 
-        IgniteConfiguration gridCfg = ctx.config();
-
         // Node doesn't have IGFS if it:
         // is daemon;
         // doesn't have configured IGFS;
         // doesn't have configured caches.
-        if (gridCfg.isDaemon() || F.isEmpty(gridCfg.getFileSystemConfiguration()) ||
-            F.isEmpty(gridCfg.getCacheConfiguration()))
+        if (igniteCfg.isDaemon() || F.isEmpty(igniteCfg.getFileSystemConfiguration()) ||
+            F.isEmpty(igniteCfg.getCacheConfiguration()))
             return;
 
         final Map<String, CacheConfiguration> cacheCfgs = new HashMap<>();
 
-        assert gridCfg.getCacheConfiguration() != null;
+        assert igniteCfg.getCacheConfiguration() != null;
 
-        for (CacheConfiguration ccfg : gridCfg.getCacheConfiguration())
+        for (CacheConfiguration ccfg : igniteCfg.getCacheConfiguration())
             cacheCfgs.put(ccfg.getName(), ccfg);
 
         Collection<IgfsAttributes> attrVals = new ArrayList<>();
 
-        assert gridCfg.getFileSystemConfiguration() != null;
+        assert igniteCfg.getFileSystemConfiguration() != null;
 
-        for (FileSystemConfiguration igfsCfg : gridCfg.getFileSystemConfiguration()) {
+        for (FileSystemConfiguration igfsCfg : igniteCfg.getFileSystemConfiguration()) {
             CacheConfiguration cacheCfg = cacheCfgs.get(igfsCfg.getDataCacheName());
 
             if (cacheCfg == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/0d623830/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
index 45d4bb6..340fb97 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
@@ -58,6 +58,7 @@ import java.util.UUID;
 import java.util.concurrent.ThreadLocalRandom;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_RETRIES_COUNT;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGFS;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
@@ -729,4 +730,24 @@ public class IgfsUtils {
     static String composeNameForTrash(IgfsPath path, IgniteUuid id) {
         return id.toString() + TRASH_NAME_SEPARATOR + path.toString();
     }
+
+    /**
+     * Check whether provided node contains IGFS with the given name.
+     *
+     * @param node Node.
+     * @param igfsName IGFS name.
+     * @return {@code True} if it contains IGFS.
+     */
+    public static boolean isIgfsNode(ClusterNode node, String igfsName) {
+        assert node != null;
+
+        IgfsAttributes[] igfs = node.attribute(ATTR_IGFS);
+
+        if (igfs != null)
+            for (IgfsAttributes attrs : igfs)
+                if (F.eq(igfsName, attrs.igfsName()))
+                    return true;
+
+        return false;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0d623830/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
new file mode 100644
index 0000000..b83ed13
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
@@ -0,0 +1,112 @@
+/*
+ * 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.igfs.client;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.internal.processors.igfs.IgfsContext;
+import org.apache.ignite.internal.processors.igfs.IgfsEx;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract callable for IGFS tasks initiated on client node and passed to data node.
+ */
+public abstract class IgfsClientAbstractCallable<T> implements IgniteCallable<T>, Binarylizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** IGFS name. */
+    protected String igfsName;
+
+    /** Injected instance. */
+    @IgniteInstanceResource
+    private transient Ignite ignite;
+
+    /**
+     * Default constructor.
+     */
+    protected IgfsClientAbstractCallable() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS name.
+     */
+    protected IgfsClientAbstractCallable(@Nullable String igfsName) {
+        this.igfsName = igfsName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public final T call() throws Exception {
+        assert ignite != null;
+
+        IgfsEx igfs = (IgfsEx)ignite.fileSystem(igfsName);
+
+        return call0(igfs.context());
+    }
+
+    /**
+     * Execute task.
+     *
+     * @param ctx IGFS ocntext.
+     * @return Result.
+     * @throws Exception If failed.
+     */
+    protected abstract T call0(IgfsContext ctx) throws Exception;
+
+    /** {@inheritDoc} */
+    @Override public final void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter rawWriter = writer.rawWriter();
+
+        rawWriter.writeString(igfsName);
+
+        writeBinary0(rawWriter);
+    }
+
+    /** {@inheritDoc} */
+    @Override public final void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader rawReader = reader.rawReader();
+
+        igfsName = rawReader.readString();
+
+        readBinary0(rawReader);
+    }
+
+    /**
+     * Write binary.
+     *
+     * @param rawWriter Raw writer.
+     */
+    protected abstract void writeBinary0(BinaryRawWriter rawWriter);
+
+    /**
+     * Read binary.
+     *
+     * @param rawReader Raw reader.
+     */
+    protected abstract void readBinary0(BinaryRawReader rawReader);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0d623830/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
index 633e9d0..76a88d9 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
@@ -69,6 +69,11 @@ public class IgniteClusterProcessProxy implements IgniteClusterEx {
     }
 
     /** {@inheritDoc} */
+    @Override public ClusterGroup forIgfsMetadataDataNodes(@Nullable String igfsName, @Nullable String metaCacheName) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
     @Override public ClusterNode localNode() {
         return compute.call(new LocalNodeTask());
     }