You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/05/27 03:45:01 UTC

[GitHub] [hbase] Apache9 commented on a change in pull request #1746: HBASE-24388 Store the locations of meta regions in master local store

Apache9 commented on a change in pull request #1746:
URL: https://github.com/apache/hbase/pull/1746#discussion_r430369088



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -225,23 +230,52 @@ public void start() throws IOException, KeeperException {
     // Start the Assignment Thread
     startAssignmentThread();
 
-    // load meta region state
-    ZKWatcher zkw = master.getZooKeeper();
-    // it could be null in some tests
-    if (zkw != null) {
-      RegionState regionState = MetaTableLocator.getMetaRegionState(zkw);
-      RegionStateNode regionNode =
-        regionStates.getOrCreateRegionStateNode(RegionInfoBuilder.FIRST_META_REGIONINFO);
-      regionNode.lock();
-      try {
-        regionNode.setRegionLocation(regionState.getServerName());
-        regionNode.setState(regionState.getState());
-        if (regionNode.getProcedure() != null) {
-          regionNode.getProcedure().stateLoaded(this, regionNode);
+    // load meta region states.
+    // notice that, here we will load all replicas, and in MasterMetaBootstrap we may assign new
+    // replicas, or remove excess replicas.
+    try (RegionScanner scanner =
+      localStore.getScanner(new Scan().addFamily(HConstants.CATALOG_FAMILY))) {
+      List<Cell> cells = new ArrayList<>();
+      boolean moreRows;
+      do {
+        moreRows = scanner.next(cells);
+        if (cells.isEmpty()) {
+          continue;
         }
-        setMetaAssigned(regionState.getRegion(), regionState.getState() == State.OPEN);
-      } finally {
-        regionNode.unlock();
+        Result result = Result.create(cells);
+        cells.clear();
+        RegionStateStore
+          .visitMetaEntry((r, regionInfo, state, regionLocation, lastHost, openSeqNum) -> {
+            RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(regionInfo);
+            regionNode.lock();
+            try {
+              regionNode.setState(state);
+              regionNode.setLastHost(lastHost);
+              regionNode.setRegionLocation(regionLocation);
+              regionNode.setOpenSeqNum(openSeqNum);
+              if (regionNode.getProcedure() != null) {
+                regionNode.getProcedure().stateLoaded(this, regionNode);
+              }
+              if (RegionReplicaUtil.isDefaultReplica(regionInfo)) {
+                setMetaAssigned(regionInfo, state == State.OPEN);
+              }
+            } finally {
+              regionNode.unlock();
+            }
+            if (regionInfo.isFirst()) {
+              // for compatibility, mirror the meta region state to zookeeper

Review comment:
       For communication with old clients, they will load the meta location from zookeeper.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -225,23 +230,52 @@ public void start() throws IOException, KeeperException {
     // Start the Assignment Thread
     startAssignmentThread();
 
-    // load meta region state
-    ZKWatcher zkw = master.getZooKeeper();
-    // it could be null in some tests
-    if (zkw != null) {
-      RegionState regionState = MetaTableLocator.getMetaRegionState(zkw);
-      RegionStateNode regionNode =
-        regionStates.getOrCreateRegionStateNode(RegionInfoBuilder.FIRST_META_REGIONINFO);
-      regionNode.lock();
-      try {
-        regionNode.setRegionLocation(regionState.getServerName());
-        regionNode.setState(regionState.getState());
-        if (regionNode.getProcedure() != null) {
-          regionNode.getProcedure().stateLoaded(this, regionNode);
+    // load meta region states.
+    // notice that, here we will load all replicas, and in MasterMetaBootstrap we may assign new
+    // replicas, or remove excess replicas.
+    try (RegionScanner scanner =
+      localStore.getScanner(new Scan().addFamily(HConstants.CATALOG_FAMILY))) {
+      List<Cell> cells = new ArrayList<>();
+      boolean moreRows;
+      do {
+        moreRows = scanner.next(cells);
+        if (cells.isEmpty()) {
+          continue;
         }
-        setMetaAssigned(regionState.getRegion(), regionState.getState() == State.OPEN);
-      } finally {
-        regionNode.unlock();
+        Result result = Result.create(cells);
+        cells.clear();
+        RegionStateStore
+          .visitMetaEntry((r, regionInfo, state, regionLocation, lastHost, openSeqNum) -> {
+            RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(regionInfo);
+            regionNode.lock();
+            try {
+              regionNode.setState(state);
+              regionNode.setLastHost(lastHost);

Review comment:
       It's just because we do not have these fields in the protobuf message which is stored on zookeeper...

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
##########
@@ -43,73 +49,103 @@
 
   private final HMaster master;
 
-  public MasterMetaBootstrap(HMaster master) {
+  private final LocalStore localStore;
+
+  public MasterMetaBootstrap(HMaster master, LocalStore localStore) {
     this.master = master;
+    this.localStore = localStore;
   }
 
   /**
    * For assigning hbase:meta replicas only.
-   * TODO: The way this assign runs, nothing but chance to stop all replicas showing up on same
-   * server as the hbase:meta region.
    */
-  void assignMetaReplicas()
-      throws IOException, InterruptedException, KeeperException {
+  void assignMetaReplicas() throws IOException, InterruptedException, KeeperException {
     int numReplicas = master.getConfiguration().getInt(HConstants.META_REPLICAS_NUM,
-           HConstants.DEFAULT_META_REPLICA_NUM);
-    if (numReplicas <= 1) {
-      // No replicaas to assign. Return.
-      return;
-    }
-    final AssignmentManager assignmentManager = master.getAssignmentManager();
-    if (!assignmentManager.isMetaLoaded()) {
-      throw new IllegalStateException("hbase:meta must be initialized first before we can " +
-          "assign out its replicas");
-    }
-    ServerName metaServername = MetaTableLocator.getMetaRegionLocation(this.master.getZooKeeper());
-    for (int i = 1; i < numReplicas; i++) {
-      // Get current meta state for replica from zk.
-      RegionState metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper(), i);
-      RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
-          RegionInfoBuilder.FIRST_META_REGIONINFO, i);
-      LOG.debug(hri.getRegionNameAsString() + " replica region state from zookeeper=" + metaState);
-      if (metaServername.equals(metaState.getServerName())) {
-        metaState = null;
-        LOG.info(hri.getRegionNameAsString() +
-          " old location is same as current hbase:meta location; setting location as null...");
+      HConstants.DEFAULT_META_REPLICA_NUM);
+    // only try to assign meta replicas when there are more than 1 replicas
+    if (numReplicas > 1) {
+      final AssignmentManager am = master.getAssignmentManager();
+      if (!am.isMetaLoaded()) {
+        throw new IllegalStateException(
+          "hbase:meta must be initialized first before we can " + "assign out its replicas");
       }
-      // These assigns run inline. All is blocked till they complete. Only interrupt is shutting
-      // down hosting server which calls AM#stop.
-      if (metaState != null && metaState.getServerName() != null) {
-        // Try to retain old assignment.
-        assignmentManager.assign(hri, metaState.getServerName());
-      } else {
-        assignmentManager.assign(hri);
+      RegionStates regionStates = am.getRegionStates();
+      for (RegionInfo regionInfo : regionStates.getRegionsOfTable(TableName.META_TABLE_NAME)) {
+        if (!RegionReplicaUtil.isDefaultReplica(regionInfo)) {
+          continue;
+        }
+        RegionState regionState = regionStates.getRegionState(regionInfo);
+        Set<ServerName> metaServerNames = new HashSet<ServerName>();
+        if (regionState.getServerName() != null) {
+          metaServerNames.add(regionState.getServerName());
+        }
+        for (int i = 1; i < numReplicas; i++) {
+          RegionInfo secondaryRegionInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, i);
+          RegionState secondaryRegionState = regionStates.getRegionState(secondaryRegionInfo);
+          ServerName sn = null;
+          if (secondaryRegionState != null) {
+            sn = secondaryRegionState.getServerName();
+            if (sn != null && !metaServerNames.add(sn)) {
+              LOG.info("{} old location {} is same with other hbase:meta replica location;" +
+                " setting location as null...", secondaryRegionInfo.getRegionNameAsString(), sn);

Review comment:
       I think the logic here is that, if we have a location, then the assign will not have effect as we assign it to the same region server. If not, it means the region is not online, then we can just assign it to a new regionserver.
   
   Anyway, I agree with you that the above logic is a bit flaky, but this is old behavior. So maybe we should fix it in another issue, not only on the feature branch?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
##########
@@ -43,73 +49,103 @@
 
   private final HMaster master;
 
-  public MasterMetaBootstrap(HMaster master) {
+  private final LocalStore localStore;
+
+  public MasterMetaBootstrap(HMaster master, LocalStore localStore) {
     this.master = master;
+    this.localStore = localStore;
   }
 
   /**
    * For assigning hbase:meta replicas only.
-   * TODO: The way this assign runs, nothing but chance to stop all replicas showing up on same
-   * server as the hbase:meta region.
    */
-  void assignMetaReplicas()
-      throws IOException, InterruptedException, KeeperException {
+  void assignMetaReplicas() throws IOException, InterruptedException, KeeperException {
     int numReplicas = master.getConfiguration().getInt(HConstants.META_REPLICAS_NUM,
-           HConstants.DEFAULT_META_REPLICA_NUM);
-    if (numReplicas <= 1) {
-      // No replicaas to assign. Return.
-      return;
-    }
-    final AssignmentManager assignmentManager = master.getAssignmentManager();
-    if (!assignmentManager.isMetaLoaded()) {
-      throw new IllegalStateException("hbase:meta must be initialized first before we can " +
-          "assign out its replicas");
-    }
-    ServerName metaServername = MetaTableLocator.getMetaRegionLocation(this.master.getZooKeeper());
-    for (int i = 1; i < numReplicas; i++) {
-      // Get current meta state for replica from zk.
-      RegionState metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper(), i);
-      RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
-          RegionInfoBuilder.FIRST_META_REGIONINFO, i);
-      LOG.debug(hri.getRegionNameAsString() + " replica region state from zookeeper=" + metaState);
-      if (metaServername.equals(metaState.getServerName())) {
-        metaState = null;
-        LOG.info(hri.getRegionNameAsString() +
-          " old location is same as current hbase:meta location; setting location as null...");
+      HConstants.DEFAULT_META_REPLICA_NUM);
+    // only try to assign meta replicas when there are more than 1 replicas
+    if (numReplicas > 1) {
+      final AssignmentManager am = master.getAssignmentManager();
+      if (!am.isMetaLoaded()) {
+        throw new IllegalStateException(
+          "hbase:meta must be initialized first before we can " + "assign out its replicas");
       }
-      // These assigns run inline. All is blocked till they complete. Only interrupt is shutting
-      // down hosting server which calls AM#stop.
-      if (metaState != null && metaState.getServerName() != null) {
-        // Try to retain old assignment.
-        assignmentManager.assign(hri, metaState.getServerName());
-      } else {
-        assignmentManager.assign(hri);
+      RegionStates regionStates = am.getRegionStates();
+      for (RegionInfo regionInfo : regionStates.getRegionsOfTable(TableName.META_TABLE_NAME)) {
+        if (!RegionReplicaUtil.isDefaultReplica(regionInfo)) {
+          continue;
+        }
+        RegionState regionState = regionStates.getRegionState(regionInfo);
+        Set<ServerName> metaServerNames = new HashSet<ServerName>();
+        if (regionState.getServerName() != null) {
+          metaServerNames.add(regionState.getServerName());
+        }
+        for (int i = 1; i < numReplicas; i++) {
+          RegionInfo secondaryRegionInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, i);
+          RegionState secondaryRegionState = regionStates.getRegionState(secondaryRegionInfo);
+          ServerName sn = null;
+          if (secondaryRegionState != null) {
+            sn = secondaryRegionState.getServerName();
+            if (sn != null && !metaServerNames.add(sn)) {
+              LOG.info("{} old location {} is same with other hbase:meta replica location;" +
+                " setting location as null...", secondaryRegionInfo.getRegionNameAsString(), sn);
+              sn = null;
+            }
+          }
+          // These assigns run inline. All is blocked till they complete. Only interrupt is shutting
+          // down hosting server which calls AM#stop.
+          if (sn != null) {
+            am.assign(secondaryRegionInfo, sn);
+          } else {
+            am.assign(secondaryRegionInfo);
+          }
+        }
       }
     }
+    // always try to remomve excess meta replicas
     unassignExcessMetaReplica(numReplicas);
   }
 
   private void unassignExcessMetaReplica(int numMetaReplicasConfigured) {
-    final ZKWatcher zooKeeper = master.getZooKeeper();
-    // unassign the unneeded replicas (for e.g., if the previous master was configured
-    // with a replication of 3 and now it is 2, we need to unassign the 1 unneeded replica)
-    try {
-      List<String> metaReplicaZnodes = zooKeeper.getMetaReplicaNodes();
-      for (String metaReplicaZnode : metaReplicaZnodes) {
-        int replicaId = zooKeeper.getZNodePaths().getMetaReplicaIdFromZnode(metaReplicaZnode);
-        if (replicaId >= numMetaReplicasConfigured) {
-          RegionState r = MetaTableLocator.getMetaRegionState(zooKeeper, replicaId);
-          LOG.info("Closing excess replica of meta region " + r.getRegion());
-          // send a close and wait for a max of 30 seconds
-          ServerManager.closeRegionSilentlyAndWait(master.getAsyncClusterConnection(),
-              r.getServerName(), r.getRegion(), 30000);
-          ZKUtil.deleteNode(zooKeeper, zooKeeper.getZNodePaths().getZNodeForReplica(replicaId));
+    ZKWatcher zooKeeper = master.getZooKeeper();
+    AssignmentManager am = master.getAssignmentManager();
+    RegionStates regionStates = am.getRegionStates();
+    Map<RegionInfo, Integer> region2MaxReplicaId = new HashMap<>();
+    for (RegionInfo regionInfo : regionStates.getRegionsOfTable(TableName.META_TABLE_NAME)) {
+      RegionInfo primaryRegionInfo = RegionReplicaUtil.getRegionInfoForDefaultReplica(regionInfo);
+      region2MaxReplicaId.compute(primaryRegionInfo,
+        (k, v) -> v == null ? regionInfo.getReplicaId() : Math.max(v, regionInfo.getReplicaId()));
+      if (regionInfo.getReplicaId() < numMetaReplicasConfigured) {
+        continue;
+      }
+      RegionState regionState = regionStates.getRegionState(regionInfo);
+      try {
+        ServerManager.closeRegionSilentlyAndWait(master.getAsyncClusterConnection(),
+          regionState.getServerName(), regionInfo, 30000);
+        if (regionInfo.isFirst()) {
+          // for compatibility, also try to remove the replicas on zk.
+          ZKUtil.deleteNode(zooKeeper,
+            zooKeeper.getZNodePaths().getZNodeForReplica(regionInfo.getReplicaId()));
         }
+      } catch (Exception e) {
+        // ignore the exception since we don't want the master to be wedged due to potential
+        // issues in the cleanup of the extra regions. We can do that cleanup via hbck or manually
+        LOG.warn("Ignoring exception " + e);
       }
-    } catch (Exception ex) {
-      // ignore the exception since we don't want the master to be wedged due to potential
-      // issues in the cleanup of the extra regions. We can do that cleanup via hbck or manually
-      LOG.warn("Ignoring exception " + ex);
+      regionStates.deleteRegion(regionInfo);
     }
+    region2MaxReplicaId.forEach((regionInfo, maxReplicaId) -> {
+      if (maxReplicaId >= numMetaReplicasConfigured) {
+        byte[] metaRow = MetaTableAccessor.getMetaKeyForRegion(regionInfo);
+        Delete delete = MetaTableAccessor.removeRegionReplica(metaRow, numMetaReplicasConfigured,

Review comment:
       If we could have multiple meta regions, maybe we will have different meta replicas for different regions, think of we fail in the middle of this process. So use another loop for safety.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
##########
@@ -43,73 +49,103 @@
 
   private final HMaster master;
 
-  public MasterMetaBootstrap(HMaster master) {
+  private final LocalStore localStore;
+
+  public MasterMetaBootstrap(HMaster master, LocalStore localStore) {
     this.master = master;
+    this.localStore = localStore;
   }
 
   /**
    * For assigning hbase:meta replicas only.
-   * TODO: The way this assign runs, nothing but chance to stop all replicas showing up on same
-   * server as the hbase:meta region.
    */
-  void assignMetaReplicas()
-      throws IOException, InterruptedException, KeeperException {
+  void assignMetaReplicas() throws IOException, InterruptedException, KeeperException {
     int numReplicas = master.getConfiguration().getInt(HConstants.META_REPLICAS_NUM,
-           HConstants.DEFAULT_META_REPLICA_NUM);
-    if (numReplicas <= 1) {
-      // No replicaas to assign. Return.
-      return;
-    }
-    final AssignmentManager assignmentManager = master.getAssignmentManager();
-    if (!assignmentManager.isMetaLoaded()) {
-      throw new IllegalStateException("hbase:meta must be initialized first before we can " +
-          "assign out its replicas");
-    }
-    ServerName metaServername = MetaTableLocator.getMetaRegionLocation(this.master.getZooKeeper());
-    for (int i = 1; i < numReplicas; i++) {
-      // Get current meta state for replica from zk.
-      RegionState metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper(), i);
-      RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
-          RegionInfoBuilder.FIRST_META_REGIONINFO, i);
-      LOG.debug(hri.getRegionNameAsString() + " replica region state from zookeeper=" + metaState);
-      if (metaServername.equals(metaState.getServerName())) {
-        metaState = null;
-        LOG.info(hri.getRegionNameAsString() +
-          " old location is same as current hbase:meta location; setting location as null...");
+      HConstants.DEFAULT_META_REPLICA_NUM);
+    // only try to assign meta replicas when there are more than 1 replicas
+    if (numReplicas > 1) {
+      final AssignmentManager am = master.getAssignmentManager();
+      if (!am.isMetaLoaded()) {
+        throw new IllegalStateException(
+          "hbase:meta must be initialized first before we can " + "assign out its replicas");
       }
-      // These assigns run inline. All is blocked till they complete. Only interrupt is shutting
-      // down hosting server which calls AM#stop.
-      if (metaState != null && metaState.getServerName() != null) {
-        // Try to retain old assignment.
-        assignmentManager.assign(hri, metaState.getServerName());
-      } else {
-        assignmentManager.assign(hri);
+      RegionStates regionStates = am.getRegionStates();
+      for (RegionInfo regionInfo : regionStates.getRegionsOfTable(TableName.META_TABLE_NAME)) {
+        if (!RegionReplicaUtil.isDefaultReplica(regionInfo)) {
+          continue;
+        }
+        RegionState regionState = regionStates.getRegionState(regionInfo);
+        Set<ServerName> metaServerNames = new HashSet<ServerName>();
+        if (regionState.getServerName() != null) {
+          metaServerNames.add(regionState.getServerName());
+        }
+        for (int i = 1; i < numReplicas; i++) {
+          RegionInfo secondaryRegionInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, i);
+          RegionState secondaryRegionState = regionStates.getRegionState(secondaryRegionInfo);
+          ServerName sn = null;
+          if (secondaryRegionState != null) {
+            sn = secondaryRegionState.getServerName();
+            if (sn != null && !metaServerNames.add(sn)) {
+              LOG.info("{} old location {} is same with other hbase:meta replica location;" +
+                " setting location as null...", secondaryRegionInfo.getRegionNameAsString(), sn);
+              sn = null;
+            }
+          }
+          // These assigns run inline. All is blocked till they complete. Only interrupt is shutting
+          // down hosting server which calls AM#stop.
+          if (sn != null) {
+            am.assign(secondaryRegionInfo, sn);
+          } else {
+            am.assign(secondaryRegionInfo);
+          }
+        }
       }
     }
+    // always try to remomve excess meta replicas
     unassignExcessMetaReplica(numReplicas);
   }
 
   private void unassignExcessMetaReplica(int numMetaReplicasConfigured) {
-    final ZKWatcher zooKeeper = master.getZooKeeper();
-    // unassign the unneeded replicas (for e.g., if the previous master was configured
-    // with a replication of 3 and now it is 2, we need to unassign the 1 unneeded replica)
-    try {
-      List<String> metaReplicaZnodes = zooKeeper.getMetaReplicaNodes();
-      for (String metaReplicaZnode : metaReplicaZnodes) {
-        int replicaId = zooKeeper.getZNodePaths().getMetaReplicaIdFromZnode(metaReplicaZnode);
-        if (replicaId >= numMetaReplicasConfigured) {
-          RegionState r = MetaTableLocator.getMetaRegionState(zooKeeper, replicaId);
-          LOG.info("Closing excess replica of meta region " + r.getRegion());
-          // send a close and wait for a max of 30 seconds
-          ServerManager.closeRegionSilentlyAndWait(master.getAsyncClusterConnection(),
-              r.getServerName(), r.getRegion(), 30000);
-          ZKUtil.deleteNode(zooKeeper, zooKeeper.getZNodePaths().getZNodeForReplica(replicaId));
+    ZKWatcher zooKeeper = master.getZooKeeper();
+    AssignmentManager am = master.getAssignmentManager();
+    RegionStates regionStates = am.getRegionStates();
+    Map<RegionInfo, Integer> region2MaxReplicaId = new HashMap<>();
+    for (RegionInfo regionInfo : regionStates.getRegionsOfTable(TableName.META_TABLE_NAME)) {
+      RegionInfo primaryRegionInfo = RegionReplicaUtil.getRegionInfoForDefaultReplica(regionInfo);
+      region2MaxReplicaId.compute(primaryRegionInfo,
+        (k, v) -> v == null ? regionInfo.getReplicaId() : Math.max(v, regionInfo.getReplicaId()));
+      if (regionInfo.getReplicaId() < numMetaReplicasConfigured) {
+        continue;
+      }
+      RegionState regionState = regionStates.getRegionState(regionInfo);
+      try {
+        ServerManager.closeRegionSilentlyAndWait(master.getAsyncClusterConnection(),
+          regionState.getServerName(), regionInfo, 30000);
+        if (regionInfo.isFirst()) {
+          // for compatibility, also try to remove the replicas on zk.
+          ZKUtil.deleteNode(zooKeeper,
+            zooKeeper.getZNodePaths().getZNodeForReplica(regionInfo.getReplicaId()));
         }
+      } catch (Exception e) {
+        // ignore the exception since we don't want the master to be wedged due to potential
+        // issues in the cleanup of the extra regions. We can do that cleanup via hbck or manually
+        LOG.warn("Ignoring exception " + e);
       }
-    } catch (Exception ex) {
-      // ignore the exception since we don't want the master to be wedged due to potential
-      // issues in the cleanup of the extra regions. We can do that cleanup via hbck or manually
-      LOG.warn("Ignoring exception " + ex);
+      regionStates.deleteRegion(regionInfo);
     }
+    region2MaxReplicaId.forEach((regionInfo, maxReplicaId) -> {
+      if (maxReplicaId >= numMetaReplicasConfigured) {
+        byte[] metaRow = MetaTableAccessor.getMetaKeyForRegion(regionInfo);
+        Delete delete = MetaTableAccessor.removeRegionReplica(metaRow, numMetaReplicasConfigured,

Review comment:
       Oh, I think the problem is that, only when we finish scanning the replicas for a region, we can know how many replicas we can remove, then we create a single Delete which deletes all the replicas. And for zk, different replicas will have different znode, so just delete it everytime...




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org