浏览代码

HDFS-4719. Remove AbstractINodeDiff.Factory and move its methods to AbstractINodeDiffList. Contributed by Arpit Agarwal

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1470759 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 年之前
父节点
当前提交
10a2d82b45
共有 12 个文件被更改,包括 84 次插入109 次删除
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
  2. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  3. 0 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  4. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  5. 4 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
  6. 14 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
  7. 57 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
  8. 0 24
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshot.java
  9. 3 28
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java
  10. 0 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionWithSnapshot.java
  11. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithSnapshot.java
  12. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt

@@ -266,3 +266,6 @@ Branch-2802 Snapshot (Unreleased)
 
   HDFS-4727. Update inodeMap after deleting files/directories/snapshots.
   (Jing Zhao via szetszwo)
+
+  HDFS-4719. Remove AbstractINodeDiff.Factory and move its methods to 
+  AbstractINodeDiffList.  (Arpit Agarwal via szetszwo)

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -53,7 +53,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiffList;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionWithSnapshot;

+ 0 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -141,13 +141,6 @@ public class INodeDirectory extends INodeWithAdditionalFields {
     return true;
   }
 
-  /**
-   * Remove the specified child and all its snapshot copies from this directory.
-   */
-  public boolean removeChildAndAllSnapshotCopies(INode child) {
-    return removeChild(child);
-  }
-
   /**
    * Replace itself with {@link INodeDirectoryWithQuota} or
    * {@link INodeDirectoryWithSnapshot} depending on the latest snapshot.

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -34,7 +34,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.Content.CountsMap.Key;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiff;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiffList;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.Util;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 

+ 4 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java

@@ -30,7 +30,7 @@ import com.google.common.base.Preconditions;
 
 /**
  * The difference of an inode between in two snapshots.
- * {@link AbstractINodeDiff2} maintains a list of snapshot diffs,
+ * {@link AbstractINodeDiffList} maintains a list of snapshot diffs,
  * <pre>
  *   d_1 -> d_2 -> ... -> d_n -> null,
  * </pre>
@@ -49,15 +49,6 @@ import com.google.common.base.Preconditions;
 abstract class AbstractINodeDiff<N extends INode,
                                  D extends AbstractINodeDiff<N, D>>
     implements Comparable<Snapshot> {
-  /** A factory for creating diff and snapshot copy of an inode. */
-  static abstract class Factory<N extends INode,
-                                D extends AbstractINodeDiff<N, D>> {
-    /** @return an {@link AbstractINodeDiff}. */
-    abstract D createDiff(Snapshot snapshot, N currentINode);
-
-    /** @return a snapshot copy of the current inode. */
-    abstract N createSnapshotCopy(N currentINode);
-  }
 
   /** The snapshot will be obtained after this diff is applied. */
   Snapshot snapshot;
@@ -105,13 +96,9 @@ abstract class AbstractINodeDiff<N extends INode,
   }
 
   /** Save the INode state to the snapshot if it is not done already. */
-  void saveSnapshotCopy(N snapshotCopy, Factory<N, D> factory, N currentINode) {
-    if (snapshotINode == null) {
-      if (snapshotCopy == null) {
-        snapshotCopy = factory.createSnapshotCopy(currentINode);
-      }
-      snapshotINode = snapshotCopy;
-    }
+  void saveSnapshotCopy(N snapshotCopy, N currentINode) {
+    Preconditions.checkState(snapshotINode == null, "Expected snapshotINode to be null");
+    snapshotINode = snapshotCopy;
   }
 
   /** @return the inode corresponding to the snapshot. */

+ 14 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java

@@ -37,15 +37,9 @@ import org.apache.hadoop.hdfs.server.namenode.Quota;
 abstract class AbstractINodeDiffList<N extends INode,
                                      D extends AbstractINodeDiff<N, D>> 
     implements Iterable<D> {
-  private AbstractINodeDiff.Factory<N, D> factory;
-
   /** Diff list sorted by snapshot IDs, i.e. in chronological order. */
   private final List<D> diffs = new ArrayList<D>();
 
-  void setFactory(AbstractINodeDiff.Factory<N, D> factory) {
-    this.factory = factory;
-  }
-
   /** @return this list as a unmodifiable {@link List}. */
   public final List<D> asList() {
     return Collections.unmodifiableList(diffs);
@@ -56,6 +50,12 @@ abstract class AbstractINodeDiffList<N extends INode,
     diffs.clear();
   }
 
+  /** @return an {@link AbstractINodeDiff}. */
+  abstract D createDiff(Snapshot snapshot, N currentINode);
+
+  /** @return a snapshot copy of the current inode. */  
+  abstract N createSnapshotCopy(N currentINode);
+
   /**
    * Delete a snapshot. The synchronization of the diff list will be done 
    * outside. If the diff to remove is not the first one in the diff list, we 
@@ -109,7 +109,7 @@ abstract class AbstractINodeDiffList<N extends INode,
   final D addDiff(Snapshot latest, N currentINode)
       throws QuotaExceededException {
     currentINode.addSpaceConsumed(1, 0, true);
-    return addLast(factory.createDiff(latest, currentINode));
+    return addLast(createDiff(latest, currentINode));
   }
 
   /** Append the diff at the end of the list. */
@@ -258,8 +258,13 @@ abstract class AbstractINodeDiffList<N extends INode,
   public void saveSelf2Snapshot(Snapshot latest, N currentINode, N snapshotCopy)
       throws QuotaExceededException {
     if (latest != null) {
-      checkAndAddLatestSnapshotDiff(latest, currentINode).saveSnapshotCopy(
-          snapshotCopy, factory, currentINode);
+      D diff = checkAndAddLatestSnapshotDiff(latest, currentINode);
+      if (diff.snapshotINode == null) {
+        if (snapshotCopy == null) {
+          snapshotCopy = createSnapshotCopy(currentINode);
+        }
+        diff.saveSnapshotCopy(snapshotCopy, currentINode);
+      }
     }
   }
   

+ 57 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java

@@ -0,0 +1,57 @@
+/**
+ * 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.hadoop.hdfs.server.namenode.snapshot;
+
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiff;
+
+/**
+ * A list of FileDiffs for storing snapshot data.
+ */
+@InterfaceAudience.Private
+public class FileDiffList
+    extends AbstractINodeDiffList<INodeFile, FileDiff> {
+
+  
+  @Override
+  FileDiff createDiff(Snapshot snapshot, INodeFile file) {
+    return new FileDiff(snapshot, file);
+  }
+
+  @Override
+  INodeFile createSnapshotCopy(INodeFile currentINode) {
+    if (currentINode instanceof INodeFileUnderConstructionWithSnapshot) {
+      final INodeFileUnderConstruction uc = 
+          (INodeFileUnderConstruction) currentINode;
+      
+      final INodeFileUnderConstruction copy = new INodeFileUnderConstruction(
+          uc, uc.getClientName(), uc.getClientMachine(), uc.getClientNode());
+      
+      copy.setBlocks(null);
+      return copy;
+    } else {
+      final INodeFile copy = new INodeFile(currentINode);
+      copy.setBlocks(null);
+      return copy;
+    }
+  }
+}

+ 0 - 24
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshot.java

@@ -116,30 +116,6 @@ public interface FileWithSnapshot {
     }
   }
 
-  static class FileDiffFactory
-      extends AbstractINodeDiff.Factory<INodeFile, FileDiff> {
-    static final FileDiffFactory INSTANCE = new FileDiffFactory();
-
-    @Override
-    FileDiff createDiff(Snapshot snapshot, INodeFile file) {
-      return new FileDiff(snapshot, file);
-    }
-
-    @Override
-    INodeFile createSnapshotCopy(INodeFile currentINode) {
-      final INodeFile copy = new INodeFile(currentINode);
-      copy.setBlocks(null);
-      return copy;
-    }
-  }
-
-  /**
-   * A list of {@link FileDiff}.
-   */
-  public static class FileDiffList
-      extends AbstractINodeDiffList<INodeFile, FileDiff> {
-  }
-
   /** @return the {@link INodeFile} view of this object. */
   public INodeFile asINodeFile();
 

+ 3 - 28
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java

@@ -417,9 +417,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     }
   }
 
-  static class DirectoryDiffFactory
-      extends AbstractINodeDiff.Factory<INodeDirectory, DirectoryDiff> {
-    static final DirectoryDiffFactory INSTANCE = new DirectoryDiffFactory();
+  /** A list of directory diffs. */
+  static class DirectoryDiffList
+      extends AbstractINodeDiffList<INodeDirectory, DirectoryDiff> {
 
     @Override
     DirectoryDiff createDiff(Snapshot snapshot, INodeDirectory currentDir) {
@@ -435,15 +435,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
       copy.clearChildren();
       return copy;
     }
-  }
 
-  /** A list of directory diffs. */
-  static class DirectoryDiffList
-      extends AbstractINodeDiffList<INodeDirectory, DirectoryDiff> {
-    DirectoryDiffList() {
-      setFactory(DirectoryDiffFactory.INSTANCE);
-    }
-    
     /** Replace the given child in the created/deleted list, if there is any. */
     private boolean replaceChild(final ListType type, final INode oldChild,
         final INode newChild) {
@@ -635,23 +627,6 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     }
     return removed;
   }
-
-  @Override
-  public boolean removeChildAndAllSnapshotCopies(INode child) {
-    if (!removeChild(child)) {
-      return false;
-    }
-
-    // remove same child from the created list, if there is any.
-    final List<DirectoryDiff> diffList = diffs.asList();
-    for(int i = diffList.size() - 1; i >= 0; i--) {
-      final ChildrenDiff diff = diffList.get(i).diff;
-      if (diff.removeChild(ListType.CREATED, child)) {
-        return true;
-      }
-    }
-    return true;
-  }
   
   @Override
   public void replaceChild(final INode oldChild, final INode newChild) {

+ 0 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionWithSnapshot.java

@@ -29,28 +29,10 @@ import org.apache.hadoop.hdfs.server.namenode.Quota;
 
 /**
  * Represent an {@link INodeFileUnderConstruction} that is snapshotted.
- * Note that snapshot files are represented by
- * {@link INodeFileUnderConstructionSnapshot}.
  */
 @InterfaceAudience.Private
 public class INodeFileUnderConstructionWithSnapshot
     extends INodeFileUnderConstruction implements FileWithSnapshot {
-  /**
-   * Factory for {@link INodeFileUnderConstruction} diff.
-   */
-  static class FileUcDiffFactory extends FileDiffFactory {
-    static final FileUcDiffFactory INSTANCE = new FileUcDiffFactory();
-
-    @Override
-    INodeFileUnderConstruction createSnapshotCopy(INodeFile file) {
-      final INodeFileUnderConstruction uc = (INodeFileUnderConstruction)file;
-      final INodeFileUnderConstruction copy = new INodeFileUnderConstruction(
-          uc, uc.getClientName(), uc.getClientMachine(), uc.getClientNode());
-      copy.setBlocks(null);
-      return copy;
-    }
-  }
-
   private final FileDiffList diffs;
   private boolean isCurrentFileDeleted = false;
 
@@ -61,7 +43,6 @@ public class INodeFileUnderConstructionWithSnapshot
       final FileDiffList diffs) {
     super(f, clientName, clientMachine, clientNode);
     this.diffs = diffs != null? diffs: new FileDiffList();
-    this.diffs.setFactory(FileUcDiffFactory.INSTANCE);
   }
 
   /**

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithSnapshot.java

@@ -28,7 +28,6 @@ import org.apache.hadoop.hdfs.server.namenode.Quota;
 
 /**
  * Represent an {@link INodeFile} that is snapshotted.
- * Note that snapshot files are represented by {@link INodeFileSnapshot}.
  */
 @InterfaceAudience.Private
 public class INodeFileWithSnapshot extends INodeFile
@@ -44,7 +43,6 @@ public class INodeFileWithSnapshot extends INodeFile
   public INodeFileWithSnapshot(INodeFile f, FileDiffList diffs) {
     super(f);
     this.diffs = diffs != null? diffs: new FileDiffList();
-    this.diffs.setFactory(FileDiffFactory.INSTANCE);
   }
 
   @Override

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java

@@ -34,7 +34,7 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiff;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiffList;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiffList;
 import org.apache.hadoop.hdfs.tools.snapshot.SnapshotDiff;