You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ji...@apache.org on 2015/07/15 01:07:43 UTC

[06/11] hadoop git commit: HDFS-8742. Inotify: Support event for OP_TRUNCATE. Contributed by Surendra Singh Lilhore.

HDFS-8742. Inotify: Support event for OP_TRUNCATE. Contributed by Surendra Singh Lilhore.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/979c9ca2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/979c9ca2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/979c9ca2

Branch: refs/heads/YARN-1197
Commit: 979c9ca2ca89e99dc7165abfa29c78d66de43d9a
Parents: 59388a8
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Jul 15 04:41:54 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Jul 15 04:42:08 2015 +0900

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/inotify/Event.java   | 37 +++++++++++++++++++-
 .../src/main/proto/inotify.proto                |  7 ++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 17 +++++++++
 .../namenode/InotifyFSEditLogOpTranslator.java  |  4 +++
 .../hdfs/TestDFSInotifyEventInputStream.java    | 17 ++++++++-
 6 files changed, 83 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/979c9ca2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java
index 53eefa0..dee17a9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java
@@ -35,7 +35,7 @@ import java.util.List;
 @InterfaceStability.Unstable
 public abstract class Event {
   public static enum EventType {
-    CREATE, CLOSE, APPEND, RENAME, METADATA, UNLINK
+    CREATE, CLOSE, APPEND, RENAME, METADATA, UNLINK, TRUNCATE
   }
 
   private EventType eventType;
@@ -542,4 +542,39 @@ public abstract class Event {
       return timestamp;
     }
   }
+
+  /**
+   * Sent when a file is truncated.
+   */
+  public static class TruncateEvent extends Event {
+    private String path;
+    private long fileSize;
+    private long timestamp;
+
+
+    public TruncateEvent(String path, long fileSize, long timestamp) {
+      super(EventType.TRUNCATE);
+      this.path = path;
+      this.fileSize = fileSize;
+      this.timestamp = timestamp;
+    }
+
+    public String getPath() {
+      return path;
+    }
+
+    /**
+     * The size of the truncated file in bytes.
+     */
+    public long getFileSize() {
+      return fileSize;
+    }
+
+    /**
+     * The time when this event occurred, in milliseconds since the epoch.
+     */
+    public long getTimestamp() {
+      return timestamp;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/979c9ca2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/inotify.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/inotify.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/inotify.proto
index 5b78fe6..5339902 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/inotify.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/inotify.proto
@@ -41,6 +41,7 @@ enum EventType {
   EVENT_RENAME = 0x3;
   EVENT_METADATA = 0x4;
   EVENT_UNLINK = 0x5;
+  EVENT_TRUNCATE = 0x6;
 }
 
 message EventProto {
@@ -87,6 +88,12 @@ message CloseEventProto {
   required int64 timestamp = 3;
 }
 
+message TruncateEventProto {
+  required string path = 1;
+  required int64 fileSize = 2;
+  required int64 timestamp = 3;
+}
+
 message AppendEventProto {
   required string path = 1;
   optional bool newBlock = 2 [default = false];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/979c9ca2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 14f3403..20bdef0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -719,6 +719,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8541. Mover should exit with NO_MOVE_PROGRESS if there is no move
     progress.  (Surendra Singh Lilhore via szetszwo)
 
+    HDFS-8742. Inotify: Support event for OP_TRUNCATE.
+    (Surendra Singh Lilhore via aajisaka)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/979c9ca2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 32d9614..4ca5b26 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -2685,6 +2685,12 @@ public class PBHelper {
                   .timestamp(unlink.getTimestamp())
                   .build());
             break;
+          case EVENT_TRUNCATE:
+            InotifyProtos.TruncateEventProto truncate =
+                InotifyProtos.TruncateEventProto.parseFrom(p.getContents());
+            events.add(new Event.TruncateEvent(truncate.getPath(),
+                truncate.getFileSize(), truncate.getTimestamp()));
+            break;
           default:
             throw new RuntimeException("Unexpected inotify event type: " +
                 p.getType());
@@ -2791,6 +2797,17 @@ public class PBHelper {
                         .setTimestamp(ue.getTimestamp()).build().toByteString()
                 ).build());
             break;
+          case TRUNCATE:
+            Event.TruncateEvent te = (Event.TruncateEvent) e;
+            events.add(InotifyProtos.EventProto.newBuilder()
+                .setType(InotifyProtos.EventType.EVENT_TRUNCATE)
+                .setContents(
+                    InotifyProtos.TruncateEventProto.newBuilder()
+                        .setPath(te.getPath())
+                        .setFileSize(te.getFileSize())
+                        .setTimestamp(te.getTimestamp()).build().toByteString()
+                ).build());
+            break;
           default:
             throw new RuntimeException("Unexpected inotify event: " + e);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/979c9ca2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/InotifyFSEditLogOpTranslator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/InotifyFSEditLogOpTranslator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/InotifyFSEditLogOpTranslator.java
index 5345b46..0918107 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/InotifyFSEditLogOpTranslator.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/InotifyFSEditLogOpTranslator.java
@@ -176,6 +176,10 @@ public class InotifyFSEditLogOpTranslator {
           .metadataType(Event.MetadataUpdateEvent.MetadataType.ACLS)
           .path(saOp.src)
           .acls(saOp.aclEntries).build() });
+    case OP_TRUNCATE:
+      FSEditLogOp.TruncateOp tOp = (FSEditLogOp.TruncateOp) op;
+      return new EventBatch(op.txid, new Event[] {
+          new Event.TruncateEvent(tOp.src, tOp.newLength, tOp.timestamp) });
     default:
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/979c9ca2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
index ba33bd3..385d653 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
@@ -102,6 +102,8 @@ public class TestDFSInotifyEventInputStream {
       DFSTestUtil.createFile(fs, new Path("/file"), BLOCK_SIZE, (short) 1, 0L);
       DFSTestUtil.createFile(fs, new Path("/file3"), BLOCK_SIZE, (short) 1, 0L);
       DFSTestUtil.createFile(fs, new Path("/file5"), BLOCK_SIZE, (short) 1, 0L);
+      DFSTestUtil.createFile(fs, new Path("/truncate_file"),
+          BLOCK_SIZE * 2, (short) 1, 0L);
       DFSInotifyEventInputStream eis = client.getInotifyEventStream();
       client.rename("/file", "/file4", null); // RenameOp -> RenameEvent
       client.rename("/file4", "/file2"); // RenameOldOp -> RenameEvent
@@ -136,7 +138,8 @@ public class TestDFSInotifyEventInputStream {
           "user::rwx,user:foo:rw-,group::r--,other::---", true));
       client.removeAcl("/file5"); // SetAclOp -> MetadataUpdateEvent
       client.rename("/file5", "/dir"); // RenameOldOp -> RenameEvent
-
+      //TruncateOp -> TruncateEvent
+      client.truncate("/truncate_file", BLOCK_SIZE);
       EventBatch batch = null;
 
       // RenameOp
@@ -354,6 +357,18 @@ public class TestDFSInotifyEventInputStream {
       Assert.assertTrue(re3.getSrcPath().equals("/file5"));
       Assert.assertTrue(re.getTimestamp() > 0);
 
+      // TruncateOp
+      batch = waitForNextEvents(eis);
+      Assert.assertEquals(1, batch.getEvents().length);
+      txid = checkTxid(batch, txid);
+      Assert
+          .assertTrue(batch.getEvents()[0].getEventType() ==
+          Event.EventType.TRUNCATE);
+      Event.TruncateEvent et = ((Event.TruncateEvent) batch.getEvents()[0]);
+      Assert.assertTrue(et.getPath().equals("/truncate_file"));
+      Assert.assertTrue(et.getFileSize() == BLOCK_SIZE);
+      Assert.assertTrue(et.getTimestamp() > 0);
+
       // Returns null when there are no further events
       Assert.assertTrue(eis.poll() == null);