package org.apache.hadoop.hdfs; import junit.framework.TestCase; import java.io.File; import java.io.IOException; import java.io.OutputStream; import java.io.RandomAccessFile; import java.lang.reflect.Method; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.util.*; import java.util.concurrent.*; import java.util.concurrent.atomic.*; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSOutputStream; import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataStorage; import org.apache.hadoop.hdfs.server.datanode.FSDataset; import org.apache.hadoop.hdfs.server.datanode.FSDatasetTestUtil; import org.apache.hadoop.hdfs.server.datanode.NameSpaceSliceStorage; import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException; import org.apache.hadoop.hdfs.server.namenode.LeaseManager; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.FSEditLog; import org.apache.hadoop.hdfs.server.namenode.FSImageAdapter; import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; import static org.apache.hadoop.hdfs.AppendTestUtil.loseLeases; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.Level; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import static org.mockito.Matchers.anyBoolean; import static org.mockito.Matchers.anyInt; import static org.mockito.Matchers.anyObject; import static org.mockito.Matchers.anyLong; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; /* File Append tests for HDFS-200 & HDFS-142, specifically focused on: * using append()/sync() to recover block information */ public class TestFileAppend4 extends TestCase { static final Log LOG = LogFactory.getLog(TestFileAppend4.class); static final long BLOCK_SIZE = 1024; static final long BBW_SIZE = 500; // don't align on bytes/checksum static final Object [] NO_ARGS = new Object []{}; Configuration conf; MiniDFSCluster cluster; Path file1; FSDataOutputStream stm; boolean simulatedStorage = false; { ((Log4JLogger)NameNode.stateChangeLog).getLogger().setLevel(Level.ALL); ((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL); ((Log4JLogger)FSNamesystem.LOG).getLogger().setLevel(Level.ALL); ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL); ((Log4JLogger)DFSClient.LOG).getLogger().setLevel(Level.ALL); } @Override public void setUp() throws Exception { this.conf = new Configuration(); if (simulatedStorage) { conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true); } conf.setBoolean("dfs.support.append", true); // lower heartbeat interval for fast recognition of DN death conf.setInt("heartbeat.recheck.interval", 1000); conf.setInt("dfs.heartbeat.interval", 1); conf.setInt("dfs.socket.timeout", 5000); // handle under-replicated blocks quickly (for replication asserts) // conf.set("dfs.replication.pending.timeout.sec", Integer.toString(5)); conf.setInt("dfs.replication.pending.timeout.sec", 5); conf.setInt("dfs.replication.interval", 1); // handle fai