|
@@ -33,6 +33,9 @@ import static org.junit.Assert.fail;
|
|
|
import java.io.IOException;
|
|
|
import java.util.concurrent.ThreadLocalRandom;
|
|
|
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector;
|
|
|
+import org.apache.hadoop.ipc.RemoteException;
|
|
|
+import org.apache.hadoop.test.LambdaTestUtils;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
@@ -218,6 +221,70 @@ public class TestFileTruncate {
|
|
|
fs.delete(dir, true);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Test truncate twice together on a file.
|
|
|
+ */
|
|
|
+ @Test(timeout=90000)
|
|
|
+ public void testTruncateTwiceTogether() throws Exception {
|
|
|
+
|
|
|
+ Path dir = new Path("/testTruncateTwiceTogether");
|
|
|
+ fs.mkdirs(dir);
|
|
|
+ final Path p = new Path(dir, "file");
|
|
|
+ final byte[] data = new byte[100 * BLOCK_SIZE];
|
|
|
+ ThreadLocalRandom.current().nextBytes(data);
|
|
|
+ writeContents(data, data.length, p);
|
|
|
+
|
|
|
+ DataNodeFaultInjector originInjector = DataNodeFaultInjector.get();
|
|
|
+ DataNodeFaultInjector injector = new DataNodeFaultInjector() {
|
|
|
+ @Override
|
|
|
+ public void delay() {
|
|
|
+ try {
|
|
|
+ // Bigger than soft lease period.
|
|
|
+ Thread.sleep(5000);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ // Ignore
|
|
|
+ }
|
|
|
+ }
|
|
|
+ };
|
|
|
+ // Delay to recovery.
|
|
|
+ DataNodeFaultInjector.set(injector);
|
|
|
+
|
|
|
+ // Truncate by using different client name.
|
|
|
+ Thread t = new Thread(() -> {
|
|
|
+ String hdfsCacheDisableKey = "fs.hdfs.impl.disable.cache";
|
|
|
+ boolean originCacheDisable =
|
|
|
+ conf.getBoolean(hdfsCacheDisableKey, false);
|
|
|
+ try {
|
|
|
+ conf.setBoolean(hdfsCacheDisableKey, true);
|
|
|
+ FileSystem fs1 = FileSystem.get(conf);
|
|
|
+ fs1.truncate(p, data.length-1);
|
|
|
+ } catch (IOException e) {
|
|
|
+ // ignore
|
|
|
+ } finally{
|
|
|
+ conf.setBoolean(hdfsCacheDisableKey, originCacheDisable);
|
|
|
+ }
|
|
|
+ });
|
|
|
+ t.start();
|
|
|
+ t.join();
|
|
|
+ NameNodeAdapter.getLeaseManager(cluster.getNamesystem())
|
|
|
+ .setLeasePeriod(LOW_SOFTLIMIT, LOW_HARDLIMIT);
|
|
|
+
|
|
|
+ LambdaTestUtils.intercept(RemoteException.class,
|
|
|
+ "/testTruncateTwiceTogether/file is being truncated",
|
|
|
+ () -> fs.truncate(p, data.length - 2));
|
|
|
+
|
|
|
+ // wait for block recovery
|
|
|
+ checkBlockRecovery(p);
|
|
|
+ assertFileLength(p, data.length - 1);
|
|
|
+
|
|
|
+ DataNodeFaultInjector.set(originInjector);
|
|
|
+ NameNodeAdapter.getLeaseManager(cluster.getNamesystem())
|
|
|
+ .setLeasePeriod(HdfsConstants.LEASE_SOFTLIMIT_PERIOD,
|
|
|
+ conf.getLong(DFSConfigKeys.DFS_LEASE_HARDLIMIT_KEY,
|
|
|
+ DFSConfigKeys.DFS_LEASE_HARDLIMIT_DEFAULT) * 1000);
|
|
|
+ fs.delete(dir, true);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Truncate files and then run other operations such as
|
|
|
* rename, set replication, set permission, etc.
|
|
@@ -631,7 +698,7 @@ public class TestFileTruncate {
|
|
|
{
|
|
|
try {
|
|
|
fs.truncate(p, 0);
|
|
|
- fail("Truncate must fail since a trancate is already in pregress.");
|
|
|
+ fail("Truncate must fail since a truncate is already in progress.");
|
|
|
} catch (IOException expected) {
|
|
|
GenericTestUtils.assertExceptionContains(
|
|
|
"Failed to TRUNCATE_FILE", expected);
|