Jelajahi Sumber

Revert "HDFS-13120. Snapshot diff could be corrupted after concat. Contributed by Xiaoyu Yao."

This reverts commit 7aec9fac4a72e7c868c9c1c7b4252a746694c936.
Kihwal Lee 7 tahun lalu
induk
melakukan
cb9e60e3d7

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

@@ -246,9 +246,7 @@ class FSDirConcatOp {
     for (INodeFile nodeToRemove : srcList) {
       if(nodeToRemove != null) {
         nodeToRemove.clearBlocks();
-        // Ensure the nodeToRemove is cleared from snapshot diff list
-        nodeToRemove.getParent().removeChild(nodeToRemove,
-            targetIIP.getLatestSnapshotId());
+        nodeToRemove.getParent().removeChild(nodeToRemove);
         fsd.getINodeMap().remove(nodeToRemove);
         count++;
       }

+ 0 - 126
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java

@@ -26,22 +26,18 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.security.PrivilegedAction;
 
-import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
@@ -65,15 +61,11 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Tests snapshot deletion.
  */
 public class TestSnapshotDeletion {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(TestSnapshotDeletion.class);
   protected static final long seed = 0;
   protected static final short REPLICATION = 3;
   protected static final short REPLICATION_1 = 2;
@@ -1240,122 +1232,4 @@ public class TestSnapshotDeletion {
     // make sure bar has been cleaned from inodeMap
     Assert.assertNull(fsdir.getInode(fileId));
   }
-
-  @Test
-  public void testSnapshotWithConcatException() throws Exception {
-    final Path st = new Path("/st");
-    hdfs.mkdirs(st);
-    hdfs.allowSnapshot(st);
-
-    Path[] files = new Path[3];
-    for (int i = 0; i < 3; i++) {
-      files[i] = new Path(st, i+ ".txt");
-    }
-
-    Path dest = new Path(st, "dest.txt");
-    hdfs.createNewFile(dest);
-    hdfs.createSnapshot(st, "ss");
-
-    for (int j = 0; j < 3; j++) {
-      FileSystem fs = cluster.getFileSystem();
-      DFSTestUtil.createFile(fs, files[j], false, 1024,
-          1024, 512, (short) 1, RandomUtils.nextLong(1, 512), true);
-    }
-
-    hdfs.createSnapshot(st, "s0");
-
-    // Verify the SnapshotException is thrown as expected for HDFS-4529
-    exception.expect(RemoteException.class);
-    String error = "Concat: the source file /st/0.txt is in snapshot";
-    exception.expectMessage(error);
-    hdfs.concat(dest, files);
-
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
-    hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
-
-    cluster.restartNameNodes();
-  }
-
-  @Test
-  public void testSnapshotDeleteWithConcat() throws Exception {
-    final Path st = new Path("/st");
-    hdfs.mkdirs(st);
-    hdfs.allowSnapshot(st);
-
-    Path[] files = new Path[3];
-    for (int i = 0; i < 3; i++) {
-      files[i] = new Path(st, i+ ".txt");
-    }
-
-    Path dest = new Path(st, "dest.txt");
-    hdfs.createNewFile(dest);
-    hdfs.createSnapshot(st, "ss");
-
-    for (int i = 0; i < 3; i++) {
-      for (int j = 0; j < 3; j++) {
-        FileSystem fs = cluster.getFileSystem();
-        DFSTestUtil.createFile(fs, files[j], false, 1024,
-            1024, 512, (short) 1, RandomUtils.nextLong(1, 512), true);
-      }
-
-      hdfs.concat(dest, files);
-
-      hdfs.createSnapshot(st, "s" + i);
-    }
-
-
-    hdfs.deleteSnapshot(st, "s1");
-
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
-    hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
-
-    cluster.restartNameNodes();
-  }
-
-  @Test
-  public void testSnapshotDiffReportWithConcat() throws Exception {
-    final Path st = new Path("/st");
-    hdfs.mkdirs(st);
-    hdfs.allowSnapshot(st);
-
-    Path[] files = new Path[3];
-    for (int i = 0; i < 3; i++) {
-      files[i] = new Path(st, i+ ".txt");
-    }
-
-    Path dest = new Path(st, "dest.txt");
-    hdfs.createNewFile(dest);
-    hdfs.createSnapshot(st, "ss");
-
-    for (int i = 0; i < 3; i++) {
-      for (int j = 0; j < 3; j++) {
-        FileSystem fs = cluster.getFileSystem();
-        DFSTestUtil.createFile(fs, files[j], false, 1024,
-            1024, 512, (short) 1, RandomUtils.nextLong(1, 512), true);
-      }
-
-      hdfs.concat(dest, files);
-
-      hdfs.createSnapshot(st, "s" + i);
-
-      SnapshotDiffReport sdr = hdfs.getSnapshotDiffReport(st, "s" + i, "ss");
-      LOG.info("Snapshot Diff s{} to ss : {}", i, sdr);
-      Assert.assertEquals(sdr.getDiffList().size(), 1);
-      Assert.assertTrue(sdr.getDiffList().get(0).getType() ==
-          SnapshotDiffReport.DiffType.MODIFY);
-      Assert.assertTrue(new Path(st, DFSUtilClient.bytes2String(
-          sdr.getDiffList().get(0).getSourcePath())).equals(dest));
-    }
-
-    hdfs.deleteSnapshot(st, "s1");
-
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
-    hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
-
-    cluster.restartNameNodes();
-  }
-
 }