瀏覽代碼

HDFS-8468. 2 RPC calls for every file read in DFSClient#open(..) resulting in double Audit log entries (Contributed by Vinayakumar B)

Vinayakumar B 10 年之前
父節點
當前提交
0b7af27b9a

+ 24 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java

@@ -24,6 +24,7 @@ import java.util.Comparator;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 
 /**
  * Collection of blocks with their locations and the file length.
@@ -37,6 +38,8 @@ public class LocatedBlocks {
   private final LocatedBlock lastLocatedBlock;
   private final boolean isLastBlockComplete;
   private final FileEncryptionInfo fileEncryptionInfo;
+  private final ECSchema ecSchema;
+  private final int stripeCellSize;
 
   public LocatedBlocks() {
     fileLength = 0;
@@ -45,17 +48,22 @@ public class LocatedBlocks {
     lastLocatedBlock = null;
     isLastBlockComplete = false;
     fileEncryptionInfo = null;
+    ecSchema = null;
+    stripeCellSize = 0;
   }
 
   public LocatedBlocks(long flength, boolean isUnderConstuction,
-    List<LocatedBlock> blks, LocatedBlock lastBlock,
-    boolean isLastBlockCompleted, FileEncryptionInfo feInfo) {
+      List<LocatedBlock> blks, LocatedBlock lastBlock,
+      boolean isLastBlockCompleted, FileEncryptionInfo feInfo,
+      ECSchema ecSchema, int stripeCellSize) {
     fileLength = flength;
     blocks = blks;
     underConstruction = isUnderConstuction;
     this.lastLocatedBlock = lastBlock;
     this.isLastBlockComplete = isLastBlockCompleted;
     this.fileEncryptionInfo = feInfo;
+    this.ecSchema = ecSchema;
+    this.stripeCellSize = stripeCellSize;
   }
 
   /**
@@ -111,6 +119,20 @@ public class LocatedBlocks {
     return fileEncryptionInfo;
   }
 
+  /**
+   * @return The ECSchema for ErasureCoded file, null otherwise.
+   */
+  public ECSchema getECSchema() {
+    return ecSchema;
+  }
+
+  /**
+   * @return Stripe Cell size for ErasureCoded file, 0 otherwise.
+   */
+  public int getStripeCellSize() {
+    return stripeCellSize;
+  }
+
   /**
    * Find block containing specified offset.
    *

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java

@@ -479,7 +479,7 @@ class JsonUtilClient {
         (Map<?, ?>) m.get("lastLocatedBlock"));
     final boolean isLastBlockComplete = (Boolean)m.get("isLastBlockComplete");
     return new LocatedBlocks(fileLength, isUnderConstruction, locatedBlocks,
-        lastLocatedBlock, isLastBlockComplete, null);
+        lastLocatedBlock, isLastBlockComplete, null, null, 0);
   }
 
 }

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt

@@ -323,3 +323,6 @@
 
     HDFS-8253. DFSStripedOutputStream.closeThreads releases cellBuffers
     multiple times.  (Kai Sasaki via szetszwo)
+
+    HDFS-8468. 2 RPC calls for every file read in DFSClient#open(..) resulting in
+    double Audit log entries (vinayakumarb)

+ 7 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -1192,15 +1192,17 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     //    Get block info from namenode
     TraceScope scope = getPathTraceScope("newDFSInputStream", src);
     try {
-      HdfsFileStatus fileInfo = getFileInfo(src);
-      if (fileInfo != null) {
-        ECSchema schema = fileInfo.getECSchema();
+      LocatedBlocks locatedBlocks = getLocatedBlocks(src, 0);
+      if (locatedBlocks != null) {
+        ECSchema schema = locatedBlocks.getECSchema();
         if (schema != null) {
           return new DFSStripedInputStream(this, src, verifyChecksum, schema,
-              fileInfo.getStripeCellSize());
+              locatedBlocks.getStripeCellSize(), locatedBlocks);
         }
+        return new DFSInputStream(this, src, verifyChecksum, locatedBlocks);
+      } else {
+        throw new IOException("Cannot open filename " + src);
       }
-      return new DFSInputStream(this, src, verifyChecksum);
     } finally {
       scope.close();
     }

+ 18 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -253,24 +253,28 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     deadNodes.put(dnInfo, dnInfo);
   }
   
-  DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum
-                 ) throws IOException, UnresolvedLinkException {
+  DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
+      LocatedBlocks locatedBlocks) throws IOException, UnresolvedLinkException {
     this.dfsClient = dfsClient;
     this.verifyChecksum = verifyChecksum;
     this.src = src;
     synchronized (infoLock) {
       this.cachingStrategy = dfsClient.getDefaultReadCachingStrategy();
     }
-    openInfo();
+    this.locatedBlocks = locatedBlocks;
+    openInfo(false);
   }
 
   /**
    * Grab the open-file info from namenode
+   * @param refreshLocatedBlocks whether to re-fetch locatedblocks
    */
-  void openInfo() throws IOException, UnresolvedLinkException {
+  void openInfo(boolean refreshLocatedBlocks) throws IOException,
+      UnresolvedLinkException {
     final DfsClientConf conf = dfsClient.getConf();
     synchronized(infoLock) {
-      lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
+      lastBlockBeingWrittenLength =
+          fetchLocatedBlocksAndGetLastBlockLength(refreshLocatedBlocks);
       int retriesForLastBlockLength = conf.getRetryTimesForGetLastBlockLength();
       while (retriesForLastBlockLength > 0) {
         // Getting last block length as -1 is a special case. When cluster
@@ -282,7 +286,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
               + "Datanodes might not have reported blocks completely."
               + " Will retry for " + retriesForLastBlockLength + " times");
           waitFor(conf.getRetryIntervalForGetLastBlockLength());
-          lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
+          lastBlockBeingWrittenLength =
+              fetchLocatedBlocksAndGetLastBlockLength(true);
         } else {
           break;
         }
@@ -303,8 +308,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     }
   }
 
-  private long fetchLocatedBlocksAndGetLastBlockLength() throws IOException {
-    final LocatedBlocks newInfo = dfsClient.getLocatedBlocks(src, 0);
+  private long fetchLocatedBlocksAndGetLastBlockLength(boolean refresh)
+      throws IOException {
+    LocatedBlocks newInfo = locatedBlocks;
+    if (locatedBlocks == null || refresh) {
+      newInfo = dfsClient.getLocatedBlocks(src, 0);
+    }
     if (DFSClient.LOG.isDebugEnabled()) {
       DFSClient.LOG.debug("newInfo = " + newInfo);
     }
@@ -1015,7 +1024,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         } catch (InterruptedException iex) {
         }
         deadNodes.clear(); //2nd option is to remove only nodes[blockId]
-        openInfo();
+        openInfo(true);
         block = refreshLocatedBlock(block);
         failures++;
       }

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
@@ -139,9 +140,10 @@ public class DFSStripedInputStream extends DFSInputStream {
   private final CompletionService<Void> readingService;
   private ReaderRetryPolicy retry;
 
-  DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
-      ECSchema schema, int cellSize) throws IOException {
-    super(dfsClient, src, verifyChecksum);
+  DFSStripedInputStream(DFSClient dfsClient, String src,
+      boolean verifyChecksum, ECSchema schema, int cellSize,
+      LocatedBlocks locatedBlocks) throws IOException {
+    super(dfsClient, src, verifyChecksum, locatedBlocks);
 
     assert schema != null;
     this.schema = schema;

+ 9 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -1338,8 +1338,9 @@ public class PBHelper {
         lb.hasLastBlock() ?
             PBHelper.convertLocatedBlockProto(lb.getLastBlock()) : null,
         lb.getIsLastBlockComplete(),
-        lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) :
-            null);
+        lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) : null,
+        lb.hasECSchema() ? convertECSchema(lb.getECSchema()) : null,
+        lb.hasStripeCellSize() ? lb.getStripeCellSize() : 0);
   }
   
   public static LocatedBlocksProto convert(LocatedBlocks lb) {
@@ -1355,6 +1356,12 @@ public class PBHelper {
     if (lb.getFileEncryptionInfo() != null) {
       builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo()));
     }
+    if (lb.getECSchema() != null) {
+      builder.setECSchema(convertECSchema(lb.getECSchema()));
+    }
+    if (lb.getStripeCellSize() != 0) {
+      builder.setStripeCellSize(lb.getStripeCellSize());
+    }
     return builder.setFileLength(lb.getFileLength())
         .setUnderConstruction(lb.isUnderConstruction())
         .addAllBlocks(PBHelper.convertLocatedBlocks2(lb.getLocatedBlocks()))

+ 9 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -938,14 +938,18 @@ public class BlockManager {
       final long fileSizeExcludeBlocksUnderConstruction,
       final boolean isFileUnderConstruction, final long offset,
       final long length, final boolean needBlockToken,
-      final boolean inSnapshot, FileEncryptionInfo feInfo)
+      final boolean inSnapshot, FileEncryptionInfo feInfo,
+      ErasureCodingZone ecZone)
       throws IOException {
     assert namesystem.hasReadLock();
+    final ECSchema schema = ecZone != null ? ecZone.getSchema() : null;
+    final int cellSize = ecZone != null ? ecZone.getCellSize() : 0;
     if (blocks == null) {
       return null;
     } else if (blocks.length == 0) {
       return new LocatedBlocks(0, isFileUnderConstruction,
-          Collections.<LocatedBlock>emptyList(), null, false, feInfo);
+          Collections.<LocatedBlock> emptyList(), null, false, feInfo, schema,
+          cellSize);
     } else {
       if (LOG.isDebugEnabled()) {
         LOG.debug("blocks = " + java.util.Arrays.asList(blocks));
@@ -968,9 +972,9 @@ public class BlockManager {
             fileSizeExcludeBlocksUnderConstruction, mode);
         isComplete = true;
       }
-      return new LocatedBlocks(
-          fileSizeExcludeBlocksUnderConstruction, isFileUnderConstruction,
-          locatedblocks, lastlb, isComplete, feInfo);
+      return new LocatedBlocks(fileSizeExcludeBlocksUnderConstruction,
+          isFileUnderConstruction, locatedblocks, lastlb, isComplete, feInfo,
+          schema, cellSize);
     }
   }
 

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

@@ -445,6 +445,8 @@ class FSDirStatAndListingOp {
     final boolean isEncrypted;
     final FileEncryptionInfo feInfo = isRawPath ? null :
         fsd.getFileEncryptionInfo(node, snapshot, iip);
+    final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
+        fsd.getFSNamesystem(), iip);
     if (node.isFile()) {
       final INodeFile fileNode = node.asFile();
       size = fileNode.computeFileSize(snapshot);
@@ -458,7 +460,7 @@ class FSDirStatAndListingOp {
 
       loc = fsd.getFSNamesystem().getBlockManager().createLocatedBlocks(
           fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size, false,
-          inSnapshot, feInfo);
+          inSnapshot, feInfo, ecZone);
       if (loc == null) {
         loc = new LocatedBlocks();
       }
@@ -469,8 +471,6 @@ class FSDirStatAndListingOp {
     }
     int childrenNum = node.isDirectory() ?
         node.asDirectory().getChildrenNum(snapshot) : 0;
-    final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
-        fsd.getFSNamesystem(), iip);
     final ECSchema schema = ecZone != null ? ecZone.getSchema() : null;
     final int cellSize = ecZone != null ? ecZone.getCellSize() : 0;
 

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

@@ -1873,10 +1873,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final FileEncryptionInfo feInfo =
         FSDirectory.isReservedRawName(srcArg) ? null
             : dir.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip);
+    final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
+        this, iip);
 
     final LocatedBlocks blocks = blockManager.createLocatedBlocks(
         inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset,
-        length, needBlockToken, iip.isSnapshot(), feInfo);
+        length, needBlockToken, iip.isSnapshot(), feInfo, ecZone);
 
     // Set caching information for the located blocks.
     for (LocatedBlock lb : blocks.getLocatedBlocks()) {

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto

@@ -301,6 +301,10 @@ message LocatedBlocksProto {
   optional LocatedBlockProto lastBlock = 4;
   required bool isLastBlockComplete = 5;
   optional FileEncryptionInfoProto fileEncryptionInfo = 6;
+
+  // Optional field for erasure coding
+  optional ECSchemaProto eCSchema = 7;
+  optional uint32 stripeCellSize = 8;
 }
 
 /**

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java

@@ -343,7 +343,7 @@ public class TestDFSClientRetries {
       // we're starting a new operation on the user level.
       doAnswer(new FailNTimesAnswer(preSpyNN, maxBlockAcquires))
         .when(spyNN).getBlockLocations(anyString(), anyLong(), anyLong());
-      is.openInfo();
+      is.openInfo(true);
       // Seek to beginning forces a reopen of the BlockReader - otherwise it'll
       // just keep reading on the existing stream and the fact that we've poisoned
       // the block info won't do anything.
@@ -496,7 +496,7 @@ public class TestDFSClientRetries {
       badBlocks.add(badLocatedBlock);
       return new LocatedBlocks(goodBlockList.getFileLength(), false,
                                badBlocks, null, true,
-                               null);
+                               null, null, 0);
     }
   }
   

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java

@@ -101,7 +101,7 @@ public class TestDFSStripedInputStream {
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
         filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks);
     final DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(),
-        filePath.toString(), false, schema, CELLSIZE);
+        filePath.toString(), false, schema, CELLSIZE, null);
 
     List<LocatedBlock> lbList = lbs.getLocatedBlocks();
     for (LocatedBlock aLbList : lbList) {
@@ -153,7 +153,7 @@ public class TestDFSStripedInputStream {
       }
     }
     DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(),
-        filePath.toString(), false, schema, CELLSIZE);
+        filePath.toString(), false, schema, CELLSIZE, null);
 
     int[] startOffsets = {0, 1, CELLSIZE - 102, CELLSIZE, CELLSIZE + 102,
         CELLSIZE*DATA_BLK_NUM, CELLSIZE*DATA_BLK_NUM + 102,
@@ -195,7 +195,7 @@ public class TestDFSStripedInputStream {
     }
     DFSStripedInputStream in =
         new DFSStripedInputStream(fs.getClient(), filePath.toString(), false,
-            ErasureCodingSchemaManager.getSystemDefaultSchema(), CELLSIZE);
+            ErasureCodingSchemaManager.getSystemDefaultSchema(), CELLSIZE, null);
     int readSize = BLOCK_GROUP_SIZE;
     byte[] readBuffer = new byte[readSize];
     byte[] expected = new byte[readSize];
@@ -293,7 +293,7 @@ public class TestDFSStripedInputStream {
 
     DFSStripedInputStream in =
         new DFSStripedInputStream(fs.getClient(), filePath.toString(),
-            false, schema, CELLSIZE);
+            false, schema, CELLSIZE, null);
 
     byte[] expected = new byte[fileSize];
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java

@@ -110,7 +110,7 @@ public class TestDFSUtil {
     l2.setCorrupt(true);
 
     List<LocatedBlock> ls = Arrays.asList(l1, l2);
-    LocatedBlocks lbs = new LocatedBlocks(10, false, ls, l2, true, null);
+    LocatedBlocks lbs = new LocatedBlocks(10, false, ls, l2, true, null, null, 0);
 
     BlockLocation[] bs = DFSUtilClient.locatedBlocks2Locations(lbs);