svn merge -c 1532860 from trunk for adding TestOpenFilesWithSnapshot.java for HDFS-5283.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1532861 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
b2e641e31f
commit
71b1436bcc
|
@ -0,0 +1,113 @@
|
||||||
|
/**
|
||||||
|
* 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 java.io.IOException;
|
||||||
|
import java.util.EnumSet;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hdfs.DFSOutputStream;
|
||||||
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||||
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||||
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||||
|
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
public class TestOpenFilesWithSnapshot {
|
||||||
|
private Configuration conf = new Configuration();
|
||||||
|
MiniDFSCluster cluster = null;
|
||||||
|
DistributedFileSystem fs = null;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setup() throws IOException {
|
||||||
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
|
||||||
|
conf.set("dfs.blocksize", "1048576");
|
||||||
|
fs = cluster.getFileSystem();
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void teardown() throws IOException {
|
||||||
|
if (fs != null) {
|
||||||
|
fs.close();
|
||||||
|
}
|
||||||
|
if (cluster != null) {
|
||||||
|
cluster.shutdown();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testUCFileDeleteWithSnapShot() throws Exception {
|
||||||
|
Path path = new Path("/test");
|
||||||
|
doWriteAndAbort(fs, path);
|
||||||
|
|
||||||
|
// delete files separately
|
||||||
|
fs.delete(new Path("/test/test/test2"), true);
|
||||||
|
fs.delete(new Path("/test/test/test3"), true);
|
||||||
|
cluster.restartNameNode();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testParentDirWithUCFileDeleteWithSnapShot() throws Exception {
|
||||||
|
Path path = new Path("/test");
|
||||||
|
doWriteAndAbort(fs, path);
|
||||||
|
|
||||||
|
// delete parent directory
|
||||||
|
fs.delete(new Path("/test/test"), true);
|
||||||
|
cluster.restartNameNode();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void doWriteAndAbort(DistributedFileSystem fs, Path path)
|
||||||
|
throws IOException {
|
||||||
|
fs.mkdirs(path);
|
||||||
|
fs.allowSnapshot(path);
|
||||||
|
DFSTestUtil
|
||||||
|
.createFile(fs, new Path("/test/test1"), 100, (short) 2, 100024L);
|
||||||
|
DFSTestUtil
|
||||||
|
.createFile(fs, new Path("/test/test2"), 100, (short) 2, 100024L);
|
||||||
|
Path file = new Path("/test/test/test2");
|
||||||
|
FSDataOutputStream out = fs.create(file);
|
||||||
|
for (int i = 0; i < 2; i++) {
|
||||||
|
long count = 0;
|
||||||
|
while (count < 1048576) {
|
||||||
|
out.writeBytes("hell");
|
||||||
|
count += 4;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
((DFSOutputStream) out.getWrappedStream()).hsync(EnumSet
|
||||||
|
.of(SyncFlag.UPDATE_LENGTH));
|
||||||
|
DFSTestUtil.abortStream((DFSOutputStream) out.getWrappedStream());
|
||||||
|
Path file2 = new Path("/test/test/test3");
|
||||||
|
FSDataOutputStream out2 = fs.create(file2);
|
||||||
|
for (int i = 0; i < 2; i++) {
|
||||||
|
long count = 0;
|
||||||
|
while (count < 1048576) {
|
||||||
|
out2.writeBytes("hell");
|
||||||
|
count += 4;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
((DFSOutputStream) out2.getWrappedStream()).hsync(EnumSet
|
||||||
|
.of(SyncFlag.UPDATE_LENGTH));
|
||||||
|
DFSTestUtil.abortStream((DFSOutputStream) out2.getWrappedStream());
|
||||||
|
fs.createSnapshot(path, "s1");
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue