You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/01/04 08:28:19 UTC

[33/50] [abbrv] ignite git commit: IGNITE-2258: IGFS: now default path modes could be optionally disabled using FileSystemConfiguration.isInitializeDefaultPathModes() property.

IGNITE-2258: IGFS: now default path modes could be optionally disabled using FileSystemConfiguration.isInitializeDefaultPathModes() property.


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

Branch: refs/heads/ignite-2218
Commit: 03ad6aa821f34eee42504d1a78c9256f7521330a
Parents: d9d5eea
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Dec 29 09:31:58 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Dec 29 09:31:58 2015 +0300

----------------------------------------------------------------------
 .../configuration/FileSystemConfiguration.java  |  54 ++++-
 .../internal/processors/igfs/IgfsImpl.java      |  12 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |   7 +
 ...condaryFileSystemInitializationSelfTest.java | 214 +++++++++++++++++++
 .../testsuites/IgniteHadoopTestSuite.java       |   2 +
 5 files changed, 278 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/03ad6aa8/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
index 888f927..3a9e55e 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
@@ -80,6 +80,9 @@ public class FileSystemConfiguration {
     /** Default IPC endpoint enabled flag. */
     public static final boolean DFLT_IPC_ENDPOINT_ENABLED = true;
 
+    /** Default value of whether to initialize default path modes. */
+    public static final boolean DFLT_INIT_DFLT_PATH_MODES = true;
+
     /** IGFS instance name. */
     private String name;
 
@@ -158,6 +161,9 @@ public class FileSystemConfiguration {
     /** Maximum range length. */
     private long maxTaskRangeLen;
 
+    /** Whether to initialize default path modes. */
+    private boolean initDfltPathModes = DFLT_INIT_DFLT_PATH_MODES;
+
     /**
      * Constructs default configuration.
      */
@@ -189,6 +195,7 @@ public class FileSystemConfiguration {
         fragmentizerThrottlingBlockLen = cfg.getFragmentizerThrottlingBlockLength();
         fragmentizerThrottlingDelay = cfg.getFragmentizerThrottlingDelay();
         secondaryFs = cfg.getSecondaryFileSystem();
+        initDfltPathModes = cfg.isInitializeDefaultPathsModes();
         ipcEndpointCfg = cfg.getIpcEndpointConfiguration();
         ipcEndpointEnabled = cfg.isIpcEndpointEnabled();
         maxSpace = cfg.getMaxSpaceSize();
@@ -507,7 +514,7 @@ public class FileSystemConfiguration {
      * Sets the secondary file system. Secondary file system is provided for pass-through, write-through,
      * and read-through purposes.
      *
-     * @param fileSystem
+     * @param fileSystem Secondary file system.
      */
     public void setSecondaryFileSystem(IgfsSecondaryFileSystem fileSystem) {
         secondaryFs = fileSystem;
@@ -519,13 +526,14 @@ public class FileSystemConfiguration {
      * If path doesn't correspond to any specified prefix or mappings are not provided, then
      * {@link #getDefaultMode()} is used.
      * <p>
-     * Several folders under {@code '/apache/ignite'} folder have predefined mappings which cannot be overridden.
-     * <li>{@code /apache/ignite/primary} and all it's sub-folders will always work in {@code PRIMARY} mode.</li>
+     * If {@link #isInitializeDefaultPathsModes()} is set to {@code true}, the following path modes will be created
+     * by default:
+     * <li>{@code /ignite/primary} and all it's sub-folders will always work in {@code PRIMARY} mode.</li>
      * <p>
      * And in case secondary file system URI is provided:
-     * <li>{@code /apache/ignite/proxy} and all it's sub-folders will always work in {@code PROXY} mode.</li>
-     * <li>{@code /apache/ignite/sync} and all it's sub-folders will always work in {@code DUAL_SYNC} mode.</li>
-     * <li>{@code /apache/ignite/async} and all it's sub-folders will always work in {@code DUAL_ASYNC} mode.</li>
+     * <li>{@code /ignite/proxy} and all it's sub-folders will always work in {@code PROXY} mode.</li>
+     * <li>{@code /ignite/sync} and all it's sub-folders will always work in {@code DUAL_SYNC} mode.</li>
+     * <li>{@code /ignite/async} and all it's sub-folders will always work in {@code DUAL_ASYNC} mode.</li>
      *
      * @return Map of paths to {@code IGFS} modes.
      */
@@ -788,6 +796,40 @@ public class FileSystemConfiguration {
         this.maxTaskRangeLen = maxTaskRangeLen;
     }
 
+    /**
+     * Get whether to initialize default path modes.
+     * <p>
+     * When set to {@code true} Ignite will automatically create the following path modes:
+     * <ul>
+     *     <li>{@code /ignite/primary} - will work in {@link IgfsMode#PRIMARY} mode;</li>
+     *     <li>{@code /ignite/sync} - will work in {@link IgfsMode#DUAL_SYNC} mode (only if secondary file system
+     *         is set);</li>
+     *     <li>{@code /ignite/async} - will work in {@link IgfsMode#DUAL_ASYNC} mode (only if secondary file system
+     *         is set);</li>
+     *     <li>{@code /ignite/proxy} - will work in {@link IgfsMode#PROXY} mode (only if secondary file system
+     *         is set).</li>
+     * </ul>
+     * See {@link #getPathModes()} for more information about path modes.
+     * <p>
+     * Defaults to {@link #DFLT_INIT_DFLT_PATH_MODES}.
+     *
+     * @return {@code True} if default path modes will be initialized.
+     */
+    public boolean isInitializeDefaultPathsModes() {
+        return initDfltPathModes;
+    }
+
+    /**
+     * Set whether to initialize default path modes.
+     * <p>
+     * See {@link #isInitializeDefaultPathsModes()} for more information.
+     *
+     * @param initDfltPathModes Whether to initialize default path modes.
+     */
+    public void setInitializeDefaultPathsModes(boolean initDfltPathModes) {
+        this.initDfltPathModes = initDfltPathModes;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(FileSystemConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/03ad6aa8/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index 0d5cda3..8534513 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -215,12 +215,14 @@ public final class IgfsImpl implements IgfsEx {
         Map<String, IgfsMode> cfgModes = new LinkedHashMap<>();
         Map<String, IgfsMode> dfltModes = new LinkedHashMap<>(4, 1.0f);
 
-        dfltModes.put("/ignite/primary", PRIMARY);
+        if (cfg.isInitializeDefaultPathsModes()) {
+            dfltModes.put("/ignite/primary", PRIMARY);
 
-        if (secondaryFs != null) {
-            dfltModes.put("/ignite/proxy", PROXY);
-            dfltModes.put("/ignite/sync", DUAL_SYNC);
-            dfltModes.put("/ignite/async", DUAL_ASYNC);
+            if (secondaryFs != null) {
+                dfltModes.put("/ignite/proxy", PROXY);
+                dfltModes.put("/ignite/sync", DUAL_SYNC);
+                dfltModes.put("/ignite/async", DUAL_ASYNC);
+            }
         }
 
         cfgModes.putAll(dfltModes);

http://git-wip-us.apache.org/repos/asf/ignite/blob/03ad6aa8/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
index 778792a..5dce67f 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
@@ -1100,6 +1100,13 @@ public class IgniteHadoopFileSystem extends FileSystem {
     }
 
     /**
+     * @return {@code true} If secondary file system is initialized.
+     */
+    public boolean hasSecondaryFileSystem() {
+        return secondaryFs != null;
+    }
+
+    /**
      * Convert the given path to path acceptable by the primary file system.
      *
      * @param path Path.

http://git-wip-us.apache.org/repos/asf/ignite/blob/03ad6aa8/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest.java
new file mode 100644
index 0000000..511c4bb
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest.java
@@ -0,0 +1,214 @@
+/*
+ * 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.igfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.FileSystemConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem;
+import org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem;
+import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+
+import java.net.URI;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.igfs.IgfsMode.PRIMARY;
+
+/**
+ * Ensures correct modes resolution for SECONDARY paths.
+ */
+public class IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest extends IgfsCommonAbstractTest {
+    /** File system. */
+    private IgniteHadoopFileSystem fs;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        U.closeQuiet(fs);
+
+        fs = null;
+
+        G.stopAll(true);
+    }
+
+    /**
+     * Perform initial startup.
+     *
+     * @param initDfltPathModes WHether to initialize default path modes.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings({"NullableProblems", "unchecked"})
+    private void startUp(boolean initDfltPathModes) throws Exception {
+        startUpSecondary();
+
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        igfsCfg.setDataCacheName("partitioned");
+        igfsCfg.setMetaCacheName("replicated");
+        igfsCfg.setName("igfs");
+        igfsCfg.setBlockSize(512 * 1024);
+        igfsCfg.setInitializeDefaultPathsModes(initDfltPathModes);
+
+        IgfsIpcEndpointConfiguration endpointCfg = new IgfsIpcEndpointConfiguration();
+
+        endpointCfg.setType(IgfsIpcEndpointType.TCP);
+        endpointCfg.setPort(10500);
+
+        igfsCfg.setIpcEndpointConfiguration(endpointCfg);
+
+        igfsCfg.setManagementPort(-1);
+        igfsCfg.setSecondaryFileSystem(new IgniteHadoopIgfsSecondaryFileSystem(
+            "igfs://igfs-secondary:igfs-grid-secondary@127.0.0.1:11500/",
+            "modules/core/src/test/config/hadoop/core-site-loopback-secondary.xml"));
+
+        CacheConfiguration cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setName("partitioned");
+        cacheCfg.setCacheMode(PARTITIONED);
+        cacheCfg.setNearConfiguration(null);
+        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
+        cacheCfg.setBackups(0);
+        cacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
+
+        metaCacheCfg.setName("replicated");
+        metaCacheCfg.setCacheMode(REPLICATED);
+        metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        IgniteConfiguration cfg = new IgniteConfiguration();
+
+        cfg.setGridName("igfs-grid");
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+
+        cfg.setDiscoverySpi(discoSpi);
+        cfg.setCacheConfiguration(metaCacheCfg, cacheCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
+
+        cfg.setLocalHost("127.0.0.1");
+
+        G.start(cfg);
+
+        Configuration fsCfg = new Configuration();
+
+        fsCfg.addResource(U.resolveIgniteUrl("modules/core/src/test/config/hadoop/core-site-loopback.xml"));
+
+        fsCfg.setBoolean("fs.igfs.impl.disable.cache", true);
+
+        fs = (IgniteHadoopFileSystem)FileSystem.get(new URI("igfs://igfs:igfs-grid@/"), fsCfg);
+    }
+
+    /**
+     * Startup secondary file system.
+     *
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    private void startUpSecondary() throws Exception {
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        igfsCfg.setDataCacheName("partitioned");
+        igfsCfg.setMetaCacheName("replicated");
+        igfsCfg.setName("igfs-secondary");
+        igfsCfg.setBlockSize(512 * 1024);
+        igfsCfg.setDefaultMode(PRIMARY);
+
+        IgfsIpcEndpointConfiguration endpointCfg = new IgfsIpcEndpointConfiguration();
+
+        endpointCfg.setType(IgfsIpcEndpointType.TCP);
+        endpointCfg.setPort(11500);
+
+        igfsCfg.setIpcEndpointConfiguration(endpointCfg);
+
+        CacheConfiguration cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setName("partitioned");
+        cacheCfg.setCacheMode(PARTITIONED);
+        cacheCfg.setNearConfiguration(null);
+        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
+        cacheCfg.setBackups(0);
+        cacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
+
+        metaCacheCfg.setName("replicated");
+        metaCacheCfg.setCacheMode(REPLICATED);
+        metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        IgniteConfiguration cfg = new IgniteConfiguration();
+
+        cfg.setGridName("igfs-grid-secondary");
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+
+        cfg.setDiscoverySpi(discoSpi);
+        cfg.setCacheConfiguration(metaCacheCfg, cacheCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
+
+        cfg.setLocalHost("127.0.0.1");
+
+        G.start(cfg);
+    }
+
+    /**
+     * Test scenario when defaults are initialized.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDefaultsInitialized() throws Exception {
+        check(true);
+    }
+
+    /**
+     * Test scenario when defaults are not initialized.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDefaultsNotInitialized() throws Exception {
+        check(false);
+    }
+
+    /**
+     * Actual check.
+     *
+     * @param initDfltPathModes Whether to initialize default path modes.
+     * @throws Exception If failed.
+     */
+    private void check(boolean initDfltPathModes) throws Exception {
+        startUp(initDfltPathModes);
+
+        assertEquals(initDfltPathModes, fs.hasSecondaryFileSystem());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/03ad6aa8/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
index 0216f4b..6641bc8 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
@@ -52,6 +52,7 @@ import org.apache.ignite.igfs.IgniteHadoopFileSystemLoopbackExternalDualAsyncSel
 import org.apache.ignite.igfs.IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest;
 import org.apache.ignite.igfs.IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest;
 import org.apache.ignite.igfs.IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest;
+import org.apache.ignite.igfs.IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest;
 import org.apache.ignite.igfs.IgniteHadoopFileSystemSecondaryModeSelfTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopCommandLineTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopDefaultMapReducePlannerSelfTest;
@@ -112,6 +113,7 @@ public class IgniteHadoopTestSuite extends TestSuite {
         suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.class.getName())));
 
         suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemSecondaryModeSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest.class.getName())));
 
         suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemClientSelfTest.class.getName())));