[17/50] [abbrv] hadoop git commit: HDFS-4266. BKJM: Separate write and ack quorum (Rakesh R via umamahesh)

2015-02-18 Thread zjshen
HDFS-4266. BKJM: Separate write and ack quorum (Rakesh R via umamahesh)


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

Branch: refs/heads/YARN-2928
Commit: f0412de1c1d42b3c2a92531f81d97a24df920523
Parents: f24a567
Author: Uma Maheswara Rao G umamah...@apache.org
Authored: Tue Feb 17 21:28:49 2015 +0530
Committer: Uma Maheswara Rao G umamah...@apache.org
Committed: Tue Feb 17 21:28:49 2015 +0530

--
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt |   2 +
 .../bkjournal/BookKeeperJournalManager.java |  15 +-
 .../bkjournal/TestBookKeeperJournalManager.java | 153 ++-
 3 files changed, 163 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0412de1/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 fcf5994..f28e41e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -639,6 +639,8 @@ Release 2.7.0 - UNRELEASED
 
 HDFS-7797. Add audit log for setQuota operation (Rakesh R via umamahesh)
 
+HDFS-4266. BKJM: Separate write and ack quorum (Rakesh R via umamahesh)
+
   OPTIMIZATIONS
 
 HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0412de1/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
--
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
index 51905c0..89fa84c 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
@@ -152,6 +152,13 @@ public class BookKeeperJournalManager implements 
JournalManager {
 = dfs.namenode.bookkeeperjournal.readEntryTimeoutSec;
   public static final int BKJM_BOOKKEEPER_READ_ENTRY_TIMEOUT_DEFAULT = 5;
 
+  public static final String BKJM_BOOKKEEPER_ACK_QUORUM_SIZE 
+= dfs.namenode.bookkeeperjournal.ack.quorum-size;
+
+  public static final String BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_SEC
+= dfs.namenode.bookkeeperjournal.addEntryTimeoutSec;
+  public static final int BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_DEFAULT = 5;
+
   private ZooKeeper zkc;
   private final Configuration conf;
   private final BookKeeper bkc;
@@ -162,6 +169,8 @@ public class BookKeeperJournalManager implements 
JournalManager {
   private final MaxTxId maxTxId;
   private final int ensembleSize;
   private final int quorumSize;
+  private final int ackQuorumSize;
+  private final int addEntryTimeout;
   private final String digestpw;
   private final int speculativeReadTimeout;
   private final int readEntryTimeout;
@@ -184,6 +193,9 @@ public class BookKeeperJournalManager implements 
JournalManager {
BKJM_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT);
 quorumSize = conf.getInt(BKJM_BOOKKEEPER_QUORUM_SIZE,
  BKJM_BOOKKEEPER_QUORUM_SIZE_DEFAULT);
+ackQuorumSize = conf.getInt(BKJM_BOOKKEEPER_ACK_QUORUM_SIZE, quorumSize);
+addEntryTimeout = conf.getInt(BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_SEC,
+ BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_DEFAULT);
 speculativeReadTimeout = conf.getInt(
  BKJM_BOOKKEEPER_SPECULATIVE_READ_TIMEOUT_MS,
  BKJM_BOOKKEEPER_SPECULATIVE_READ_TIMEOUT_DEFAULT);
@@ -216,6 +228,7 @@ public class BookKeeperJournalManager implements 
JournalManager {
   ClientConfiguration clientConf = new ClientConfiguration();
   clientConf.setSpeculativeReadTimeout(speculativeReadTimeout);
   clientConf.setReadEntryTimeout(readEntryTimeout);
+  clientConf.setAddEntryTimeout(addEntryTimeout);
   bkc = new BookKeeper(clientConf, zkc);
 } catch (KeeperException e) {
   throw new IOException(Error initializing zk, e);
@@ -403,7 +416,7 @@ public class BookKeeperJournalManager implements 
JournalManager {
 // bookkeeper errored on last stream, clean up ledger
 currentLedger.close();
   }
-  currentLedger = 

hadoop git commit: HDFS-4266. BKJM: Separate write and ack quorum (Rakesh R via umamahesh)

2015-02-17 Thread umamahesh
Repository: hadoop
Updated Branches:
  refs/heads/trunk f24a56787 - f0412de1c


HDFS-4266. BKJM: Separate write and ack quorum (Rakesh R via umamahesh)


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

Branch: refs/heads/trunk
Commit: f0412de1c1d42b3c2a92531f81d97a24df920523
Parents: f24a567
Author: Uma Maheswara Rao G umamah...@apache.org
Authored: Tue Feb 17 21:28:49 2015 +0530
Committer: Uma Maheswara Rao G umamah...@apache.org
Committed: Tue Feb 17 21:28:49 2015 +0530

--
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt |   2 +
 .../bkjournal/BookKeeperJournalManager.java |  15 +-
 .../bkjournal/TestBookKeeperJournalManager.java | 153 ++-
 3 files changed, 163 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0412de1/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 fcf5994..f28e41e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -639,6 +639,8 @@ Release 2.7.0 - UNRELEASED
 
 HDFS-7797. Add audit log for setQuota operation (Rakesh R via umamahesh)
 
+HDFS-4266. BKJM: Separate write and ack quorum (Rakesh R via umamahesh)
+
   OPTIMIZATIONS
 
 HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0412de1/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
--
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
index 51905c0..89fa84c 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
@@ -152,6 +152,13 @@ public class BookKeeperJournalManager implements 
JournalManager {
 = dfs.namenode.bookkeeperjournal.readEntryTimeoutSec;
   public static final int BKJM_BOOKKEEPER_READ_ENTRY_TIMEOUT_DEFAULT = 5;
 
+  public static final String BKJM_BOOKKEEPER_ACK_QUORUM_SIZE 
+= dfs.namenode.bookkeeperjournal.ack.quorum-size;
+
+  public static final String BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_SEC
+= dfs.namenode.bookkeeperjournal.addEntryTimeoutSec;
+  public static final int BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_DEFAULT = 5;
+
   private ZooKeeper zkc;
   private final Configuration conf;
   private final BookKeeper bkc;
@@ -162,6 +169,8 @@ public class BookKeeperJournalManager implements 
JournalManager {
   private final MaxTxId maxTxId;
   private final int ensembleSize;
   private final int quorumSize;
+  private final int ackQuorumSize;
+  private final int addEntryTimeout;
   private final String digestpw;
   private final int speculativeReadTimeout;
   private final int readEntryTimeout;
@@ -184,6 +193,9 @@ public class BookKeeperJournalManager implements 
JournalManager {
BKJM_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT);
 quorumSize = conf.getInt(BKJM_BOOKKEEPER_QUORUM_SIZE,
  BKJM_BOOKKEEPER_QUORUM_SIZE_DEFAULT);
+ackQuorumSize = conf.getInt(BKJM_BOOKKEEPER_ACK_QUORUM_SIZE, quorumSize);
+addEntryTimeout = conf.getInt(BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_SEC,
+ BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_DEFAULT);
 speculativeReadTimeout = conf.getInt(
  BKJM_BOOKKEEPER_SPECULATIVE_READ_TIMEOUT_MS,
  BKJM_BOOKKEEPER_SPECULATIVE_READ_TIMEOUT_DEFAULT);
@@ -216,6 +228,7 @@ public class BookKeeperJournalManager implements 
JournalManager {
   ClientConfiguration clientConf = new ClientConfiguration();
   clientConf.setSpeculativeReadTimeout(speculativeReadTimeout);
   clientConf.setReadEntryTimeout(readEntryTimeout);
+  clientConf.setAddEntryTimeout(addEntryTimeout);
   bkc = new BookKeeper(clientConf, zkc);
 } catch (KeeperException e) {
   throw new IOException(Error initializing zk, e);
@@ -403,7 +416,7 @@ public class BookKeeperJournalManager implements 
JournalManager {
 // bookkeeper errored on last stream, clean up 

hadoop git commit: HDFS-4266. BKJM: Separate write and ack quorum (Rakesh R via umamahesh)

2015-02-17 Thread umamahesh
Repository: hadoop
Updated Branches:
  refs/heads/branch-2 35fecb530 - 2cbac36fd


HDFS-4266. BKJM: Separate write and ack quorum (Rakesh R via umamahesh)

(cherry picked from commit f0412de1c1d42b3c2a92531f81d97a24df920523)


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

Branch: refs/heads/branch-2
Commit: 2cbac36fd3eb1160baf53f643223f96d53d111df
Parents: 35fecb5
Author: Uma Maheswara Rao G umamah...@apache.org
Authored: Tue Feb 17 21:28:49 2015 +0530
Committer: Uma Maheswara Rao G umamah...@apache.org
Committed: Tue Feb 17 21:31:43 2015 +0530

--
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt |   2 +
 .../bkjournal/BookKeeperJournalManager.java |  15 +-
 .../bkjournal/TestBookKeeperJournalManager.java | 153 ++-
 3 files changed, 163 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2cbac36f/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 58561d9..b95eded 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -346,6 +346,8 @@ Release 2.7.0 - UNRELEASED
 
 HDFS-7797. Add audit log for setQuota operation (Rakesh R via umamahesh)
 
+HDFS-4266. BKJM: Separate write and ack quorum (Rakesh R via umamahesh)
+
   OPTIMIZATIONS
 
 HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2cbac36f/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
--
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
index aecc464..16ffe52 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
@@ -152,6 +152,13 @@ public class BookKeeperJournalManager implements 
JournalManager {
 = dfs.namenode.bookkeeperjournal.readEntryTimeoutSec;
   public static final int BKJM_BOOKKEEPER_READ_ENTRY_TIMEOUT_DEFAULT = 5;
 
+  public static final String BKJM_BOOKKEEPER_ACK_QUORUM_SIZE 
+= dfs.namenode.bookkeeperjournal.ack.quorum-size;
+
+  public static final String BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_SEC
+= dfs.namenode.bookkeeperjournal.addEntryTimeoutSec;
+  public static final int BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_DEFAULT = 5;
+
   private ZooKeeper zkc;
   private final Configuration conf;
   private final BookKeeper bkc;
@@ -162,6 +169,8 @@ public class BookKeeperJournalManager implements 
JournalManager {
   private final MaxTxId maxTxId;
   private final int ensembleSize;
   private final int quorumSize;
+  private final int ackQuorumSize;
+  private final int addEntryTimeout;
   private final String digestpw;
   private final int speculativeReadTimeout;
   private final int readEntryTimeout;
@@ -184,6 +193,9 @@ public class BookKeeperJournalManager implements 
JournalManager {
BKJM_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT);
 quorumSize = conf.getInt(BKJM_BOOKKEEPER_QUORUM_SIZE,
  BKJM_BOOKKEEPER_QUORUM_SIZE_DEFAULT);
+ackQuorumSize = conf.getInt(BKJM_BOOKKEEPER_ACK_QUORUM_SIZE, quorumSize);
+addEntryTimeout = conf.getInt(BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_SEC,
+ BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_DEFAULT);
 speculativeReadTimeout = conf.getInt(
  BKJM_BOOKKEEPER_SPECULATIVE_READ_TIMEOUT_MS,
  BKJM_BOOKKEEPER_SPECULATIVE_READ_TIMEOUT_DEFAULT);
@@ -216,6 +228,7 @@ public class BookKeeperJournalManager implements 
JournalManager {
   ClientConfiguration clientConf = new ClientConfiguration();
   clientConf.setSpeculativeReadTimeout(speculativeReadTimeout);
   clientConf.setReadEntryTimeout(readEntryTimeout);
+  clientConf.setAddEntryTimeout(addEntryTimeout);
   bkc = new BookKeeper(clientConf, zkc);
 } catch (KeeperException e) {
   throw new IOException(Error initializing zk, e);
@@ -403,7 +416,7 @@ public class BookKeeperJournalManager implements