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/03/17 20:14:03 UTC

ignite git commit: Reworked delete.

Repository: ignite
Updated Branches:
  refs/heads/igfs-refactoring [created] e15ff973e


Reworked delete.


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

Branch: refs/heads/igfs-refactoring
Commit: e15ff973e28e206af10ec5a3a059361bcba9da53
Parents: 3e53f17
Author: thatcoach <pp...@list.ru>
Authored: Thu Mar 17 22:07:06 2016 +0300
Committer: thatcoach <pp...@list.ru>
Committed: Thu Mar 17 22:07:06 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsMetaManager.java        | 159 +++++++++--------
 .../internal/processors/igfs/IgfsPathIds.java   | 175 +++++++++++++++++++
 .../internal/processors/igfs/IgfsUtils.java     |  29 ++-
 3 files changed, 279 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e15ff973/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 b4774f2..626881b 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
@@ -17,31 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.concurrent.CountDownLatch;
-import javax.cache.processor.EntryProcessor;
-import javax.cache.processor.EntryProcessorException;
-import javax.cache.processor.MutableEntry;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteInterruptedException;
@@ -84,14 +59,35 @@ import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_CREATED;
-import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_RENAMED;
-import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CREATED;
-import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_RENAMED;
-import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_WRITE;
-import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.builder;
-import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
-import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+
+import static org.apache.ignite.events.EventType.*;
+import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
 
 /**
  * Cache based structure (meta data) manager.
@@ -334,6 +330,34 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
+     * Gets all file IDs for components of specified path. Result cannot be empty - there is at least root element.
+     * But each element (except the first) can be {@code null} if such files don't exist.
+     *
+     * @param path Path.
+     * @return Collection of file IDs for components of specified path.
+     * @throws IgniteCheckedException If failed.
+     */
+    public IgfsPathIds pathIds(IgfsPath path) throws IgniteCheckedException {
+        if (busyLock.enterBusy()) {
+            try {
+                validTxState(false);
+
+                List<IgniteUuid> ids = fileIds(path, false);
+
+                List<String> parts = new ArrayList<>(path.components());
+                parts.add(0, null);
+
+                return new IgfsPathIds(path, parts.toArray(new String[0]), ids.toArray(new IgniteUuid[0]));
+            }
+            finally {
+                busyLock.leaveBusy();
+            }
+        }
+        else
+            throw new IllegalStateException("Failed to get file IDS because Grid is stopping: " + path);
+    }
+
+    /**
      * Gets all file IDs for components of specified path possibly skipping existing transaction. Result cannot
      * be empty - there is at least root element. But each element (except the first) can be {@code null} if such
      * files don't exist.
@@ -1117,72 +1141,57 @@ public class IgfsMetaManager extends IgfsManager {
             try {
                 validTxState(false);
 
-                final SortedSet<IgniteUuid> allIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
-
-                List<IgniteUuid> pathIdList = fileIds(path);
+                IgfsPathIds pathIds = pathIds(path);
 
-                assert pathIdList.size() > 1;
+                // Continue only if the whole path present.
+                if (!pathIds.allPartsFound())
+                    return null; // A fragment of the path no longer exists.
 
-                final IgniteUuid victimId = pathIdList.get(pathIdList.size() - 1);
+                IgniteUuid victimId = pathIds.lastId();
+                String victimName = pathIds.lastPart();
 
-                assert !IgfsUtils.isRootOrTrashId(victimId) : "Cannot delete root or trash directories.";
+                if (IgfsUtils.isRootId(victimId))
+                    throw new IgfsException("Cannot remove root directory");
 
-                allIds.addAll(pathIdList);
+                // Prepare IDs to lock.
+                SortedSet<IgniteUuid> allIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
 
-                if (allIds.remove(null))
-                    return null; // A fragment of the path no longer exists.
+                pathIds.addExistingIds(allIds);
 
                 IgniteUuid trashId = IgfsUtils.randomTrashId();
 
-                boolean added = allIds.add(trashId);
-                assert added;
+                allIds.add(trashId);
 
-                final IgniteInternalTx tx = startTx();
+                IgniteInternalTx tx = startTx();
 
                 try {
-                    final Map<IgniteUuid, IgfsFileInfo> infoMap = lockIds(allIds);
+                    // Lock participants.
+                    Map<IgniteUuid, IgfsFileInfo> lockInfos = lockIds(allIds);
 
-                    // Directory starure was changed concurrently, so the original path no longer exists:
-                    if (!verifyPathIntegrity(path, pathIdList, infoMap))
+                    // Ensure that all participants are still in place.
+                    if (!pathIds.verifyIntergrity(lockInfos))
                         return null;
 
-                    final IgfsFileInfo victimInfo = infoMap.get(victimId);
+                    IgfsFileInfo victimInfo = lockInfos.get(victimId);
 
+                    // Cannot delete non-empty directory if recursive flag is not set.
                     if (!recursive && victimInfo.hasChildren())
-                        // Throw exception if not empty and not recursive.
                         throw new IgfsDirectoryNotEmptyException("Failed to remove directory (directory is not " +
                             "empty and recursive flag is not set).");
 
-                    IgfsFileInfo destInfo = infoMap.get(trashId);
-
-                    assert destInfo != null;
-
-                    final String srcFileName = path.name();
-
-                    final String destFileName = victimId.toString();
-
-                    assert !destInfo.hasChild(destFileName) : "Failed to add file name into the " +
-                        "destination directory (file already exists) [destName=" + destFileName + ']';
-
-                    IgfsFileInfo srcParentInfo = infoMap.get(pathIdList.get(pathIdList.size() - 2));
-
-                    assert srcParentInfo != null;
-
-                    IgniteUuid srcParentId = srcParentInfo.id();
-                    assert srcParentId.equals(pathIdList.get(pathIdList.size() - 2));
-
-                    IgfsListingEntry srcEntry = srcParentInfo.listing().get(srcFileName);
+                    // Prepare trash data.
+                    IgfsFileInfo trashInfo = lockInfos.get(trashId);
+                    final String trashName = victimId.toString();
 
-                    assert srcEntry != null : "Deletion victim not found in parent listing [path=" + path +
-                        ", name=" + srcFileName + ", listing=" + srcParentInfo.listing() + ']';
+                    assert !trashInfo.hasChild(trashName) : "Failed to add file name into the " +
+                        "destination directory (file already exists) [destName=" + trashName + ']';
 
-                    assert victimId.equals(srcEntry.fileId());
+                    IgniteUuid parentId = pathIds.lastParentId();
+                    IgfsFileInfo parentInfo = lockInfos.get(parentId);
 
-                    transferEntry(srcEntry, srcParentId, srcFileName, trashId, destFileName);
+                    transferEntry(parentInfo.listing().get(victimName), parentId, victimName, trashId, trashName);
 
                     if (victimInfo.isFile())
-                        // Update a file info of the removed file with a file path,
-                        // which will be used by delete worker for event notifications.
                         invokeUpdatePath(victimId, path);
 
                     tx.commit();

http://git-wip-us.apache.org/repos/asf/ignite/blob/e15ff973/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
new file mode 100644
index 0000000..63f281d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
@@ -0,0 +1,175 @@
+/*
+ * 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.igfs.IgfsPath;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.Collection;
+import java.util.Map;
+
+/**
+ * Path IDs abstraction. Contains path and corresponding IDs.
+ */
+public class IgfsPathIds {
+    /** Original path. */
+    private final IgfsPath path;
+
+    /** Path parts. */
+    private final String[] parts;
+
+    /** IDs. */
+    private final IgniteUuid[] ids;
+
+    /** Index of last existing ID. */
+    private final int lastExistingIdx;
+
+    /**
+     * Constructor.
+     *
+     * @param path Path.
+     * @param parts Path parts.
+     * @param ids IDs.
+     */
+    public IgfsPathIds(IgfsPath path, String[] parts, IgniteUuid[] ids) {
+        assert path != null;
+        assert parts.length == ids.length;
+
+        this.path = path;
+        this.parts = parts;
+        this.ids = ids;
+
+        int lastExistingIdx0 = -1;
+
+        for (int i = parts.length - 1; i >= 0; i--) {
+            if (ids[i] != null) {
+                lastExistingIdx0 = i;
+
+                break;
+            }
+        }
+
+        assert lastExistingIdx0 >= 0;
+
+        lastExistingIdx = lastExistingIdx0;
+    }
+
+    /**
+     * @return Path.
+     */
+    public IgfsPath path() {
+        return path;
+    }
+
+    /**
+     * @return Last part.
+     */
+    public String lastPart() {
+        return parts[parts.length - 1];
+    }
+
+    /**
+     * @return Last ID.
+     */
+    @Nullable public IgniteUuid lastId() {
+        return ids[ids.length - 1];
+    }
+
+    /**
+     * @return Whether last path part has parent.
+     */
+    public boolean lastHasParent() {
+        return ids.length > 1;
+    }
+
+    /**
+     * @return Before last ID.
+     */
+    @Nullable public IgniteUuid lastParentId() {
+        assert lastHasParent();
+
+        return ids[ids.length - 2];
+    }
+
+    /**
+     * @return Last existing ID.
+     */
+    public IgniteUuid lastExistingId() {
+        return ids[lastExistingIdx];
+    }
+
+    /**
+     * @return Whether last existing path part has parent.
+     */
+    public boolean lastExistingHasParent() {
+        return lastExistingIdx > 0;
+    }
+
+    /**
+     * @return Last existing parent ID.
+     */
+    public IgniteUuid lastExistingParentId() {
+        return ids[lastExistingIdx - 1];
+    }
+
+    /**
+     * @return {@code True} if all parts were found.
+     */
+    public boolean allPartsFound() {
+        return parts.length == lastExistingIdx + 1;
+    }
+
+    /**
+     * Add existing IDs to provided collection.
+     *
+     * @param col Collection.
+     */
+    public void addExistingIds(Collection<IgniteUuid> col) {
+        for (int i = 0; i <= lastExistingIdx; i++)
+            col.add(ids[i]);
+    }
+
+    /**
+     * Verify that observed paths are found in provided infos in the right order.
+     *
+     * @param infos Info.
+     * @return {@code True} if full integrity is preserved.
+     */
+    public boolean verifyIntergrity(Map<IgniteUuid, IgfsFileInfo> infos) {
+        for (int i = 0; i <= lastExistingIdx; i++) {
+            IgniteUuid curId = ids[i];
+            IgfsFileInfo curInfo = infos.get(curId);
+
+            // Check if required ID is there.
+            if (curInfo == null)
+                return false;
+
+            // For non-leaf entry we check if child exists.
+            if (i < lastExistingIdx) {
+                String childName = parts[i + 1];
+                IgniteUuid childId = ids[i + 1];
+
+                if (!curInfo.hasChild(childName, childId))
+                    return false;
+            }
+        }
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e15ff973/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 edded2f..799852c 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
@@ -17,10 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.lang.reflect.Constructor;
-import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
@@ -43,9 +39,13 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.transactions.Transaction;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_RETRIES_COUNT;
-import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
-import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+import java.lang.reflect.Constructor;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.ignite.IgniteSystemProperties.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
 
 /**
  * Common IGFS utility methods.
@@ -104,7 +104,17 @@ public class IgfsUtils {
      * @return {@code True} if this is root ID or trash ID.
      */
     public static boolean isRootOrTrashId(@Nullable IgniteUuid id) {
-        return id != null && (ROOT_ID.equals(id) || isTrashId(id));
+        return isRootId(id) || isTrashId(id);
+    }
+
+    /**
+     * Check whether provided ID is root ID.
+     *
+     * @param id ID.
+     * @return {@code True} if this is root ID.
+     */
+    public static boolean isRootId(@Nullable IgniteUuid id) {
+        return id != null && ROOT_ID.equals(id);
     }
 
     /**
@@ -114,7 +124,8 @@ public class IgfsUtils {
      * @return {@code True} if this is trash ID.
      */
     private static boolean isTrashId(IgniteUuid id) {
-        assert id != null;
+        if (id == null)
+            return false;
 
         UUID gid = id.globalId();