HDFS-3689. Add support for variable length block. Contributed by Jing Zhao.
(cherry picked from commit 2848db814a
)
Conflicts:
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java
This commit is contained in:
parent
9d57392f5b
commit
2156e38d5e
|
@ -47,6 +47,10 @@ import org.apache.hadoop.classification.InterfaceStability;
|
||||||
* <li> SYNC_BLOCK - to force closed blocks to the disk device.
|
* <li> SYNC_BLOCK - to force closed blocks to the disk device.
|
||||||
* In addition {@link Syncable#hsync()} should be called after each write,
|
* In addition {@link Syncable#hsync()} should be called after each write,
|
||||||
* if true synchronous behavior is required.</li>
|
* if true synchronous behavior is required.</li>
|
||||||
|
* <li> LAZY_PERSIST - Create the block on transient storage (RAM) if
|
||||||
|
* available.</li>
|
||||||
|
* <li> APPEND_NEWBLOCK - Append data to a new block instead of end of the last
|
||||||
|
* partial block.</li>
|
||||||
* </ol>
|
* </ol>
|
||||||
*
|
*
|
||||||
* Following combination is not valid and will result in
|
* Following combination is not valid and will result in
|
||||||
|
@ -93,7 +97,13 @@ public enum CreateFlag {
|
||||||
* This flag must only be used for intermediate data whose loss can be
|
* This flag must only be used for intermediate data whose loss can be
|
||||||
* tolerated by the application.
|
* tolerated by the application.
|
||||||
*/
|
*/
|
||||||
LAZY_PERSIST((short) 0x10);
|
LAZY_PERSIST((short) 0x10),
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Append data to a new block instead of the end of the last partial block.
|
||||||
|
* This is only useful for APPEND.
|
||||||
|
*/
|
||||||
|
NEW_BLOCK((short) 0x20);
|
||||||
|
|
||||||
private final short mode;
|
private final short mode;
|
||||||
|
|
||||||
|
@ -149,4 +159,16 @@ public enum CreateFlag {
|
||||||
+ ". Create option is not specified in " + flag);
|
+ ". Create option is not specified in " + flag);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Validate the CreateFlag for the append operation. The flag must contain
|
||||||
|
* APPEND, and cannot contain OVERWRITE.
|
||||||
|
*/
|
||||||
|
public static void validateForAppend(EnumSet<CreateFlag> flag) {
|
||||||
|
validate(flag);
|
||||||
|
if (!flag.contains(APPEND)) {
|
||||||
|
throw new HadoopIllegalArgumentException(flag
|
||||||
|
+ " does not contain APPEND");
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -18,10 +18,12 @@
|
||||||
package org.apache.hadoop.hdfs.nfs.nfs3;
|
package org.apache.hadoop.hdfs.nfs.nfs3;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.EnumSet;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
||||||
|
import org.apache.hadoop.fs.CreateFlag;
|
||||||
import org.apache.hadoop.hdfs.DFSClient;
|
import org.apache.hadoop.hdfs.DFSClient;
|
||||||
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
||||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
|
import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
|
||||||
|
@ -147,7 +149,8 @@ public class WriteManager {
|
||||||
CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
|
CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
|
||||||
CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
|
CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
|
||||||
|
|
||||||
fos = dfsClient.append(fileIdPath, bufferSize, null, null);
|
fos = dfsClient.append(fileIdPath, bufferSize,
|
||||||
|
EnumSet.of(CreateFlag.APPEND), null, null);
|
||||||
|
|
||||||
latestAttr = Nfs3Utils.getFileAttr(dfsClient, fileIdPath, iug);
|
latestAttr = Nfs3Utils.getFileAttr(dfsClient, fileIdPath, iug);
|
||||||
} catch (RemoteException e) {
|
} catch (RemoteException e) {
|
||||||
|
|
|
@ -33,6 +33,8 @@ Release 2.7.0 - UNRELEASED
|
||||||
HDFS-6133. Add a feature for replica pinning so that a pinned replica
|
HDFS-6133. Add a feature for replica pinning so that a pinned replica
|
||||||
will not be moved by Balancer/Mover. (zhaoyunjiong via szetszwo)
|
will not be moved by Balancer/Mover. (zhaoyunjiong via szetszwo)
|
||||||
|
|
||||||
|
HDFS-3689. Add support for variable length block. Contributed by Jing Zhao.
|
||||||
|
|
||||||
IMPROVEMENTS
|
IMPROVEMENTS
|
||||||
|
|
||||||
HDFS-7055. Add tracing to DFSInputStream (cmccabe)
|
HDFS-7055. Add tracing to DFSInputStream (cmccabe)
|
||||||
|
|
|
@ -1651,8 +1651,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
||||||
*
|
*
|
||||||
* @return output stream
|
* @return output stream
|
||||||
*
|
*
|
||||||
* @see ClientProtocol#create(String, FsPermission, String, EnumSetWritable,
|
* @see ClientProtocol#create for detailed description of exceptions thrown
|
||||||
* boolean, short, long) for detailed description of exceptions thrown
|
|
||||||
*/
|
*/
|
||||||
public DFSOutputStream create(String src,
|
public DFSOutputStream create(String src,
|
||||||
FsPermission permission,
|
FsPermission permission,
|
||||||
|
@ -1726,7 +1725,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
||||||
}
|
}
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
return callAppend(src, buffersize, progress);
|
return callAppend(src, buffersize, flag, progress);
|
||||||
}
|
}
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
@ -1804,11 +1803,16 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Method to get stream returned by append call */
|
/** Method to get stream returned by append call */
|
||||||
private DFSOutputStream callAppend(String src,
|
private DFSOutputStream callAppend(String src, int buffersize,
|
||||||
int buffersize, Progressable progress) throws IOException {
|
EnumSet<CreateFlag> flag, Progressable progress) throws IOException {
|
||||||
LastBlockWithStatus lastBlockWithStatus = null;
|
CreateFlag.validateForAppend(flag);
|
||||||
try {
|
try {
|
||||||
lastBlockWithStatus = namenode.append(src, clientName);
|
LastBlockWithStatus blkWithStatus = namenode.append(src, clientName,
|
||||||
|
new EnumSetWritable<>(flag, CreateFlag.class));
|
||||||
|
return DFSOutputStream.newStreamForAppend(this, src,
|
||||||
|
flag.contains(CreateFlag.NEW_BLOCK),
|
||||||
|
buffersize, progress, blkWithStatus.getLastBlock(),
|
||||||
|
blkWithStatus.getFileStatus(), dfsClientConf.createChecksum());
|
||||||
} catch(RemoteException re) {
|
} catch(RemoteException re) {
|
||||||
throw re.unwrapRemoteException(AccessControlException.class,
|
throw re.unwrapRemoteException(AccessControlException.class,
|
||||||
FileNotFoundException.class,
|
FileNotFoundException.class,
|
||||||
|
@ -1818,10 +1822,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
||||||
UnresolvedPathException.class,
|
UnresolvedPathException.class,
|
||||||
SnapshotAccessControlException.class);
|
SnapshotAccessControlException.class);
|
||||||
}
|
}
|
||||||
HdfsFileStatus newStat = lastBlockWithStatus.getFileStatus();
|
|
||||||
return DFSOutputStream.newStreamForAppend(this, src, buffersize, progress,
|
|
||||||
lastBlockWithStatus.getLastBlock(), newStat,
|
|
||||||
dfsClientConf.createChecksum());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1829,23 +1829,25 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
||||||
*
|
*
|
||||||
* @param src file name
|
* @param src file name
|
||||||
* @param buffersize buffer size
|
* @param buffersize buffer size
|
||||||
|
* @param flag indicates whether to append data to a new block instead of
|
||||||
|
* the last block
|
||||||
* @param progress for reporting write-progress; null is acceptable.
|
* @param progress for reporting write-progress; null is acceptable.
|
||||||
* @param statistics file system statistics; null is acceptable.
|
* @param statistics file system statistics; null is acceptable.
|
||||||
* @return an output stream for writing into the file
|
* @return an output stream for writing into the file
|
||||||
*
|
*
|
||||||
* @see ClientProtocol#append(String, String)
|
* @see ClientProtocol#append(String, String, EnumSetWritable)
|
||||||
*/
|
*/
|
||||||
public HdfsDataOutputStream append(final String src, final int buffersize,
|
public HdfsDataOutputStream append(final String src, final int buffersize,
|
||||||
final Progressable progress, final FileSystem.Statistics statistics
|
EnumSet<CreateFlag> flag, final Progressable progress,
|
||||||
) throws IOException {
|
final FileSystem.Statistics statistics) throws IOException {
|
||||||
final DFSOutputStream out = append(src, buffersize, progress);
|
final DFSOutputStream out = append(src, buffersize, flag, progress);
|
||||||
return createWrappedOutputStream(out, statistics, out.getInitialLen());
|
return createWrappedOutputStream(out, statistics, out.getInitialLen());
|
||||||
}
|
}
|
||||||
|
|
||||||
private DFSOutputStream append(String src, int buffersize, Progressable progress)
|
private DFSOutputStream append(String src, int buffersize,
|
||||||
throws IOException {
|
EnumSet<CreateFlag> flag, Progressable progress) throws IOException {
|
||||||
checkOpen();
|
checkOpen();
|
||||||
final DFSOutputStream result = callAppend(src, buffersize, progress);
|
final DFSOutputStream result = callAppend(src, buffersize, flag, progress);
|
||||||
beginFileLease(result.getFileId(), result);
|
beginFileLease(result.getFileId(), result);
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
@ -1932,7 +1934,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Move blocks from src to trg and delete src
|
* Move blocks from src to trg and delete src
|
||||||
* See {@link ClientProtocol#concat(String, String [])}.
|
* See {@link ClientProtocol#concat}.
|
||||||
*/
|
*/
|
||||||
public void concat(String trg, String [] srcs) throws IOException {
|
public void concat(String trg, String [] srcs) throws IOException {
|
||||||
checkOpen();
|
checkOpen();
|
||||||
|
@ -1974,7 +1976,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Truncate a file to an indicated size
|
* Truncate a file to an indicated size
|
||||||
* See {@link ClientProtocol#truncate(String, long)}.
|
* See {@link ClientProtocol#truncate}.
|
||||||
*/
|
*/
|
||||||
public boolean truncate(String src, long newLength) throws IOException {
|
public boolean truncate(String src, long newLength) throws IOException {
|
||||||
checkOpen();
|
checkOpen();
|
||||||
|
@ -2996,7 +2998,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get {@link ContentSummary} rooted at the specified directory.
|
* Get {@link ContentSummary} rooted at the specified directory.
|
||||||
* @param path The string representation of the path
|
* @param src The string representation of the path
|
||||||
*
|
*
|
||||||
* @see ClientProtocol#getContentSummary(String)
|
* @see ClientProtocol#getContentSummary(String)
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -426,15 +426,16 @@ public class DFSOutputStream extends FSOutputSummer
|
||||||
/**
|
/**
|
||||||
* construction with tracing info
|
* construction with tracing info
|
||||||
*/
|
*/
|
||||||
private DataStreamer(HdfsFileStatus stat, Span span) {
|
private DataStreamer(HdfsFileStatus stat, ExtendedBlock block, Span span) {
|
||||||
isAppend = false;
|
isAppend = false;
|
||||||
isLazyPersistFile = isLazyPersist(stat);
|
isLazyPersistFile = isLazyPersist(stat);
|
||||||
|
this.block = block;
|
||||||
stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
|
stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
|
||||||
traceSpan = span;
|
traceSpan = span;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Construct a data streamer for append
|
* Construct a data streamer for appending to the last partial block
|
||||||
* @param lastBlock last block of the file to be appended
|
* @param lastBlock last block of the file to be appended
|
||||||
* @param stat status of the file to be appended
|
* @param stat status of the file to be appended
|
||||||
* @param bytesPerChecksum number of bytes per checksum
|
* @param bytesPerChecksum number of bytes per checksum
|
||||||
|
@ -1737,7 +1738,7 @@ public class DFSOutputStream extends FSOutputSummer
|
||||||
if (Trace.isTracing()) {
|
if (Trace.isTracing()) {
|
||||||
traceSpan = Trace.startSpan(this.getClass().getSimpleName()).detach();
|
traceSpan = Trace.startSpan(this.getClass().getSimpleName()).detach();
|
||||||
}
|
}
|
||||||
streamer = new DataStreamer(stat, traceSpan);
|
streamer = new DataStreamer(stat, null, traceSpan);
|
||||||
if (favoredNodes != null && favoredNodes.length != 0) {
|
if (favoredNodes != null && favoredNodes.length != 0) {
|
||||||
streamer.setFavoredNodes(favoredNodes);
|
streamer.setFavoredNodes(favoredNodes);
|
||||||
}
|
}
|
||||||
|
@ -1794,7 +1795,7 @@ public class DFSOutputStream extends FSOutputSummer
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Construct a new output stream for append. */
|
/** Construct a new output stream for append. */
|
||||||
private DFSOutputStream(DFSClient dfsClient, String src,
|
private DFSOutputStream(DFSClient dfsClient, String src, boolean toNewBlock,
|
||||||
Progressable progress, LocatedBlock lastBlock, HdfsFileStatus stat,
|
Progressable progress, LocatedBlock lastBlock, HdfsFileStatus stat,
|
||||||
DataChecksum checksum) throws IOException {
|
DataChecksum checksum) throws IOException {
|
||||||
this(dfsClient, src, progress, stat, checksum);
|
this(dfsClient, src, progress, stat, checksum);
|
||||||
|
@ -1806,21 +1807,24 @@ public class DFSOutputStream extends FSOutputSummer
|
||||||
}
|
}
|
||||||
|
|
||||||
// The last partial block of the file has to be filled.
|
// The last partial block of the file has to be filled.
|
||||||
if (lastBlock != null) {
|
if (!toNewBlock && lastBlock != null) {
|
||||||
// indicate that we are appending to an existing block
|
// indicate that we are appending to an existing block
|
||||||
bytesCurBlock = lastBlock.getBlockSize();
|
bytesCurBlock = lastBlock.getBlockSize();
|
||||||
streamer = new DataStreamer(lastBlock, stat, bytesPerChecksum, traceSpan);
|
streamer = new DataStreamer(lastBlock, stat, bytesPerChecksum, traceSpan);
|
||||||
} else {
|
} else {
|
||||||
computePacketChunkSize(dfsClient.getConf().writePacketSize, bytesPerChecksum);
|
computePacketChunkSize(dfsClient.getConf().writePacketSize,
|
||||||
streamer = new DataStreamer(stat, traceSpan);
|
bytesPerChecksum);
|
||||||
|
streamer = new DataStreamer(stat,
|
||||||
|
lastBlock != null ? lastBlock.getBlock() : null, traceSpan);
|
||||||
}
|
}
|
||||||
this.fileEncryptionInfo = stat.getFileEncryptionInfo();
|
this.fileEncryptionInfo = stat.getFileEncryptionInfo();
|
||||||
}
|
}
|
||||||
|
|
||||||
static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src,
|
static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src,
|
||||||
int buffersize, Progressable progress, LocatedBlock lastBlock,
|
boolean toNewBlock, int bufferSize, Progressable progress,
|
||||||
HdfsFileStatus stat, DataChecksum checksum) throws IOException {
|
LocatedBlock lastBlock, HdfsFileStatus stat, DataChecksum checksum)
|
||||||
final DFSOutputStream out = new DFSOutputStream(dfsClient, src,
|
throws IOException {
|
||||||
|
final DFSOutputStream out = new DFSOutputStream(dfsClient, src, toNewBlock,
|
||||||
progress, lastBlock, stat, checksum);
|
progress, lastBlock, stat, checksum);
|
||||||
out.start();
|
out.start();
|
||||||
return out;
|
return out;
|
||||||
|
@ -1959,7 +1963,6 @@ public class DFSOutputStream extends FSOutputSummer
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
@Deprecated
|
@Deprecated
|
||||||
public void sync() throws IOException {
|
public void sync() throws IOException {
|
||||||
hflush();
|
hflush();
|
||||||
|
@ -2022,35 +2025,37 @@ public class DFSOutputStream extends FSOutputSummer
|
||||||
long toWaitFor;
|
long toWaitFor;
|
||||||
long lastBlockLength = -1L;
|
long lastBlockLength = -1L;
|
||||||
boolean updateLength = syncFlags.contains(SyncFlag.UPDATE_LENGTH);
|
boolean updateLength = syncFlags.contains(SyncFlag.UPDATE_LENGTH);
|
||||||
|
boolean endBlock = syncFlags.contains(SyncFlag.END_BLOCK);
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
// flush checksum buffer, but keep checksum buffer intact
|
// flush checksum buffer, but keep checksum buffer intact if we do not
|
||||||
int numKept = flushBuffer(true, true);
|
// need to end the current block
|
||||||
|
int numKept = flushBuffer(!endBlock, true);
|
||||||
// bytesCurBlock potentially incremented if there was buffered data
|
// bytesCurBlock potentially incremented if there was buffered data
|
||||||
|
|
||||||
if (DFSClient.LOG.isDebugEnabled()) {
|
if (DFSClient.LOG.isDebugEnabled()) {
|
||||||
DFSClient.LOG.debug(
|
DFSClient.LOG.debug("DFSClient flush():"
|
||||||
"DFSClient flush(): " +
|
+ " bytesCurBlock=" + bytesCurBlock
|
||||||
" bytesCurBlock " + bytesCurBlock +
|
+ " lastFlushOffset=" + lastFlushOffset
|
||||||
" lastFlushOffset " + lastFlushOffset);
|
+ " createNewBlock=" + endBlock);
|
||||||
}
|
}
|
||||||
// Flush only if we haven't already flushed till this offset.
|
// Flush only if we haven't already flushed till this offset.
|
||||||
if (lastFlushOffset != bytesCurBlock) {
|
if (lastFlushOffset != bytesCurBlock) {
|
||||||
assert bytesCurBlock > lastFlushOffset;
|
assert bytesCurBlock > lastFlushOffset;
|
||||||
// record the valid offset of this flush
|
// record the valid offset of this flush
|
||||||
lastFlushOffset = bytesCurBlock;
|
lastFlushOffset = bytesCurBlock;
|
||||||
if (isSync && currentPacket == null) {
|
if (isSync && currentPacket == null && !endBlock) {
|
||||||
// Nothing to send right now,
|
// Nothing to send right now,
|
||||||
// but sync was requested.
|
// but sync was requested.
|
||||||
// Send an empty packet
|
// Send an empty packet if we do not end the block right now
|
||||||
currentPacket = createPacket(packetSize, chunksPerPacket,
|
currentPacket = createPacket(packetSize, chunksPerPacket,
|
||||||
bytesCurBlock, currentSeqno++);
|
bytesCurBlock, currentSeqno++);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
if (isSync && bytesCurBlock > 0) {
|
if (isSync && bytesCurBlock > 0 && !endBlock) {
|
||||||
// Nothing to send right now,
|
// Nothing to send right now,
|
||||||
// and the block was partially written,
|
// and the block was partially written,
|
||||||
// and sync was requested.
|
// and sync was requested.
|
||||||
// So send an empty sync packet.
|
// So send an empty sync packet if we do not end the block right now
|
||||||
currentPacket = createPacket(packetSize, chunksPerPacket,
|
currentPacket = createPacket(packetSize, chunksPerPacket,
|
||||||
bytesCurBlock, currentSeqno++);
|
bytesCurBlock, currentSeqno++);
|
||||||
} else if (currentPacket != null) {
|
} else if (currentPacket != null) {
|
||||||
|
@ -2063,10 +2068,21 @@ public class DFSOutputStream extends FSOutputSummer
|
||||||
currentPacket.syncBlock = isSync;
|
currentPacket.syncBlock = isSync;
|
||||||
waitAndQueueCurrentPacket();
|
waitAndQueueCurrentPacket();
|
||||||
}
|
}
|
||||||
|
if (endBlock && bytesCurBlock > 0) {
|
||||||
|
// Need to end the current block, thus send an empty packet to
|
||||||
|
// indicate this is the end of the block and reset bytesCurBlock
|
||||||
|
currentPacket = createPacket(0, 0, bytesCurBlock, currentSeqno++);
|
||||||
|
currentPacket.lastPacketInBlock = true;
|
||||||
|
currentPacket.syncBlock = shouldSyncBlock || isSync;
|
||||||
|
waitAndQueueCurrentPacket();
|
||||||
|
bytesCurBlock = 0;
|
||||||
|
lastFlushOffset = 0;
|
||||||
|
} else {
|
||||||
// Restore state of stream. Record the last flush offset
|
// Restore state of stream. Record the last flush offset
|
||||||
// of the last full chunk that was flushed.
|
// of the last full chunk that was flushed.
|
||||||
//
|
|
||||||
bytesCurBlock -= numKept;
|
bytesCurBlock -= numKept;
|
||||||
|
}
|
||||||
|
|
||||||
toWaitFor = lastQueuedSeqno;
|
toWaitFor = lastQueuedSeqno;
|
||||||
} // end synchronized
|
} // end synchronized
|
||||||
|
|
||||||
|
@ -2085,8 +2101,8 @@ public class DFSOutputStream extends FSOutputSummer
|
||||||
// namenode.
|
// namenode.
|
||||||
if (persistBlocks.getAndSet(false) || updateLength) {
|
if (persistBlocks.getAndSet(false) || updateLength) {
|
||||||
try {
|
try {
|
||||||
dfsClient.namenode.fsync(src, fileId,
|
dfsClient.namenode.fsync(src, fileId, dfsClient.clientName,
|
||||||
dfsClient.clientName, lastBlockLength);
|
lastBlockLength);
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src, ioe);
|
DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src, ioe);
|
||||||
// If we got an error here, it might be because some other thread called
|
// If we got an error here, it might be because some other thread called
|
||||||
|
|
|
@ -314,13 +314,19 @@ public class DistributedFileSystem extends FileSystem {
|
||||||
@Override
|
@Override
|
||||||
public FSDataOutputStream append(Path f, final int bufferSize,
|
public FSDataOutputStream append(Path f, final int bufferSize,
|
||||||
final Progressable progress) throws IOException {
|
final Progressable progress) throws IOException {
|
||||||
|
return append(f, EnumSet.of(CreateFlag.APPEND), bufferSize, progress);
|
||||||
|
}
|
||||||
|
|
||||||
|
public FSDataOutputStream append(Path f, final EnumSet<CreateFlag> flag,
|
||||||
|
final int bufferSize, final Progressable progress) throws IOException {
|
||||||
statistics.incrementWriteOps(1);
|
statistics.incrementWriteOps(1);
|
||||||
Path absF = fixRelativePart(f);
|
Path absF = fixRelativePart(f);
|
||||||
return new FileSystemLinkResolver<FSDataOutputStream>() {
|
return new FileSystemLinkResolver<FSDataOutputStream>() {
|
||||||
@Override
|
@Override
|
||||||
public FSDataOutputStream doCall(final Path p)
|
public FSDataOutputStream doCall(final Path p)
|
||||||
throws IOException, UnresolvedLinkException {
|
throws IOException {
|
||||||
return dfs.append(getPathName(p), bufferSize, progress, statistics);
|
return dfs.append(getPathName(p), bufferSize, flag, progress,
|
||||||
|
statistics);
|
||||||
}
|
}
|
||||||
@Override
|
@Override
|
||||||
public FSDataOutputStream next(final FileSystem fs, final Path p)
|
public FSDataOutputStream next(final FileSystem fs, final Path p)
|
||||||
|
|
|
@ -101,6 +101,12 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
|
||||||
* When doing sync to DataNodes, also update the metadata (block length) in
|
* When doing sync to DataNodes, also update the metadata (block length) in
|
||||||
* the NameNode.
|
* the NameNode.
|
||||||
*/
|
*/
|
||||||
UPDATE_LENGTH;
|
UPDATE_LENGTH,
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sync the data to DataNode, close the current block, and allocate a new
|
||||||
|
* block
|
||||||
|
*/
|
||||||
|
END_BLOCK;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -463,15 +463,22 @@ public abstract class Event {
|
||||||
*/
|
*/
|
||||||
public static class AppendEvent extends Event {
|
public static class AppendEvent extends Event {
|
||||||
private String path;
|
private String path;
|
||||||
|
private boolean newBlock;
|
||||||
|
|
||||||
public static class Builder {
|
public static class Builder {
|
||||||
private String path;
|
private String path;
|
||||||
|
private boolean newBlock;
|
||||||
|
|
||||||
public Builder path(String path) {
|
public Builder path(String path) {
|
||||||
this.path = path;
|
this.path = path;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder newBlock(boolean newBlock) {
|
||||||
|
this.newBlock = newBlock;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
public AppendEvent build() {
|
public AppendEvent build() {
|
||||||
return new AppendEvent(this);
|
return new AppendEvent(this);
|
||||||
}
|
}
|
||||||
|
@ -480,11 +487,16 @@ public abstract class Event {
|
||||||
private AppendEvent(Builder b) {
|
private AppendEvent(Builder b) {
|
||||||
super(EventType.APPEND);
|
super(EventType.APPEND);
|
||||||
this.path = b.path;
|
this.path = b.path;
|
||||||
|
this.newBlock = b.newBlock;
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getPath() {
|
public String getPath() {
|
||||||
return path;
|
return path;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean toNewBlock() {
|
||||||
|
return newBlock;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -204,6 +204,7 @@ public interface ClientProtocol {
|
||||||
* Append to the end of the file.
|
* Append to the end of the file.
|
||||||
* @param src path of the file being created.
|
* @param src path of the file being created.
|
||||||
* @param clientName name of the current client.
|
* @param clientName name of the current client.
|
||||||
|
* @param flag indicates whether the data is appended to a new block.
|
||||||
* @return wrapper with information about the last partial block and file
|
* @return wrapper with information about the last partial block and file
|
||||||
* status if any
|
* status if any
|
||||||
* @throws AccessControlException if permission to append file is
|
* @throws AccessControlException if permission to append file is
|
||||||
|
@ -226,10 +227,10 @@ public interface ClientProtocol {
|
||||||
* @throws UnsupportedOperationException if append is not supported
|
* @throws UnsupportedOperationException if append is not supported
|
||||||
*/
|
*/
|
||||||
@AtMostOnce
|
@AtMostOnce
|
||||||
public LastBlockWithStatus append(String src, String clientName)
|
public LastBlockWithStatus append(String src, String clientName,
|
||||||
throws AccessControlException, DSQuotaExceededException,
|
EnumSetWritable<CreateFlag> flag) throws AccessControlException,
|
||||||
FileNotFoundException, SafeModeException, UnresolvedLinkException,
|
DSQuotaExceededException, FileNotFoundException, SafeModeException,
|
||||||
SnapshotAccessControlException, IOException;
|
UnresolvedLinkException, SnapshotAccessControlException, IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set replication for an existing file.
|
* Set replication for an existing file.
|
||||||
|
|
|
@ -18,12 +18,14 @@
|
||||||
package org.apache.hadoop.hdfs.protocolPB;
|
package org.apache.hadoop.hdfs.protocolPB;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.EnumSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
|
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
|
||||||
import org.apache.hadoop.fs.ContentSummary;
|
import org.apache.hadoop.fs.ContentSummary;
|
||||||
|
import org.apache.hadoop.fs.CreateFlag;
|
||||||
import org.apache.hadoop.fs.FsServerDefaults;
|
import org.apache.hadoop.fs.FsServerDefaults;
|
||||||
import org.apache.hadoop.fs.Options.Rename;
|
import org.apache.hadoop.fs.Options.Rename;
|
||||||
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
|
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
|
||||||
|
@ -65,6 +67,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowS
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotResponseProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotResponseProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessResponseProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteResponseProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteResponseProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
|
||||||
|
@ -187,8 +191,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto;
|
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessResponseProto;
|
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneResponseProto;
|
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneResponseProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathResponseProto;
|
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathResponseProto;
|
||||||
|
@ -209,6 +211,7 @@ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrResponseProto;
|
||||||
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
|
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
|
||||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INodeId;
|
import org.apache.hadoop.hdfs.server.namenode.INodeId;
|
||||||
|
import org.apache.hadoop.io.EnumSetWritable;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
|
import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
|
||||||
import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenResponseProto;
|
import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenResponseProto;
|
||||||
|
@ -412,8 +415,11 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
||||||
public AppendResponseProto append(RpcController controller,
|
public AppendResponseProto append(RpcController controller,
|
||||||
AppendRequestProto req) throws ServiceException {
|
AppendRequestProto req) throws ServiceException {
|
||||||
try {
|
try {
|
||||||
|
EnumSetWritable<CreateFlag> flags = req.hasFlag() ?
|
||||||
|
PBHelper.convertCreateFlag(req.getFlag()) :
|
||||||
|
new EnumSetWritable<>(EnumSet.of(CreateFlag.APPEND));
|
||||||
LastBlockWithStatus result = server.append(req.getSrc(),
|
LastBlockWithStatus result = server.append(req.getSrc(),
|
||||||
req.getClientName());
|
req.getClientName(), flags);
|
||||||
AppendResponseProto.Builder builder = AppendResponseProto.newBuilder();
|
AppendResponseProto.Builder builder = AppendResponseProto.newBuilder();
|
||||||
if (result.getLastBlock() != null) {
|
if (result.getLastBlock() != null) {
|
||||||
builder.setBlock(PBHelper.convert(result.getLastBlock()));
|
builder.setBlock(PBHelper.convert(result.getLastBlock()));
|
||||||
|
|
|
@ -28,7 +28,6 @@ import com.google.common.collect.Lists;
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.crypto.CipherSuite;
|
|
||||||
import org.apache.hadoop.crypto.CryptoProtocolVersion;
|
import org.apache.hadoop.crypto.CryptoProtocolVersion;
|
||||||
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
|
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
|
||||||
import org.apache.hadoop.fs.CacheFlag;
|
import org.apache.hadoop.fs.CacheFlag;
|
||||||
|
@ -85,6 +84,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowS
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
|
||||||
|
@ -158,13 +158,11 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTim
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.TruncateRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.TruncateRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto;
|
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
|
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
|
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto;
|
||||||
|
@ -319,13 +317,12 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public LastBlockWithStatus append(String src, String clientName)
|
public LastBlockWithStatus append(String src, String clientName,
|
||||||
throws AccessControlException, DSQuotaExceededException,
|
EnumSetWritable<CreateFlag> flag) throws AccessControlException,
|
||||||
FileNotFoundException, SafeModeException, UnresolvedLinkException,
|
DSQuotaExceededException, FileNotFoundException, SafeModeException,
|
||||||
IOException {
|
UnresolvedLinkException, IOException {
|
||||||
AppendRequestProto req = AppendRequestProto.newBuilder()
|
AppendRequestProto req = AppendRequestProto.newBuilder().setSrc(src)
|
||||||
.setSrc(src)
|
.setClientName(clientName).setFlag(PBHelper.convertCreateFlag(flag))
|
||||||
.setClientName(clientName)
|
|
||||||
.build();
|
.build();
|
||||||
try {
|
try {
|
||||||
AppendResponseProto res = rpcProxy.append(null, req);
|
AppendResponseProto res = rpcProxy.append(null, req);
|
||||||
|
|
|
@ -1374,6 +1374,9 @@ public class PBHelper {
|
||||||
if (flag.contains(CreateFlag.LAZY_PERSIST)) {
|
if (flag.contains(CreateFlag.LAZY_PERSIST)) {
|
||||||
value |= CreateFlagProto.LAZY_PERSIST.getNumber();
|
value |= CreateFlagProto.LAZY_PERSIST.getNumber();
|
||||||
}
|
}
|
||||||
|
if (flag.contains(CreateFlag.NEW_BLOCK)) {
|
||||||
|
value |= CreateFlagProto.NEW_BLOCK.getNumber();
|
||||||
|
}
|
||||||
return value;
|
return value;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1394,7 +1397,11 @@ public class PBHelper {
|
||||||
== CreateFlagProto.LAZY_PERSIST_VALUE) {
|
== CreateFlagProto.LAZY_PERSIST_VALUE) {
|
||||||
result.add(CreateFlag.LAZY_PERSIST);
|
result.add(CreateFlag.LAZY_PERSIST);
|
||||||
}
|
}
|
||||||
return new EnumSetWritable<CreateFlag>(result);
|
if ((flag & CreateFlagProto.NEW_BLOCK_VALUE)
|
||||||
|
== CreateFlagProto.NEW_BLOCK_VALUE) {
|
||||||
|
result.add(CreateFlag.NEW_BLOCK);
|
||||||
|
}
|
||||||
|
return new EnumSetWritable<CreateFlag>(result, CreateFlag.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static int convertCacheFlags(EnumSet<CacheFlag> flags) {
|
public static int convertCacheFlags(EnumSet<CacheFlag> flags) {
|
||||||
|
@ -2606,10 +2613,10 @@ public class PBHelper {
|
||||||
.build());
|
.build());
|
||||||
break;
|
break;
|
||||||
case EVENT_APPEND:
|
case EVENT_APPEND:
|
||||||
InotifyProtos.AppendEventProto reopen =
|
InotifyProtos.AppendEventProto append =
|
||||||
InotifyProtos.AppendEventProto.parseFrom(p.getContents());
|
InotifyProtos.AppendEventProto.parseFrom(p.getContents());
|
||||||
events.add(new Event.AppendEvent.Builder()
|
events.add(new Event.AppendEvent.Builder().path(append.getPath())
|
||||||
.path(reopen.getPath())
|
.newBlock(append.hasNewBlock() && append.getNewBlock())
|
||||||
.build());
|
.build());
|
||||||
break;
|
break;
|
||||||
case EVENT_UNLINK:
|
case EVENT_UNLINK:
|
||||||
|
@ -2711,10 +2718,10 @@ public class PBHelper {
|
||||||
Event.AppendEvent re2 = (Event.AppendEvent) e;
|
Event.AppendEvent re2 = (Event.AppendEvent) e;
|
||||||
events.add(InotifyProtos.EventProto.newBuilder()
|
events.add(InotifyProtos.EventProto.newBuilder()
|
||||||
.setType(InotifyProtos.EventType.EVENT_APPEND)
|
.setType(InotifyProtos.EventType.EVENT_APPEND)
|
||||||
.setContents(
|
.setContents(InotifyProtos.AppendEventProto.newBuilder()
|
||||||
InotifyProtos.AppendEventProto.newBuilder()
|
.setPath(re2.getPath())
|
||||||
.setPath(re2.getPath()).build().toByteString()
|
.setNewBlock(re2.toNewBlock()).build().toByteString())
|
||||||
).build());
|
.build());
|
||||||
break;
|
break;
|
||||||
case UNLINK:
|
case UNLINK:
|
||||||
Event.UnlinkEvent ue = (Event.UnlinkEvent) e;
|
Event.UnlinkEvent ue = (Event.UnlinkEvent) e;
|
||||||
|
|
|
@ -176,7 +176,8 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> {
|
||||||
final int bufferSize = params.bufferSize();
|
final int bufferSize = params.bufferSize();
|
||||||
|
|
||||||
DFSClient dfsClient = newDfsClient(nnId, conf);
|
DFSClient dfsClient = newDfsClient(nnId, conf);
|
||||||
OutputStream out = dfsClient.append(path, bufferSize, null, null);
|
OutputStream out = dfsClient.append(path, bufferSize,
|
||||||
|
EnumSet.of(CreateFlag.APPEND), null, null);
|
||||||
DefaultHttpResponse resp = new DefaultHttpResponse(HTTP_1_1, OK);
|
DefaultHttpResponse resp = new DefaultHttpResponse(HTTP_1_1, OK);
|
||||||
resp.headers().set(CONTENT_LENGTH, 0);
|
resp.headers().set(CONTENT_LENGTH, 0);
|
||||||
ctx.pipeline().replace(this, HdfsWriter.class.getSimpleName(),
|
ctx.pipeline().replace(this, HdfsWriter.class.getSimpleName(),
|
||||||
|
|
|
@ -19,11 +19,10 @@ package org.apache.hadoop.hdfs.server.namenode;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.hadoop.HadoopIllegalArgumentException;
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.fs.permission.FsAction;
|
import org.apache.hadoop.fs.permission.FsAction;
|
||||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
|
||||||
import org.apache.hadoop.hdfs.protocol.SnapshotException;
|
import org.apache.hadoop.hdfs.protocol.SnapshotException;
|
||||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
@ -33,127 +32,29 @@ import java.util.Set;
|
||||||
import static org.apache.hadoop.util.Time.now;
|
import static org.apache.hadoop.util.Time.now;
|
||||||
|
|
||||||
class FSDirConcatOp {
|
class FSDirConcatOp {
|
||||||
static HdfsFileStatus concat(
|
|
||||||
FSDirectory fsd, String target, String[] srcs,
|
static HdfsFileStatus concat(FSDirectory fsd, String target, String[] srcs,
|
||||||
boolean logRetryCache) throws IOException {
|
boolean logRetryCache) throws IOException {
|
||||||
Preconditions.checkArgument(!target.isEmpty(), "Target file name is empty");
|
Preconditions.checkArgument(!target.isEmpty(), "Target file name is empty");
|
||||||
Preconditions.checkArgument(srcs != null && srcs.length > 0,
|
Preconditions.checkArgument(srcs != null && srcs.length > 0,
|
||||||
"No sources given");
|
"No sources given");
|
||||||
assert srcs != null;
|
assert srcs != null;
|
||||||
|
if (FSDirectory.LOG.isDebugEnabled()) {
|
||||||
FSDirectory.LOG.debug("concat {} to {}", Arrays.toString(srcs), target);
|
FSDirectory.LOG.debug("concat {} to {}", Arrays.toString(srcs), target);
|
||||||
// We require all files be in the same directory
|
|
||||||
String trgParent =
|
|
||||||
target.substring(0, target.lastIndexOf(Path.SEPARATOR_CHAR));
|
|
||||||
for (String s : srcs) {
|
|
||||||
String srcParent = s.substring(0, s.lastIndexOf(Path.SEPARATOR_CHAR));
|
|
||||||
if (!srcParent.equals(trgParent)) {
|
|
||||||
throw new IllegalArgumentException(
|
|
||||||
"Sources and target are not in the same directory");
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
final INodesInPath trgIip = fsd.getINodesInPath4Write(target);
|
final INodesInPath targetIIP = fsd.getINodesInPath4Write(target);
|
||||||
// write permission for the target
|
// write permission for the target
|
||||||
|
FSPermissionChecker pc = null;
|
||||||
if (fsd.isPermissionEnabled()) {
|
if (fsd.isPermissionEnabled()) {
|
||||||
FSPermissionChecker pc = fsd.getPermissionChecker();
|
pc = fsd.getPermissionChecker();
|
||||||
fsd.checkPathAccess(pc, trgIip, FsAction.WRITE);
|
fsd.checkPathAccess(pc, targetIIP, FsAction.WRITE);
|
||||||
|
|
||||||
// and srcs
|
|
||||||
for(String aSrc: srcs) {
|
|
||||||
final INodesInPath srcIip = fsd.getINodesInPath4Write(aSrc);
|
|
||||||
fsd.checkPathAccess(pc, srcIip, FsAction.READ); // read the file
|
|
||||||
fsd.checkParentAccess(pc, srcIip, FsAction.WRITE); // for delete
|
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
// to make sure no two files are the same
|
|
||||||
Set<INode> si = new HashSet<INode>();
|
|
||||||
|
|
||||||
// we put the following prerequisite for the operation
|
|
||||||
// replication and blocks sizes should be the same for ALL the blocks
|
|
||||||
|
|
||||||
// check the target
|
// check the target
|
||||||
if (fsd.getEZForPath(trgIip) != null) {
|
verifyTargetFile(fsd, target, targetIIP);
|
||||||
throw new HadoopIllegalArgumentException(
|
// check the srcs
|
||||||
"concat can not be called for files in an encryption zone.");
|
INodeFile[] srcFiles = verifySrcFiles(fsd, srcs, targetIIP, pc);
|
||||||
}
|
|
||||||
final INodeFile trgInode = INodeFile.valueOf(trgIip.getLastINode(), target);
|
|
||||||
if(trgInode.isUnderConstruction()) {
|
|
||||||
throw new HadoopIllegalArgumentException("concat: target file "
|
|
||||||
+ target + " is under construction");
|
|
||||||
}
|
|
||||||
// per design target shouldn't be empty and all the blocks same size
|
|
||||||
if(trgInode.numBlocks() == 0) {
|
|
||||||
throw new HadoopIllegalArgumentException("concat: target file "
|
|
||||||
+ target + " is empty");
|
|
||||||
}
|
|
||||||
if (trgInode.isWithSnapshot()) {
|
|
||||||
throw new HadoopIllegalArgumentException("concat: target file "
|
|
||||||
+ target + " is in a snapshot");
|
|
||||||
}
|
|
||||||
|
|
||||||
long blockSize = trgInode.getPreferredBlockSize();
|
|
||||||
|
|
||||||
// check the end block to be full
|
|
||||||
final BlockInfoContiguous last = trgInode.getLastBlock();
|
|
||||||
if(blockSize != last.getNumBytes()) {
|
|
||||||
throw new HadoopIllegalArgumentException("The last block in " + target
|
|
||||||
+ " is not full; last block size = " + last.getNumBytes()
|
|
||||||
+ " but file block size = " + blockSize);
|
|
||||||
}
|
|
||||||
|
|
||||||
si.add(trgInode);
|
|
||||||
final short repl = trgInode.getFileReplication();
|
|
||||||
|
|
||||||
// now check the srcs
|
|
||||||
boolean endSrc = false; // final src file doesn't have to have full end block
|
|
||||||
for(int i=0; i< srcs.length; i++) {
|
|
||||||
String src = srcs[i];
|
|
||||||
if(i== srcs.length-1)
|
|
||||||
endSrc=true;
|
|
||||||
|
|
||||||
final INodeFile srcInode = INodeFile.valueOf(fsd.getINode4Write(src), src);
|
|
||||||
if(src.isEmpty()
|
|
||||||
|| srcInode.isUnderConstruction()
|
|
||||||
|| srcInode.numBlocks() == 0) {
|
|
||||||
throw new HadoopIllegalArgumentException("concat: source file " + src
|
|
||||||
+ " is invalid or empty or underConstruction");
|
|
||||||
}
|
|
||||||
|
|
||||||
// check replication and blocks size
|
|
||||||
if(repl != srcInode.getBlockReplication()) {
|
|
||||||
throw new HadoopIllegalArgumentException("concat: the source file "
|
|
||||||
+ src + " and the target file " + target
|
|
||||||
+ " should have the same replication: source replication is "
|
|
||||||
+ srcInode.getBlockReplication()
|
|
||||||
+ " but target replication is " + repl);
|
|
||||||
}
|
|
||||||
|
|
||||||
//boolean endBlock=false;
|
|
||||||
// verify that all the blocks are of the same length as target
|
|
||||||
// should be enough to check the end blocks
|
|
||||||
final BlockInfoContiguous[] srcBlocks = srcInode.getBlocks();
|
|
||||||
int idx = srcBlocks.length-1;
|
|
||||||
if(endSrc)
|
|
||||||
idx = srcBlocks.length-2; // end block of endSrc is OK not to be full
|
|
||||||
if(idx >= 0 && srcBlocks[idx].getNumBytes() != blockSize) {
|
|
||||||
throw new HadoopIllegalArgumentException("concat: the source file "
|
|
||||||
+ src + " and the target file " + target
|
|
||||||
+ " should have the same blocks sizes: target block size is "
|
|
||||||
+ blockSize + " but the size of source block " + idx + " is "
|
|
||||||
+ srcBlocks[idx].getNumBytes());
|
|
||||||
}
|
|
||||||
|
|
||||||
si.add(srcInode);
|
|
||||||
}
|
|
||||||
|
|
||||||
// make sure no two files are the same
|
|
||||||
if(si.size() < srcs.length+1) { // trg + srcs
|
|
||||||
// it means at least two files are the same
|
|
||||||
throw new HadoopIllegalArgumentException(
|
|
||||||
"concat: at least two of the source files are the same");
|
|
||||||
}
|
|
||||||
|
|
||||||
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
||||||
NameNode.stateChangeLog.debug("DIR* NameSystem.concat: " +
|
NameNode.stateChangeLog.debug("DIR* NameSystem.concat: " +
|
||||||
|
@ -163,71 +64,139 @@ class FSDirConcatOp {
|
||||||
long timestamp = now();
|
long timestamp = now();
|
||||||
fsd.writeLock();
|
fsd.writeLock();
|
||||||
try {
|
try {
|
||||||
unprotectedConcat(fsd, target, srcs, timestamp);
|
unprotectedConcat(fsd, targetIIP, srcFiles, timestamp);
|
||||||
} finally {
|
} finally {
|
||||||
fsd.writeUnlock();
|
fsd.writeUnlock();
|
||||||
}
|
}
|
||||||
fsd.getEditLog().logConcat(target, srcs, timestamp, logRetryCache);
|
fsd.getEditLog().logConcat(target, srcs, timestamp, logRetryCache);
|
||||||
return fsd.getAuditFileInfo(trgIip);
|
return fsd.getAuditFileInfo(targetIIP);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void verifyTargetFile(FSDirectory fsd, final String target,
|
||||||
|
final INodesInPath targetIIP) throws IOException {
|
||||||
|
// check the target
|
||||||
|
if (fsd.getEZForPath(targetIIP) != null) {
|
||||||
|
throw new HadoopIllegalArgumentException(
|
||||||
|
"concat can not be called for files in an encryption zone.");
|
||||||
|
}
|
||||||
|
final INodeFile targetINode = INodeFile.valueOf(targetIIP.getLastINode(),
|
||||||
|
target);
|
||||||
|
if(targetINode.isUnderConstruction()) {
|
||||||
|
throw new HadoopIllegalArgumentException("concat: target file "
|
||||||
|
+ target + " is under construction");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static INodeFile[] verifySrcFiles(FSDirectory fsd, String[] srcs,
|
||||||
|
INodesInPath targetIIP, FSPermissionChecker pc) throws IOException {
|
||||||
|
// to make sure no two files are the same
|
||||||
|
Set<INodeFile> si = new HashSet<>();
|
||||||
|
final INodeFile targetINode = targetIIP.getLastINode().asFile();
|
||||||
|
final INodeDirectory targetParent = targetINode.getParent();
|
||||||
|
// now check the srcs
|
||||||
|
for(String src : srcs) {
|
||||||
|
final INodesInPath iip = fsd.getINodesInPath4Write(src);
|
||||||
|
// permission check for srcs
|
||||||
|
if (pc != null) {
|
||||||
|
fsd.checkPathAccess(pc, iip, FsAction.READ); // read the file
|
||||||
|
fsd.checkParentAccess(pc, iip, FsAction.WRITE); // for delete
|
||||||
|
}
|
||||||
|
final INode srcINode = iip.getLastINode();
|
||||||
|
final INodeFile srcINodeFile = INodeFile.valueOf(srcINode, src);
|
||||||
|
// make sure the src file and the target file are in the same dir
|
||||||
|
if (srcINodeFile.getParent() != targetParent) {
|
||||||
|
throw new HadoopIllegalArgumentException("Source file " + src
|
||||||
|
+ " is not in the same directory with the target "
|
||||||
|
+ targetIIP.getPath());
|
||||||
|
}
|
||||||
|
// make sure all the source files are not in snapshot
|
||||||
|
if (srcINode.isInLatestSnapshot(iip.getLatestSnapshotId())) {
|
||||||
|
throw new SnapshotException("Concat: the source file " + src
|
||||||
|
+ " is in snapshot");
|
||||||
|
}
|
||||||
|
// check if the file has other references.
|
||||||
|
if (srcINode.isReference() && ((INodeReference.WithCount)
|
||||||
|
srcINode.asReference().getReferredINode()).getReferenceCount() > 1) {
|
||||||
|
throw new SnapshotException("Concat: the source file " + src
|
||||||
|
+ " is referred by some other reference in some snapshot.");
|
||||||
|
}
|
||||||
|
if (srcINode == targetINode) {
|
||||||
|
throw new HadoopIllegalArgumentException("concat: the src file " + src
|
||||||
|
+ " is the same with the target file " + targetIIP.getPath());
|
||||||
|
}
|
||||||
|
if(srcINodeFile.isUnderConstruction() || srcINodeFile.numBlocks() == 0) {
|
||||||
|
throw new HadoopIllegalArgumentException("concat: source file " + src
|
||||||
|
+ " is invalid or empty or underConstruction");
|
||||||
|
}
|
||||||
|
si.add(srcINodeFile);
|
||||||
|
}
|
||||||
|
|
||||||
|
// make sure no two files are the same
|
||||||
|
if(si.size() < srcs.length) {
|
||||||
|
// it means at least two files are the same
|
||||||
|
throw new HadoopIllegalArgumentException(
|
||||||
|
"concat: at least two of the source files are the same");
|
||||||
|
}
|
||||||
|
return si.toArray(new INodeFile[si.size()]);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static long computeQuotaDelta(INodeFile target, INodeFile[] srcList) {
|
||||||
|
long delta = 0;
|
||||||
|
short targetRepl = target.getBlockReplication();
|
||||||
|
for (INodeFile src : srcList) {
|
||||||
|
if (targetRepl != src.getBlockReplication()) {
|
||||||
|
delta += src.computeFileSize() *
|
||||||
|
(targetRepl - src.getBlockReplication());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return delta;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void verifyQuota(FSDirectory fsd, INodesInPath targetIIP,
|
||||||
|
long delta) throws QuotaExceededException {
|
||||||
|
if (!fsd.getFSNamesystem().isImageLoaded() || fsd.shouldSkipQuotaChecks()) {
|
||||||
|
// Do not check quota if editlog is still being processed
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
FSDirectory.verifyQuota(targetIIP, targetIIP.length() - 1, 0, delta, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Concat all the blocks from srcs to trg and delete the srcs files
|
* Concat all the blocks from srcs to trg and delete the srcs files
|
||||||
* @param fsd FSDirectory
|
* @param fsd FSDirectory
|
||||||
* @param target target file to move the blocks to
|
|
||||||
* @param srcs list of file to move the blocks from
|
|
||||||
*/
|
*/
|
||||||
static void unprotectedConcat(
|
static void unprotectedConcat(FSDirectory fsd, INodesInPath targetIIP,
|
||||||
FSDirectory fsd, String target, String[] srcs, long timestamp)
|
INodeFile[] srcList, long timestamp) throws IOException {
|
||||||
throws IOException {
|
|
||||||
assert fsd.hasWriteLock();
|
assert fsd.hasWriteLock();
|
||||||
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
||||||
NameNode.stateChangeLog.debug("DIR* FSNamesystem.concat to "+target);
|
NameNode.stateChangeLog.debug("DIR* FSNamesystem.concat to "
|
||||||
}
|
+ targetIIP.getPath());
|
||||||
// do the move
|
|
||||||
|
|
||||||
final INodesInPath trgIIP = fsd.getINodesInPath4Write(target, true);
|
|
||||||
final INodeFile trgInode = trgIIP.getLastINode().asFile();
|
|
||||||
INodeDirectory trgParent = trgIIP.getINode(-2).asDirectory();
|
|
||||||
final int trgLatestSnapshot = trgIIP.getLatestSnapshotId();
|
|
||||||
|
|
||||||
final INodeFile [] allSrcInodes = new INodeFile[srcs.length];
|
|
||||||
for(int i = 0; i < srcs.length; i++) {
|
|
||||||
final INodesInPath iip = fsd.getINodesInPath4Write(srcs[i]);
|
|
||||||
final int latest = iip.getLatestSnapshotId();
|
|
||||||
final INode inode = iip.getLastINode();
|
|
||||||
|
|
||||||
// check if the file in the latest snapshot
|
|
||||||
if (inode.isInLatestSnapshot(latest)) {
|
|
||||||
throw new SnapshotException("Concat: the source file " + srcs[i]
|
|
||||||
+ " is in snapshot " + latest);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// check if the file has other references.
|
final INodeFile trgInode = targetIIP.getLastINode().asFile();
|
||||||
if (inode.isReference() && ((INodeReference.WithCount)
|
long delta = computeQuotaDelta(trgInode, srcList);
|
||||||
inode.asReference().getReferredINode()).getReferenceCount() > 1) {
|
verifyQuota(fsd, targetIIP, delta);
|
||||||
throw new SnapshotException("Concat: the source file " + srcs[i]
|
|
||||||
+ " is referred by some other reference in some snapshot.");
|
|
||||||
}
|
|
||||||
|
|
||||||
allSrcInodes[i] = inode.asFile();
|
// the target file can be included in a snapshot
|
||||||
}
|
trgInode.recordModification(targetIIP.getLatestSnapshotId());
|
||||||
trgInode.concatBlocks(allSrcInodes);
|
INodeDirectory trgParent = targetIIP.getINode(-2).asDirectory();
|
||||||
|
trgInode.concatBlocks(srcList);
|
||||||
|
|
||||||
// since we are in the same dir - we can use same parent to remove files
|
// since we are in the same dir - we can use same parent to remove files
|
||||||
int count = 0;
|
int count = 0;
|
||||||
for(INodeFile nodeToRemove: allSrcInodes) {
|
for (INodeFile nodeToRemove : srcList) {
|
||||||
if(nodeToRemove == null) continue;
|
if(nodeToRemove != null) {
|
||||||
|
|
||||||
nodeToRemove.setBlocks(null);
|
nodeToRemove.setBlocks(null);
|
||||||
trgParent.removeChild(nodeToRemove, trgLatestSnapshot);
|
nodeToRemove.getParent().removeChild(nodeToRemove);
|
||||||
fsd.getINodeMap().remove(nodeToRemove);
|
fsd.getINodeMap().remove(nodeToRemove);
|
||||||
count++;
|
count++;
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
trgInode.setModificationTime(timestamp, trgLatestSnapshot);
|
trgInode.setModificationTime(timestamp, targetIIP.getLatestSnapshotId());
|
||||||
trgParent.updateModificationTime(timestamp, trgLatestSnapshot);
|
trgParent.updateModificationTime(timestamp, targetIIP.getLatestSnapshotId());
|
||||||
// update quota on the parent directory ('count' files removed, 0 space)
|
// update quota on the parent directory ('count' files removed, 0 space)
|
||||||
FSDirectory.unprotectedUpdateCount(trgIIP, trgIIP.length() - 1, -count, 0);
|
FSDirectory.unprotectedUpdateCount(targetIIP, targetIIP.length() - 1,
|
||||||
|
-count, delta);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -452,7 +452,7 @@ public class FSDirectory implements Closeable {
|
||||||
Preconditions.checkState(fileINode.isUnderConstruction());
|
Preconditions.checkState(fileINode.isUnderConstruction());
|
||||||
|
|
||||||
// check quota limits and updated space consumed
|
// check quota limits and updated space consumed
|
||||||
updateCount(inodesInPath, 0, fileINode.getBlockDiskspace(), true);
|
updateCount(inodesInPath, 0, fileINode.getPreferredBlockDiskspace(), true);
|
||||||
|
|
||||||
// associate new last block for the file
|
// associate new last block for the file
|
||||||
BlockInfoContiguousUnderConstruction blockInfo =
|
BlockInfoContiguousUnderConstruction blockInfo =
|
||||||
|
@ -508,7 +508,7 @@ public class FSDirectory implements Closeable {
|
||||||
}
|
}
|
||||||
|
|
||||||
// update space consumed
|
// update space consumed
|
||||||
updateCount(iip, 0, -fileNode.getBlockDiskspace(), true);
|
updateCount(iip, 0, -fileNode.getPreferredBlockDiskspace(), true);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -34,10 +34,10 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.Options;
|
import org.apache.hadoop.fs.Options;
|
||||||
|
import org.apache.hadoop.fs.XAttr;
|
||||||
import org.apache.hadoop.fs.permission.AclEntry;
|
import org.apache.hadoop.fs.permission.AclEntry;
|
||||||
import org.apache.hadoop.fs.permission.FsPermission;
|
import org.apache.hadoop.fs.permission.FsPermission;
|
||||||
import org.apache.hadoop.fs.permission.PermissionStatus;
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
||||||
import org.apache.hadoop.fs.XAttr;
|
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.protocol.Block;
|
import org.apache.hadoop.hdfs.protocol.Block;
|
||||||
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
|
||||||
|
@ -52,9 +52,11 @@ import org.apache.hadoop.hdfs.server.common.StorageInfo;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AppendOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CloseOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CloseOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ConcatDeleteOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ConcatDeleteOp;
|
||||||
|
@ -703,6 +705,18 @@ public class FSEditLog implements LogsPurgeable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void logAppendFile(String path, INodeFile file, boolean newBlock,
|
||||||
|
boolean toLogRpcIds) {
|
||||||
|
FileUnderConstructionFeature uc = file.getFileUnderConstructionFeature();
|
||||||
|
assert uc != null;
|
||||||
|
AppendOp op = AppendOp.getInstance(cache.get()).setPath(path)
|
||||||
|
.setClientName(uc.getClientName())
|
||||||
|
.setClientMachine(uc.getClientMachine())
|
||||||
|
.setNewBlock(newBlock);
|
||||||
|
logRpcIds(op, toLogRpcIds);
|
||||||
|
logEdit(op);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Add open lease record to edit log.
|
* Add open lease record to edit log.
|
||||||
* Records the block locations of the last block.
|
* Records the block locations of the last block.
|
||||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.XAttrSetFlag;
|
import org.apache.hadoop.fs.XAttrSetFlag;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
|
||||||
import org.apache.hadoop.hdfs.protocol.Block;
|
import org.apache.hadoop.hdfs.protocol.Block;
|
||||||
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
|
||||||
|
@ -54,6 +55,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AppendOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.BlockListUpdatingOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.BlockListUpdatingOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ClearNSQuotaOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ClearNSQuotaOp;
|
||||||
|
@ -69,6 +71,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCachePoolOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCacheDirectiveInfoOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCacheDirectiveInfoOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCachePoolOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCachePoolOp;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveXAttrOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp;
|
||||||
|
@ -84,7 +87,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetStoragePolicyOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetStoragePolicyOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetXAttrOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetXAttrOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveXAttrOp;
|
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
|
||||||
|
@ -332,16 +334,16 @@ public class FSEditLogLoader {
|
||||||
AddCloseOp addCloseOp = (AddCloseOp)op;
|
AddCloseOp addCloseOp = (AddCloseOp)op;
|
||||||
final String path =
|
final String path =
|
||||||
renameReservedPathsOnUpgrade(addCloseOp.path, logVersion);
|
renameReservedPathsOnUpgrade(addCloseOp.path, logVersion);
|
||||||
if (LOG.isDebugEnabled()) {
|
if (FSNamesystem.LOG.isDebugEnabled()) {
|
||||||
LOG.debug(op.opCode + ": " + path +
|
FSNamesystem.LOG.debug(op.opCode + ": " + path +
|
||||||
" numblocks : " + addCloseOp.blocks.length +
|
" numblocks : " + addCloseOp.blocks.length +
|
||||||
" clientHolder " + addCloseOp.clientName +
|
" clientHolder " + addCloseOp.clientName +
|
||||||
" clientMachine " + addCloseOp.clientMachine);
|
" clientMachine " + addCloseOp.clientMachine);
|
||||||
}
|
}
|
||||||
// There three cases here:
|
// There are 3 cases here:
|
||||||
// 1. OP_ADD to create a new file
|
// 1. OP_ADD to create a new file
|
||||||
// 2. OP_ADD to update file blocks
|
// 2. OP_ADD to update file blocks
|
||||||
// 3. OP_ADD to open file for append
|
// 3. OP_ADD to open file for append (old append)
|
||||||
|
|
||||||
// See if the file already exists (persistBlocks call)
|
// See if the file already exists (persistBlocks call)
|
||||||
INodesInPath iip = fsDir.getINodesInPath(path, true);
|
INodesInPath iip = fsDir.getINodesInPath(path, true);
|
||||||
|
@ -384,19 +386,17 @@ public class FSEditLogLoader {
|
||||||
fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
|
fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
|
||||||
addCloseOp.rpcCallId, stat);
|
addCloseOp.rpcCallId, stat);
|
||||||
}
|
}
|
||||||
} else { // This is OP_ADD on an existing file
|
} else { // This is OP_ADD on an existing file (old append)
|
||||||
if (!oldFile.isUnderConstruction()) {
|
if (!oldFile.isUnderConstruction()) {
|
||||||
// This is case 3: a call to append() on an already-closed file.
|
// This is case 3: a call to append() on an already-closed file.
|
||||||
if (FSNamesystem.LOG.isDebugEnabled()) {
|
if (FSNamesystem.LOG.isDebugEnabled()) {
|
||||||
FSNamesystem.LOG.debug("Reopening an already-closed file " +
|
FSNamesystem.LOG.debug("Reopening an already-closed file " +
|
||||||
"for append");
|
"for append");
|
||||||
}
|
}
|
||||||
// Note we do not replace the INodeFile when converting it to
|
LocatedBlock lb = fsNamesys.prepareFileForAppend(path, iip,
|
||||||
// under-construction
|
addCloseOp.clientName, addCloseOp.clientMachine, false, false,
|
||||||
LocatedBlock lb = fsNamesys.prepareFileForWrite(path, iip,
|
false);
|
||||||
addCloseOp.clientName, addCloseOp.clientMachine, false, false);
|
// add the op into retry cache if necessary
|
||||||
|
|
||||||
// add the op into retry cache is necessary
|
|
||||||
if (toAddRetryCache) {
|
if (toAddRetryCache) {
|
||||||
HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
|
HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
|
||||||
fsNamesys.dir,
|
fsNamesys.dir,
|
||||||
|
@ -454,6 +454,34 @@ public class FSEditLogLoader {
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
case OP_APPEND: {
|
||||||
|
AppendOp appendOp = (AppendOp) op;
|
||||||
|
final String path = renameReservedPathsOnUpgrade(appendOp.path,
|
||||||
|
logVersion);
|
||||||
|
if (FSNamesystem.LOG.isDebugEnabled()) {
|
||||||
|
FSNamesystem.LOG.debug(op.opCode + ": " + path +
|
||||||
|
" clientName " + appendOp.clientName +
|
||||||
|
" clientMachine " + appendOp.clientMachine +
|
||||||
|
" newBlock " + appendOp.newBlock);
|
||||||
|
}
|
||||||
|
INodesInPath iip = fsDir.getINodesInPath4Write(path);
|
||||||
|
INodeFile file = INodeFile.valueOf(iip.getLastINode(), path);
|
||||||
|
if (!file.isUnderConstruction()) {
|
||||||
|
LocatedBlock lb = fsNamesys.prepareFileForAppend(path, iip,
|
||||||
|
appendOp.clientName, appendOp.clientMachine, appendOp.newBlock,
|
||||||
|
false, false);
|
||||||
|
// add the op into retry cache if necessary
|
||||||
|
if (toAddRetryCache) {
|
||||||
|
HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
|
||||||
|
fsNamesys.dir, HdfsFileStatus.EMPTY_NAME, file,
|
||||||
|
BlockStoragePolicySuite.ID_UNSPECIFIED,
|
||||||
|
Snapshot.CURRENT_STATE_ID, false, iip);
|
||||||
|
fsNamesys.addCacheEntryWithPayload(appendOp.rpcClientId,
|
||||||
|
appendOp.rpcCallId, new LastBlockWithStatus(lb, stat));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
}
|
||||||
case OP_UPDATE_BLOCKS: {
|
case OP_UPDATE_BLOCKS: {
|
||||||
UpdateBlocksOp updateOp = (UpdateBlocksOp)op;
|
UpdateBlocksOp updateOp = (UpdateBlocksOp)op;
|
||||||
final String path =
|
final String path =
|
||||||
|
@ -500,7 +528,14 @@ public class FSEditLogLoader {
|
||||||
srcs[i] =
|
srcs[i] =
|
||||||
renameReservedPathsOnUpgrade(concatDeleteOp.srcs[i], logVersion);
|
renameReservedPathsOnUpgrade(concatDeleteOp.srcs[i], logVersion);
|
||||||
}
|
}
|
||||||
FSDirConcatOp.unprotectedConcat(fsDir, trg, srcs, concatDeleteOp.timestamp);
|
INodesInPath targetIIP = fsDir.getINodesInPath4Write(trg);
|
||||||
|
INodeFile[] srcFiles = new INodeFile[srcs.length];
|
||||||
|
for (int i = 0; i < srcs.length; i++) {
|
||||||
|
INodesInPath srcIIP = fsDir.getINodesInPath4Write(srcs[i]);
|
||||||
|
srcFiles[i] = srcIIP.getLastINode().asFile();
|
||||||
|
}
|
||||||
|
FSDirConcatOp.unprotectedConcat(fsDir, targetIIP, srcFiles,
|
||||||
|
concatDeleteOp.timestamp);
|
||||||
|
|
||||||
if (toAddRetryCache) {
|
if (toAddRetryCache) {
|
||||||
fsNamesys.addCacheEntry(concatDeleteOp.rpcClientId,
|
fsNamesys.addCacheEntry(concatDeleteOp.rpcClientId,
|
||||||
|
|
|
@ -18,6 +18,7 @@
|
||||||
package org.apache.hadoop.hdfs.server.namenode;
|
package org.apache.hadoop.hdfs.server.namenode;
|
||||||
|
|
||||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD;
|
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD;
|
||||||
|
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_APPEND;
|
||||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_BLOCK;
|
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_BLOCK;
|
||||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_CACHE_DIRECTIVE;
|
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_CACHE_DIRECTIVE;
|
||||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_CACHE_POOL;
|
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_CACHE_POOL;
|
||||||
|
@ -209,6 +210,7 @@ public abstract class FSEditLogOp {
|
||||||
inst.put(OP_SET_XATTR, new SetXAttrOp());
|
inst.put(OP_SET_XATTR, new SetXAttrOp());
|
||||||
inst.put(OP_REMOVE_XATTR, new RemoveXAttrOp());
|
inst.put(OP_REMOVE_XATTR, new RemoveXAttrOp());
|
||||||
inst.put(OP_SET_STORAGE_POLICY, new SetStoragePolicyOp());
|
inst.put(OP_SET_STORAGE_POLICY, new SetStoragePolicyOp());
|
||||||
|
inst.put(OP_APPEND, new AppendOp());
|
||||||
}
|
}
|
||||||
|
|
||||||
public FSEditLogOp get(FSEditLogOpCodes opcode) {
|
public FSEditLogOp get(FSEditLogOpCodes opcode) {
|
||||||
|
@ -430,7 +432,7 @@ public abstract class FSEditLogOp {
|
||||||
private AddCloseOp(FSEditLogOpCodes opCode) {
|
private AddCloseOp(FSEditLogOpCodes opCode) {
|
||||||
super(opCode);
|
super(opCode);
|
||||||
storagePolicyId = BlockStoragePolicySuite.ID_UNSPECIFIED;
|
storagePolicyId = BlockStoragePolicySuite.ID_UNSPECIFIED;
|
||||||
assert(opCode == OP_ADD || opCode == OP_CLOSE);
|
assert(opCode == OP_ADD || opCode == OP_CLOSE || opCode == OP_APPEND);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -772,7 +774,7 @@ public abstract class FSEditLogOp {
|
||||||
}
|
}
|
||||||
|
|
||||||
static AddOp getInstance(OpInstanceCache cache) {
|
static AddOp getInstance(OpInstanceCache cache) {
|
||||||
return (AddOp)cache.get(OP_ADD);
|
return (AddOp) cache.get(OP_ADD);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -790,7 +792,7 @@ public abstract class FSEditLogOp {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Although {@link ClientProtocol#appendFile} may also log a close op, we do
|
* Although {@link ClientProtocol#append} may also log a close op, we do
|
||||||
* not need to record the rpc ids here since a successful appendFile op will
|
* not need to record the rpc ids here since a successful appendFile op will
|
||||||
* finally log an AddOp.
|
* finally log an AddOp.
|
||||||
*/
|
*/
|
||||||
|
@ -817,6 +819,97 @@ public abstract class FSEditLogOp {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static class AppendOp extends FSEditLogOp {
|
||||||
|
String path;
|
||||||
|
String clientName;
|
||||||
|
String clientMachine;
|
||||||
|
boolean newBlock;
|
||||||
|
|
||||||
|
private AppendOp() {
|
||||||
|
super(OP_APPEND);
|
||||||
|
}
|
||||||
|
|
||||||
|
static AppendOp getInstance(OpInstanceCache cache) {
|
||||||
|
return (AppendOp) cache.get(OP_APPEND);
|
||||||
|
}
|
||||||
|
|
||||||
|
AppendOp setPath(String path) {
|
||||||
|
this.path = path;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
AppendOp setClientName(String clientName) {
|
||||||
|
this.clientName = clientName;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
AppendOp setClientMachine(String clientMachine) {
|
||||||
|
this.clientMachine = clientMachine;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
AppendOp setNewBlock(boolean newBlock) {
|
||||||
|
this.newBlock = newBlock;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
StringBuilder builder = new StringBuilder();
|
||||||
|
builder.append("AppendOp ");
|
||||||
|
builder.append("[path=").append(path);
|
||||||
|
builder.append(", clientName=").append(clientName);
|
||||||
|
builder.append(", clientMachine=").append(clientMachine);
|
||||||
|
builder.append(", newBlock=").append(newBlock).append("]");
|
||||||
|
return builder.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
void resetSubFields() {
|
||||||
|
this.path = null;
|
||||||
|
this.clientName = null;
|
||||||
|
this.clientMachine = null;
|
||||||
|
this.newBlock = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
void readFields(DataInputStream in, int logVersion) throws IOException {
|
||||||
|
this.path = FSImageSerialization.readString(in);
|
||||||
|
this.clientName = FSImageSerialization.readString(in);
|
||||||
|
this.clientMachine = FSImageSerialization.readString(in);
|
||||||
|
this.newBlock = FSImageSerialization.readBoolean(in);
|
||||||
|
readRpcIds(in, logVersion);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeFields(DataOutputStream out) throws IOException {
|
||||||
|
FSImageSerialization.writeString(path, out);
|
||||||
|
FSImageSerialization.writeString(clientName, out);
|
||||||
|
FSImageSerialization.writeString(clientMachine, out);
|
||||||
|
FSImageSerialization.writeBoolean(newBlock, out);
|
||||||
|
writeRpcIds(rpcClientId, rpcCallId, out);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void toXml(ContentHandler contentHandler) throws SAXException {
|
||||||
|
XMLUtils.addSaxString(contentHandler, "PATH", path);
|
||||||
|
XMLUtils.addSaxString(contentHandler, "CLIENT_NAME", clientName);
|
||||||
|
XMLUtils.addSaxString(contentHandler, "CLIENT_MACHINE", clientMachine);
|
||||||
|
XMLUtils.addSaxString(contentHandler, "NEWBLOCK",
|
||||||
|
Boolean.toString(newBlock));
|
||||||
|
appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
void fromXml(Stanza st) throws InvalidXmlException {
|
||||||
|
this.path = st.getValue("PATH");
|
||||||
|
this.clientName = st.getValue("CLIENT_NAME");
|
||||||
|
this.clientMachine = st.getValue("CLIENT_MACHINE");
|
||||||
|
this.newBlock = Boolean.parseBoolean(st.getValue("NEWBLOCK"));
|
||||||
|
readRpcIdsFromXml(st);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
static class AddBlockOp extends FSEditLogOp {
|
static class AddBlockOp extends FSEditLogOp {
|
||||||
private String path;
|
private String path;
|
||||||
private Block penultimateBlock;
|
private Block penultimateBlock;
|
||||||
|
@ -1645,7 +1738,7 @@ public abstract class FSEditLogOp {
|
||||||
* {@link ClientProtocol#updateBlockForPipeline},
|
* {@link ClientProtocol#updateBlockForPipeline},
|
||||||
* {@link ClientProtocol#recoverLease}, {@link ClientProtocol#addBlock}) or
|
* {@link ClientProtocol#recoverLease}, {@link ClientProtocol#addBlock}) or
|
||||||
* already bound with other editlog op which records rpc ids (
|
* already bound with other editlog op which records rpc ids (
|
||||||
* {@link ClientProtocol#startFile}). Thus no need to record rpc ids here.
|
* {@link ClientProtocol#create}). Thus no need to record rpc ids here.
|
||||||
*/
|
*/
|
||||||
static class SetGenstampV1Op extends FSEditLogOp {
|
static class SetGenstampV1Op extends FSEditLogOp {
|
||||||
long genStampV1;
|
long genStampV1;
|
||||||
|
|
|
@ -74,6 +74,7 @@ public enum FSEditLogOpCodes {
|
||||||
OP_REMOVE_XATTR ((byte) 44),
|
OP_REMOVE_XATTR ((byte) 44),
|
||||||
OP_SET_STORAGE_POLICY ((byte) 45),
|
OP_SET_STORAGE_POLICY ((byte) 45),
|
||||||
OP_TRUNCATE ((byte) 46),
|
OP_TRUNCATE ((byte) 46),
|
||||||
|
OP_APPEND ((byte) 47),
|
||||||
|
|
||||||
// Note that the current range of the valid OP code is 0~127
|
// Note that the current range of the valid OP code is 0~127
|
||||||
OP_INVALID ((byte) -1);
|
OP_INVALID ((byte) -1);
|
||||||
|
|
|
@ -254,6 +254,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
|
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
||||||
|
import org.apache.hadoop.io.EnumSetWritable;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.ipc.RetriableException;
|
import org.apache.hadoop.ipc.RetriableException;
|
||||||
|
@ -2581,12 +2582,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
||||||
* <p>
|
* <p>
|
||||||
*
|
*
|
||||||
* For description of parameters and exceptions thrown see
|
* For description of parameters and exceptions thrown see
|
||||||
* {@link ClientProtocol#append(String, String)}
|
* {@link ClientProtocol#append(String, String, EnumSetWritable)}
|
||||||
*
|
*
|
||||||
* @return the last block locations if the block is partial or null otherwise
|
* @return the last block locations if the block is partial or null otherwise
|
||||||
*/
|
*/
|
||||||
private LocatedBlock appendFileInternal(FSPermissionChecker pc,
|
private LocatedBlock appendFileInternal(FSPermissionChecker pc,
|
||||||
INodesInPath iip, String holder, String clientMachine,
|
INodesInPath iip, String holder, String clientMachine, boolean newBlock,
|
||||||
boolean logRetryCache) throws IOException {
|
boolean logRetryCache) throws IOException {
|
||||||
assert hasWriteLock();
|
assert hasWriteLock();
|
||||||
// Verify that the destination does not exist as a directory already.
|
// Verify that the destination does not exist as a directory already.
|
||||||
|
@ -2608,7 +2609,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
||||||
INodeFile myFile = INodeFile.valueOf(inode, src, true);
|
INodeFile myFile = INodeFile.valueOf(inode, src, true);
|
||||||
final BlockStoragePolicy lpPolicy =
|
final BlockStoragePolicy lpPolicy =
|
||||||
blockManager.getStoragePolicy("LAZY_PERSIST");
|
blockManager.getStoragePolicy("LAZY_PERSIST");
|
||||||
|
|
||||||
if (lpPolicy != null &&
|
if (lpPolicy != null &&
|
||||||
lpPolicy.getId() == myFile.getStoragePolicyID()) {
|
lpPolicy.getId() == myFile.getStoragePolicyID()) {
|
||||||
throw new UnsupportedOperationException(
|
throw new UnsupportedOperationException(
|
||||||
|
@ -2625,8 +2625,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
||||||
throw new IOException("append: lastBlock=" + lastBlock +
|
throw new IOException("append: lastBlock=" + lastBlock +
|
||||||
" of src=" + src + " is not sufficiently replicated yet.");
|
" of src=" + src + " is not sufficiently replicated yet.");
|
||||||
}
|
}
|
||||||
return prepareFileForWrite(src, iip, holder, clientMachine, true,
|
return prepareFileForAppend(src, iip, holder, clientMachine, newBlock,
|
||||||
logRetryCache);
|
true, logRetryCache);
|
||||||
} catch (IOException ie) {
|
} catch (IOException ie) {
|
||||||
NameNode.stateChangeLog.warn("DIR* NameSystem.append: " +ie.getMessage());
|
NameNode.stateChangeLog.warn("DIR* NameSystem.append: " +ie.getMessage());
|
||||||
throw ie;
|
throw ie;
|
||||||
|
@ -2640,6 +2640,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
||||||
* @param src path to the file
|
* @param src path to the file
|
||||||
* @param leaseHolder identifier of the lease holder on this file
|
* @param leaseHolder identifier of the lease holder on this file
|
||||||
* @param clientMachine identifier of the client machine
|
* @param clientMachine identifier of the client machine
|
||||||
|
* @param newBlock if the data is appended to a new block
|
||||||
* @param writeToEditLog whether to persist this change to the edit log
|
* @param writeToEditLog whether to persist this change to the edit log
|
||||||
* @param logRetryCache whether to record RPC ids in editlog for retry cache
|
* @param logRetryCache whether to record RPC ids in editlog for retry cache
|
||||||
* rebuilding
|
* rebuilding
|
||||||
|
@ -2647,9 +2648,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
||||||
* @throws UnresolvedLinkException
|
* @throws UnresolvedLinkException
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
LocatedBlock prepareFileForWrite(String src, INodesInPath iip,
|
LocatedBlock prepareFileForAppend(String src, INodesInPath iip,
|
||||||
String leaseHolder, String clientMachine, boolean writeToEditLog,
|
String leaseHolder, String clientMachine, boolean newBlock,
|
||||||
boolean logRetryCache) throws IOException {
|
boolean writeToEditLog, boolean logRetryCache) throws IOException {
|
||||||
final INodeFile file = iip.getLastINode().asFile();
|
final INodeFile file = iip.getLastINode().asFile();
|
||||||
file.recordModification(iip.getLatestSnapshotId());
|
file.recordModification(iip.getLatestSnapshotId());
|
||||||
file.toUnderConstruction(leaseHolder, clientMachine);
|
file.toUnderConstruction(leaseHolder, clientMachine);
|
||||||
|
@ -2657,16 +2658,24 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
||||||
leaseManager.addLease(
|
leaseManager.addLease(
|
||||||
file.getFileUnderConstructionFeature().getClientName(), src);
|
file.getFileUnderConstructionFeature().getClientName(), src);
|
||||||
|
|
||||||
LocatedBlock ret =
|
LocatedBlock ret = null;
|
||||||
blockManager.convertLastBlockToUnderConstruction(file, 0);
|
if (!newBlock) {
|
||||||
|
ret = blockManager.convertLastBlockToUnderConstruction(file, 0);
|
||||||
if (ret != null) {
|
if (ret != null) {
|
||||||
// update the quota: use the preferred block size for UC block
|
// update the quota: use the preferred block size for UC block
|
||||||
final long diff = file.getPreferredBlockSize() - ret.getBlockSize();
|
final long diff = file.getPreferredBlockSize() - ret.getBlockSize();
|
||||||
dir.updateSpaceConsumed(iip, 0, diff * file.getBlockReplication());
|
dir.updateSpaceConsumed(iip, 0, diff * file.getBlockReplication());
|
||||||
}
|
}
|
||||||
|
} else {
|
||||||
|
BlockInfoContiguous lastBlock = file.getLastBlock();
|
||||||
|
if (lastBlock != null) {
|
||||||
|
ExtendedBlock blk = new ExtendedBlock(this.getBlockPoolId(), lastBlock);
|
||||||
|
ret = new LocatedBlock(blk, new DatanodeInfo[0]);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
if (writeToEditLog) {
|
if (writeToEditLog) {
|
||||||
getEditLog().logOpenFile(src, file, false, logRetryCache);
|
getEditLog().logAppendFile(src, file, newBlock, logRetryCache);
|
||||||
}
|
}
|
||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
@ -2812,11 +2821,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
||||||
/**
|
/**
|
||||||
* Append to an existing file in the namespace.
|
* Append to an existing file in the namespace.
|
||||||
*/
|
*/
|
||||||
LastBlockWithStatus appendFile(
|
LastBlockWithStatus appendFile(String src, String holder,
|
||||||
String src, String holder, String clientMachine, boolean logRetryCache)
|
String clientMachine, EnumSet<CreateFlag> flag, boolean logRetryCache)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
try {
|
try {
|
||||||
return appendFileInt(src, holder, clientMachine, logRetryCache);
|
return appendFileInt(src, holder, clientMachine,
|
||||||
|
flag.contains(CreateFlag.NEW_BLOCK), logRetryCache);
|
||||||
} catch (AccessControlException e) {
|
} catch (AccessControlException e) {
|
||||||
logAuditEvent(false, "append", src);
|
logAuditEvent(false, "append", src);
|
||||||
throw e;
|
throw e;
|
||||||
|
@ -2824,7 +2834,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
||||||
}
|
}
|
||||||
|
|
||||||
private LastBlockWithStatus appendFileInt(final String srcArg, String holder,
|
private LastBlockWithStatus appendFileInt(final String srcArg, String holder,
|
||||||
String clientMachine, boolean logRetryCache) throws IOException {
|
String clientMachine, boolean newBlock, boolean logRetryCache)
|
||||||
|
throws IOException {
|
||||||
String src = srcArg;
|
String src = srcArg;
|
||||||
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
||||||
NameNode.stateChangeLog.debug("DIR* NameSystem.appendFile: src=" + src
|
NameNode.stateChangeLog.debug("DIR* NameSystem.appendFile: src=" + src
|
||||||
|
@ -2849,7 +2860,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
||||||
checkNameNodeSafeMode("Cannot append to file" + src);
|
checkNameNodeSafeMode("Cannot append to file" + src);
|
||||||
src = dir.resolvePath(pc, src, pathComponents);
|
src = dir.resolvePath(pc, src, pathComponents);
|
||||||
final INodesInPath iip = dir.getINodesInPath4Write(src);
|
final INodesInPath iip = dir.getINodesInPath4Write(src);
|
||||||
lb = appendFileInternal(pc, iip, holder, clientMachine, logRetryCache);
|
lb = appendFileInternal(pc, iip, holder, clientMachine, newBlock,
|
||||||
|
logRetryCache);
|
||||||
stat = FSDirStatAndListingOp.getFileInfo(dir, src, false,
|
stat = FSDirStatAndListingOp.getFileInfo(dir, src, false,
|
||||||
FSDirectory.isReservedRawName(srcArg), true);
|
FSDirectory.isReservedRawName(srcArg), true);
|
||||||
} catch (StandbyException se) {
|
} catch (StandbyException se) {
|
||||||
|
|
|
@ -411,7 +411,7 @@ public class INodeFile extends INodeWithAdditionalFields
|
||||||
}
|
}
|
||||||
|
|
||||||
/** @return the diskspace required for a full block. */
|
/** @return the diskspace required for a full block. */
|
||||||
final long getBlockDiskspace() {
|
final long getPreferredBlockDiskspace() {
|
||||||
return getPreferredBlockSize() * getBlockReplication();
|
return getPreferredBlockSize() * getBlockReplication();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -65,6 +65,10 @@ public class InotifyFSEditLogOpTranslator {
|
||||||
FSEditLogOp.CloseOp cOp = (FSEditLogOp.CloseOp) op;
|
FSEditLogOp.CloseOp cOp = (FSEditLogOp.CloseOp) op;
|
||||||
return new EventBatch(op.txid, new Event[] {
|
return new EventBatch(op.txid, new Event[] {
|
||||||
new Event.CloseEvent(cOp.path, getSize(cOp), cOp.mtime) });
|
new Event.CloseEvent(cOp.path, getSize(cOp), cOp.mtime) });
|
||||||
|
case OP_APPEND:
|
||||||
|
FSEditLogOp.AppendOp appendOp = (FSEditLogOp.AppendOp) op;
|
||||||
|
return new EventBatch(op.txid, new Event[] {new Event.AppendEvent
|
||||||
|
.Builder().path(appendOp.path).newBlock(appendOp.newBlock).build()});
|
||||||
case OP_SET_REPLICATION:
|
case OP_SET_REPLICATION:
|
||||||
FSEditLogOp.SetReplicationOp setRepOp = (FSEditLogOp.SetReplicationOp) op;
|
FSEditLogOp.SetReplicationOp setRepOp = (FSEditLogOp.SetReplicationOp) op;
|
||||||
return new EventBatch(op.txid,
|
return new EventBatch(op.txid,
|
||||||
|
|
|
@ -70,7 +70,8 @@ public class NameNodeLayoutVersion {
|
||||||
"creating file with overwrite"),
|
"creating file with overwrite"),
|
||||||
XATTRS_NAMESPACE_EXT(-59, "Increase number of xattr namespaces"),
|
XATTRS_NAMESPACE_EXT(-59, "Increase number of xattr namespaces"),
|
||||||
BLOCK_STORAGE_POLICY(-60, "Block Storage policy"),
|
BLOCK_STORAGE_POLICY(-60, "Block Storage policy"),
|
||||||
TRUNCATE(-61, "Truncate");
|
TRUNCATE(-61, "Truncate"),
|
||||||
|
APPEND_NEW_BLOCK(-62, "Support appending to new block");
|
||||||
|
|
||||||
private final FeatureInfo info;
|
private final FeatureInfo info;
|
||||||
|
|
||||||
|
|
|
@ -630,15 +630,16 @@ class NameNodeRpcServer implements NamenodeProtocols {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override // ClientProtocol
|
@Override // ClientProtocol
|
||||||
public LastBlockWithStatus append(String src, String clientName)
|
public LastBlockWithStatus append(String src, String clientName,
|
||||||
throws IOException {
|
EnumSetWritable<CreateFlag> flag) throws IOException {
|
||||||
checkNNStartup();
|
checkNNStartup();
|
||||||
String clientMachine = getClientMachine();
|
String clientMachine = getClientMachine();
|
||||||
if (stateChangeLog.isDebugEnabled()) {
|
if (stateChangeLog.isDebugEnabled()) {
|
||||||
stateChangeLog.debug("*DIR* NameNode.append: file "
|
stateChangeLog.debug("*DIR* NameNode.append: file "
|
||||||
+src+" for "+clientName+" at "+clientMachine);
|
+src+" for "+clientName+" at "+clientMachine);
|
||||||
}
|
}
|
||||||
CacheEntryWithPayload cacheEntry = RetryCache.waitForCompletion(retryCache, null);
|
CacheEntryWithPayload cacheEntry = RetryCache.waitForCompletion(retryCache,
|
||||||
|
null);
|
||||||
if (cacheEntry != null && cacheEntry.isSuccess()) {
|
if (cacheEntry != null && cacheEntry.isSuccess()) {
|
||||||
return (LastBlockWithStatus) cacheEntry.getPayload();
|
return (LastBlockWithStatus) cacheEntry.getPayload();
|
||||||
}
|
}
|
||||||
|
@ -646,7 +647,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
|
||||||
LastBlockWithStatus info = null;
|
LastBlockWithStatus info = null;
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
try {
|
try {
|
||||||
info = namesystem.appendFile(src, clientName, clientMachine,
|
info = namesystem.appendFile(src, clientName, clientMachine, flag.get(),
|
||||||
cacheEntry != null);
|
cacheEntry != null);
|
||||||
success = true;
|
success = true;
|
||||||
} finally {
|
} finally {
|
||||||
|
|
|
@ -66,6 +66,7 @@ enum CreateFlagProto {
|
||||||
OVERWRITE = 0x02; // Truncate/overwrite a file. Same as POSIX O_TRUNC
|
OVERWRITE = 0x02; // Truncate/overwrite a file. Same as POSIX O_TRUNC
|
||||||
APPEND = 0x04; // Append to a file
|
APPEND = 0x04; // Append to a file
|
||||||
LAZY_PERSIST = 0x10; // File with reduced durability guarantees.
|
LAZY_PERSIST = 0x10; // File with reduced durability guarantees.
|
||||||
|
NEW_BLOCK = 0x20; // Write data to a new block when appending
|
||||||
}
|
}
|
||||||
|
|
||||||
message CreateRequestProto {
|
message CreateRequestProto {
|
||||||
|
@ -86,6 +87,7 @@ message CreateResponseProto {
|
||||||
message AppendRequestProto {
|
message AppendRequestProto {
|
||||||
required string src = 1;
|
required string src = 1;
|
||||||
required string clientName = 2;
|
required string clientName = 2;
|
||||||
|
optional uint32 flag = 3; // bits set using CreateFlag
|
||||||
}
|
}
|
||||||
|
|
||||||
message AppendResponseProto {
|
message AppendResponseProto {
|
||||||
|
|
|
@ -89,6 +89,7 @@ message CloseEventProto {
|
||||||
|
|
||||||
message AppendEventProto {
|
message AppendEventProto {
|
||||||
required string path = 1;
|
required string path = 1;
|
||||||
|
optional bool newBlock = 2 [default = false];
|
||||||
}
|
}
|
||||||
|
|
||||||
message RenameEventProto {
|
message RenameEventProto {
|
||||||
|
|
|
@ -159,6 +159,22 @@ public class AppendTestUtil {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static void check(DistributedFileSystem fs, Path p, int position,
|
||||||
|
int length) throws IOException {
|
||||||
|
byte[] buf = new byte[length];
|
||||||
|
int i = 0;
|
||||||
|
try {
|
||||||
|
FSDataInputStream in = fs.open(p);
|
||||||
|
in.read(position, buf, 0, buf.length);
|
||||||
|
for(i = position; i < length + position; i++) {
|
||||||
|
assertEquals((byte) i, buf[i - position]);
|
||||||
|
}
|
||||||
|
in.close();
|
||||||
|
} catch(IOException ioe) {
|
||||||
|
throw new IOException("p=" + p + ", length=" + length + ", i=" + i, ioe);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* create a buffer that contains the entire test file data.
|
* create a buffer that contains the entire test file data.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -1178,6 +1178,9 @@ public class DFSTestUtil {
|
||||||
FSDataOutputStream s = filesystem.create(pathFileCreate);
|
FSDataOutputStream s = filesystem.create(pathFileCreate);
|
||||||
// OP_CLOSE 9
|
// OP_CLOSE 9
|
||||||
s.close();
|
s.close();
|
||||||
|
// OP_APPEND 47
|
||||||
|
FSDataOutputStream s2 = filesystem.append(pathFileCreate, 4096, null);
|
||||||
|
s2.close();
|
||||||
// OP_SET_STORAGE_POLICY 45
|
// OP_SET_STORAGE_POLICY 45
|
||||||
filesystem.setStoragePolicy(pathFileCreate,
|
filesystem.setStoragePolicy(pathFileCreate,
|
||||||
HdfsConstants.HOT_STORAGE_POLICY_NAME);
|
HdfsConstants.HOT_STORAGE_POLICY_NAME);
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs;
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.CreateFlag;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.XAttrSetFlag;
|
import org.apache.hadoop.fs.XAttrSetFlag;
|
||||||
|
@ -71,7 +72,7 @@ public class TestDFSInotifyEventInputStream {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testOpcodeCount() {
|
public void testOpcodeCount() {
|
||||||
Assert.assertEquals(48, FSEditLogOpCodes.values().length);
|
Assert.assertEquals(49, FSEditLogOpCodes.values().length);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -109,7 +110,8 @@ public class TestDFSInotifyEventInputStream {
|
||||||
os.write(new byte[BLOCK_SIZE]);
|
os.write(new byte[BLOCK_SIZE]);
|
||||||
os.close(); // CloseOp -> CloseEvent
|
os.close(); // CloseOp -> CloseEvent
|
||||||
// AddOp -> AppendEvent
|
// AddOp -> AppendEvent
|
||||||
os = client.append("/file2", BLOCK_SIZE, null, null);
|
os = client.append("/file2", BLOCK_SIZE, EnumSet.of(CreateFlag.APPEND),
|
||||||
|
null, null);
|
||||||
os.write(new byte[BLOCK_SIZE]);
|
os.write(new byte[BLOCK_SIZE]);
|
||||||
os.close(); // CloseOp -> CloseEvent
|
os.close(); // CloseOp -> CloseEvent
|
||||||
Thread.sleep(10); // so that the atime will get updated on the next line
|
Thread.sleep(10); // so that the atime will get updated on the next line
|
||||||
|
@ -182,13 +184,14 @@ public class TestDFSInotifyEventInputStream {
|
||||||
Assert.assertTrue(ce2.getFileSize() > 0);
|
Assert.assertTrue(ce2.getFileSize() > 0);
|
||||||
Assert.assertTrue(ce2.getTimestamp() > 0);
|
Assert.assertTrue(ce2.getTimestamp() > 0);
|
||||||
|
|
||||||
// AddOp
|
// AppendOp
|
||||||
batch = waitForNextEvents(eis);
|
batch = waitForNextEvents(eis);
|
||||||
Assert.assertEquals(1, batch.getEvents().length);
|
Assert.assertEquals(1, batch.getEvents().length);
|
||||||
txid = checkTxid(batch, txid);
|
txid = checkTxid(batch, txid);
|
||||||
Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.APPEND);
|
Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.APPEND);
|
||||||
Event.AppendEvent append2 = (Event.AppendEvent)batch.getEvents()[0];
|
Event.AppendEvent append2 = (Event.AppendEvent)batch.getEvents()[0];
|
||||||
Assert.assertEquals("/file2", append2.getPath());
|
Assert.assertEquals("/file2", append2.getPath());
|
||||||
|
Assert.assertFalse(append2.toNewBlock());
|
||||||
|
|
||||||
// CloseOp
|
// CloseOp
|
||||||
batch = waitForNextEvents(eis);
|
batch = waitForNextEvents(eis);
|
||||||
|
|
|
@ -25,10 +25,12 @@ import java.io.File;
|
||||||
import java.io.FileNotFoundException;
|
import java.io.FileNotFoundException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
|
import java.util.EnumSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.BlockLocation;
|
import org.apache.hadoop.fs.BlockLocation;
|
||||||
|
import org.apache.hadoop.fs.CreateFlag;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.HardLink;
|
import org.apache.hadoop.fs.HardLink;
|
||||||
|
@ -346,6 +348,45 @@ public class TestFileAppend{
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Test two consecutive appends on a file with a full block. */
|
||||||
|
@Test
|
||||||
|
public void testAppend2Twice() throws Exception {
|
||||||
|
Configuration conf = new HdfsConfiguration();
|
||||||
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
|
||||||
|
final DistributedFileSystem fs1 = cluster.getFileSystem();
|
||||||
|
final FileSystem fs2 = AppendTestUtil.createHdfsWithDifferentUsername(conf);
|
||||||
|
try {
|
||||||
|
final Path p = new Path("/testAppendTwice/foo");
|
||||||
|
final int len = 1 << 16;
|
||||||
|
final byte[] fileContents = AppendTestUtil.initBuffer(len);
|
||||||
|
|
||||||
|
{
|
||||||
|
// create a new file with a full block.
|
||||||
|
FSDataOutputStream out = fs2.create(p, true, 4096, (short)1, len);
|
||||||
|
out.write(fileContents, 0, len);
|
||||||
|
out.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
//1st append does not add any data so that the last block remains full
|
||||||
|
//and the last block in INodeFileUnderConstruction is a BlockInfo
|
||||||
|
//but not BlockInfoUnderConstruction.
|
||||||
|
((DistributedFileSystem) fs2).append(p,
|
||||||
|
EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null);
|
||||||
|
|
||||||
|
// 2nd append should get AlreadyBeingCreatedException
|
||||||
|
fs1.append(p);
|
||||||
|
Assert.fail();
|
||||||
|
} catch(RemoteException re) {
|
||||||
|
AppendTestUtil.LOG.info("Got an exception:", re);
|
||||||
|
Assert.assertEquals(AlreadyBeingCreatedException.class.getName(),
|
||||||
|
re.getClassName());
|
||||||
|
} finally {
|
||||||
|
fs2.close();
|
||||||
|
fs1.close();
|
||||||
|
cluster.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/** Tests appending after soft-limit expires. */
|
/** Tests appending after soft-limit expires. */
|
||||||
@Test
|
@Test
|
||||||
public void testAppendAfterSoftLimit()
|
public void testAppendAfterSoftLimit()
|
||||||
|
@ -388,6 +429,54 @@ public class TestFileAppend{
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Tests appending after soft-limit expires. */
|
||||||
|
@Test
|
||||||
|
public void testAppend2AfterSoftLimit() throws Exception {
|
||||||
|
Configuration conf = new HdfsConfiguration();
|
||||||
|
conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
|
||||||
|
//Set small soft-limit for lease
|
||||||
|
final long softLimit = 1L;
|
||||||
|
final long hardLimit = 9999999L;
|
||||||
|
|
||||||
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
|
||||||
|
.build();
|
||||||
|
cluster.setLeasePeriod(softLimit, hardLimit);
|
||||||
|
cluster.waitActive();
|
||||||
|
|
||||||
|
DistributedFileSystem fs = cluster.getFileSystem();
|
||||||
|
DistributedFileSystem fs2 = new DistributedFileSystem();
|
||||||
|
fs2.initialize(fs.getUri(), conf);
|
||||||
|
|
||||||
|
final Path testPath = new Path("/testAppendAfterSoftLimit");
|
||||||
|
final byte[] fileContents = AppendTestUtil.initBuffer(32);
|
||||||
|
|
||||||
|
// create a new file without closing
|
||||||
|
FSDataOutputStream out = fs.create(testPath);
|
||||||
|
out.write(fileContents);
|
||||||
|
|
||||||
|
//Wait for > soft-limit
|
||||||
|
Thread.sleep(250);
|
||||||
|
|
||||||
|
try {
|
||||||
|
FSDataOutputStream appendStream2 = fs2.append(testPath,
|
||||||
|
EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null);
|
||||||
|
appendStream2.write(fileContents);
|
||||||
|
appendStream2.close();
|
||||||
|
assertEquals(fileContents.length, fs.getFileStatus(testPath).getLen());
|
||||||
|
// make sure we now have 1 block since the first writer was revoked
|
||||||
|
LocatedBlocks blks = fs.getClient().getLocatedBlocks(testPath.toString(),
|
||||||
|
0L);
|
||||||
|
assertEquals(1, blks.getLocatedBlocks().size());
|
||||||
|
for (LocatedBlock blk : blks.getLocatedBlocks()) {
|
||||||
|
assertEquals(fileContents.length, blk.getBlockSize());
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
fs.close();
|
||||||
|
fs2.close();
|
||||||
|
cluster.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Old replica of the block should not be accepted as valid for append/read
|
* Old replica of the block should not be accepted as valid for append/read
|
||||||
*/
|
*/
|
||||||
|
@ -441,4 +530,77 @@ public class TestFileAppend{
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Old replica of the block should not be accepted as valid for append/read
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testMultiAppend2() throws Exception {
|
||||||
|
Configuration conf = new HdfsConfiguration();
|
||||||
|
conf.set("dfs.client.block.write.replace-datanode-on-failure.enable",
|
||||||
|
"false");
|
||||||
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
|
||||||
|
.build();
|
||||||
|
DistributedFileSystem fs = null;
|
||||||
|
final String hello = "hello\n";
|
||||||
|
try {
|
||||||
|
fs = cluster.getFileSystem();
|
||||||
|
Path path = new Path("/test");
|
||||||
|
FSDataOutputStream out = fs.create(path);
|
||||||
|
out.writeBytes(hello);
|
||||||
|
out.close();
|
||||||
|
|
||||||
|
// stop one datanode
|
||||||
|
DataNodeProperties dnProp = cluster.stopDataNode(0);
|
||||||
|
String dnAddress = dnProp.datanode.getXferAddress().toString();
|
||||||
|
if (dnAddress.startsWith("/")) {
|
||||||
|
dnAddress = dnAddress.substring(1);
|
||||||
|
}
|
||||||
|
|
||||||
|
// append again to bump genstamps
|
||||||
|
for (int i = 0; i < 2; i++) {
|
||||||
|
out = fs.append(path,
|
||||||
|
EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null);
|
||||||
|
out.writeBytes(hello);
|
||||||
|
out.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
// re-open and make the block state as underconstruction
|
||||||
|
out = fs.append(path, EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK),
|
||||||
|
4096, null);
|
||||||
|
cluster.restartDataNode(dnProp, true);
|
||||||
|
// wait till the block report comes
|
||||||
|
Thread.sleep(2000);
|
||||||
|
out.writeBytes(hello);
|
||||||
|
out.close();
|
||||||
|
// check the block locations
|
||||||
|
LocatedBlocks blocks = fs.getClient().getLocatedBlocks(path.toString(), 0L);
|
||||||
|
// since we append the file 3 time, we should be 4 blocks
|
||||||
|
assertEquals(4, blocks.getLocatedBlocks().size());
|
||||||
|
for (LocatedBlock block : blocks.getLocatedBlocks()) {
|
||||||
|
assertEquals(hello.length(), block.getBlockSize());
|
||||||
|
}
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
for (int i = 0; i < 4; i++) {
|
||||||
|
sb.append(hello);
|
||||||
|
}
|
||||||
|
final byte[] content = sb.toString().getBytes();
|
||||||
|
AppendTestUtil.checkFullFile(fs, path, content.length, content,
|
||||||
|
"Read /test");
|
||||||
|
|
||||||
|
// restart namenode to make sure the editlog can be properly applied
|
||||||
|
cluster.restartNameNode(true);
|
||||||
|
cluster.waitActive();
|
||||||
|
AppendTestUtil.checkFullFile(fs, path, content.length, content,
|
||||||
|
"Read /test");
|
||||||
|
blocks = fs.getClient().getLocatedBlocks(path.toString(), 0L);
|
||||||
|
// since we append the file 3 time, we should be 4 blocks
|
||||||
|
assertEquals(4, blocks.getLocatedBlocks().size());
|
||||||
|
for (LocatedBlock block : blocks.getLocatedBlocks()) {
|
||||||
|
assertEquals(hello.length(), block.getBlockSize());
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
IOUtils.closeStream(fs);
|
||||||
|
cluster.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hdfs;
|
package org.apache.hadoop.hdfs;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertFalse;
|
import static org.junit.Assert.assertFalse;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
import static org.junit.Assert.fail;
|
import static org.junit.Assert.fail;
|
||||||
|
@ -24,13 +25,17 @@ import static org.junit.Assert.fail;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.EnumSet;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.commons.logging.impl.Log4JLogger;
|
import org.apache.commons.logging.impl.Log4JLogger;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.CreateFlag;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.permission.FsPermission;
|
import org.apache.hadoop.fs.permission.FsPermission;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||||
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
|
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
|
@ -61,11 +66,7 @@ public class TestFileAppend2 {
|
||||||
final int numberOfFiles = 50;
|
final int numberOfFiles = 50;
|
||||||
final int numThreads = 10;
|
final int numThreads = 10;
|
||||||
final int numAppendsPerThread = 20;
|
final int numAppendsPerThread = 20;
|
||||||
/***
|
|
||||||
int numberOfFiles = 1;
|
|
||||||
int numThreads = 1;
|
|
||||||
int numAppendsPerThread = 2000;
|
|
||||||
****/
|
|
||||||
Workload[] workload = null;
|
Workload[] workload = null;
|
||||||
final ArrayList<Path> testFiles = new ArrayList<Path>();
|
final ArrayList<Path> testFiles = new ArrayList<Path>();
|
||||||
volatile static boolean globalStatus = true;
|
volatile static boolean globalStatus = true;
|
||||||
|
@ -223,16 +224,170 @@ public class TestFileAppend2 {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates one file, writes a few bytes to it and then closed it.
|
||||||
|
* Reopens the same file for appending using append2 API, write all blocks and
|
||||||
|
* then close. Verify that all data exists in file.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testSimpleAppend2() throws Exception {
|
||||||
|
final Configuration conf = new HdfsConfiguration();
|
||||||
|
if (simulatedStorage) {
|
||||||
|
SimulatedFSDataset.setFactory(conf);
|
||||||
|
}
|
||||||
|
conf.setInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 50);
|
||||||
|
fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
|
||||||
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
|
||||||
|
DistributedFileSystem fs = cluster.getFileSystem();
|
||||||
|
try {
|
||||||
|
{ // test appending to a file.
|
||||||
|
// create a new file.
|
||||||
|
Path file1 = new Path("/simpleAppend.dat");
|
||||||
|
FSDataOutputStream stm = AppendTestUtil.createFile(fs, file1, 1);
|
||||||
|
System.out.println("Created file simpleAppend.dat");
|
||||||
|
|
||||||
|
// write to file
|
||||||
|
int mid = 186; // io.bytes.per.checksum bytes
|
||||||
|
System.out.println("Writing " + mid + " bytes to file " + file1);
|
||||||
|
stm.write(fileContents, 0, mid);
|
||||||
|
stm.close();
|
||||||
|
System.out.println("Wrote and Closed first part of file.");
|
||||||
|
|
||||||
|
// write to file
|
||||||
|
int mid2 = 607; // io.bytes.per.checksum bytes
|
||||||
|
System.out.println("Writing " + mid + " bytes to file " + file1);
|
||||||
|
stm = fs.append(file1,
|
||||||
|
EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null);
|
||||||
|
stm.write(fileContents, mid, mid2-mid);
|
||||||
|
stm.close();
|
||||||
|
System.out.println("Wrote and Closed second part of file.");
|
||||||
|
|
||||||
|
// write the remainder of the file
|
||||||
|
stm = fs.append(file1,
|
||||||
|
EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null);
|
||||||
|
// ensure getPos is set to reflect existing size of the file
|
||||||
|
assertTrue(stm.getPos() > 0);
|
||||||
|
System.out.println("Writing " + (AppendTestUtil.FILE_SIZE - mid2) +
|
||||||
|
" bytes to file " + file1);
|
||||||
|
stm.write(fileContents, mid2, AppendTestUtil.FILE_SIZE - mid2);
|
||||||
|
System.out.println("Written second part of file");
|
||||||
|
stm.close();
|
||||||
|
System.out.println("Wrote and Closed second part of file.");
|
||||||
|
|
||||||
|
// verify that entire file is good
|
||||||
|
AppendTestUtil.checkFullFile(fs, file1, AppendTestUtil.FILE_SIZE,
|
||||||
|
fileContents, "Read 2");
|
||||||
|
// also make sure there three different blocks for the file
|
||||||
|
List<LocatedBlock> blocks = fs.getClient().getLocatedBlocks(
|
||||||
|
file1.toString(), 0L).getLocatedBlocks();
|
||||||
|
assertEquals(12, blocks.size()); // the block size is 1024
|
||||||
|
assertEquals(mid, blocks.get(0).getBlockSize());
|
||||||
|
assertEquals(mid2 - mid, blocks.get(1).getBlockSize());
|
||||||
|
for (int i = 2; i < 11; i++) {
|
||||||
|
assertEquals(AppendTestUtil.BLOCK_SIZE, blocks.get(i).getBlockSize());
|
||||||
|
}
|
||||||
|
assertEquals((AppendTestUtil.FILE_SIZE - mid2)
|
||||||
|
% AppendTestUtil.BLOCK_SIZE, blocks.get(11).getBlockSize());
|
||||||
|
}
|
||||||
|
|
||||||
|
{ // test appending to an non-existing file.
|
||||||
|
FSDataOutputStream out = null;
|
||||||
|
try {
|
||||||
|
out = fs.append(new Path("/non-existing.dat"),
|
||||||
|
EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null);
|
||||||
|
fail("Expected to have FileNotFoundException");
|
||||||
|
} catch(java.io.FileNotFoundException fnfe) {
|
||||||
|
System.out.println("Good: got " + fnfe);
|
||||||
|
fnfe.printStackTrace(System.out);
|
||||||
|
} finally {
|
||||||
|
IOUtils.closeStream(out);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
{ // test append permission.
|
||||||
|
// set root to all writable
|
||||||
|
Path root = new Path("/");
|
||||||
|
fs.setPermission(root, new FsPermission((short)0777));
|
||||||
|
fs.close();
|
||||||
|
|
||||||
|
// login as a different user
|
||||||
|
final UserGroupInformation superuser =
|
||||||
|
UserGroupInformation.getCurrentUser();
|
||||||
|
String username = "testappenduser";
|
||||||
|
String group = "testappendgroup";
|
||||||
|
assertFalse(superuser.getShortUserName().equals(username));
|
||||||
|
assertFalse(Arrays.asList(superuser.getGroupNames()).contains(group));
|
||||||
|
UserGroupInformation appenduser = UserGroupInformation
|
||||||
|
.createUserForTesting(username, new String[] { group });
|
||||||
|
|
||||||
|
fs = (DistributedFileSystem) DFSTestUtil.getFileSystemAs(appenduser,
|
||||||
|
conf);
|
||||||
|
|
||||||
|
// create a file
|
||||||
|
Path dir = new Path(root, getClass().getSimpleName());
|
||||||
|
Path foo = new Path(dir, "foo.dat");
|
||||||
|
FSDataOutputStream out = null;
|
||||||
|
int offset = 0;
|
||||||
|
try {
|
||||||
|
out = fs.create(foo);
|
||||||
|
int len = 10 + AppendTestUtil.nextInt(100);
|
||||||
|
out.write(fileContents, offset, len);
|
||||||
|
offset += len;
|
||||||
|
} finally {
|
||||||
|
IOUtils.closeStream(out);
|
||||||
|
}
|
||||||
|
|
||||||
|
// change dir and foo to minimal permissions.
|
||||||
|
fs.setPermission(dir, new FsPermission((short)0100));
|
||||||
|
fs.setPermission(foo, new FsPermission((short)0200));
|
||||||
|
|
||||||
|
// try append, should success
|
||||||
|
out = null;
|
||||||
|
try {
|
||||||
|
out = fs.append(foo,
|
||||||
|
EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null);
|
||||||
|
int len = 10 + AppendTestUtil.nextInt(100);
|
||||||
|
out.write(fileContents, offset, len);
|
||||||
|
offset += len;
|
||||||
|
} finally {
|
||||||
|
IOUtils.closeStream(out);
|
||||||
|
}
|
||||||
|
|
||||||
|
// change dir and foo to all but no write on foo.
|
||||||
|
fs.setPermission(foo, new FsPermission((short)0577));
|
||||||
|
fs.setPermission(dir, new FsPermission((short)0777));
|
||||||
|
|
||||||
|
// try append, should fail
|
||||||
|
out = null;
|
||||||
|
try {
|
||||||
|
out = fs.append(foo,
|
||||||
|
EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null);
|
||||||
|
fail("Expected to have AccessControlException");
|
||||||
|
} catch(AccessControlException ace) {
|
||||||
|
System.out.println("Good: got " + ace);
|
||||||
|
ace.printStackTrace(System.out);
|
||||||
|
} finally {
|
||||||
|
IOUtils.closeStream(out);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
fs.close();
|
||||||
|
cluster.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
//
|
//
|
||||||
// an object that does a bunch of appends to files
|
// an object that does a bunch of appends to files
|
||||||
//
|
//
|
||||||
class Workload extends Thread {
|
class Workload extends Thread {
|
||||||
private final int id;
|
private final int id;
|
||||||
private final MiniDFSCluster cluster;
|
private final MiniDFSCluster cluster;
|
||||||
|
private final boolean appendToNewBlock;
|
||||||
|
|
||||||
Workload(MiniDFSCluster cluster, int threadIndex) {
|
Workload(MiniDFSCluster cluster, int threadIndex, boolean append2) {
|
||||||
id = threadIndex;
|
id = threadIndex;
|
||||||
this.cluster = cluster;
|
this.cluster = cluster;
|
||||||
|
this.appendToNewBlock = append2;
|
||||||
}
|
}
|
||||||
|
|
||||||
// create a bunch of files. Write to them and then verify.
|
// create a bunch of files. Write to them and then verify.
|
||||||
|
@ -255,7 +410,7 @@ public class TestFileAppend2 {
|
||||||
long len = 0;
|
long len = 0;
|
||||||
int sizeToAppend = 0;
|
int sizeToAppend = 0;
|
||||||
try {
|
try {
|
||||||
FileSystem fs = cluster.getFileSystem();
|
DistributedFileSystem fs = cluster.getFileSystem();
|
||||||
|
|
||||||
// add a random number of bytes to file
|
// add a random number of bytes to file
|
||||||
len = fs.getFileStatus(testfile).getLen();
|
len = fs.getFileStatus(testfile).getLen();
|
||||||
|
@ -279,7 +434,9 @@ public class TestFileAppend2 {
|
||||||
" appending " + sizeToAppend + " bytes " +
|
" appending " + sizeToAppend + " bytes " +
|
||||||
" to file " + testfile +
|
" to file " + testfile +
|
||||||
" of size " + len);
|
" of size " + len);
|
||||||
FSDataOutputStream stm = fs.append(testfile);
|
FSDataOutputStream stm = appendToNewBlock ? fs.append(testfile,
|
||||||
|
EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null)
|
||||||
|
: fs.append(testfile);
|
||||||
stm.write(fileContents, (int)len, sizeToAppend);
|
stm.write(fileContents, (int)len, sizeToAppend);
|
||||||
stm.close();
|
stm.close();
|
||||||
|
|
||||||
|
@ -292,7 +449,7 @@ public class TestFileAppend2 {
|
||||||
" expected size " + (len + sizeToAppend) +
|
" expected size " + (len + sizeToAppend) +
|
||||||
" waiting for namenode metadata update.");
|
" waiting for namenode metadata update.");
|
||||||
Thread.sleep(5000);
|
Thread.sleep(5000);
|
||||||
} catch (InterruptedException e) {;}
|
} catch (InterruptedException e) {}
|
||||||
}
|
}
|
||||||
|
|
||||||
assertTrue("File " + testfile + " size is " +
|
assertTrue("File " + testfile + " size is " +
|
||||||
|
@ -300,7 +457,7 @@ public class TestFileAppend2 {
|
||||||
" but expected " + (len + sizeToAppend),
|
" but expected " + (len + sizeToAppend),
|
||||||
fs.getFileStatus(testfile).getLen() == (len + sizeToAppend));
|
fs.getFileStatus(testfile).getLen() == (len + sizeToAppend));
|
||||||
|
|
||||||
AppendTestUtil.checkFullFile(fs, testfile, (int)(len + sizeToAppend),
|
AppendTestUtil.checkFullFile(fs, testfile, (int) (len + sizeToAppend),
|
||||||
fileContents, "Read 2");
|
fileContents, "Read 2");
|
||||||
} catch (Throwable e) {
|
} catch (Throwable e) {
|
||||||
globalStatus = false;
|
globalStatus = false;
|
||||||
|
@ -325,10 +482,8 @@ public class TestFileAppend2 {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test that appends to files at random offsets.
|
* Test that appends to files at random offsets.
|
||||||
* @throws IOException an exception might be thrown
|
|
||||||
*/
|
*/
|
||||||
@Test
|
private void testComplexAppend(boolean appendToNewBlock) throws IOException {
|
||||||
public void testComplexAppend() throws IOException {
|
|
||||||
fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
|
fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
|
||||||
Configuration conf = new HdfsConfiguration();
|
Configuration conf = new HdfsConfiguration();
|
||||||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 2000);
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 2000);
|
||||||
|
@ -360,7 +515,7 @@ public class TestFileAppend2 {
|
||||||
// Create threads and make them run workload concurrently.
|
// Create threads and make them run workload concurrently.
|
||||||
workload = new Workload[numThreads];
|
workload = new Workload[numThreads];
|
||||||
for (int i = 0; i < numThreads; i++) {
|
for (int i = 0; i < numThreads; i++) {
|
||||||
workload[i] = new Workload(cluster, i);
|
workload[i] = new Workload(cluster, i, appendToNewBlock);
|
||||||
workload[i].start();
|
workload[i].start();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -384,4 +539,14 @@ public class TestFileAppend2 {
|
||||||
//
|
//
|
||||||
assertTrue("testComplexAppend Worker encountered exceptions.", globalStatus);
|
assertTrue("testComplexAppend Worker encountered exceptions.", globalStatus);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testComplexAppend() throws IOException {
|
||||||
|
testComplexAppend(false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testComplexAppend2() throws IOException {
|
||||||
|
testComplexAppend(true);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,20 +24,20 @@ import static org.junit.Assert.fail;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.RandomAccessFile;
|
import java.io.RandomAccessFile;
|
||||||
|
import java.util.EnumSet;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
|
import org.apache.hadoop.fs.CreateFlag;
|
||||||
import org.mockito.invocation.InvocationOnMock;
|
import org.mockito.invocation.InvocationOnMock;
|
||||||
import static org.mockito.Mockito.spy;
|
import static org.mockito.Mockito.spy;
|
||||||
import static org.mockito.Mockito.when;
|
import static org.mockito.Mockito.when;
|
||||||
import org.mockito.stubbing.Answer;
|
import org.mockito.stubbing.Answer;
|
||||||
|
|
||||||
import org.apache.commons.logging.impl.Log4JLogger;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hdfs.DFSClient;
|
|
||||||
import org.apache.hadoop.hdfs.DFSClientAdapter;
|
|
||||||
import org.apache.hadoop.hdfs.protocol.Block;
|
import org.apache.hadoop.hdfs.protocol.Block;
|
||||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
|
@ -49,6 +49,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
|
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
|
||||||
import org.apache.log4j.Level;
|
import org.apache.log4j.Level;
|
||||||
import org.junit.AfterClass;
|
import org.junit.AfterClass;
|
||||||
|
import org.junit.Assert;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -116,6 +117,32 @@ public class TestFileAppend3 {
|
||||||
AppendTestUtil.check(fs, p, len1 + len2);
|
AppendTestUtil.check(fs, p, len1 + len2);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTC1ForAppend2() throws Exception {
|
||||||
|
final Path p = new Path("/TC1/foo2");
|
||||||
|
|
||||||
|
//a. Create file and write one block of data. Close file.
|
||||||
|
final int len1 = (int) BLOCK_SIZE;
|
||||||
|
{
|
||||||
|
FSDataOutputStream out = fs.create(p, false, buffersize, REPLICATION,
|
||||||
|
BLOCK_SIZE);
|
||||||
|
AppendTestUtil.write(out, 0, len1);
|
||||||
|
out.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
// Reopen file to append. Append half block of data. Close file.
|
||||||
|
final int len2 = (int) BLOCK_SIZE / 2;
|
||||||
|
{
|
||||||
|
FSDataOutputStream out = fs.append(p,
|
||||||
|
EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null);
|
||||||
|
AppendTestUtil.write(out, len1, len2);
|
||||||
|
out.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
// b. Reopen file and read 1.5 blocks worth of data. Close file.
|
||||||
|
AppendTestUtil.check(fs, p, len1 + len2);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* TC2: Append on non-block boundary.
|
* TC2: Append on non-block boundary.
|
||||||
* @throws IOException an exception might be thrown
|
* @throws IOException an exception might be thrown
|
||||||
|
@ -147,6 +174,40 @@ public class TestFileAppend3 {
|
||||||
AppendTestUtil.check(fs, p, len1 + len2);
|
AppendTestUtil.check(fs, p, len1 + len2);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTC2ForAppend2() throws Exception {
|
||||||
|
final Path p = new Path("/TC2/foo2");
|
||||||
|
|
||||||
|
//a. Create file with one and a half block of data. Close file.
|
||||||
|
final int len1 = (int) (BLOCK_SIZE + BLOCK_SIZE / 2);
|
||||||
|
{
|
||||||
|
FSDataOutputStream out = fs.create(p, false, buffersize, REPLICATION,
|
||||||
|
BLOCK_SIZE);
|
||||||
|
AppendTestUtil.write(out, 0, len1);
|
||||||
|
out.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
AppendTestUtil.check(fs, p, len1);
|
||||||
|
|
||||||
|
// Reopen file to append quarter block of data. Close file.
|
||||||
|
final int len2 = (int) BLOCK_SIZE / 4;
|
||||||
|
{
|
||||||
|
FSDataOutputStream out = fs.append(p, EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK),
|
||||||
|
4096, null);
|
||||||
|
AppendTestUtil.write(out, len1, len2);
|
||||||
|
out.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
// b. Reopen file and read 1.75 blocks of data. Close file.
|
||||||
|
AppendTestUtil.check(fs, p, len1 + len2);
|
||||||
|
List<LocatedBlock> blocks = fs.getClient().getLocatedBlocks(
|
||||||
|
p.toString(), 0L).getLocatedBlocks();
|
||||||
|
Assert.assertEquals(3, blocks.size());
|
||||||
|
Assert.assertEquals(BLOCK_SIZE, blocks.get(0).getBlockSize());
|
||||||
|
Assert.assertEquals(BLOCK_SIZE / 2, blocks.get(1).getBlockSize());
|
||||||
|
Assert.assertEquals(BLOCK_SIZE / 4, blocks.get(2).getBlockSize());
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* TC5: Only one simultaneous append.
|
* TC5: Only one simultaneous append.
|
||||||
* @throws IOException an exception might be thrown
|
* @throws IOException an exception might be thrown
|
||||||
|
@ -174,18 +235,63 @@ public class TestFileAppend3 {
|
||||||
AppendTestUtil.LOG.info("GOOD: got an exception", ioe);
|
AppendTestUtil.LOG.info("GOOD: got an exception", ioe);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
((DistributedFileSystem) AppendTestUtil
|
||||||
|
.createHdfsWithDifferentUsername(conf)).append(p,
|
||||||
|
EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null);
|
||||||
|
fail("This should fail.");
|
||||||
|
} catch(IOException ioe) {
|
||||||
|
AppendTestUtil.LOG.info("GOOD: got an exception", ioe);
|
||||||
|
}
|
||||||
|
|
||||||
//d. On Machine M1, close file.
|
//d. On Machine M1, close file.
|
||||||
out.close();
|
out.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTC5ForAppend2() throws Exception {
|
||||||
|
final Path p = new Path("/TC5/foo2");
|
||||||
|
|
||||||
|
// a. Create file on Machine M1. Write half block to it. Close file.
|
||||||
|
{
|
||||||
|
FSDataOutputStream out = fs.create(p, false, buffersize, REPLICATION,
|
||||||
|
BLOCK_SIZE);
|
||||||
|
AppendTestUtil.write(out, 0, (int)(BLOCK_SIZE/2));
|
||||||
|
out.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
// b. Reopen file in "append" mode on Machine M1.
|
||||||
|
FSDataOutputStream out = fs.append(p, EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK),
|
||||||
|
4096, null);
|
||||||
|
|
||||||
|
// c. On Machine M2, reopen file in "append" mode. This should fail.
|
||||||
|
try {
|
||||||
|
((DistributedFileSystem) AppendTestUtil
|
||||||
|
.createHdfsWithDifferentUsername(conf)).append(p,
|
||||||
|
EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null);
|
||||||
|
fail("This should fail.");
|
||||||
|
} catch(IOException ioe) {
|
||||||
|
AppendTestUtil.LOG.info("GOOD: got an exception", ioe);
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
AppendTestUtil.createHdfsWithDifferentUsername(conf).append(p);
|
||||||
|
fail("This should fail.");
|
||||||
|
} catch(IOException ioe) {
|
||||||
|
AppendTestUtil.LOG.info("GOOD: got an exception", ioe);
|
||||||
|
}
|
||||||
|
|
||||||
|
// d. On Machine M1, close file.
|
||||||
|
out.close();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* TC7: Corrupted replicas are present.
|
* TC7: Corrupted replicas are present.
|
||||||
* @throws IOException an exception might be thrown
|
* @throws IOException an exception might be thrown
|
||||||
*/
|
*/
|
||||||
@Test
|
private void testTC7(boolean appendToNewBlock) throws Exception {
|
||||||
public void testTC7() throws Exception {
|
|
||||||
final short repl = 2;
|
final short repl = 2;
|
||||||
final Path p = new Path("/TC7/foo");
|
final Path p = new Path("/TC7/foo" + (appendToNewBlock ? "0" : "1"));
|
||||||
System.out.println("p=" + p);
|
System.out.println("p=" + p);
|
||||||
|
|
||||||
//a. Create file with replication factor of 2. Write half block of data. Close file.
|
//a. Create file with replication factor of 2. Write half block of data. Close file.
|
||||||
|
@ -219,7 +325,8 @@ public class TestFileAppend3 {
|
||||||
//c. Open file in "append mode". Append a new block worth of data. Close file.
|
//c. Open file in "append mode". Append a new block worth of data. Close file.
|
||||||
final int len2 = (int)BLOCK_SIZE;
|
final int len2 = (int)BLOCK_SIZE;
|
||||||
{
|
{
|
||||||
FSDataOutputStream out = fs.append(p);
|
FSDataOutputStream out = appendToNewBlock ?
|
||||||
|
fs.append(p, EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null) : fs.append(p);
|
||||||
AppendTestUtil.write(out, len1, len2);
|
AppendTestUtil.write(out, len1, len2);
|
||||||
out.close();
|
out.close();
|
||||||
}
|
}
|
||||||
|
@ -228,13 +335,21 @@ public class TestFileAppend3 {
|
||||||
AppendTestUtil.check(fs, p, len1 + len2);
|
AppendTestUtil.check(fs, p, len1 + len2);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTC7() throws Exception {
|
||||||
|
testTC7(false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTC7ForAppend2() throws Exception {
|
||||||
|
testTC7(true);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* TC11: Racing rename
|
* TC11: Racing rename
|
||||||
* @throws IOException an exception might be thrown
|
|
||||||
*/
|
*/
|
||||||
@Test
|
private void testTC11(boolean appendToNewBlock) throws Exception {
|
||||||
public void testTC11() throws Exception {
|
final Path p = new Path("/TC11/foo" + (appendToNewBlock ? "0" : "1"));
|
||||||
final Path p = new Path("/TC11/foo");
|
|
||||||
System.out.println("p=" + p);
|
System.out.println("p=" + p);
|
||||||
|
|
||||||
//a. Create file and write one block of data. Close file.
|
//a. Create file and write one block of data. Close file.
|
||||||
|
@ -246,7 +361,9 @@ public class TestFileAppend3 {
|
||||||
}
|
}
|
||||||
|
|
||||||
//b. Reopen file in "append" mode. Append half block of data.
|
//b. Reopen file in "append" mode. Append half block of data.
|
||||||
FSDataOutputStream out = fs.append(p);
|
FSDataOutputStream out = appendToNewBlock ?
|
||||||
|
fs.append(p, EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null) :
|
||||||
|
fs.append(p);
|
||||||
final int len2 = (int)BLOCK_SIZE/2;
|
final int len2 = (int)BLOCK_SIZE/2;
|
||||||
AppendTestUtil.write(out, len1, len2);
|
AppendTestUtil.write(out, len1, len2);
|
||||||
out.hflush();
|
out.hflush();
|
||||||
|
@ -278,13 +395,21 @@ public class TestFileAppend3 {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTC11() throws Exception {
|
||||||
|
testTC11(false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTC11ForAppend2() throws Exception {
|
||||||
|
testTC11(true);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* TC12: Append to partial CRC chunk
|
* TC12: Append to partial CRC chunk
|
||||||
* @throws IOException an exception might be thrown
|
|
||||||
*/
|
*/
|
||||||
@Test
|
private void testTC12(boolean appendToNewBlock) throws Exception {
|
||||||
public void testTC12() throws Exception {
|
final Path p = new Path("/TC12/foo" + (appendToNewBlock ? "0" : "1"));
|
||||||
final Path p = new Path("/TC12/foo");
|
|
||||||
System.out.println("p=" + p);
|
System.out.println("p=" + p);
|
||||||
|
|
||||||
//a. Create file with a block size of 64KB
|
//a. Create file with a block size of 64KB
|
||||||
|
@ -300,23 +425,43 @@ public class TestFileAppend3 {
|
||||||
//b. Reopen file in "append" mode. Append another 5877 bytes of data. Close file.
|
//b. Reopen file in "append" mode. Append another 5877 bytes of data. Close file.
|
||||||
final int len2 = 5877;
|
final int len2 = 5877;
|
||||||
{
|
{
|
||||||
FSDataOutputStream out = fs.append(p);
|
FSDataOutputStream out = appendToNewBlock ?
|
||||||
|
fs.append(p, EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null) :
|
||||||
|
fs.append(p);
|
||||||
AppendTestUtil.write(out, len1, len2);
|
AppendTestUtil.write(out, len1, len2);
|
||||||
out.close();
|
out.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
//c. Reopen file and read 25687+5877 bytes of data from file. Close file.
|
//c. Reopen file and read 25687+5877 bytes of data from file. Close file.
|
||||||
AppendTestUtil.check(fs, p, len1 + len2);
|
AppendTestUtil.check(fs, p, len1 + len2);
|
||||||
|
if (appendToNewBlock) {
|
||||||
|
LocatedBlocks blks = fs.dfs.getLocatedBlocks(p.toString(), 0);
|
||||||
|
Assert.assertEquals(2, blks.getLocatedBlocks().size());
|
||||||
|
Assert.assertEquals(len1, blks.getLocatedBlocks().get(0).getBlockSize());
|
||||||
|
Assert.assertEquals(len2, blks.getLocatedBlocks().get(1).getBlockSize());
|
||||||
|
AppendTestUtil.check(fs, p, 0, len1);
|
||||||
|
AppendTestUtil.check(fs, p, len1, len2);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Append to a partial CRC chunk and
|
|
||||||
* the first write does not fill up the partial CRC trunk
|
|
||||||
* *
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
@Test
|
@Test
|
||||||
public void testAppendToPartialChunk() throws IOException {
|
public void testTC12() throws Exception {
|
||||||
final Path p = new Path("/partialChunk/foo");
|
testTC12(false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTC12ForAppend2() throws Exception {
|
||||||
|
testTC12(true);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Append to a partial CRC chunk and the first write does not fill up the
|
||||||
|
* partial CRC trunk
|
||||||
|
*/
|
||||||
|
private void testAppendToPartialChunk(boolean appendToNewBlock)
|
||||||
|
throws IOException {
|
||||||
|
final Path p = new Path("/partialChunk/foo"
|
||||||
|
+ (appendToNewBlock ? "0" : "1"));
|
||||||
final int fileLen = 513;
|
final int fileLen = 513;
|
||||||
System.out.println("p=" + p);
|
System.out.println("p=" + p);
|
||||||
|
|
||||||
|
@ -331,7 +476,9 @@ public class TestFileAppend3 {
|
||||||
System.out.println("Wrote 1 byte and closed the file " + p);
|
System.out.println("Wrote 1 byte and closed the file " + p);
|
||||||
|
|
||||||
// append to file
|
// append to file
|
||||||
stm = fs.append(p);
|
stm = appendToNewBlock ?
|
||||||
|
fs.append(p, EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null) :
|
||||||
|
fs.append(p);
|
||||||
// Append to a partial CRC trunk
|
// Append to a partial CRC trunk
|
||||||
stm.write(fileContents, 1, 1);
|
stm.write(fileContents, 1, 1);
|
||||||
stm.hflush();
|
stm.hflush();
|
||||||
|
@ -340,7 +487,9 @@ public class TestFileAppend3 {
|
||||||
System.out.println("Append 1 byte and closed the file " + p);
|
System.out.println("Append 1 byte and closed the file " + p);
|
||||||
|
|
||||||
// write the remainder of the file
|
// write the remainder of the file
|
||||||
stm = fs.append(p);
|
stm = appendToNewBlock ?
|
||||||
|
fs.append(p, EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null) :
|
||||||
|
fs.append(p);
|
||||||
|
|
||||||
// ensure getPos is set to reflect existing size of the file
|
// ensure getPos is set to reflect existing size of the file
|
||||||
assertEquals(2, stm.getPos());
|
assertEquals(2, stm.getPos());
|
||||||
|
@ -439,4 +588,14 @@ public class TestFileAppend3 {
|
||||||
// if append was called with a stale file stat.
|
// if append was called with a stale file stat.
|
||||||
doSmallAppends(file, fs, 20);
|
doSmallAppends(file, fs, 20);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAppendToPartialChunk() throws IOException {
|
||||||
|
testAppendToPartialChunk(false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAppendToPartialChunkforAppend2() throws IOException {
|
||||||
|
testAppendToPartialChunk(true);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -99,10 +99,11 @@ public class TestFileAppendRestart {
|
||||||
// OP_ADD to create file
|
// OP_ADD to create file
|
||||||
// OP_ADD_BLOCK for first block
|
// OP_ADD_BLOCK for first block
|
||||||
// OP_CLOSE to close file
|
// OP_CLOSE to close file
|
||||||
// OP_ADD to reopen file
|
// OP_APPEND to reopen file
|
||||||
// OP_ADD_BLOCK for second block
|
// OP_ADD_BLOCK for second block
|
||||||
// OP_CLOSE to close file
|
// OP_CLOSE to close file
|
||||||
assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
|
assertEquals(1, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
|
||||||
|
assertEquals(1, (int)counts.get(FSEditLogOpCodes.OP_APPEND).held);
|
||||||
assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_ADD_BLOCK).held);
|
assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_ADD_BLOCK).held);
|
||||||
assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_CLOSE).held);
|
assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_CLOSE).held);
|
||||||
|
|
||||||
|
@ -112,13 +113,14 @@ public class TestFileAppendRestart {
|
||||||
// OP_ADD to create file
|
// OP_ADD to create file
|
||||||
// OP_ADD_BLOCK for first block
|
// OP_ADD_BLOCK for first block
|
||||||
// OP_CLOSE to close file
|
// OP_CLOSE to close file
|
||||||
// OP_ADD to re-establish the lease
|
// OP_APPEND to re-establish the lease
|
||||||
// OP_UPDATE_BLOCKS from the updatePipeline call (increments genstamp of last block)
|
// OP_UPDATE_BLOCKS from the updatePipeline call (increments genstamp of last block)
|
||||||
// OP_ADD_BLOCK at the start of the second block
|
// OP_ADD_BLOCK at the start of the second block
|
||||||
// OP_CLOSE to close file
|
// OP_CLOSE to close file
|
||||||
// Total: 2 OP_ADDs, 1 OP_UPDATE_BLOCKS, 2 OP_ADD_BLOCKs, and 2 OP_CLOSEs
|
// Total: 2 OP_ADDs, 1 OP_UPDATE_BLOCKS, 2 OP_ADD_BLOCKs, and 2 OP_CLOSEs
|
||||||
// in addition to the ones above
|
// in addition to the ones above
|
||||||
assertEquals(2+2, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
|
assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
|
||||||
|
assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_APPEND).held);
|
||||||
assertEquals(1, (int)counts.get(FSEditLogOpCodes.OP_UPDATE_BLOCKS).held);
|
assertEquals(1, (int)counts.get(FSEditLogOpCodes.OP_UPDATE_BLOCKS).held);
|
||||||
assertEquals(2+2, (int)counts.get(FSEditLogOpCodes.OP_ADD_BLOCK).held);
|
assertEquals(2+2, (int)counts.get(FSEditLogOpCodes.OP_ADD_BLOCK).held);
|
||||||
assertEquals(2+2, (int)counts.get(FSEditLogOpCodes.OP_CLOSE).held);
|
assertEquals(2+2, (int)counts.get(FSEditLogOpCodes.OP_CLOSE).held);
|
||||||
|
|
|
@ -31,7 +31,9 @@ import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
|
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||||
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.log4j.Level;
|
import org.apache.log4j.Level;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -122,6 +124,65 @@ public class TestHFlush {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test hsync with END_BLOCK flag.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void hSyncEndBlock_00() throws IOException {
|
||||||
|
final int preferredBlockSize = 1024;
|
||||||
|
Configuration conf = new HdfsConfiguration();
|
||||||
|
conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, preferredBlockSize);
|
||||||
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2)
|
||||||
|
.build();
|
||||||
|
DistributedFileSystem fileSystem = cluster.getFileSystem();
|
||||||
|
FSDataOutputStream stm = null;
|
||||||
|
try {
|
||||||
|
Path path = new Path("/" + fName);
|
||||||
|
stm = fileSystem.create(path, true, 4096, (short) 2,
|
||||||
|
AppendTestUtil.BLOCK_SIZE);
|
||||||
|
System.out.println("Created file " + path.toString());
|
||||||
|
((DFSOutputStream) stm.getWrappedStream()).hsync(EnumSet
|
||||||
|
.of(SyncFlag.END_BLOCK));
|
||||||
|
long currentFileLength = fileSystem.getFileStatus(path).getLen();
|
||||||
|
assertEquals(0L, currentFileLength);
|
||||||
|
LocatedBlocks blocks = fileSystem.dfs.getLocatedBlocks(path.toString(), 0);
|
||||||
|
assertEquals(0, blocks.getLocatedBlocks().size());
|
||||||
|
|
||||||
|
// write a block and call hsync(end_block) at the block boundary
|
||||||
|
stm.write(new byte[preferredBlockSize]);
|
||||||
|
((DFSOutputStream) stm.getWrappedStream()).hsync(EnumSet
|
||||||
|
.of(SyncFlag.END_BLOCK));
|
||||||
|
currentFileLength = fileSystem.getFileStatus(path).getLen();
|
||||||
|
assertEquals(preferredBlockSize, currentFileLength);
|
||||||
|
blocks = fileSystem.dfs.getLocatedBlocks(path.toString(), 0);
|
||||||
|
assertEquals(1, blocks.getLocatedBlocks().size());
|
||||||
|
|
||||||
|
// call hsync then call hsync(end_block) immediately
|
||||||
|
stm.write(new byte[preferredBlockSize / 2]);
|
||||||
|
stm.hsync();
|
||||||
|
((DFSOutputStream) stm.getWrappedStream()).hsync(EnumSet
|
||||||
|
.of(SyncFlag.END_BLOCK));
|
||||||
|
currentFileLength = fileSystem.getFileStatus(path).getLen();
|
||||||
|
assertEquals(preferredBlockSize + preferredBlockSize / 2,
|
||||||
|
currentFileLength);
|
||||||
|
blocks = fileSystem.dfs.getLocatedBlocks(path.toString(), 0);
|
||||||
|
assertEquals(2, blocks.getLocatedBlocks().size());
|
||||||
|
|
||||||
|
stm.write(new byte[preferredBlockSize / 4]);
|
||||||
|
stm.hsync();
|
||||||
|
currentFileLength = fileSystem.getFileStatus(path).getLen();
|
||||||
|
assertEquals(preferredBlockSize + preferredBlockSize / 2
|
||||||
|
+ preferredBlockSize / 4, currentFileLength);
|
||||||
|
blocks = fileSystem.dfs.getLocatedBlocks(path.toString(), 0);
|
||||||
|
assertEquals(3, blocks.getLocatedBlocks().size());
|
||||||
|
} finally {
|
||||||
|
IOUtils.cleanup(null, stm, fileSystem);
|
||||||
|
if (cluster != null) {
|
||||||
|
cluster.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The test calls
|
* The test calls
|
||||||
* {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
|
* {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
|
||||||
|
@ -133,6 +194,29 @@ public class TestHFlush {
|
||||||
(short) 2, true, EnumSet.of(SyncFlag.UPDATE_LENGTH));
|
(short) 2, true, EnumSet.of(SyncFlag.UPDATE_LENGTH));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The test calls
|
||||||
|
* {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
|
||||||
|
* while requiring the semantic of {@link SyncFlag#END_BLOCK}.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void hSyncEndBlock_01() throws IOException {
|
||||||
|
doTheJob(new HdfsConfiguration(), fName, AppendTestUtil.BLOCK_SIZE,
|
||||||
|
(short) 2, true, EnumSet.of(SyncFlag.END_BLOCK));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The test calls
|
||||||
|
* {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
|
||||||
|
* while requiring the semantic of {@link SyncFlag#END_BLOCK} and
|
||||||
|
* {@link SyncFlag#UPDATE_LENGTH}.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void hSyncEndBlockAndUpdateLength() throws IOException {
|
||||||
|
doTheJob(new HdfsConfiguration(), fName, AppendTestUtil.BLOCK_SIZE,
|
||||||
|
(short) 2, true, EnumSet.of(SyncFlag.END_BLOCK, SyncFlag.UPDATE_LENGTH));
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The test calls
|
* The test calls
|
||||||
* {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
|
* {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
|
||||||
|
@ -153,6 +237,19 @@ public class TestHFlush {
|
||||||
EnumSet.of(SyncFlag.UPDATE_LENGTH));
|
EnumSet.of(SyncFlag.UPDATE_LENGTH));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void hSyncEndBlock_02() throws IOException {
|
||||||
|
Configuration conf = new HdfsConfiguration();
|
||||||
|
int customPerChecksumSize = 512;
|
||||||
|
int customBlockSize = customPerChecksumSize * 3;
|
||||||
|
// Modify defaul filesystem settings
|
||||||
|
conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
|
||||||
|
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
|
||||||
|
|
||||||
|
doTheJob(conf, fName, customBlockSize, (short) 2, true,
|
||||||
|
EnumSet.of(SyncFlag.END_BLOCK));
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The test calls
|
* The test calls
|
||||||
* {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
|
* {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
|
||||||
|
@ -174,6 +271,19 @@ public class TestHFlush {
|
||||||
EnumSet.of(SyncFlag.UPDATE_LENGTH));
|
EnumSet.of(SyncFlag.UPDATE_LENGTH));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void hSyncEndBlock_03() throws IOException {
|
||||||
|
Configuration conf = new HdfsConfiguration();
|
||||||
|
int customPerChecksumSize = 400;
|
||||||
|
int customBlockSize = customPerChecksumSize * 3;
|
||||||
|
// Modify defaul filesystem settings
|
||||||
|
conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
|
||||||
|
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
|
||||||
|
|
||||||
|
doTheJob(conf, fName, customBlockSize, (short) 2, true,
|
||||||
|
EnumSet.of(SyncFlag.END_BLOCK));
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The method starts new cluster with defined Configuration; creates a file
|
* The method starts new cluster with defined Configuration; creates a file
|
||||||
* with specified block_size and writes 10 equal sections in it; it also calls
|
* with specified block_size and writes 10 equal sections in it; it also calls
|
||||||
|
@ -197,12 +307,13 @@ public class TestHFlush {
|
||||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
||||||
.numDataNodes(replicas).build();
|
.numDataNodes(replicas).build();
|
||||||
// Make sure we work with DFS in order to utilize all its functionality
|
// Make sure we work with DFS in order to utilize all its functionality
|
||||||
DistributedFileSystem fileSystem =
|
DistributedFileSystem fileSystem = cluster.getFileSystem();
|
||||||
cluster.getFileSystem();
|
|
||||||
|
|
||||||
FSDataInputStream is;
|
FSDataInputStream is;
|
||||||
try {
|
try {
|
||||||
Path path = new Path(fileName);
|
Path path = new Path(fileName);
|
||||||
|
final String pathName = new Path(fileSystem.getWorkingDirectory(), path)
|
||||||
|
.toUri().getPath();
|
||||||
FSDataOutputStream stm = fileSystem.create(path, false, 4096, replicas,
|
FSDataOutputStream stm = fileSystem.create(path, false, 4096, replicas,
|
||||||
block_size);
|
block_size);
|
||||||
System.out.println("Created file " + fileName);
|
System.out.println("Created file " + fileName);
|
||||||
|
@ -210,7 +321,8 @@ public class TestHFlush {
|
||||||
int tenth = AppendTestUtil.FILE_SIZE/SECTIONS;
|
int tenth = AppendTestUtil.FILE_SIZE/SECTIONS;
|
||||||
int rounding = AppendTestUtil.FILE_SIZE - tenth * SECTIONS;
|
int rounding = AppendTestUtil.FILE_SIZE - tenth * SECTIONS;
|
||||||
for (int i=0; i<SECTIONS; i++) {
|
for (int i=0; i<SECTIONS; i++) {
|
||||||
System.out.println("Writing " + (tenth * i) + " to " + (tenth * (i+1)) + " section to file " + fileName);
|
System.out.println("Writing " + (tenth * i) + " to "
|
||||||
|
+ (tenth * (i + 1)) + " section to file " + fileName);
|
||||||
// write to the file
|
// write to the file
|
||||||
stm.write(fileContent, tenth * i, tenth);
|
stm.write(fileContent, tenth * i, tenth);
|
||||||
|
|
||||||
|
@ -227,7 +339,11 @@ public class TestHFlush {
|
||||||
assertEquals(
|
assertEquals(
|
||||||
"File size doesn't match for hsync/hflush with updating the length",
|
"File size doesn't match for hsync/hflush with updating the length",
|
||||||
tenth * (i + 1), currentFileLength);
|
tenth * (i + 1), currentFileLength);
|
||||||
|
} else if (isSync && syncFlags.contains(SyncFlag.END_BLOCK)) {
|
||||||
|
LocatedBlocks blocks = fileSystem.dfs.getLocatedBlocks(pathName, 0);
|
||||||
|
assertEquals(i + 1, blocks.getLocatedBlocks().size());
|
||||||
}
|
}
|
||||||
|
|
||||||
byte [] toRead = new byte[tenth];
|
byte [] toRead = new byte[tenth];
|
||||||
byte [] expected = new byte[tenth];
|
byte [] expected = new byte[tenth];
|
||||||
System.arraycopy(fileContent, tenth * i, expected, 0, tenth);
|
System.arraycopy(fileContent, tenth * i, expected, 0, tenth);
|
||||||
|
|
|
@ -22,8 +22,10 @@ import static org.junit.Assert.assertTrue;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.RandomAccessFile;
|
import java.io.RandomAccessFile;
|
||||||
|
import java.util.EnumSet;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.CreateFlag;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
@ -40,6 +42,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
||||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.TestInterDatanodeProtocol;
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.TestInterDatanodeProtocol;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
|
import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
||||||
|
import org.apache.hadoop.io.EnumSetWritable;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -124,7 +127,8 @@ public class TestLeaseRecovery {
|
||||||
}
|
}
|
||||||
|
|
||||||
DataNode.LOG.info("dfs.dfs.clientName=" + dfs.dfs.clientName);
|
DataNode.LOG.info("dfs.dfs.clientName=" + dfs.dfs.clientName);
|
||||||
cluster.getNameNodeRpc().append(filestr, dfs.dfs.clientName);
|
cluster.getNameNodeRpc().append(filestr, dfs.dfs.clientName,
|
||||||
|
new EnumSetWritable<>(EnumSet.of(CreateFlag.APPEND)));
|
||||||
|
|
||||||
// expire lease to trigger block recovery.
|
// expire lease to trigger block recovery.
|
||||||
waitLeaseRecovery(cluster);
|
waitLeaseRecovery(cluster);
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
|
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
|
||||||
import com.google.common.util.concurrent.Uninterruptibles;
|
import com.google.common.util.concurrent.Uninterruptibles;
|
||||||
|
import org.apache.hadoop.fs.CreateFlag;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||||
|
@ -28,6 +29,7 @@ import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.EnumSet;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
|
@ -234,7 +236,8 @@ public class TestLazyPersistFiles extends LazyPersistTestCase {
|
||||||
makeTestFile(path, BLOCK_SIZE, true);
|
makeTestFile(path, BLOCK_SIZE, true);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
client.append(path.toString(), BUFFER_LENGTH, null, null).close();
|
client.append(path.toString(), BUFFER_LENGTH,
|
||||||
|
EnumSet.of(CreateFlag.APPEND), null, null).close();
|
||||||
fail("Append to LazyPersist file did not fail as expected");
|
fail("Append to LazyPersist file did not fail as expected");
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
LOG.info("Got expected exception ", t);
|
LOG.info("Got expected exception ", t);
|
||||||
|
|
|
@ -40,9 +40,12 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
|
import org.apache.hadoop.ipc.RemoteException;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -99,7 +102,7 @@ public class TestHDFSConcat {
|
||||||
HdfsFileStatus fStatus;
|
HdfsFileStatus fStatus;
|
||||||
FSDataInputStream stm;
|
FSDataInputStream stm;
|
||||||
|
|
||||||
String trg = new String("/trg");
|
String trg = "/trg";
|
||||||
Path trgPath = new Path(trg);
|
Path trgPath = new Path(trg);
|
||||||
DFSTestUtil.createFile(dfs, trgPath, fileLen, REPL_FACTOR, 1);
|
DFSTestUtil.createFile(dfs, trgPath, fileLen, REPL_FACTOR, 1);
|
||||||
fStatus = nn.getFileInfo(trg);
|
fStatus = nn.getFileInfo(trg);
|
||||||
|
@ -112,7 +115,7 @@ public class TestHDFSConcat {
|
||||||
long [] lens = new long [numFiles];
|
long [] lens = new long [numFiles];
|
||||||
|
|
||||||
|
|
||||||
int i = 0;
|
int i;
|
||||||
for(i=0; i<files.length; i++) {
|
for(i=0; i<files.length; i++) {
|
||||||
files[i] = new Path("/file"+i);
|
files[i] = new Path("/file"+i);
|
||||||
Path path = files[i];
|
Path path = files[i];
|
||||||
|
@ -385,6 +388,75 @@ public class TestHDFSConcat {
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
// exspected
|
// exspected
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* make sure we update the quota correctly after concat
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testConcatWithQuotaDecrease() throws IOException {
|
||||||
|
final short srcRepl = 3; // note this is different with REPL_FACTOR
|
||||||
|
final int srcNum = 10;
|
||||||
|
final Path foo = new Path("/foo");
|
||||||
|
final Path[] srcs = new Path[srcNum];
|
||||||
|
final Path target = new Path(foo, "target");
|
||||||
|
DFSTestUtil.createFile(dfs, target, blockSize, REPL_FACTOR, 0L);
|
||||||
|
|
||||||
|
dfs.setQuota(foo, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
|
||||||
|
|
||||||
|
for (int i = 0; i < srcNum; i++) {
|
||||||
|
srcs[i] = new Path(foo, "src" + i);
|
||||||
|
DFSTestUtil.createFile(dfs, srcs[i], blockSize * 2, srcRepl, 0L);
|
||||||
|
}
|
||||||
|
|
||||||
|
ContentSummary summary = dfs.getContentSummary(foo);
|
||||||
|
Assert.assertEquals(11, summary.getFileCount());
|
||||||
|
Assert.assertEquals(blockSize * REPL_FACTOR +
|
||||||
|
blockSize * 2 * srcRepl * srcNum, summary.getSpaceConsumed());
|
||||||
|
|
||||||
|
dfs.concat(target, srcs);
|
||||||
|
summary = dfs.getContentSummary(foo);
|
||||||
|
Assert.assertEquals(1, summary.getFileCount());
|
||||||
|
Assert.assertEquals(
|
||||||
|
blockSize * REPL_FACTOR + blockSize * 2 * REPL_FACTOR * srcNum,
|
||||||
|
summary.getSpaceConsumed());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testConcatWithQuotaIncrease() throws IOException {
|
||||||
|
final short repl = 3;
|
||||||
|
final int srcNum = 10;
|
||||||
|
final Path foo = new Path("/foo");
|
||||||
|
final Path bar = new Path(foo, "bar");
|
||||||
|
final Path[] srcs = new Path[srcNum];
|
||||||
|
final Path target = new Path(bar, "target");
|
||||||
|
DFSTestUtil.createFile(dfs, target, blockSize, repl, 0L);
|
||||||
|
|
||||||
|
final long dsQuota = blockSize * repl + blockSize * srcNum * REPL_FACTOR;
|
||||||
|
dfs.setQuota(foo, Long.MAX_VALUE - 1, dsQuota);
|
||||||
|
|
||||||
|
for (int i = 0; i < srcNum; i++) {
|
||||||
|
srcs[i] = new Path(bar, "src" + i);
|
||||||
|
DFSTestUtil.createFile(dfs, srcs[i], blockSize, REPL_FACTOR, 0L);
|
||||||
|
}
|
||||||
|
|
||||||
|
ContentSummary summary = dfs.getContentSummary(bar);
|
||||||
|
Assert.assertEquals(11, summary.getFileCount());
|
||||||
|
Assert.assertEquals(dsQuota, summary.getSpaceConsumed());
|
||||||
|
|
||||||
|
try {
|
||||||
|
dfs.concat(target, srcs);
|
||||||
|
fail("QuotaExceededException expected");
|
||||||
|
} catch (RemoteException e) {
|
||||||
|
Assert.assertTrue(
|
||||||
|
e.unwrapRemoteException() instanceof QuotaExceededException);
|
||||||
|
}
|
||||||
|
|
||||||
|
dfs.setQuota(foo, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
|
||||||
|
dfs.concat(target, srcs);
|
||||||
|
summary = dfs.getContentSummary(bar);
|
||||||
|
Assert.assertEquals(1, summary.getFileCount());
|
||||||
|
Assert.assertEquals(blockSize * repl * (srcNum + 1),
|
||||||
|
summary.getSpaceConsumed());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -232,14 +232,18 @@ public class TestNamenodeRetryCache {
|
||||||
|
|
||||||
// Retried append requests succeed
|
// Retried append requests succeed
|
||||||
newCall();
|
newCall();
|
||||||
LastBlockWithStatus b = nnRpc.append(src, "holder");
|
LastBlockWithStatus b = nnRpc.append(src, "holder",
|
||||||
Assert.assertEquals(b, nnRpc.append(src, "holder"));
|
new EnumSetWritable<>(EnumSet.of(CreateFlag.APPEND)));
|
||||||
Assert.assertEquals(b, nnRpc.append(src, "holder"));
|
Assert.assertEquals(b, nnRpc.append(src, "holder",
|
||||||
|
new EnumSetWritable<>(EnumSet.of(CreateFlag.APPEND))));
|
||||||
|
Assert.assertEquals(b, nnRpc.append(src, "holder",
|
||||||
|
new EnumSetWritable<>(EnumSet.of(CreateFlag.APPEND))));
|
||||||
|
|
||||||
// non-retried call fails
|
// non-retried call fails
|
||||||
newCall();
|
newCall();
|
||||||
try {
|
try {
|
||||||
nnRpc.append(src, "holder");
|
nnRpc.append(src, "holder",
|
||||||
|
new EnumSetWritable<>(EnumSet.of(CreateFlag.APPEND)));
|
||||||
Assert.fail("testAppend - expected exception is not thrown");
|
Assert.fail("testAppend - expected exception is not thrown");
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
// Expected
|
// Expected
|
||||||
|
@ -409,7 +413,7 @@ public class TestNamenodeRetryCache {
|
||||||
|
|
||||||
LightWeightCache<CacheEntry, CacheEntry> cacheSet =
|
LightWeightCache<CacheEntry, CacheEntry> cacheSet =
|
||||||
(LightWeightCache<CacheEntry, CacheEntry>) namesystem.getRetryCache().getCacheSet();
|
(LightWeightCache<CacheEntry, CacheEntry>) namesystem.getRetryCache().getCacheSet();
|
||||||
assertEquals(24, cacheSet.size());
|
assertEquals(25, cacheSet.size());
|
||||||
|
|
||||||
Map<CacheEntry, CacheEntry> oldEntries =
|
Map<CacheEntry, CacheEntry> oldEntries =
|
||||||
new HashMap<CacheEntry, CacheEntry>();
|
new HashMap<CacheEntry, CacheEntry>();
|
||||||
|
@ -428,7 +432,7 @@ public class TestNamenodeRetryCache {
|
||||||
assertTrue(namesystem.hasRetryCache());
|
assertTrue(namesystem.hasRetryCache());
|
||||||
cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) namesystem
|
cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) namesystem
|
||||||
.getRetryCache().getCacheSet();
|
.getRetryCache().getCacheSet();
|
||||||
assertEquals(24, cacheSet.size());
|
assertEquals(25, cacheSet.size());
|
||||||
iter = cacheSet.iterator();
|
iter = cacheSet.iterator();
|
||||||
while (iter.hasNext()) {
|
while (iter.hasNext()) {
|
||||||
CacheEntry entry = iter.next();
|
CacheEntry entry = iter.next();
|
||||||
|
|
|
@ -163,7 +163,7 @@ public class TestRetryCacheWithHA {
|
||||||
FSNamesystem fsn0 = cluster.getNamesystem(0);
|
FSNamesystem fsn0 = cluster.getNamesystem(0);
|
||||||
LightWeightCache<CacheEntry, CacheEntry> cacheSet =
|
LightWeightCache<CacheEntry, CacheEntry> cacheSet =
|
||||||
(LightWeightCache<CacheEntry, CacheEntry>) fsn0.getRetryCache().getCacheSet();
|
(LightWeightCache<CacheEntry, CacheEntry>) fsn0.getRetryCache().getCacheSet();
|
||||||
assertEquals(24, cacheSet.size());
|
assertEquals(25, cacheSet.size());
|
||||||
|
|
||||||
Map<CacheEntry, CacheEntry> oldEntries =
|
Map<CacheEntry, CacheEntry> oldEntries =
|
||||||
new HashMap<CacheEntry, CacheEntry>();
|
new HashMap<CacheEntry, CacheEntry>();
|
||||||
|
@ -184,7 +184,7 @@ public class TestRetryCacheWithHA {
|
||||||
FSNamesystem fsn1 = cluster.getNamesystem(1);
|
FSNamesystem fsn1 = cluster.getNamesystem(1);
|
||||||
cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) fsn1
|
cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) fsn1
|
||||||
.getRetryCache().getCacheSet();
|
.getRetryCache().getCacheSet();
|
||||||
assertEquals(24, cacheSet.size());
|
assertEquals(25, cacheSet.size());
|
||||||
iter = cacheSet.iterator();
|
iter = cacheSet.iterator();
|
||||||
while (iter.hasNext()) {
|
while (iter.hasNext()) {
|
||||||
CacheEntry entry = iter.next();
|
CacheEntry entry = iter.next();
|
||||||
|
@ -438,7 +438,8 @@ public class TestRetryCacheWithHA {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void invoke() throws Exception {
|
void invoke() throws Exception {
|
||||||
lbk = client.getNamenode().append(fileName, client.getClientName());
|
lbk = client.getNamenode().append(fileName, client.getClientName(),
|
||||||
|
new EnumSetWritable<>(EnumSet.of(CreateFlag.APPEND)));
|
||||||
}
|
}
|
||||||
|
|
||||||
// check if the inode of the file is under construction
|
// check if the inode of the file is under construction
|
||||||
|
@ -701,7 +702,8 @@ public class TestRetryCacheWithHA {
|
||||||
final Path filePath = new Path(file);
|
final Path filePath = new Path(file);
|
||||||
DFSTestUtil.createFile(dfs, filePath, BlockSize, DataNodes, 0);
|
DFSTestUtil.createFile(dfs, filePath, BlockSize, DataNodes, 0);
|
||||||
// append to the file and leave the last block under construction
|
// append to the file and leave the last block under construction
|
||||||
out = this.client.append(file, BlockSize, null, null);
|
out = this.client.append(file, BlockSize, EnumSet.of(CreateFlag.APPEND),
|
||||||
|
null, null);
|
||||||
byte[] appendContent = new byte[100];
|
byte[] appendContent = new byte[100];
|
||||||
new Random().nextBytes(appendContent);
|
new Random().nextBytes(appendContent);
|
||||||
out.write(appendContent);
|
out.write(appendContent);
|
||||||
|
|
Binary file not shown.
File diff suppressed because it is too large
Load Diff
Loading…
Reference in New Issue