You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2018/08/14 18:04:15 UTC

[01/15] hbase git commit: HBASE-21018 RS crashed because AsyncFS was unable to update HDFS data encryption key [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-20387 f7ce2f743 -> cc3efceca (forced update)


HBASE-21018 RS crashed because AsyncFS was unable to update HDFS data encryption key


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

Branch: refs/heads/HBASE-20387
Commit: a3ab9306a6a1b044a8558814c5e21a38e0cb8b03
Parents: 86821de
Author: Wei-Chiu Chuang <we...@cloudera.com>
Authored: Mon Aug 6 18:00:58 2018 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Fri Aug 10 19:53:41 2018 -0700

----------------------------------------------------------------------
 .../FanOutOneBlockAsyncDFSOutputSaslHelper.java  | 19 ++++++++++++++-----
 1 file changed, 14 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a3ab9306/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java
index ef6c1ca..a56c3d7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java
@@ -367,16 +367,20 @@ public final class FanOutOneBlockAsyncDFSOutputSaslHelper {
 
     private final Promise<Void> promise;
 
+    private final DFSClient dfsClient;
+
     private int step = 0;
 
     public SaslNegotiateHandler(Configuration conf, String username, char[] password,
-        Map<String, String> saslProps, int timeoutMs, Promise<Void> promise) throws SaslException {
+        Map<String, String> saslProps, int timeoutMs, Promise<Void> promise,
+        DFSClient dfsClient) throws SaslException {
       this.conf = conf;
       this.saslProps = saslProps;
       this.saslClient = Sasl.createSaslClient(new String[] { MECHANISM }, username, PROTOCOL,
         SERVER_NAME, saslProps, new SaslClientCallbackHandler(username, password));
       this.timeoutMs = timeoutMs;
       this.promise = promise;
+      this.dfsClient = dfsClient;
     }
 
     private void sendSaslMessage(ChannelHandlerContext ctx, byte[] payload) throws IOException {
@@ -434,6 +438,7 @@ public final class FanOutOneBlockAsyncDFSOutputSaslHelper {
 
     private void check(DataTransferEncryptorMessageProto proto) throws IOException {
       if (proto.getStatus() == DataTransferEncryptorStatus.ERROR_UNKNOWN_KEY) {
+        dfsClient.clearDataEncryptionKey();
         throw new InvalidEncryptionKeyException(proto.getMessage());
       } else if (proto.getStatus() == DataTransferEncryptorStatus.ERROR) {
         throw new IOException(proto.getMessage());
@@ -737,12 +742,14 @@ public final class FanOutOneBlockAsyncDFSOutputSaslHelper {
   }
 
   private static void doSaslNegotiation(Configuration conf, Channel channel, int timeoutMs,
-      String username, char[] password, Map<String, String> saslProps, Promise<Void> saslPromise) {
+      String username, char[] password, Map<String, String> saslProps, Promise<Void> saslPromise,
+      DFSClient dfsClient) {
     try {
       channel.pipeline().addLast(new IdleStateHandler(timeoutMs, 0, 0, TimeUnit.MILLISECONDS),
         new ProtobufVarint32FrameDecoder(),
         new ProtobufDecoder(DataTransferEncryptorMessageProto.getDefaultInstance()),
-        new SaslNegotiateHandler(conf, username, password, saslProps, timeoutMs, saslPromise));
+        new SaslNegotiateHandler(conf, username, password, saslProps, timeoutMs, saslPromise,
+            dfsClient));
     } catch (SaslException e) {
       saslPromise.tryFailure(e);
     }
@@ -769,7 +776,8 @@ public final class FanOutOneBlockAsyncDFSOutputSaslHelper {
       }
       doSaslNegotiation(conf, channel, timeoutMs, getUserNameFromEncryptionKey(encryptionKey),
         encryptionKeyToPassword(encryptionKey.encryptionKey),
-        createSaslPropertiesForEncryption(encryptionKey.encryptionAlgorithm), saslPromise);
+        createSaslPropertiesForEncryption(encryptionKey.encryptionAlgorithm), saslPromise,
+          client);
     } else if (!UserGroupInformation.isSecurityEnabled()) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("SASL client skipping handshake in unsecured configuration for addr = " + addr
@@ -794,7 +802,8 @@ public final class FanOutOneBlockAsyncDFSOutputSaslHelper {
           "SASL client doing general handshake for addr = " + addr + ", datanodeId = " + dnInfo);
       }
       doSaslNegotiation(conf, channel, timeoutMs, buildUsername(accessToken),
-        buildClientPassword(accessToken), saslPropsResolver.getClientProperties(addr), saslPromise);
+        buildClientPassword(accessToken), saslPropsResolver.getClientProperties(addr), saslPromise,
+          client);
     } else {
       // It's a secured cluster using non-privileged ports, but no SASL. The only way this can
       // happen is if the DataNode has ignore.secure.ports.for.testing configured, so this is a rare


[08/15] hbase git commit: HBASE-20985 add two attributes when we do normalization

Posted by bu...@apache.org.
HBASE-20985 add two attributes when we do normalization

Signed-off-by: Guanghao Zhang <zg...@apache.org>


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

Branch: refs/heads/HBASE-20387
Commit: 21e02813fedc6846c9ef381e1ff9b49534f20ec9
Parents: 911a6ef
Author: jingyuntian <ti...@gmail.com>
Authored: Wed Aug 8 19:33:03 2018 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Mon Aug 13 16:43:54 2018 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/HTableDescriptor.java   |  24 ++++
 .../hadoop/hbase/client/TableDescriptor.java    |  16 +++
 .../hbase/client/TableDescriptorBuilder.java    |  60 ++++++++++
 .../normalizer/SimpleRegionNormalizer.java      |  28 ++++-
 .../normalizer/TestSimpleRegionNormalizer.java  | 109 +++++++++++++++++++
 hbase-shell/src/main/ruby/hbase/admin.rb        |   2 +
 .../src/main/ruby/shell/commands/alter.rb       |   3 +-
 hbase-shell/src/test/ruby/hbase/admin_test.rb   |   7 ++
 8 files changed, 247 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/21e02813/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index cab1acf..e239966 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -65,6 +65,10 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
   public static final String REGION_REPLICATION = TableDescriptorBuilder.REGION_REPLICATION;
   public static final String REGION_MEMSTORE_REPLICATION = TableDescriptorBuilder.REGION_MEMSTORE_REPLICATION;
   public static final String NORMALIZATION_ENABLED = TableDescriptorBuilder.NORMALIZATION_ENABLED;
+  public static final String NORMALIZER_TARGET_REGION_COUNT =
+      TableDescriptorBuilder.NORMALIZER_TARGET_REGION_COUNT;
+  public static final String NORMALIZER_TARGET_REGION_SIZE =
+      TableDescriptorBuilder.NORMALIZER_TARGET_REGION_SIZE;
   public static final String PRIORITY = TableDescriptorBuilder.PRIORITY;
   public static final boolean DEFAULT_READONLY = TableDescriptorBuilder.DEFAULT_READONLY;
   public static final boolean DEFAULT_COMPACTION_ENABLED = TableDescriptorBuilder.DEFAULT_COMPACTION_ENABLED;
@@ -288,6 +292,26 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
     return this;
   }
 
+  @Override
+  public int getNormalizerTargetRegionCount() {
+    return getDelegateeForModification().getNormalizerTargetRegionCount();
+  }
+
+  public HTableDescriptor setNormalizerTargetRegionCount(final int regionCount) {
+    getDelegateeForModification().setNormalizerTargetRegionCount(regionCount);
+    return this;
+  }
+
+  @Override
+  public long getNormalizerTargetRegionSize() {
+    return getDelegateeForModification().getNormalizerTargetRegionSize();
+  }
+
+  public HTableDescriptor setNormalizerTargetRegionSize(final long regionSize) {
+    getDelegateeForModification().setNormalizerTargetRegionSize(regionSize);
+    return this;
+  }
+
   /**
    * Sets the {@link Durability} setting for the table. This defaults to Durability.USE_DEFAULT.
    * @param durability enum value

http://git-wip-us.apache.org/repos/asf/hbase/blob/21e02813/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
index 4f713d1..82e08c3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
@@ -261,6 +261,22 @@ public interface TableDescriptor {
   boolean isNormalizationEnabled();
 
   /**
+   * Check if there is the target region count. If so, the normalize plan will
+   * be calculated based on the target region count.
+   *
+   * @return target region count after normalize done
+   */
+  int getNormalizerTargetRegionCount();
+
+  /**
+   * Check if there is the target region size. If so, the normalize plan will
+   * be calculated based on the target region size.
+   *
+   * @return target region size after normalize done
+   */
+  long getNormalizerTargetRegionSize();
+
+  /**
    * Check if the readOnly flag of the table is set. If the readOnly flag is set
    * then the contents of the table can only be read from but not modified.
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/21e02813/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
index 78f25ec..d1c3f78 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
@@ -150,6 +150,17 @@ public class TableDescriptorBuilder {
   private static final Bytes NORMALIZATION_ENABLED_KEY
           = new Bytes(Bytes.toBytes(NORMALIZATION_ENABLED));
 
+  @InterfaceAudience.Private
+  public static final String NORMALIZER_TARGET_REGION_COUNT =
+      "NORMALIZER_TARGET_REGION_COUNT";
+  private static final Bytes NORMALIZER_TARGET_REGION_COUNT_KEY =
+      new Bytes(Bytes.toBytes(NORMALIZER_TARGET_REGION_COUNT));
+
+  @InterfaceAudience.Private
+  public static final String NORMALIZER_TARGET_REGION_SIZE = "NORMALIZER_TARGET_REGION_SIZE";
+  private static final Bytes NORMALIZER_TARGET_REGION_SIZE_KEY =
+      new Bytes(Bytes.toBytes(NORMALIZER_TARGET_REGION_SIZE));
+
   /**
    * Default durability for HTD is USE_DEFAULT, which defaults to HBase-global
    * default value
@@ -380,6 +391,16 @@ public class TableDescriptorBuilder {
     return this;
   }
 
+  public TableDescriptorBuilder setNormalizerTargetRegionCount(final int regionCount) {
+    desc.setNormalizerTargetRegionCount(regionCount);
+    return this;
+  }
+
+  public TableDescriptorBuilder setNormalizerTargetRegionSize(final long regionSize) {
+    desc.setNormalizerTargetRegionSize(regionSize);
+    return this;
+  }
+
   public TableDescriptorBuilder setNormalizationEnabled(final boolean isEnable) {
     desc.setNormalizationEnabled(isEnable);
     return this;
@@ -710,6 +731,27 @@ public class TableDescriptorBuilder {
     }
 
     /**
+     * Check if there is the target region count. If so, the normalize plan will be calculated based
+     * on the target region count.
+     * @return target region count after normalize done
+     */
+    @Override
+    public int getNormalizerTargetRegionCount() {
+      return getOrDefault(NORMALIZER_TARGET_REGION_COUNT_KEY, Integer::valueOf,
+        Integer.valueOf(-1));
+    }
+
+    /**
+     * Check if there is the target region size. If so, the normalize plan will be calculated based
+     * on the target region size.
+     * @return target region size after normalize done
+     */
+    @Override
+    public long getNormalizerTargetRegionSize() {
+      return getOrDefault(NORMALIZER_TARGET_REGION_SIZE_KEY, Long::valueOf, Long.valueOf(-1));
+    }
+
+    /**
      * Setting the table normalization enable flag.
      *
      * @param isEnable True if enable normalization.
@@ -720,6 +762,24 @@ public class TableDescriptorBuilder {
     }
 
     /**
+     * Setting the target region count of table normalization .
+     * @param regionCount the target region count.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setNormalizerTargetRegionCount(final int regionCount) {
+      return setValue(NORMALIZER_TARGET_REGION_COUNT_KEY, Integer.toString(regionCount));
+    }
+
+    /**
+     * Setting the target region size of table normalization.
+     * @param regionSize the target region size.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setNormalizerTargetRegionSize(final long regionSize) {
+      return setValue(NORMALIZER_TARGET_REGION_SIZE_KEY, Long.toString(regionSize));
+    }
+
+    /**
      * Sets the {@link Durability} setting for the table. This defaults to
      * Durability.USE_DEFAULT.
      *

http://git-wip-us.apache.org/repos/asf/hbase/blob/21e02813/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
index f02ccf4..a30a13b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.master.normalizer;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
@@ -30,6 +31,7 @@ import org.apache.hadoop.hbase.Size;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
@@ -156,8 +158,32 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
         totalSizeMb += regionSize;
       }
     }
+    int targetRegionCount = -1;
+    long targetRegionSize = -1;
+    try {
+      TableDescriptor tableDescriptor = masterServices.getTableDescriptors().get(table);
+      if(tableDescriptor != null) {
+        targetRegionCount =
+            tableDescriptor.getNormalizerTargetRegionCount();
+        targetRegionSize =
+            tableDescriptor.getNormalizerTargetRegionSize();
+        LOG.debug("Table {}:  target region count is {}, target region size is {}", table,
+            targetRegionCount, targetRegionSize);
+      }
+    } catch (IOException e) {
+      LOG.warn(
+        "cannot get the target number and target size of table {}, they will be default value -1.",
+        table);
+    }
 
-    double avgRegionSize = acutalRegionCnt == 0 ? 0 : totalSizeMb / (double) acutalRegionCnt;
+    double avgRegionSize;
+    if (targetRegionSize > 0) {
+      avgRegionSize = targetRegionSize;
+    } else if (targetRegionCount > 0) {
+      avgRegionSize = totalSizeMb / (double) targetRegionCount;
+    } else {
+      avgRegionSize = acutalRegionCnt == 0 ? 0 : totalSizeMb / (double) acutalRegionCnt;
+    }
 
     LOG.debug("Table " + table + ", total aggregated regions size: " + totalSizeMb);
     LOG.debug("Table " + table + ", average region size: " + avgRegionSize);

http://git-wip-us.apache.org/repos/asf/hbase/blob/21e02813/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
index 34489ad..743ec2b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
@@ -367,6 +368,114 @@ public class TestSimpleRegionNormalizer {
     assertEquals(hri4, ((SplitNormalizationPlan) plan).getRegionInfo());
   }
 
+  @Test
+  public void testSplitWithTargetRegionCount() throws Exception {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    List<RegionInfo> RegionInfo = new ArrayList<>();
+    Map<byte[], Integer> regionSizes = new HashMap<>();
+
+    RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("aaa"))
+        .setEndKey(Bytes.toBytes("bbb")).build();
+    RegionInfo.add(hri1);
+    regionSizes.put(hri1.getRegionName(), 20);
+
+    RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("bbb"))
+        .setEndKey(Bytes.toBytes("ccc")).build();
+    RegionInfo.add(hri2);
+    regionSizes.put(hri2.getRegionName(), 40);
+
+    RegionInfo hri3 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("ccc"))
+        .setEndKey(Bytes.toBytes("ddd")).build();
+    RegionInfo.add(hri3);
+    regionSizes.put(hri3.getRegionName(), 60);
+
+    RegionInfo hri4 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("ddd"))
+        .setEndKey(Bytes.toBytes("eee")).build();
+    RegionInfo.add(hri4);
+    regionSizes.put(hri4.getRegionName(), 80);
+
+    RegionInfo hri5 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("eee"))
+        .setEndKey(Bytes.toBytes("fff")).build();
+    RegionInfo.add(hri5);
+    regionSizes.put(hri5.getRegionName(), 100);
+
+    RegionInfo hri6 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("fff"))
+        .setEndKey(Bytes.toBytes("ggg")).build();
+    RegionInfo.add(hri6);
+    regionSizes.put(hri6.getRegionName(), 120);
+
+    setupMocksForNormalizer(regionSizes, RegionInfo);
+
+    // test when target region size is 20
+    when(masterServices.getTableDescriptors().get(any()).getNormalizerTargetRegionSize())
+        .thenReturn(20L);
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
+    Assert.assertEquals(4, plans.size());
+
+    for (NormalizationPlan plan : plans) {
+      assertTrue(plan instanceof SplitNormalizationPlan);
+    }
+
+    // test when target region size is 200
+    when(masterServices.getTableDescriptors().get(any()).getNormalizerTargetRegionSize())
+        .thenReturn(200L);
+    plans = normalizer.computePlanForTable(tableName);
+    Assert.assertEquals(2, plans.size());
+    NormalizationPlan plan = plans.get(0);
+    assertTrue(plan instanceof MergeNormalizationPlan);
+    assertEquals(hri1, ((MergeNormalizationPlan) plan).getFirstRegion());
+    assertEquals(hri2, ((MergeNormalizationPlan) plan).getSecondRegion());
+  }
+
+  @Test
+  public void testSplitWithTargetRegionSize() throws Exception {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    List<RegionInfo> RegionInfo = new ArrayList<>();
+    Map<byte[], Integer> regionSizes = new HashMap<>();
+
+    RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("aaa"))
+        .setEndKey(Bytes.toBytes("bbb")).build();
+    RegionInfo.add(hri1);
+    regionSizes.put(hri1.getRegionName(), 20);
+
+    RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("bbb"))
+        .setEndKey(Bytes.toBytes("ccc")).build();
+    RegionInfo.add(hri2);
+    regionSizes.put(hri2.getRegionName(), 40);
+
+    RegionInfo hri3 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("ccc"))
+        .setEndKey(Bytes.toBytes("ddd")).build();
+    RegionInfo.add(hri3);
+    regionSizes.put(hri3.getRegionName(), 60);
+
+    RegionInfo hri4 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("ddd"))
+        .setEndKey(Bytes.toBytes("eee")).build();
+    RegionInfo.add(hri4);
+    regionSizes.put(hri4.getRegionName(), 80);
+
+    setupMocksForNormalizer(regionSizes, RegionInfo);
+
+    // test when target region count is 8
+    when(masterServices.getTableDescriptors().get(any()).getNormalizerTargetRegionCount())
+        .thenReturn(8);
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
+    Assert.assertEquals(2, plans.size());
+
+    for (NormalizationPlan plan : plans) {
+      assertTrue(plan instanceof SplitNormalizationPlan);
+    }
+
+    // test when target region count is 3
+    when(masterServices.getTableDescriptors().get(any()).getNormalizerTargetRegionCount())
+        .thenReturn(3);
+    plans = normalizer.computePlanForTable(tableName);
+    Assert.assertEquals(1, plans.size());
+    NormalizationPlan plan = plans.get(0);
+    assertTrue(plan instanceof MergeNormalizationPlan);
+    assertEquals(hri1, ((MergeNormalizationPlan) plan).getFirstRegion());
+    assertEquals(hri2, ((MergeNormalizationPlan) plan).getSecondRegion());
+  }
+
   @SuppressWarnings("MockitoCast")
   protected void setupMocksForNormalizer(Map<byte[], Integer> regionSizes,
                                          List<RegionInfo> RegionInfo) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/21e02813/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 75d2de3..2be059d 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -1250,6 +1250,8 @@ module Hbase
       htd.setReadOnly(JBoolean.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::READONLY))) if arg.include?(org.apache.hadoop.hbase.HTableDescriptor::READONLY)
       htd.setCompactionEnabled(JBoolean.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::COMPACTION_ENABLED))) if arg.include?(org.apache.hadoop.hbase.HTableDescriptor::COMPACTION_ENABLED)
       htd.setNormalizationEnabled(JBoolean.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::NORMALIZATION_ENABLED))) if arg.include?(org.apache.hadoop.hbase.HTableDescriptor::NORMALIZATION_ENABLED)
+      htd.setNormalizerTargetRegionCount(JInteger.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::NORMALIZER_TARGET_REGION_COUNT))) if arg.include?(org.apache.hadoop.hbase.HTableDescriptor::NORMALIZER_TARGET_REGION_COUNT)
+      htd.setNormalizerTargetRegionSize(JLong.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::NORMALIZER_TARGET_REGION_SIZE))) if arg.include?(org.apache.hadoop.hbase.HTableDescriptor::NORMALIZER_TARGET_REGION_SIZE)
       htd.setMemStoreFlushSize(JLong.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::MEMSTORE_FLUSHSIZE))) if arg.include?(org.apache.hadoop.hbase.HTableDescriptor::MEMSTORE_FLUSHSIZE)
       htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::DURABILITY))) if arg.include?(org.apache.hadoop.hbase.HTableDescriptor::DURABILITY)
       htd.setPriority(JInteger.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::PRIORITY))) if arg.include?(org.apache.hadoop.hbase.HTableDescriptor::PRIORITY)

http://git-wip-us.apache.org/repos/asf/hbase/blob/21e02813/hbase-shell/src/main/ruby/shell/commands/alter.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/alter.rb b/hbase-shell/src/main/ruby/shell/commands/alter.rb
index 4aef28c..5ba7551 100644
--- a/hbase-shell/src/main/ruby/shell/commands/alter.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/alter.rb
@@ -47,7 +47,8 @@ To delete the 'f1' column family in table 'ns1:t1', use one of:
   hbase> alter 'ns1:t1', 'delete' => 'f1'
 
 You can also change table-scope attributes like MAX_FILESIZE, READONLY,
-MEMSTORE_FLUSHSIZE, DURABILITY, etc. These can be put at the end;
+MEMSTORE_FLUSHSIZE, NORMALIZATION_ENABLED, NORMALIZER_TARGET_REGION_COUNT,
+NORMALIZER_TARGET_REGION_SIZE(MB), DURABILITY, etc. These can be put at the end;
 for example, to change the max size of a region to 128MB, do:
 
   hbase> alter 't1', MAX_FILESIZE => '134217728'

http://git-wip-us.apache.org/repos/asf/hbase/blob/21e02813/hbase-shell/src/test/ruby/hbase/admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/admin_test.rb b/hbase-shell/src/test/ruby/hbase/admin_test.rb
index 69dde45..1a12921 100644
--- a/hbase-shell/src/test/ruby/hbase/admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/admin_test.rb
@@ -516,6 +516,13 @@ module Hbase
       assert_match(/12345678/, admin.describe(@test_name))
     end
 
+    define_test 'alter should be able to set the TargetRegionSize and TargetRegionCount' do
+      command(:alter, @test_name, 'NORMALIZER_TARGET_REGION_COUNT' => 156)
+      assert_match(/156/, admin.describe(@test_name))
+      command(:alter, @test_name, 'NORMALIZER_TARGET_REGION_SIZE' => 234)
+      assert_match(/234/, admin.describe(@test_name))
+    end
+
     define_test 'alter should support shortcut DELETE alter specs' do
       assert_equal(['x:', 'y:'], table(@test_name).get_all_columns.sort)
       command(:alter, @test_name, 'delete' => 'y')


[15/15] hbase git commit: WIP turn on command echo fo run-flaky.

Posted by bu...@apache.org.
WIP turn on command echo fo run-flaky.


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

Branch: refs/heads/HBASE-20387
Commit: cc3efcecac63652d3b5d8287031fa53ee97406cf
Parents: 736b43a
Author: Sean Busbey <bu...@apache.org>
Authored: Tue Aug 14 13:03:44 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Tue Aug 14 13:03:44 2018 -0500

----------------------------------------------------------------------
 dev-support/flaky-tests/run-flaky-tests.Jenkinsfile | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cc3efcec/dev-support/flaky-tests/run-flaky-tests.Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/flaky-tests/run-flaky-tests.Jenkinsfile b/dev-support/flaky-tests/run-flaky-tests.Jenkinsfile
index cacb175..9c369c1 100644
--- a/dev-support/flaky-tests/run-flaky-tests.Jenkinsfile
+++ b/dev-support/flaky-tests/run-flaky-tests.Jenkinsfile
@@ -42,6 +42,8 @@ pipeline {
       steps {
         sh '''#!/usr/bin/env bash
           set -e
+          // temp because this fails when not in debug
+          set -x
           if [ "${DEBUG}" = "true" ]; then
             curl_debug="-v"
             mvn_debug="-X"


[03/15] hbase git commit: Add brfrn169 (Toshihiro Suzuki) as a committer

Posted by bu...@apache.org.
Add brfrn169 (Toshihiro Suzuki) as a committer


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

Branch: refs/heads/HBASE-20387
Commit: 7f3d9f6f9c3ead6116ebede9730ca0ee9e0ac7f6
Parents: 699ea4c
Author: Toshihiro Suzuki <br...@gmail.com>
Authored: Sun Aug 12 01:30:29 2018 +0900
Committer: Toshihiro Suzuki <br...@gmail.com>
Committed: Sun Aug 12 01:30:29 2018 +0900

----------------------------------------------------------------------
 pom.xml | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7f3d9f6f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d0320db..30a18e9 100755
--- a/pom.xml
+++ b/pom.xml
@@ -225,6 +225,12 @@
       <timezone>+8</timezone>
     </developer>
     <developer>
+      <id>brfrn169</id>
+      <name>Toshihiro Suzuki</name>
+      <email>brfrn169@apache.org</email>
+      <timezone>+9</timezone>
+    </developer>
+    <developer>
       <id>busbey</id>
       <name>Sean Busbey</name>
       <email>busbey@apache.org</email>


[09/15] hbase git commit: HBASE-20975 Lock may not be taken or released while rolling back procedure

Posted by bu...@apache.org.
HBASE-20975 Lock may not be taken or released while rolling back procedure


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

Branch: refs/heads/HBASE-20387
Commit: a07e755625382d3904c935c21a4f240ede6d2f43
Parents: 21e0281
Author: Allan Yang <al...@apache.org>
Authored: Mon Aug 13 20:23:04 2018 +0800
Committer: Allan Yang <al...@apache.org>
Committed: Mon Aug 13 20:23:04 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/procedure2/ProcedureExecutor.java  | 16 ++++------------
 1 file changed, 4 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a07e7556/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index 2ee80d7..464eaeb 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -1363,29 +1363,21 @@ public class ProcedureExecutor<TEnvironment> {
     assert subprocStack != null : "Called rollback with no steps executed rootProc=" + rootProc;
 
     int stackTail = subprocStack.size();
-    boolean reuseLock = false;
-    while (stackTail --> 0) {
+    while (stackTail-- > 0) {
       Procedure<TEnvironment> proc = subprocStack.get(stackTail);
 
-      LockState lockState;
-      if (!reuseLock && (lockState = acquireLock(proc)) != LockState.LOCK_ACQUIRED) {
+      LockState lockState = acquireLock(proc);
+      if (lockState != LockState.LOCK_ACQUIRED) {
         // can't take a lock on the procedure, add the root-proc back on the
         // queue waiting for the lock availability
         return lockState;
       }
 
       lockState = executeRollback(proc);
+      releaseLock(proc, false);
       boolean abortRollback = lockState != LockState.LOCK_ACQUIRED;
       abortRollback |= !isRunning() || !store.isRunning();
 
-      // If the next procedure is the same to this one
-      // (e.g. StateMachineProcedure reuse the same instance)
-      // we can avoid to lock/unlock each step
-      reuseLock = stackTail > 0 && (subprocStack.get(stackTail - 1) == proc) && !abortRollback;
-      if (!reuseLock && proc.hasLock()) {
-        releaseLock(proc, false);
-      }
-
       // allows to kill the executor before something is stored to the wal.
       // useful to test the procedure recovery.
       if (abortRollback) {


[07/15] hbase git commit: HBASE-21025 Addendum missed a 'succ = true'

Posted by bu...@apache.org.
HBASE-21025 Addendum missed a 'succ = true'

Signed-off-by: Guanghao Zhang <zg...@apache.org>


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

Branch: refs/heads/HBASE-20387
Commit: 911a6ef4123f0e2118a5853890d4514007306047
Parents: 067fbaa
Author: Duo Zhang <zh...@apache.org>
Authored: Mon Aug 13 10:50:45 2018 +0800
Committer: Duo Zhang <zh...@apache.org>
Committed: Mon Aug 13 10:53:47 2018 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/master/TableStateManager.java | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/911a6ef4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
index ecd4a7d..6e34618 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
@@ -233,6 +233,7 @@ public class TableStateManager {
     try {
       MetaTableAccessor.updateTableState(master.getConnection(), tableName, newState);
       tableName2State.put(tableName, newState);
+      succ = true;
     } finally {
       if (!succ) {
         tableName2State.remove(tableName);


[06/15] hbase git commit: HBASE-21038 SAXParseException when hbase.spark.use.hbasecontext=false (Ajith S)

Posted by bu...@apache.org.
HBASE-21038 SAXParseException when hbase.spark.use.hbasecontext=false (Ajith S)


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

Branch: refs/heads/HBASE-20387
Commit: 067fbaa5b5b56dd68700f805f192457f0a0d6120
Parents: 953e5aa
Author: tedyu <yu...@gmail.com>
Authored: Sun Aug 12 09:29:37 2018 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Sun Aug 12 09:29:37 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/spark/DefaultSource.scala      |  4 ++--
 .../hadoop/hbase/spark/DefaultSourceSuite.scala | 23 ++++++++++++++++++++
 2 files changed, 25 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/067fbaa5/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
index 08df635..9e13954 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
@@ -105,7 +105,7 @@ case class HBaseRelation (
 
   val catalog = HBaseTableCatalog(parameters)
   def tableName = catalog.name
-  val configResources = parameters.getOrElse(HBaseSparkConf.HBASE_CONFIG_LOCATION, "")
+  val configResources = parameters.get(HBaseSparkConf.HBASE_CONFIG_LOCATION)
   val useHBaseContext =  parameters.get(HBaseSparkConf.USE_HBASECONTEXT).map(_.toBoolean).getOrElse(HBaseSparkConf.DEFAULT_USE_HBASECONTEXT)
   val usePushDownColumnFilter = parameters.get(HBaseSparkConf.PUSHDOWN_COLUMN_FILTER)
     .map(_.toBoolean).getOrElse(HBaseSparkConf.DEFAULT_PUSHDOWN_COLUMN_FILTER)
@@ -132,7 +132,7 @@ case class HBaseRelation (
     LatestHBaseContextCache.latest
   } else {
     val config = HBaseConfiguration.create()
-    configResources.split(",").foreach( r => config.addResource(r))
+    configResources.map(resource => resource.split(",").foreach(r => config.addResource(r)))
     new HBaseContext(sqlContext.sparkContext, config)
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/067fbaa5/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala
index 6298ed0..89b68f4 100644
--- a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala
+++ b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala
@@ -28,6 +28,7 @@ import org.apache.spark.sql.functions._
 import org.apache.spark.sql.{DataFrame, SQLContext}
 import org.apache.spark.{SparkConf, SparkContext}
 import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
+import org.xml.sax.SAXParseException
 
 case class HBaseRecord(
   col0: String,
@@ -1038,4 +1039,26 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging {
     s.show()
     assert(s.count() == 7)
   }
+
+  test("test create HBaseRelation with new context throws SAXParseException") {
+    val catalog = s"""{
+                     |"table":{"namespace":"default", "name":"t1NotThere"},
+                     |"rowkey":"key",
+                     |"columns":{
+                     |"KEY_FIELD":{"cf":"rowkey", "col":"key", "type":"string"},
+                     |"A_FIELD":{"cf":"c", "col":"a", "type":"string"},
+                     |"B_FIELD":{"cf":"c", "col":"c", "type":"string"}
+                     |}
+                     |}""".stripMargin
+    try {
+      HBaseRelation(Map(HBaseTableCatalog.tableCatalog -> catalog,
+        HBaseSparkConf.USE_HBASECONTEXT -> "false"), None)(sqlContext)
+    } catch {
+        case e: Throwable => if(e.getCause.isInstanceOf[SAXParseException]) {
+          fail("SAXParseException due to configuration loading empty resource")
+        } else {
+          println("Failed due to some other exception, ignore " + e.getMessage)
+        }
+    }
+  }
 }


[02/15] hbase git commit: HBASE-21012 Revert the change of serializing TimeRangeTracker

Posted by bu...@apache.org.
HBASE-21012 Revert the change of serializing TimeRangeTracker

Signed-off-by: Michael Stack <st...@apache.org>
Signed-off-by: zhangduo <zh...@apache.org>
Signed-off-by: Chia-Ping Tsai <ch...@gmail.com>


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

Branch: refs/heads/HBASE-20387
Commit: 699ea4c7d02c4871a57c1cb1a984c3451c54cc6b
Parents: a3ab930
Author: brandboat <br...@gmail.com>
Authored: Thu Aug 9 12:27:25 2018 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Sat Aug 11 22:17:04 2018 +0800

----------------------------------------------------------------------
 .../hbase/regionserver/TimeRangeTracker.java    | 27 +++++++++++++++-----
 src/main/asciidoc/_chapters/upgrading.adoc      | 12 +++++++++
 2 files changed, 32 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/699ea4c7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java
index 5c0eee5..d995fc6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java
@@ -19,7 +19,9 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -206,13 +208,24 @@ public abstract class TimeRangeTracker {
     }
   }
 
-  public static byte[] toByteArray(TimeRangeTracker tracker) {
-    return ProtobufUtil.prependPBMagic(
-        HBaseProtos.TimeRangeTracker.newBuilder()
-          .setFrom(tracker.getMin())
-          .setTo(tracker.getMax())
-          .build()
-          .toByteArray());
+  /**
+   * This method used to serialize TimeRangeTracker (TRT) by protobuf while this breaks the
+   * forward compatibility on HFile.(See HBASE-21008) In previous hbase version ( < 2.0.0 ) we use
+   * DataOutput to serialize TRT, these old versions don't have capability to deserialize TRT
+   * which is serialized by protobuf. So we need to revert the change of serializing
+   * TimeRangeTracker back to DataOutput. For more information, please check HBASE-21012.
+   * @param tracker TimeRangeTracker needed to be serialized.
+   * @return byte array filled with serialized TimeRangeTracker.
+   * @throws IOException if something goes wrong in writeLong.
+   */
+  public static byte[] toByteArray(TimeRangeTracker tracker) throws IOException {
+    try (ByteArrayOutputStream bos = new ByteArrayOutputStream()) {
+      try (DataOutputStream dos = new DataOutputStream(bos)) {
+        dos.writeLong(tracker.getMin());
+        dos.writeLong(tracker.getMax());
+        return bos.toByteArray();
+      }
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/699ea4c7/src/main/asciidoc/_chapters/upgrading.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/upgrading.adoc b/src/main/asciidoc/_chapters/upgrading.adoc
index bc2ec1c..6dc788a 100644
--- a/src/main/asciidoc/_chapters/upgrading.adoc
+++ b/src/main/asciidoc/_chapters/upgrading.adoc
@@ -588,6 +588,18 @@ The internal changes to HBase during this upgrade were sufficient for compilatio
 
 If you previously relied on client side tracing integrated with HBase operations, it is recommended that you upgrade your usage to HTrace 4 as well.
 
+[[upgrade2.0.hfile.compatability]]
+.HFile lose forward compatability
+
+HFiles generated by 2.0.0, 2.0.1, 2.1.0 are not forward compatible to 1.4.6-, 1.3.2.1-, 1.2.6.1-,
+and other inactive releases. Why HFile lose compatability is hbase in new versions
+(2.0.0, 2.0.1, 2.1.0) use protobuf to serialize/deserialize TimeRangeTracker (TRT) while old
+versions use DataInput/DataOutput. To solve this, We have to put
+link:https://jira.apache.org/jira/browse/HBASE-21012[HBASE-21012]
+to 2.x and put link:https://jira.apache.org/jira/browse/HBASE-21013[HBASE-21013] in 1.x.
+For more information, please check
+link:https://jira.apache.org/jira/browse/HBASE-21008[HBASE-21008].
+
 [[upgrade2.0.perf]]
 .Performance
 


[14/15] hbase git commit: HBASE-20387 turn flaky test tracking infra into per-branch pipeline.

Posted by bu...@apache.org.
HBASE-20387 turn flaky test tracking infra into per-branch pipeline.

* gather up all the flaky test stuff into a directory
* create Jenkins Pipeline DSL for the report generation and the flaky re-testing
* have the nightly per-branch job consume the results of flaky reporting


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

Branch: refs/heads/HBASE-20387
Commit: 736b43a45be6a60b2566e284888635b9a3c23c33
Parents: 67b7976
Author: Sean Busbey <bu...@apache.org>
Authored: Fri Aug 10 11:28:10 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Tue Aug 14 12:46:39 2018 -0500

----------------------------------------------------------------------
 dev-support/Jenkinsfile                         |   6 +-
 dev-support/findHangingTests.py                 | 115 --------
 dev-support/flaky-dashboard-template.html       | 199 -------------
 dev-support/flaky-tests/findHangingTests.py     | 115 ++++++++
 .../flaky-tests/flaky-dashboard-template.html   | 199 +++++++++++++
 .../flaky-tests/flaky-reporting.Jenkinsfile     |  66 +++++
 dev-support/flaky-tests/report-flakies.py       | 280 +++++++++++++++++++
 .../flaky-tests/run-flaky-tests.Jenkinsfile     |  65 +++++
 dev-support/hbase_nightly_yetus.sh              |   4 -
 dev-support/report-flakies.py                   | 280 -------------------
 10 files changed, 726 insertions(+), 603 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/736b43a4/dev-support/Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index 59d3227..7c3538a 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -47,11 +47,7 @@ pipeline {
     ARCHIVE_PATTERN_LIST = 'TEST-*.xml,org.apache.h*.txt,*.dumpstream,*.dump'
     // These tests currently have known failures. Once they burn down to 0, remove from here so that new problems will cause a failure.
     TESTS_FILTER = 'cc,checkstyle,javac,javadoc,pylint,shellcheck,whitespace,perlcritic,ruby-lint,rubocop,mvnsite'
-    // Flaky urls for different branches. Replace '-' and '.' in branch name by '_' because those
-    // characters are not allowed in bash variable name.
-    // Not excluding flakies from the nightly build for now.
-    // EXCLUDE_TESTS_URL_master = 'https://builds.apache.org/job/HBase-Find-Flaky-Tests/lastSuccessfulBuild/artifact/excludes/'
-    // EXCLUDE_TESTS_URL_branch_2 = 'https://builds.apache.org/job/HBase-Find-Flaky-Tests-branch2.0/lastSuccessfulBuild/artifact/excludes/'
+    EXCLUDE_TESTS_URL = "${JENKINS_URL}/job/HBase-Find-Flaky-Tests-per-branch/job/${BRANCH_NAME}/lastSuccessfulBuild/artifact/excludes"
   }
   parameters {
     booleanParam(name: 'USE_YETUS_PRERELEASE', defaultValue: false, description: '''Check to use the current HEAD of apache/yetus rather than our configured release.

http://git-wip-us.apache.org/repos/asf/hbase/blob/736b43a4/dev-support/findHangingTests.py
----------------------------------------------------------------------
diff --git a/dev-support/findHangingTests.py b/dev-support/findHangingTests.py
deleted file mode 100755
index 328516e..0000000
--- a/dev-support/findHangingTests.py
+++ /dev/null
@@ -1,115 +0,0 @@
-#!/usr/bin/env python
-##
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# pylint: disable=invalid-name
-# To disable 'invalid constant name' warnings.
-
-"""
-# Script to find hanging test from Jenkins build output
-# usage: ./findHangingTests.py <url of Jenkins build console>
-"""
-
-import re
-import sys
-import requests
-
-# If any of these strings appear in the console output, it's a build one should probably ignore
-# for analyzing failed/hanging tests.
-BAD_RUN_STRINGS = [
-    "Slave went offline during the build",  # Machine went down, can't do anything about it.
-    "The forked VM terminated without properly saying goodbye",  # JVM crashed.
-]
-
-
-def get_bad_tests(console_url):
-    """
-    Returns [[all tests], [failed tests], [timeout tests], [hanging tests]] if successfully gets
-    the build information.
-    If there is error getting console text or if there are blacklisted strings in console text,
-    then returns None.
-    """
-    response = requests.get(console_url)
-    if response.status_code != 200:
-        print "Error getting consoleText. Response = {} {}".format(
-            response.status_code, response.reason)
-        return
-
-    # All tests: All testcases which were run.
-    # Hanging test: A testcase which started but never finished.
-    # Failed test: Testcase which encountered any kind of failure. It can be failing atomic tests,
-    #   timed out tests, etc
-    # Timeout test: A Testcase which encountered timeout. Naturally, all timeout tests will be
-    #   included in failed tests.
-    all_tests_set = set()
-    hanging_tests_set = set()
-    failed_tests_set = set()
-    timeout_tests_set = set()
-    for line in response.content.splitlines():
-        result1 = re.findall("Running org.apache.hadoop.hbase.(.*)", line)
-        if len(result1) == 1:
-            test_case = result1[0]
-            if test_case in all_tests_set:
-                print ("ERROR! Multiple tests with same name '{}'. Might get wrong results "
-                       "for this test.".format(test_case))
-            else:
-                hanging_tests_set.add(test_case)
-                all_tests_set.add(test_case)
-        result2 = re.findall("Tests run:.*?- in org.apache.hadoop.hbase.(.*)", line)
-        if len(result2) == 1:
-            test_case = result2[0]
-            if "FAILURE!" in line:
-                failed_tests_set.add(test_case)
-            if test_case not in hanging_tests_set:
-                print ("ERROR! No test '{}' found in hanging_tests. Might get wrong results "
-                       "for this test. This may also happen if maven is set to retry failing "
-                       "tests.".format(test_case))
-            else:
-                hanging_tests_set.remove(test_case)
-        result3 = re.match("^\\s+(\\w*).*\\sTestTimedOut", line)
-        if result3:
-            test_case = result3.group(1)
-            timeout_tests_set.add(test_case)
-        for bad_string in BAD_RUN_STRINGS:
-            if re.match(".*" + bad_string + ".*", line):
-                print "Bad string found in build:\n > {}".format(line)
-    print "Result > total tests: {:4}   failed : {:4}  timedout : {:4}  hanging : {:4}".format(
-        len(all_tests_set), len(failed_tests_set), len(timeout_tests_set), len(hanging_tests_set))
-    return [all_tests_set, failed_tests_set, timeout_tests_set, hanging_tests_set]
-
-if __name__ == "__main__":
-    if len(sys.argv) != 2:
-        print "ERROR : Provide the jenkins job console URL as the only argument."
-        sys.exit(1)
-
-    print "Fetching {}".format(sys.argv[1])
-    result = get_bad_tests(sys.argv[1])
-    if not result:
-        sys.exit(1)
-    [all_tests, failed_tests, timedout_tests, hanging_tests] = result
-
-    print "Found {} hanging tests:".format(len(hanging_tests))
-    for test in hanging_tests:
-        print test
-    print "\n"
-    print "Found {} failed tests of which {} timed out:".format(
-        len(failed_tests), len(timedout_tests))
-    for test in failed_tests:
-        print "{0} {1}".format(test, ("(Timed Out)" if test in timedout_tests else ""))
-
-    print ("\nA test may have had 0 or more atomic test failures before it timed out. So a "
-           "'Timed Out' test may have other errors too.")

http://git-wip-us.apache.org/repos/asf/hbase/blob/736b43a4/dev-support/flaky-dashboard-template.html
----------------------------------------------------------------------
diff --git a/dev-support/flaky-dashboard-template.html b/dev-support/flaky-dashboard-template.html
deleted file mode 100644
index f37c7d5..0000000
--- a/dev-support/flaky-dashboard-template.html
+++ /dev/null
@@ -1,199 +0,0 @@
-<!--
- - Licensed to the Apache Software Foundation (ASF) under one
- - or more contributor license agreements.  See the NOTICE file
- - distributed with this work for additional information
- - regarding copyright ownership.  The ASF licenses this file
- - to you under the Apache License, Version 2.0 (the
- - "License"); you may not use this file except in compliance
- - with the License.  You may obtain a copy of the License at
- -
- -     http://www.apache.org/licenses/LICENSE-2.0
- -
- - Unless required by applicable law or agreed to in writing, software
- - distributed under the License is distributed on an "AS IS" BASIS,
- - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- - See the License for the specific language governing permissions and
- - limitations under the License.
- -->
-<!DOCTYPE html>
-<html>
-<head>
-    <title>Apache HBase Flaky Dashboard</title>
-    <style type="text/css">
-        table {
-            table-layout: fixed;
-        }
-        th {
-            font-size: 15px;
-        }
-        td {
-            font-size: 18px;
-            vertical-align: text-top;
-            overflow: hidden;
-            white-space: nowrap;
-        }
-        .show_hide_button {
-            font-size: 100%;
-            padding: .5em 1em;
-            border: 0 rgba(0,0,0,0);
-            border-radius: 10px;
-        }
-    </style>
-</head>
-<body>
-    <script src="https://d3js.org/d3.v3.min.js"></script>
-    <script>
-        var width = 300;
-        var height = 25;
-        var x = d3.scale.linear().range([0, width]);
-
-        function csvToArray(csv) {
-            if (csv.length == 0)
-                return [];
-            splits = csv.split(",");
-            ret = [];
-            for (i = 0; i < splits.length; i++) {
-                ret.push(parseInt(splits[i]));
-            }
-            return ret;
-        }
-
-        function sortNumber(a,b) {
-            return a - b;
-        }
-
-        function sparkline(elemId, failed, timeout, hanging, success, domain_min, domain_max) {
-            failed = csvToArray(failed);
-            timeout = csvToArray(timeout);
-            hanging = csvToArray(hanging);
-            success = csvToArray(success);
-            all = failed.concat(timeout).concat(hanging).concat(success);
-            all.sort(sortNumber);
-            x.domain([domain_min, domain_max + 1]);
-            rect_width = x(domain_min + 1) - x(domain_min) - 1;
-            svg = d3.select("#" + elemId).append('svg').attr('width', width).attr('height', height);
-            svg.selectAll("dot")
-                    .data(all)
-                    .enter()
-                    .append("svg:rect")
-                    .attr("x", function(d) { return x(d); })
-                    .attr("y", 3)
-                    .attr("height", height- 6)
-                    .attr("width", rect_width)
-                    .attr("fill", function(d) {
-                        if (success.includes(d)) return "green";
-                        else if (timeout.includes(d)) return "gold";
-                        else if (hanging.includes(d)) return "blue";
-                        else if (failed.includes(d)) return "red";
-                        else return "black";
-                    })
-                    .append('svg:title')
-                    .text(function(d) { return d; });
-        }
-    </script>
-<p>
-    <img style="vertical-align:middle; display:inline-block;" height="80px"
-         src="https://hbase.apache.org/images/hbase_logo_with_orca_large.png">
-    &nbsp;&nbsp;&nbsp;&nbsp;
-              <span style="font-size:50px; vertical-align:middle; display:inline-block;">
-                  Apache HBase Flaky Tests Dashboard
-              </span>
-</p>
-<span>Last updated: <b>{{datetime}}</b></span><br>
-<span>Count of flaky tests (cumulated from all jobs):
-    <b>{{bad_tests_count}}</b></span><br>
-<br><br>
-<span style="font-size:20px;"><b>List of Jobs</b></span><br>
-<br>
-{% for url in results %}
-<a href="#job_{{ loop.index }}">{{ url |e }}</a>
-<br>
-{% endfor %}
-<br>
-<br>
-<span style="font-size:20px;"><b>Results</b></span><br>
-<br>
-{% for url in results %}
-{% set result = results[url] %}
-{% set url_counter = loop.index %}
-{# Dedup ids since test names may duplicate across urls #}
-<span id="job_{{ url_counter }}" style="font-weight:bold;">
-    {{ url |e }}<br>
-    <a href="{{ url |e }}">
-        Go to <img height="16px" src="https://jenkins.io/sites/default/files/jenkins_favicon.ico">
-    </a>
-    &nbsp;&nbsp;&nbsp;&nbsp;
-    <a href="#">Go to top</a>
-</span>
-<br/><br/>
-Legend : green: success, red: failed, yellow: timeout, blue: hanging
-<table>
-    <tr>
-        <th width="400px">Test Name</th>
-        <th width="150px">Flakyness</th>
-        <th width="200px">Failed/Timeout/Hanging</th>
-        <th width="300px">Trends</th>
-        <th>Run Ids</th>
-    </tr>
-    {% for test in result %}
-    {% set all = result[test]['all'] %}
-    {% set failed = result[test]['failed'] %}
-    {% set timeout = result[test]['timeout'] %}
-    {% set hanging = result[test]['hanging'] %}
-    {% set success = result[test]['success'] %}
-    <tr>
-        <td>{{ test |e }}</td>
-        {% set flakyness =
-        (failed|length + hanging|length) * 100 / all|length %}
-        {% if flakyness == 100 %}
-        <td align="middle" style="background-color:#FF9999;">
-            {% else %}
-        <td align="middle">
-            {% endif %}
-            {{ "{:.1f}% ({} / {})".format(
-            flakyness, failed|length + hanging|length, all|length) }}
-        </td>
-        <td align="middle">
-            {{ failed|length }} / {{ timeout|length }} / {{ hanging|length }}
-        </td>
-        {# Replace '.' in test names with '_' because dots are part of css selectors.  #}
-        {% set sparkline_id = "sparkline_" ~ test|replace(".","_") ~ "_" ~ url_counter  %}
-        <td id="{{ sparkline_id }}" align="middle">
-        </td>
-        <script>sparkline("{{ sparkline_id }}", "{{ failed|join(',') }}", "{{ timeout|join(',') }}",
-                "{{ hanging|join(',') }}", "{{ success|join(',') }}", {{ build_ids[url][0] }},
-                {{ build_ids[url][-1] }});</script>
-        <td>
-            {% set id = "details_" ~ test ~ "_" ~ url_counter %}
-            <button class="show_hide_button" onclick="toggle('{{ id }}')">
-                show/hide</button>
-            <br/>
-            <div id="{{ id }}"
-                 style="display: none; width:300px; white-space: normal">
-                {% macro print_run_ids(url, run_ids) -%}
-                {% for i in run_ids %}
-                <a href="{{ url }}/{{ i }}">{{ i }}</a>&nbsp;
-                {% endfor %}
-                {%- endmacro %}
-                Failed : {{ print_run_ids(url, failed) }}<br/>
-                Timed Out : {{ print_run_ids(url, timeout) }}<br/>
-                Hanging : {{ print_run_ids(url, hanging) }}<br/>
-                Succeeded : {{ print_run_ids(url, success) }}
-            </div>
-        </td>
-    </tr>
-    {% endfor %}
-</table>
-<br><br><br>
-{% endfor %}
-<script type="text/javascript">
-    function toggle(id) {
-        if (document.getElementById(id).style["display"] == "none") {
-            document.getElementById(id).style["display"]  = "block";
-        } else {
-            document.getElementById(id).style["display"] = "none";
-        }
-    }
-</script>
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/hbase/blob/736b43a4/dev-support/flaky-tests/findHangingTests.py
----------------------------------------------------------------------
diff --git a/dev-support/flaky-tests/findHangingTests.py b/dev-support/flaky-tests/findHangingTests.py
new file mode 100755
index 0000000..328516e
--- /dev/null
+++ b/dev-support/flaky-tests/findHangingTests.py
@@ -0,0 +1,115 @@
+#!/usr/bin/env python
+##
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# pylint: disable=invalid-name
+# To disable 'invalid constant name' warnings.
+
+"""
+# Script to find hanging test from Jenkins build output
+# usage: ./findHangingTests.py <url of Jenkins build console>
+"""
+
+import re
+import sys
+import requests
+
+# If any of these strings appear in the console output, it's a build one should probably ignore
+# for analyzing failed/hanging tests.
+BAD_RUN_STRINGS = [
+    "Slave went offline during the build",  # Machine went down, can't do anything about it.
+    "The forked VM terminated without properly saying goodbye",  # JVM crashed.
+]
+
+
+def get_bad_tests(console_url):
+    """
+    Returns [[all tests], [failed tests], [timeout tests], [hanging tests]] if successfully gets
+    the build information.
+    If there is error getting console text or if there are blacklisted strings in console text,
+    then returns None.
+    """
+    response = requests.get(console_url)
+    if response.status_code != 200:
+        print "Error getting consoleText. Response = {} {}".format(
+            response.status_code, response.reason)
+        return
+
+    # All tests: All testcases which were run.
+    # Hanging test: A testcase which started but never finished.
+    # Failed test: Testcase which encountered any kind of failure. It can be failing atomic tests,
+    #   timed out tests, etc
+    # Timeout test: A Testcase which encountered timeout. Naturally, all timeout tests will be
+    #   included in failed tests.
+    all_tests_set = set()
+    hanging_tests_set = set()
+    failed_tests_set = set()
+    timeout_tests_set = set()
+    for line in response.content.splitlines():
+        result1 = re.findall("Running org.apache.hadoop.hbase.(.*)", line)
+        if len(result1) == 1:
+            test_case = result1[0]
+            if test_case in all_tests_set:
+                print ("ERROR! Multiple tests with same name '{}'. Might get wrong results "
+                       "for this test.".format(test_case))
+            else:
+                hanging_tests_set.add(test_case)
+                all_tests_set.add(test_case)
+        result2 = re.findall("Tests run:.*?- in org.apache.hadoop.hbase.(.*)", line)
+        if len(result2) == 1:
+            test_case = result2[0]
+            if "FAILURE!" in line:
+                failed_tests_set.add(test_case)
+            if test_case not in hanging_tests_set:
+                print ("ERROR! No test '{}' found in hanging_tests. Might get wrong results "
+                       "for this test. This may also happen if maven is set to retry failing "
+                       "tests.".format(test_case))
+            else:
+                hanging_tests_set.remove(test_case)
+        result3 = re.match("^\\s+(\\w*).*\\sTestTimedOut", line)
+        if result3:
+            test_case = result3.group(1)
+            timeout_tests_set.add(test_case)
+        for bad_string in BAD_RUN_STRINGS:
+            if re.match(".*" + bad_string + ".*", line):
+                print "Bad string found in build:\n > {}".format(line)
+    print "Result > total tests: {:4}   failed : {:4}  timedout : {:4}  hanging : {:4}".format(
+        len(all_tests_set), len(failed_tests_set), len(timeout_tests_set), len(hanging_tests_set))
+    return [all_tests_set, failed_tests_set, timeout_tests_set, hanging_tests_set]
+
+if __name__ == "__main__":
+    if len(sys.argv) != 2:
+        print "ERROR : Provide the jenkins job console URL as the only argument."
+        sys.exit(1)
+
+    print "Fetching {}".format(sys.argv[1])
+    result = get_bad_tests(sys.argv[1])
+    if not result:
+        sys.exit(1)
+    [all_tests, failed_tests, timedout_tests, hanging_tests] = result
+
+    print "Found {} hanging tests:".format(len(hanging_tests))
+    for test in hanging_tests:
+        print test
+    print "\n"
+    print "Found {} failed tests of which {} timed out:".format(
+        len(failed_tests), len(timedout_tests))
+    for test in failed_tests:
+        print "{0} {1}".format(test, ("(Timed Out)" if test in timedout_tests else ""))
+
+    print ("\nA test may have had 0 or more atomic test failures before it timed out. So a "
+           "'Timed Out' test may have other errors too.")

http://git-wip-us.apache.org/repos/asf/hbase/blob/736b43a4/dev-support/flaky-tests/flaky-dashboard-template.html
----------------------------------------------------------------------
diff --git a/dev-support/flaky-tests/flaky-dashboard-template.html b/dev-support/flaky-tests/flaky-dashboard-template.html
new file mode 100644
index 0000000..f37c7d5
--- /dev/null
+++ b/dev-support/flaky-tests/flaky-dashboard-template.html
@@ -0,0 +1,199 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one
+ - or more contributor license agreements.  See the NOTICE file
+ - distributed with this work for additional information
+ - regarding copyright ownership.  The ASF licenses this file
+ - to you under the Apache License, Version 2.0 (the
+ - "License"); you may not use this file except in compliance
+ - with the License.  You may obtain a copy of the License at
+ -
+ -     http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+<!DOCTYPE html>
+<html>
+<head>
+    <title>Apache HBase Flaky Dashboard</title>
+    <style type="text/css">
+        table {
+            table-layout: fixed;
+        }
+        th {
+            font-size: 15px;
+        }
+        td {
+            font-size: 18px;
+            vertical-align: text-top;
+            overflow: hidden;
+            white-space: nowrap;
+        }
+        .show_hide_button {
+            font-size: 100%;
+            padding: .5em 1em;
+            border: 0 rgba(0,0,0,0);
+            border-radius: 10px;
+        }
+    </style>
+</head>
+<body>
+    <script src="https://d3js.org/d3.v3.min.js"></script>
+    <script>
+        var width = 300;
+        var height = 25;
+        var x = d3.scale.linear().range([0, width]);
+
+        function csvToArray(csv) {
+            if (csv.length == 0)
+                return [];
+            splits = csv.split(",");
+            ret = [];
+            for (i = 0; i < splits.length; i++) {
+                ret.push(parseInt(splits[i]));
+            }
+            return ret;
+        }
+
+        function sortNumber(a,b) {
+            return a - b;
+        }
+
+        function sparkline(elemId, failed, timeout, hanging, success, domain_min, domain_max) {
+            failed = csvToArray(failed);
+            timeout = csvToArray(timeout);
+            hanging = csvToArray(hanging);
+            success = csvToArray(success);
+            all = failed.concat(timeout).concat(hanging).concat(success);
+            all.sort(sortNumber);
+            x.domain([domain_min, domain_max + 1]);
+            rect_width = x(domain_min + 1) - x(domain_min) - 1;
+            svg = d3.select("#" + elemId).append('svg').attr('width', width).attr('height', height);
+            svg.selectAll("dot")
+                    .data(all)
+                    .enter()
+                    .append("svg:rect")
+                    .attr("x", function(d) { return x(d); })
+                    .attr("y", 3)
+                    .attr("height", height- 6)
+                    .attr("width", rect_width)
+                    .attr("fill", function(d) {
+                        if (success.includes(d)) return "green";
+                        else if (timeout.includes(d)) return "gold";
+                        else if (hanging.includes(d)) return "blue";
+                        else if (failed.includes(d)) return "red";
+                        else return "black";
+                    })
+                    .append('svg:title')
+                    .text(function(d) { return d; });
+        }
+    </script>
+<p>
+    <img style="vertical-align:middle; display:inline-block;" height="80px"
+         src="https://hbase.apache.org/images/hbase_logo_with_orca_large.png">
+    &nbsp;&nbsp;&nbsp;&nbsp;
+              <span style="font-size:50px; vertical-align:middle; display:inline-block;">
+                  Apache HBase Flaky Tests Dashboard
+              </span>
+</p>
+<span>Last updated: <b>{{datetime}}</b></span><br>
+<span>Count of flaky tests (cumulated from all jobs):
+    <b>{{bad_tests_count}}</b></span><br>
+<br><br>
+<span style="font-size:20px;"><b>List of Jobs</b></span><br>
+<br>
+{% for url in results %}
+<a href="#job_{{ loop.index }}">{{ url |e }}</a>
+<br>
+{% endfor %}
+<br>
+<br>
+<span style="font-size:20px;"><b>Results</b></span><br>
+<br>
+{% for url in results %}
+{% set result = results[url] %}
+{% set url_counter = loop.index %}
+{# Dedup ids since test names may duplicate across urls #}
+<span id="job_{{ url_counter }}" style="font-weight:bold;">
+    {{ url |e }}<br>
+    <a href="{{ url |e }}">
+        Go to <img height="16px" src="https://jenkins.io/sites/default/files/jenkins_favicon.ico">
+    </a>
+    &nbsp;&nbsp;&nbsp;&nbsp;
+    <a href="#">Go to top</a>
+</span>
+<br/><br/>
+Legend : green: success, red: failed, yellow: timeout, blue: hanging
+<table>
+    <tr>
+        <th width="400px">Test Name</th>
+        <th width="150px">Flakyness</th>
+        <th width="200px">Failed/Timeout/Hanging</th>
+        <th width="300px">Trends</th>
+        <th>Run Ids</th>
+    </tr>
+    {% for test in result %}
+    {% set all = result[test]['all'] %}
+    {% set failed = result[test]['failed'] %}
+    {% set timeout = result[test]['timeout'] %}
+    {% set hanging = result[test]['hanging'] %}
+    {% set success = result[test]['success'] %}
+    <tr>
+        <td>{{ test |e }}</td>
+        {% set flakyness =
+        (failed|length + hanging|length) * 100 / all|length %}
+        {% if flakyness == 100 %}
+        <td align="middle" style="background-color:#FF9999;">
+            {% else %}
+        <td align="middle">
+            {% endif %}
+            {{ "{:.1f}% ({} / {})".format(
+            flakyness, failed|length + hanging|length, all|length) }}
+        </td>
+        <td align="middle">
+            {{ failed|length }} / {{ timeout|length }} / {{ hanging|length }}
+        </td>
+        {# Replace '.' in test names with '_' because dots are part of css selectors.  #}
+        {% set sparkline_id = "sparkline_" ~ test|replace(".","_") ~ "_" ~ url_counter  %}
+        <td id="{{ sparkline_id }}" align="middle">
+        </td>
+        <script>sparkline("{{ sparkline_id }}", "{{ failed|join(',') }}", "{{ timeout|join(',') }}",
+                "{{ hanging|join(',') }}", "{{ success|join(',') }}", {{ build_ids[url][0] }},
+                {{ build_ids[url][-1] }});</script>
+        <td>
+            {% set id = "details_" ~ test ~ "_" ~ url_counter %}
+            <button class="show_hide_button" onclick="toggle('{{ id }}')">
+                show/hide</button>
+            <br/>
+            <div id="{{ id }}"
+                 style="display: none; width:300px; white-space: normal">
+                {% macro print_run_ids(url, run_ids) -%}
+                {% for i in run_ids %}
+                <a href="{{ url }}/{{ i }}">{{ i }}</a>&nbsp;
+                {% endfor %}
+                {%- endmacro %}
+                Failed : {{ print_run_ids(url, failed) }}<br/>
+                Timed Out : {{ print_run_ids(url, timeout) }}<br/>
+                Hanging : {{ print_run_ids(url, hanging) }}<br/>
+                Succeeded : {{ print_run_ids(url, success) }}
+            </div>
+        </td>
+    </tr>
+    {% endfor %}
+</table>
+<br><br><br>
+{% endfor %}
+<script type="text/javascript">
+    function toggle(id) {
+        if (document.getElementById(id).style["display"] == "none") {
+            document.getElementById(id).style["display"]  = "block";
+        } else {
+            document.getElementById(id).style["display"] = "none";
+        }
+    }
+</script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/hbase/blob/736b43a4/dev-support/flaky-tests/flaky-reporting.Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/flaky-tests/flaky-reporting.Jenkinsfile b/dev-support/flaky-tests/flaky-reporting.Jenkinsfile
new file mode 100644
index 0000000..0d20336
--- /dev/null
+++ b/dev-support/flaky-tests/flaky-reporting.Jenkinsfile
@@ -0,0 +1,66 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+pipeline {
+  agent {
+    node {
+      label 'Hadoop'
+    }
+  }
+  triggers {
+    cron('@daily')
+  }
+  options {
+    buildDiscarder(logRotator(numToKeepStr: '100'))
+    timeout (time: 15, unit: 'MINUTES')
+    timestamps()
+  }
+  parameters {
+    booleanParam(name: 'DEBUG', defaultValue: false, description: 'Produce a lot more meta-information.')
+  }
+  stages {
+    stage ('build flaky report') {
+      steps {
+        sh '''#!/usr/bin/env bash
+          set -e
+          if [ "${DEBUG}" = "true" ]; then
+            set -x
+          fi
+          declare -a flaky_args
+          flaky_args=("${flaky_args[@]}" --urls "${JENKINS_URL}/job/HBase%20Nightly/job/${BRANCH_NAME}" --is-yetus True --max-builds 5)
+          flaky_args=("${flaky_args[@]}" --urls "${JENKINS_URL}/job/HBase-Flaky-Tests-per-branch/job/${BRANCH_NAME}" --is-yetus False --max-builds 40)
+          docker build -t hbase-dev-support dev-support
+          docker run -v "${WORKSPACE}":/hbase --workdir=/hbase hbase-dev-support python dev-support/flaky-tests/report-flakies.py --mvn -v "${flaky_args[@]}"
+'''
+      }
+    }
+  }
+  post {
+    always {
+      // Has to be relative to WORKSPACE.
+      archive "includes,excludes,dashboard.html"
+      publishHTML target: [
+        allowMissing: true,
+        keepAll: true,
+        alwaysLinkToLastBuild: true,
+        // Has to be relative to WORKSPACE
+        reportDir: ".",
+        reportFiles: 'dashboard.html',
+        reportName: 'Flaky Test Report'
+      ]
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/736b43a4/dev-support/flaky-tests/report-flakies.py
----------------------------------------------------------------------
diff --git a/dev-support/flaky-tests/report-flakies.py b/dev-support/flaky-tests/report-flakies.py
new file mode 100755
index 0000000..1b3161a
--- /dev/null
+++ b/dev-support/flaky-tests/report-flakies.py
@@ -0,0 +1,280 @@
+#!/usr/bin/env python
+##
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# pylint: disable=invalid-name
+# To disable 'invalid constant name' warnings.
+# pylint: disable=import-error
+# Testing environment may not have all dependencies.
+
+"""
+This script uses Jenkins REST api to collect test result(s) of given build/builds and generates
+flakyness data about unittests.
+Print help: report-flakies.py -h
+"""
+
+import argparse
+import logging
+import os
+import time
+from collections import OrderedDict
+from jinja2 import Template
+
+import requests
+
+import findHangingTests
+
+parser = argparse.ArgumentParser()
+parser.add_argument(
+    '--urls', metavar='URL', action='append', required=True,
+    help='Urls to analyze, which can refer to simple projects, multi-configuration projects or '
+         'individual build run.')
+parser.add_argument('--excluded-builds', metavar='n1,n2', action='append',
+                    help='List of build numbers to exclude (or "None"). Not required, '
+                         'but if specified, number of uses should be same as that of --urls '
+                         'since the values are matched.')
+parser.add_argument('--max-builds', metavar='n', action='append', type=int,
+                    help='The maximum number of builds to use (if available on jenkins). Specify '
+                         '0 to analyze all builds. Not required, but if specified, number of uses '
+                         'should be same as that of --urls since the values are matched.')
+parser.add_argument('--is-yetus', metavar='True/False', action='append', choices=['True', 'False'],
+                    help='True, if build is yetus style i.e. look for maven output in artifacts; '
+                         'False, if maven output is in <url>/consoleText itself.')
+parser.add_argument(
+    "--mvn", action="store_true",
+    help="Writes two strings for including/excluding these flaky tests using maven flags. These "
+         "strings are written to files so they can be saved as artifacts and easily imported in "
+         "other projects. Also writes timeout and failing tests in separate files for "
+         "reference.")
+parser.add_argument("-v", "--verbose", help="Prints more logs.", action="store_true")
+args = parser.parse_args()
+
+logging.basicConfig()
+logger = logging.getLogger(__name__)
+if args.verbose:
+    logger.setLevel(logging.INFO)
+
+
+def get_bad_tests(build_url, is_yetus):
+    """
+    Given url of an executed build, analyzes its maven output, and returns
+    [list of all tests, list of timeout tests, list of failed tests].
+    Returns None if can't get maven output from the build or if there is any other error.
+    """
+    logger.info("Analyzing %s", build_url)
+    needed_fields="_class,building"
+    if is_yetus:
+        needed_fields+=",artifacts[fileName,relativePath]"
+    response = requests.get(build_url + "/api/json?tree=" + needed_fields).json()
+    if response["building"]:
+        logger.info("Skipping this build since it is in progress.")
+        return {}
+    console_url = None
+    if is_yetus:
+        for artifact in response["artifacts"]:
+            if artifact["fileName"] == "patch-unit-root.txt":
+                console_url = build_url + "/artifact/" + artifact["relativePath"]
+                break
+        if console_url is None:
+            logger.info("Can't find 'patch-unit-root.txt' artifact for Yetus build %s\n. Ignoring "
+                        "this build.", build_url)
+            return
+    else:
+        console_url = build_url + "/consoleText"
+    build_result = findHangingTests.get_bad_tests(console_url)
+    if not build_result:
+        logger.info("Ignoring build %s", build_url)
+        return
+    return build_result
+
+
+def expand_multi_config_projects(cli_args):
+    """
+    If any url is of type multi-configuration project (i.e. has key 'activeConfigurations'),
+    get urls for individual jobs.
+    """
+    job_urls = cli_args.urls
+    excluded_builds_arg = cli_args.excluded_builds
+    max_builds_arg = cli_args.max_builds
+    is_yetus_arg = cli_args.is_yetus
+    if excluded_builds_arg is not None and len(excluded_builds_arg) != len(job_urls):
+        raise Exception("Number of --excluded-builds arguments should be same as that of --urls "
+                        "since values are matched.")
+    if max_builds_arg is not None and len(max_builds_arg) != len(job_urls):
+        raise Exception("Number of --max-builds arguments should be same as that of --urls "
+                        "since values are matched.")
+    final_expanded_urls = []
+    for (i, job_url) in enumerate(job_urls):
+        max_builds = 10000  # Some high number
+        is_yetus = False
+        if is_yetus_arg is not None:
+            is_yetus = is_yetus_arg[i] == "True"
+        if max_builds_arg is not None and max_builds_arg[i] != 0:
+            max_builds = int(max_builds_arg[i])
+        excluded_builds = []
+        if excluded_builds_arg is not None and excluded_builds_arg[i] != "None":
+            excluded_builds = [int(x) for x in excluded_builds_arg[i].split(",")]
+        request = requests.get(job_url + "/api/json?tree=_class,activeConfigurations%5Burl%5D")
+        if request.status_code != 200:
+            raise Exception("Failed to get job information from jenkins for url '" + job_url +
+                            "'. Jenkins returned HTTP status " + str(request.status_code))
+        response = request.json()
+        if response.has_key("activeConfigurations"):
+            for config in response["activeConfigurations"]:
+                final_expanded_urls.append({'url':config["url"], 'max_builds': max_builds,
+                                            'excludes': excluded_builds, 'is_yetus': is_yetus})
+        else:
+            final_expanded_urls.append({'url':job_url, 'max_builds': max_builds,
+                                        'excludes': excluded_builds, 'is_yetus': is_yetus})
+    return final_expanded_urls
+
+
+# Set of timeout/failed tests across all given urls.
+all_timeout_tests = set()
+all_failed_tests = set()
+all_hanging_tests = set()
+# Contains { <url> : { <bad_test> : { 'all': [<build ids>], 'failed': [<build ids>],
+#                                     'timeout': [<build ids>], 'hanging': [<builds ids>] } } }
+url_to_bad_test_results = OrderedDict()
+# Contains { <url> : [run_ids] }
+# Used for common min/max build ids when generating sparklines.
+url_to_build_ids = OrderedDict()
+
+# Iterates over each url, gets test results and prints flaky tests.
+expanded_urls = expand_multi_config_projects(args)
+for url_max_build in expanded_urls:
+    url = url_max_build["url"]
+    excludes = url_max_build["excludes"]
+    json_response = requests.get(url + "/api/json?tree=id,builds%5Bnumber,url%5D").json()
+    if json_response.has_key("builds"):
+        builds = json_response["builds"]
+        logger.info("Analyzing job: %s", url)
+    else:
+        builds = [{'number': json_response["id"], 'url': url}]
+        logger.info("Analyzing build : %s", url)
+    build_id_to_results = {}
+    num_builds = 0
+    url_to_build_ids[url] = []
+    build_ids_without_tests_run = []
+    for build in builds:
+        build_id = build["number"]
+        if build_id in excludes:
+            continue
+        result = get_bad_tests(build["url"], url_max_build['is_yetus'])
+        if not result:
+            continue
+        if len(result[0]) > 0:
+            build_id_to_results[build_id] = result
+        else:
+            build_ids_without_tests_run.append(build_id)
+        num_builds += 1
+        url_to_build_ids[url].append(build_id)
+        if num_builds == url_max_build["max_builds"]:
+            break
+    url_to_build_ids[url].sort()
+
+    # Collect list of bad tests.
+    bad_tests = set()
+    for build in build_id_to_results:
+        [_, failed_tests, timeout_tests, hanging_tests] = build_id_to_results[build]
+        all_timeout_tests.update(timeout_tests)
+        all_failed_tests.update(failed_tests)
+        all_hanging_tests.update(hanging_tests)
+        # Note that timedout tests are already included in failed tests.
+        bad_tests.update(failed_tests.union(hanging_tests))
+
+    # For each bad test, get build ids where it ran, timed out, failed or hanged.
+    test_to_build_ids = {key : {'all' : set(), 'timeout': set(), 'failed': set(),
+                                'hanging' : set(), 'bad_count' : 0}
+                         for key in bad_tests}
+    for build in build_id_to_results:
+        [all_tests, failed_tests, timeout_tests, hanging_tests] = build_id_to_results[build]
+        for bad_test in test_to_build_ids:
+            is_bad = False
+            if all_tests.issuperset([bad_test]):
+                test_to_build_ids[bad_test]["all"].add(build)
+            if timeout_tests.issuperset([bad_test]):
+                test_to_build_ids[bad_test]['timeout'].add(build)
+                is_bad = True
+            if failed_tests.issuperset([bad_test]):
+                test_to_build_ids[bad_test]['failed'].add(build)
+                is_bad = True
+            if hanging_tests.issuperset([bad_test]):
+                test_to_build_ids[bad_test]['hanging'].add(build)
+                is_bad = True
+            if is_bad:
+                test_to_build_ids[bad_test]['bad_count'] += 1
+
+    # Calculate flakyness % and successful builds for each test. Also sort build ids.
+    for bad_test in test_to_build_ids:
+        test_result = test_to_build_ids[bad_test]
+        test_result['flakyness'] = test_result['bad_count'] * 100.0 / len(test_result['all'])
+        test_result['success'] = (test_result['all'].difference(
+            test_result['failed'].union(test_result['hanging'])))
+        for key in ['all', 'timeout', 'failed', 'hanging', 'success']:
+            test_result[key] = sorted(test_result[key])
+
+
+    # Sort tests in descending order by flakyness.
+    sorted_test_to_build_ids = OrderedDict(
+        sorted(test_to_build_ids.iteritems(), key=lambda x: x[1]['flakyness'], reverse=True))
+    url_to_bad_test_results[url] = sorted_test_to_build_ids
+
+    if len(sorted_test_to_build_ids) > 0:
+        print "URL: {}".format(url)
+        print "{:>60}  {:10}  {:25}  {}".format(
+            "Test Name", "Total Runs", "Bad Runs(failed/timeout/hanging)", "Flakyness")
+        for bad_test in sorted_test_to_build_ids:
+            test_status = sorted_test_to_build_ids[bad_test]
+            print "{:>60}  {:10}  {:7} ( {:4} / {:5} / {:5} )  {:2.0f}%".format(
+                bad_test, len(test_status['all']), test_status['bad_count'],
+                len(test_status['failed']), len(test_status['timeout']),
+                len(test_status['hanging']), test_status['flakyness'])
+    else:
+        print "No flaky tests founds."
+        if len(url_to_build_ids[url]) == len(build_ids_without_tests_run):
+            print "None of the analyzed builds have test result."
+
+    print "Builds analyzed: {}".format(url_to_build_ids[url])
+    print "Builds without any test runs: {}".format(build_ids_without_tests_run)
+    print ""
+
+
+all_bad_tests = all_hanging_tests.union(all_failed_tests)
+if args.mvn:
+    includes = ",".join(all_bad_tests)
+    with open("./includes", "w") as inc_file:
+        inc_file.write(includes)
+
+    excludes = ["**/{0}.java".format(bad_test) for bad_test in all_bad_tests]
+    with open("./excludes", "w") as exc_file:
+        exc_file.write(",".join(excludes))
+
+    with open("./timeout", "w") as timeout_file:
+        timeout_file.write(",".join(all_timeout_tests))
+
+    with open("./failed", "w") as failed_file:
+        failed_file.write(",".join(all_failed_tests))
+
+dev_support_dir = os.path.dirname(os.path.abspath(__file__))
+with open(os.path.join(dev_support_dir, "flaky-dashboard-template.html"), "r") as f:
+    template = Template(f.read())
+
+with open("dashboard.html", "w") as f:
+    datetime = time.strftime("%m/%d/%Y %H:%M:%S")
+    f.write(template.render(datetime=datetime, bad_tests_count=len(all_bad_tests),
+                            results=url_to_bad_test_results, build_ids=url_to_build_ids))

http://git-wip-us.apache.org/repos/asf/hbase/blob/736b43a4/dev-support/flaky-tests/run-flaky-tests.Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/flaky-tests/run-flaky-tests.Jenkinsfile b/dev-support/flaky-tests/run-flaky-tests.Jenkinsfile
new file mode 100644
index 0000000..cacb175
--- /dev/null
+++ b/dev-support/flaky-tests/run-flaky-tests.Jenkinsfile
@@ -0,0 +1,65 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+pipeline {
+  agent {
+    node {
+      label 'Hadoop'
+    }
+  }
+  triggers {
+    cron('@hourly')
+  }
+  options {
+    // this should roughly match how long we tell the flaky dashboard to look at
+    buildDiscarder(logRotator(numToKeepStr: '80'))
+    timeout (time: 2, unit: 'HOURS')
+    timestamps()
+  }
+  parameters {
+    booleanParam(name: 'DEBUG', defaultValue: false, description: 'Produce a lot more meta-information.')
+  }
+  tools {
+    // this should match what the yetus nightly job for the branch will use
+    maven 'Maven (latest)'
+    jdk "JDK 1.8 (latest)"
+  }
+  stages {
+    stage ('run flaky tests') {
+      steps {
+        sh '''#!/usr/bin/env bash
+          set -e
+          if [ "${DEBUG}" = "true" ]; then
+            curl_debug="-v"
+            mvn_debug="-X"
+            set -x
+          fi
+          ulimit -a
+          rm -rf local-repository/org/apache/hbase
+          curl --fail "${curl_debug}" -o includes.txt "${JENKINS_URL}/job/HBase-Find-Flaky-Tests-per-branch/job/${BRANCH_NAME}/lastSuccessfulBuild/artifact/includes"
+          mvn clean package "${mvn_debug}" --batch-mode -fn -Dbuild.id="${BUILD_ID}" -Dtest="$(cat includes.txt)" -Dmaven.test.redirectTestOutputToFile=true -Dmaven.repo.local="${WORKSPACE}/local-repository" -Dsurefire.firstPartForkCount=3 -Dsurefire.secondPartForkCount=3
+'''
+      }
+    }
+  }
+  post {
+    always {
+      junit testResults: "**/surefire-reports/*.xml", allowEmptyResults: true
+      // TODO compress these logs
+      archive 'includes.txt,**/surefire-reports/*,**/test-data/*'
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/736b43a4/dev-support/hbase_nightly_yetus.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase_nightly_yetus.sh b/dev-support/hbase_nightly_yetus.sh
index bba5f4d..185a288 100755
--- a/dev-support/hbase_nightly_yetus.sh
+++ b/dev-support/hbase_nightly_yetus.sh
@@ -71,10 +71,6 @@ YETUS_ARGS=("--tests-filter=${TESTS_FILTER}" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--proclimit=10000" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--dockermemlimit=20g" "${YETUS_ARGS[@]}")
 
-# Currently, flaky list is calculated only for master branch.
-UNDERSCORED_BRANCH_NAME=$(echo ${BRANCH_NAME} | tr '.-' '_')
-EXCLUDE_TESTS_URL=$(eval echo "\$EXCLUDE_TESTS_URL_${UNDERSCORED_BRANCH_NAME}")
-INCLUDE_TESTS_URL=$(eval echo "\$INCLUDE_TESTS_URL_${UNDERSCORED_BRANCH_NAME}")
 if [[ -n "${EXCLUDE_TESTS_URL}" ]]; then
   YETUS_ARGS=("--exclude-tests-url=${EXCLUDE_TESTS_URL}" "${YETUS_ARGS[@]}")
 fi

http://git-wip-us.apache.org/repos/asf/hbase/blob/736b43a4/dev-support/report-flakies.py
----------------------------------------------------------------------
diff --git a/dev-support/report-flakies.py b/dev-support/report-flakies.py
deleted file mode 100755
index 1b3161a..0000000
--- a/dev-support/report-flakies.py
+++ /dev/null
@@ -1,280 +0,0 @@
-#!/usr/bin/env python
-##
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# pylint: disable=invalid-name
-# To disable 'invalid constant name' warnings.
-# pylint: disable=import-error
-# Testing environment may not have all dependencies.
-
-"""
-This script uses Jenkins REST api to collect test result(s) of given build/builds and generates
-flakyness data about unittests.
-Print help: report-flakies.py -h
-"""
-
-import argparse
-import logging
-import os
-import time
-from collections import OrderedDict
-from jinja2 import Template
-
-import requests
-
-import findHangingTests
-
-parser = argparse.ArgumentParser()
-parser.add_argument(
-    '--urls', metavar='URL', action='append', required=True,
-    help='Urls to analyze, which can refer to simple projects, multi-configuration projects or '
-         'individual build run.')
-parser.add_argument('--excluded-builds', metavar='n1,n2', action='append',
-                    help='List of build numbers to exclude (or "None"). Not required, '
-                         'but if specified, number of uses should be same as that of --urls '
-                         'since the values are matched.')
-parser.add_argument('--max-builds', metavar='n', action='append', type=int,
-                    help='The maximum number of builds to use (if available on jenkins). Specify '
-                         '0 to analyze all builds. Not required, but if specified, number of uses '
-                         'should be same as that of --urls since the values are matched.')
-parser.add_argument('--is-yetus', metavar='True/False', action='append', choices=['True', 'False'],
-                    help='True, if build is yetus style i.e. look for maven output in artifacts; '
-                         'False, if maven output is in <url>/consoleText itself.')
-parser.add_argument(
-    "--mvn", action="store_true",
-    help="Writes two strings for including/excluding these flaky tests using maven flags. These "
-         "strings are written to files so they can be saved as artifacts and easily imported in "
-         "other projects. Also writes timeout and failing tests in separate files for "
-         "reference.")
-parser.add_argument("-v", "--verbose", help="Prints more logs.", action="store_true")
-args = parser.parse_args()
-
-logging.basicConfig()
-logger = logging.getLogger(__name__)
-if args.verbose:
-    logger.setLevel(logging.INFO)
-
-
-def get_bad_tests(build_url, is_yetus):
-    """
-    Given url of an executed build, analyzes its maven output, and returns
-    [list of all tests, list of timeout tests, list of failed tests].
-    Returns None if can't get maven output from the build or if there is any other error.
-    """
-    logger.info("Analyzing %s", build_url)
-    needed_fields="_class,building"
-    if is_yetus:
-        needed_fields+=",artifacts[fileName,relativePath]"
-    response = requests.get(build_url + "/api/json?tree=" + needed_fields).json()
-    if response["building"]:
-        logger.info("Skipping this build since it is in progress.")
-        return {}
-    console_url = None
-    if is_yetus:
-        for artifact in response["artifacts"]:
-            if artifact["fileName"] == "patch-unit-root.txt":
-                console_url = build_url + "/artifact/" + artifact["relativePath"]
-                break
-        if console_url is None:
-            logger.info("Can't find 'patch-unit-root.txt' artifact for Yetus build %s\n. Ignoring "
-                        "this build.", build_url)
-            return
-    else:
-        console_url = build_url + "/consoleText"
-    build_result = findHangingTests.get_bad_tests(console_url)
-    if not build_result:
-        logger.info("Ignoring build %s", build_url)
-        return
-    return build_result
-
-
-def expand_multi_config_projects(cli_args):
-    """
-    If any url is of type multi-configuration project (i.e. has key 'activeConfigurations'),
-    get urls for individual jobs.
-    """
-    job_urls = cli_args.urls
-    excluded_builds_arg = cli_args.excluded_builds
-    max_builds_arg = cli_args.max_builds
-    is_yetus_arg = cli_args.is_yetus
-    if excluded_builds_arg is not None and len(excluded_builds_arg) != len(job_urls):
-        raise Exception("Number of --excluded-builds arguments should be same as that of --urls "
-                        "since values are matched.")
-    if max_builds_arg is not None and len(max_builds_arg) != len(job_urls):
-        raise Exception("Number of --max-builds arguments should be same as that of --urls "
-                        "since values are matched.")
-    final_expanded_urls = []
-    for (i, job_url) in enumerate(job_urls):
-        max_builds = 10000  # Some high number
-        is_yetus = False
-        if is_yetus_arg is not None:
-            is_yetus = is_yetus_arg[i] == "True"
-        if max_builds_arg is not None and max_builds_arg[i] != 0:
-            max_builds = int(max_builds_arg[i])
-        excluded_builds = []
-        if excluded_builds_arg is not None and excluded_builds_arg[i] != "None":
-            excluded_builds = [int(x) for x in excluded_builds_arg[i].split(",")]
-        request = requests.get(job_url + "/api/json?tree=_class,activeConfigurations%5Burl%5D")
-        if request.status_code != 200:
-            raise Exception("Failed to get job information from jenkins for url '" + job_url +
-                            "'. Jenkins returned HTTP status " + str(request.status_code))
-        response = request.json()
-        if response.has_key("activeConfigurations"):
-            for config in response["activeConfigurations"]:
-                final_expanded_urls.append({'url':config["url"], 'max_builds': max_builds,
-                                            'excludes': excluded_builds, 'is_yetus': is_yetus})
-        else:
-            final_expanded_urls.append({'url':job_url, 'max_builds': max_builds,
-                                        'excludes': excluded_builds, 'is_yetus': is_yetus})
-    return final_expanded_urls
-
-
-# Set of timeout/failed tests across all given urls.
-all_timeout_tests = set()
-all_failed_tests = set()
-all_hanging_tests = set()
-# Contains { <url> : { <bad_test> : { 'all': [<build ids>], 'failed': [<build ids>],
-#                                     'timeout': [<build ids>], 'hanging': [<builds ids>] } } }
-url_to_bad_test_results = OrderedDict()
-# Contains { <url> : [run_ids] }
-# Used for common min/max build ids when generating sparklines.
-url_to_build_ids = OrderedDict()
-
-# Iterates over each url, gets test results and prints flaky tests.
-expanded_urls = expand_multi_config_projects(args)
-for url_max_build in expanded_urls:
-    url = url_max_build["url"]
-    excludes = url_max_build["excludes"]
-    json_response = requests.get(url + "/api/json?tree=id,builds%5Bnumber,url%5D").json()
-    if json_response.has_key("builds"):
-        builds = json_response["builds"]
-        logger.info("Analyzing job: %s", url)
-    else:
-        builds = [{'number': json_response["id"], 'url': url}]
-        logger.info("Analyzing build : %s", url)
-    build_id_to_results = {}
-    num_builds = 0
-    url_to_build_ids[url] = []
-    build_ids_without_tests_run = []
-    for build in builds:
-        build_id = build["number"]
-        if build_id in excludes:
-            continue
-        result = get_bad_tests(build["url"], url_max_build['is_yetus'])
-        if not result:
-            continue
-        if len(result[0]) > 0:
-            build_id_to_results[build_id] = result
-        else:
-            build_ids_without_tests_run.append(build_id)
-        num_builds += 1
-        url_to_build_ids[url].append(build_id)
-        if num_builds == url_max_build["max_builds"]:
-            break
-    url_to_build_ids[url].sort()
-
-    # Collect list of bad tests.
-    bad_tests = set()
-    for build in build_id_to_results:
-        [_, failed_tests, timeout_tests, hanging_tests] = build_id_to_results[build]
-        all_timeout_tests.update(timeout_tests)
-        all_failed_tests.update(failed_tests)
-        all_hanging_tests.update(hanging_tests)
-        # Note that timedout tests are already included in failed tests.
-        bad_tests.update(failed_tests.union(hanging_tests))
-
-    # For each bad test, get build ids where it ran, timed out, failed or hanged.
-    test_to_build_ids = {key : {'all' : set(), 'timeout': set(), 'failed': set(),
-                                'hanging' : set(), 'bad_count' : 0}
-                         for key in bad_tests}
-    for build in build_id_to_results:
-        [all_tests, failed_tests, timeout_tests, hanging_tests] = build_id_to_results[build]
-        for bad_test in test_to_build_ids:
-            is_bad = False
-            if all_tests.issuperset([bad_test]):
-                test_to_build_ids[bad_test]["all"].add(build)
-            if timeout_tests.issuperset([bad_test]):
-                test_to_build_ids[bad_test]['timeout'].add(build)
-                is_bad = True
-            if failed_tests.issuperset([bad_test]):
-                test_to_build_ids[bad_test]['failed'].add(build)
-                is_bad = True
-            if hanging_tests.issuperset([bad_test]):
-                test_to_build_ids[bad_test]['hanging'].add(build)
-                is_bad = True
-            if is_bad:
-                test_to_build_ids[bad_test]['bad_count'] += 1
-
-    # Calculate flakyness % and successful builds for each test. Also sort build ids.
-    for bad_test in test_to_build_ids:
-        test_result = test_to_build_ids[bad_test]
-        test_result['flakyness'] = test_result['bad_count'] * 100.0 / len(test_result['all'])
-        test_result['success'] = (test_result['all'].difference(
-            test_result['failed'].union(test_result['hanging'])))
-        for key in ['all', 'timeout', 'failed', 'hanging', 'success']:
-            test_result[key] = sorted(test_result[key])
-
-
-    # Sort tests in descending order by flakyness.
-    sorted_test_to_build_ids = OrderedDict(
-        sorted(test_to_build_ids.iteritems(), key=lambda x: x[1]['flakyness'], reverse=True))
-    url_to_bad_test_results[url] = sorted_test_to_build_ids
-
-    if len(sorted_test_to_build_ids) > 0:
-        print "URL: {}".format(url)
-        print "{:>60}  {:10}  {:25}  {}".format(
-            "Test Name", "Total Runs", "Bad Runs(failed/timeout/hanging)", "Flakyness")
-        for bad_test in sorted_test_to_build_ids:
-            test_status = sorted_test_to_build_ids[bad_test]
-            print "{:>60}  {:10}  {:7} ( {:4} / {:5} / {:5} )  {:2.0f}%".format(
-                bad_test, len(test_status['all']), test_status['bad_count'],
-                len(test_status['failed']), len(test_status['timeout']),
-                len(test_status['hanging']), test_status['flakyness'])
-    else:
-        print "No flaky tests founds."
-        if len(url_to_build_ids[url]) == len(build_ids_without_tests_run):
-            print "None of the analyzed builds have test result."
-
-    print "Builds analyzed: {}".format(url_to_build_ids[url])
-    print "Builds without any test runs: {}".format(build_ids_without_tests_run)
-    print ""
-
-
-all_bad_tests = all_hanging_tests.union(all_failed_tests)
-if args.mvn:
-    includes = ",".join(all_bad_tests)
-    with open("./includes", "w") as inc_file:
-        inc_file.write(includes)
-
-    excludes = ["**/{0}.java".format(bad_test) for bad_test in all_bad_tests]
-    with open("./excludes", "w") as exc_file:
-        exc_file.write(",".join(excludes))
-
-    with open("./timeout", "w") as timeout_file:
-        timeout_file.write(",".join(all_timeout_tests))
-
-    with open("./failed", "w") as failed_file:
-        failed_file.write(",".join(all_failed_tests))
-
-dev_support_dir = os.path.dirname(os.path.abspath(__file__))
-with open(os.path.join(dev_support_dir, "flaky-dashboard-template.html"), "r") as f:
-    template = Template(f.read())
-
-with open("dashboard.html", "w") as f:
-    datetime = time.strftime("%m/%d/%Y %H:%M:%S")
-    f.write(template.render(datetime=datetime, bad_tests_count=len(all_bad_tests),
-                            results=url_to_bad_test_results, build_ids=url_to_build_ids))


[13/15] hbase git commit: HBASE-20979 Flaky test reporting should specify what JSON it needs and handle HTTP errors

Posted by bu...@apache.org.
HBASE-20979 Flaky test reporting should specify what JSON it needs and handle HTTP errors


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

Branch: refs/heads/HBASE-20387
Commit: 67b79764868b3c406632e553f667275a7a3d8656
Parents: e705cf1
Author: Sean Busbey <bu...@apache.org>
Authored: Mon Jul 30 12:36:54 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Tue Aug 14 12:46:39 2018 -0500

----------------------------------------------------------------------
 dev-support/report-flakies.py | 13 ++++++++++---
 1 file changed, 10 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/67b79764/dev-support/report-flakies.py
----------------------------------------------------------------------
diff --git a/dev-support/report-flakies.py b/dev-support/report-flakies.py
index 201980d..1b3161a 100755
--- a/dev-support/report-flakies.py
+++ b/dev-support/report-flakies.py
@@ -76,7 +76,10 @@ def get_bad_tests(build_url, is_yetus):
     Returns None if can't get maven output from the build or if there is any other error.
     """
     logger.info("Analyzing %s", build_url)
-    response = requests.get(build_url + "/api/json").json()
+    needed_fields="_class,building"
+    if is_yetus:
+        needed_fields+=",artifacts[fileName,relativePath]"
+    response = requests.get(build_url + "/api/json?tree=" + needed_fields).json()
     if response["building"]:
         logger.info("Skipping this build since it is in progress.")
         return {}
@@ -125,7 +128,11 @@ def expand_multi_config_projects(cli_args):
         excluded_builds = []
         if excluded_builds_arg is not None and excluded_builds_arg[i] != "None":
             excluded_builds = [int(x) for x in excluded_builds_arg[i].split(",")]
-        response = requests.get(job_url + "/api/json").json()
+        request = requests.get(job_url + "/api/json?tree=_class,activeConfigurations%5Burl%5D")
+        if request.status_code != 200:
+            raise Exception("Failed to get job information from jenkins for url '" + job_url +
+                            "'. Jenkins returned HTTP status " + str(request.status_code))
+        response = request.json()
         if response.has_key("activeConfigurations"):
             for config in response["activeConfigurations"]:
                 final_expanded_urls.append({'url':config["url"], 'max_builds': max_builds,
@@ -152,7 +159,7 @@ expanded_urls = expand_multi_config_projects(args)
 for url_max_build in expanded_urls:
     url = url_max_build["url"]
     excludes = url_max_build["excludes"]
-    json_response = requests.get(url + "/api/json").json()
+    json_response = requests.get(url + "/api/json?tree=id,builds%5Bnumber,url%5D").json()
     if json_response.has_key("builds"):
         builds = json_response["builds"]
         logger.info("Analyzing job: %s", url)


[10/15] hbase git commit: HBASE-20257 hbase-spark should not depend on com.google.code.findbugs.jsr305

Posted by bu...@apache.org.
HBASE-20257 hbase-spark should not depend on com.google.code.findbugs.jsr305

Signed-off-by: tedyu <yu...@gmail.com>
Signed-off-by: Sean Busbey <bu...@apache.org>


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

Branch: refs/heads/HBASE-20387
Commit: d6e1646bea8e1b534c736c672f0da69c8cea3c47
Parents: a07e755
Author: Artem Ervits <ge...@gmail.com>
Authored: Mon Aug 13 11:35:22 2018 -0400
Committer: tedyu <yu...@gmail.com>
Committed: Mon Aug 13 11:01:44 2018 -0700

----------------------------------------------------------------------
 hbase-spark-it/pom.xml | 10 ----------
 hbase-spark/pom.xml    | 45 ++++++++++++++++++++++++---------------------
 2 files changed, 24 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d6e1646b/hbase-spark-it/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-spark-it/pom.xml b/hbase-spark-it/pom.xml
index bfd2906..2f29fa3 100644
--- a/hbase-spark-it/pom.xml
+++ b/hbase-spark-it/pom.xml
@@ -123,16 +123,6 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-enforcer-plugin</artifactId>
         <executions>
-          <!-- purposefully have jsr 305 exclusion only warn in this module -->
-          <execution>
-            <id>banned-jsr305</id>
-            <goals>
-              <goal>enforce</goal>
-            </goals>
-            <configuration>
-              <fail>false</fail>
-            </configuration>
-          </execution>
           <!-- hbase-spark is ok in this modules -->
           <execution>
             <id>banned-hbase-spark</id>

http://git-wip-us.apache.org/repos/asf/hbase/blob/d6e1646b/hbase-spark/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-spark/pom.xml b/hbase-spark/pom.xml
index a5f96b4..1042663 100644
--- a/hbase-spark/pom.xml
+++ b/hbase-spark/pom.xml
@@ -55,10 +55,6 @@
       <version>${scala.version}</version>
       <scope>provided</scope>
     </dependency>
-    <!-- we exclude jsr305 below and then expressly relist it as
-              provided / optional to avoid dependency resolution possibly
-              bringing it back into runtime scope.
-         -->
     <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-core_${scala.binary.version}</artifactId>
@@ -92,13 +88,6 @@
       <version>1.1.4</version>
     </dependency>
     <dependency>
-      <groupId>com.google.code.findbugs</groupId>
-      <artifactId>jsr305</artifactId>
-      <version>1.3.9</version>
-      <scope>provided</scope>
-      <optional>true</optional>
-    </dependency>
-    <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-sql_${scala.binary.version}</artifactId>
       <version>${spark.version}</version>
@@ -487,16 +476,6 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-enforcer-plugin</artifactId>
         <executions>
-          <!-- purposefully have jsr 305 exclusion only warn in this module -->
-          <execution>
-            <id>banned-jsr305</id>
-            <goals>
-              <goal>enforce</goal>
-            </goals>
-            <configuration>
-              <fail>false</fail>
-            </configuration>
-          </execution>
           <!-- scala is ok in the spark modules -->
           <execution>
             <id>banned-scala</id>
@@ -581,6 +560,12 @@
           <version>${hadoop-two.version}</version>
           <type>test-jar</type>
           <scope>test</scope>
+          <exclusions>
+            <exclusion>
+              <groupId>com.google.code.findbugs</groupId>
+              <artifactId>jsr305</artifactId>
+            </exclusion>
+          </exclusions>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
@@ -588,6 +573,12 @@
           <version>${hadoop-two.version}</version>
           <type>test-jar</type>
           <scope>test</scope>
+          <exclusions>
+            <exclusion>
+              <groupId>com.google.code.findbugs</groupId>
+              <artifactId>jsr305</artifactId>
+            </exclusion>
+          </exclusions>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
@@ -629,6 +620,12 @@
           <version>${hadoop-three.version}</version>
           <type>test-jar</type>
           <scope>test</scope>
+          <exclusions>
+            <exclusion>
+              <groupId>com.google.code.findbugs</groupId>
+              <artifactId>jsr305</artifactId>
+            </exclusion>
+          </exclusions>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
@@ -636,6 +633,12 @@
           <version>${hadoop-three.version}</version>
           <type>test-jar</type>
           <scope>test</scope>
+          <exclusions>
+            <exclusion>
+              <groupId>com.google.code.findbugs</groupId>
+              <artifactId>jsr305</artifactId>
+            </exclusion>
+          </exclusions>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>


[11/15] hbase git commit: HBASE-21005 Remove local maven repo and fluido-skin jar

Posted by bu...@apache.org.
HBASE-21005 Remove local maven repo and fluido-skin jar

Signed-off-by: Mike Drob <md...@apache.org>
Signed-off-by: Sean Busbey <bu...@apache.org>


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

Branch: refs/heads/HBASE-20387
Commit: a2d3f161fef26ccc579f5d3ce77e837353f21ab8
Parents: d6e1646
Author: Josh Elser <el...@apache.org>
Authored: Mon Aug 6 11:43:39 2018 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon Aug 13 18:21:19 2018 -0400

----------------------------------------------------------------------
 pom.xml                                         |   7 -
 .../1.5-HBASE/maven-fluido-skin-1.5-HBASE.jar   | Bin 344936 -> 0 bytes
 .../1.5-HBASE/maven-fluido-skin-1.5-HBASE.pom   | 718 -------------------
 .../maven-fluido-skin/maven-metadata-local.xml  |  12 -
 src/site/site.xml                               |  14 +-
 5 files changed, 12 insertions(+), 739 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a2d3f161/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 30a18e9..540c1f9 100755
--- a/pom.xml
+++ b/pom.xml
@@ -3805,11 +3805,4 @@
       <url>file:///tmp</url>
     </site>
   </distributionManagement>
-  <repositories>
-    <repository>
-        <id>project.local</id>
-        <name>project</name>
-        <url>file:${project.basedir}/src/site/resources/repo</url>
-    </repository>
-</repositories>
 </project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/a2d3f161/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.jar
----------------------------------------------------------------------
diff --git a/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.jar b/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.jar
deleted file mode 100644
index 5b93209..0000000
Binary files a/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.jar and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/a2d3f161/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.pom
----------------------------------------------------------------------
diff --git a/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.pom b/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.pom
deleted file mode 100644
index d12092b..0000000
--- a/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.pom
+++ /dev/null
@@ -1,718 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-
-  <parent>
-    <groupId>org.apache.maven.skins</groupId>
-    <artifactId>maven-skins</artifactId>
-    <version>10</version>
-    <relativePath>../maven-skins/pom.xml</relativePath>
-  </parent>
-
-  <artifactId>maven-fluido-skin</artifactId>
-  <version>1.5-HBASE</version>
-
-  <name>Apache Maven Fluido Skin</name>
-  <description>The Apache Maven Fluido Skin is an Apache Maven site skin
-    built on top of Twitter's bootstrap.</description>
-  <inceptionYear>2011</inceptionYear>
-
-  <scm>
-    <connection>scm:svn:http://svn.apache.org/repos/asf/maven/skins/trunk/maven-fluido-skin/</connection>
-    <developerConnection>scm:svn:https://svn.apache.org/repos/asf/maven/skins/trunk/maven-fluido-skin/</developerConnection>
-    <url>http://svn.apache.org/viewvc/maven/skins/trunk/maven-fluido-skin/</url>
-  </scm>
-  <issueManagement>
-    <system>jira</system>
-    <url>https://issues.apache.org/jira/browse/MSKINS/component/12326474</url>
-  </issueManagement>
-  <distributionManagement>
-    <site>
-      <id>apache.website</id>
-      <url>scm:svn:https://svn.apache.org/repos/infra/websites/production/maven/components/${maven.site.path}</url>
-    </site>
-  </distributionManagement>
-
-  <contributors>
-    <!-- in alphabetical order -->
-    <contributor>
-      <name>Bruno P. Kinoshita</name>
-      <email>brunodepaulak AT yahoo DOT com DOT br</email>
-    </contributor>
-    <contributor>
-      <name>Carlos Villaronga</name>
-      <email>cvillaronga AT gmail DOT com</email>
-    </contributor>
-    <contributor>
-      <name>Christian Grobmeier</name>
-      <email>grobmeier AT apache DOT org</email>
-    </contributor>
-    <contributor>
-      <name>Conny Kreyssel</name>
-      <email>dev AT kreyssel DOT org</email>
-    </contributor>
-    <contributor>
-      <name>Michael Koch</name>
-      <email>tensberg AT gmx DOT net</email>
-    </contributor>
-    <contributor>
-      <name>Emmanuel Hugonnet</name>
-      <email>emmanuel DOT hugonnet AT gmail DOT com</email>
-    </contributor>
-    <contributor>
-      <name>Ivan Habunek</name>
-      <email>ihabunek AT apache DOT org</email>
-    </contributor>
-    <contributor>
-      <name>Eric Barboni</name>
-    </contributor>
-    <contributor>
-      <name>Michael Osipov</name>
-      <email>michaelo AT apache DOT org</email>
-    </contributor>
-  </contributors>
-
-  <properties>
-    <bootstrap.version>2.3.2</bootstrap.version>
-    <jquery.version>1.11.2</jquery.version>
-  </properties>
-
-  <build>
-    <resources>
-      <resource>
-        <directory>.</directory>
-        <targetPath>META-INF</targetPath>
-        <includes>
-          <include>NOTICE</include>
-          <include>LICENSE</include>
-        </includes>
-      </resource>
-
-      <!-- exclude css and js since will include the minified version -->
-      <resource>
-        <directory>${basedir}/src/main/resources</directory>
-        <excludes>
-          <exclude>css/**</exclude>
-          <exclude>js/**</exclude>
-        </excludes>
-        <filtering>true</filtering> <!-- add skin-info -->
-      </resource>
-
-      <!-- include the print.css -->
-      <resource>
-        <directory>${basedir}/src/main/resources</directory>
-        <includes>
-          <include>css/print.css</include>
-        </includes>
-      </resource>
-
-      <!-- include minified only -->
-      <resource>
-        <directory>${project.build.directory}/${project.build.finalName}</directory>
-        <includes>
-          <include>css/apache-maven-fluido-${project.version}.min.css</include>
-          <include>js/apache-maven-fluido-${project.version}.min.js</include>
-        </includes>
-      </resource>
-    </resources>
-
-    <pluginManagement>
-      <plugins>
-        <plugin>
-          <groupId>org.apache.rat</groupId>
-          <artifactId>apache-rat-plugin</artifactId>
-          <configuration>
-            <excludes combine.children="append">
-              <exclude>src/main/resources/fonts/glyphicons-halflings-regular.svg</exclude>
-              <exclude>src/main/resources/js/prettify.js</exclude>
-              <exclude>src/main/resources/js/jquery-*.js</exclude>
-            </excludes>
-          </configuration>
-        </plugin>
-      </plugins>
-    </pluginManagement>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-resources-plugin</artifactId>
-        <dependencies><!-- TODO remove when upgrading to version 2.8: see MSHARED-325 / MRESOURCES-192 -->
-          <dependency>
-              <groupId>org.apache.maven.shared</groupId>
-              <artifactId>maven-filtering</artifactId>
-              <version>1.3</version>
-          </dependency>
-        </dependencies>
-        <configuration>
-          <delimiters>
-            <delimiter>@</delimiter>
-          </delimiters>
-          <useDefaultDelimiters>false</useDefaultDelimiters>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>com.samaxes.maven</groupId>
-        <artifactId>maven-minify-plugin</artifactId>
-        <version>1.3.5</version>
-        <executions>
-          <execution>
-            <id>default-minify</id>
-            <phase>generate-resources</phase>
-            <configuration>
-              <webappSourceDir>${basedir}/src/main/resources</webappSourceDir>
-              <cssSourceDir>css</cssSourceDir>
-              <cssSourceFiles>
-                <cssSourceFile>bootstrap-${bootstrap.version}.css</cssSourceFile>
-                <cssSourceFile>maven-base.css</cssSourceFile>
-                <cssSourceFile>maven-theme.css</cssSourceFile>
-                <cssSourceFile>prettify.css</cssSourceFile>
-              </cssSourceFiles>
-              <cssFinalFile>apache-maven-fluido-${project.version}.css</cssFinalFile>
-              <jsSourceDir>js</jsSourceDir>
-              <jsSourceFiles>
-                <jsSourceFile>jquery-${jquery.version}.js</jsSourceFile>
-                <jsSourceFile>bootstrap-${bootstrap.version}.js</jsSourceFile>
-                <jsSourceFile>prettify.js</jsSourceFile>
-                <jsSourceFile>fluido.js</jsSourceFile>
-              </jsSourceFiles>
-              <jsFinalFile>apache-maven-fluido-${project.version}.js</jsFinalFile>
-            </configuration>
-            <goals>
-              <goal>minify</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-
-  <profiles>
-    <profile>
-      <id>run-its</id>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-invoker-plugin</artifactId>
-            <configuration>
-              <debug>true</debug>
-              <projectsDirectory>src/it</projectsDirectory>
-              <cloneProjectsTo>${project.build.directory}/it</cloneProjectsTo>
-              <preBuildHookScript>setup</preBuildHookScript>
-              <postBuildHookScript>verify</postBuildHookScript>
-              <localRepositoryPath>${project.build.directory}/local-repo</localRepositoryPath>
-              <settingsFile>src/it/settings.xml</settingsFile>
-              <pomIncludes>
-                <pomInclude>*/pom.xml</pomInclude>
-              </pomIncludes>
-              <goals>
-                <goal>site</goal>
-              </goals>
-            </configuration>
-            <executions>
-              <execution>
-                <id>integration-test</id>
-                <goals>
-                  <goal>install</goal>
-                  <goal>integration-test</goal>
-                  <goal>verify</goal>
-                </goals>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-    <profile>
-      <id>reporting</id>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-resources-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>copy-sidebar</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/sidebar/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/sidebar/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-topbar</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/topbar/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/topbar/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-topbar-inverse</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/topbar-inverse/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/topbar-inverse/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-10</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-10/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-10/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-13</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-13/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-13/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-14</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-14/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-14/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-14_sitesearch</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-14_sitesearch/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-14_sitesearch/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-15</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-15/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-15/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-16</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-16/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-16/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-17</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-17/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-17/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-21</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-21/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-21/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-22</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-22/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-22/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-22_default</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-22_default/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-22_default/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-22_topbar</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-22_topbar/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-22_topbar/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-23</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-23/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-23/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-24</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-24/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-24/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-24_topbar</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-24_topbar/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-24_topbar/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-25</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-25/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-25/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-28</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-28/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-28/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-31</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-31/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-31/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-33</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-33/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-33/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-33_topbar</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-33_topbar/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-33_topbar/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-34</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-34/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-34/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-34_topbar</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-34_topbar/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-34_topbar/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-41</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-41/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-41/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-72</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-72/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-72/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-75</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-75/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-75/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-76</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-76/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-76/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-76_topbar</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-76_topbar/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-76_topbar/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-85</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-85/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-85/</outputDirectory>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-      <reporting>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-invoker-plugin</artifactId>
-            <version>1.8</version>
-          </plugin>
-        </plugins>
-      </reporting>
-    </profile>
-  </profiles>
-</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/a2d3f161/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/maven-metadata-local.xml
----------------------------------------------------------------------
diff --git a/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/maven-metadata-local.xml b/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/maven-metadata-local.xml
deleted file mode 100644
index 65791e8..0000000
--- a/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/maven-metadata-local.xml
+++ /dev/null
@@ -1,12 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<metadata>
-  <groupId>org.apache.maven.skins</groupId>
-  <artifactId>maven-fluido-skin</artifactId>
-  <versioning>
-    <release>1.5-HBASE</release>
-    <versions>
-      <version>1.5-HBASE</version>
-    </versions>
-    <lastUpdated>20151111033340</lastUpdated>
-  </versioning>
-</metadata>

http://git-wip-us.apache.org/repos/asf/hbase/blob/a2d3f161/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
index 122c947..907c06d 100644
--- a/src/site/site.xml
+++ b/src/site/site.xml
@@ -23,9 +23,19 @@
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
     xsi:schemaLocation="http://maven.apache.org/DECORATION/1.0.0 http://maven.apache.org/xsd/decoration-1.0.0.xsd">
   <skin>
-    <groupId>org.apache.maven.skins</groupId>
+    <!-- We tried to get some changes to the fluido-skin committed upstream
+         but with little success. This is a custom artifact that contains
+         some commits over the official maven-fluido-skin 1.4 release. The
+         actual code is available at:
+         https://github.com/joshelser/maven-fluido-skin/tree/1.4-HBase-patched.
+         The hope is that we can get these fixes included upstream and
+         remove the need for this custom fork. This is published to
+         my end-user's GAV for ease of releasing this to maven central
+         until the upstream update happens and is released.
+         See HBASE-14785 and HBASE-21005 for more info. -->
+    <groupId>com.github.joshelser</groupId>
     <artifactId>maven-fluido-skin</artifactId>
-    <version>1.5-HBASE</version>
+    <version>1.4-HBase</version>
   </skin>
   <custom>
     <fluidoSkin>


[04/15] hbase git commit: HBASE-20512 document change to running tests on secure clusters Signed-off-by: Tak Lon (Stephen) Wu Signed-off-by: Sean Busbey

Posted by bu...@apache.org.
HBASE-20512 document change to running tests on secure clusters
Signed-off-by: Tak Lon (Stephen) Wu
Signed-off-by: Sean Busbey <bu...@apache.org>


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

Branch: refs/heads/HBASE-20387
Commit: 695182bfe32ea67e0e14fb17581b198c48cf8e10
Parents: 7f3d9f6
Author: Michael Stack <st...@apache.org>
Authored: Tue Jul 31 13:42:26 2018 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Sat Aug 11 11:19:57 2018 -0700

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/upgrading.adoc | 18 ++++++++++++++++++
 1 file changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/695182bf/src/main/asciidoc/_chapters/upgrading.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/upgrading.adoc b/src/main/asciidoc/_chapters/upgrading.adoc
index 6dc788a..a556123 100644
--- a/src/main/asciidoc/_chapters/upgrading.adoc
+++ b/src/main/asciidoc/_chapters/upgrading.adoc
@@ -611,6 +611,19 @@ Performance is also an area that is now under active review so look forward to
 improvement in coming releases (See
 link:https://issues.apache.org/jira/browse/HBASE-20188[HBASE-20188 TESTING Performance]).
 
+[[upgrade2.0.it.kerberos]]
+.Integration Tests and Kerberos
+Integration Tests (`IntegrationTests*`) used to rely on the Kerberos credential cache
+for authentication against secured clusters. This used to lead to tests failing due
+to authentication failures when the tickets in the credential cache expired.
+As of hbase-2.0.0 (and hbase-1.3.0+), the integration test clients will make use
+of the configuration properties `hbase.client.keytab.file` and
+`hbase.client.kerberos.principal`. They are required. The clients will perform a
+login from the configured keytab file and automatically refresh the credentials
+in the background for the process lifetime (See
+link:https://issues.apache.org/jira/browse/HBASE-16231[HBASE-16231]).
+
+
 ////
 This would be a good place to link to an appendix on migrating applications
 ////
@@ -731,6 +744,11 @@ Notes:
 
 Doing a raw scan will now return results that have expired according to TTL settings.
 
+[[upgrade1.3]]
+=== Upgrading from pre-1.3 to 1.3+
+If running Integration Tests under Kerberos, see <<upgrade2.0.it.kerberos>>.
+
+
 [[upgrade1.0]]
 === Upgrading to 1.x
 


[12/15] hbase git commit: HBASE-20772 Controlled shutdown fills Master log with the disturbing message 'No matching procedure found for rit=OPEN, location=ZZZZ, table=YYYYY, region=XXXX transition to CLOSED'

Posted by bu...@apache.org.
HBASE-20772 Controlled shutdown fills Master log with the disturbing message 'No matching procedure found for rit=OPEN, location=ZZZZ, table=YYYYY, region=XXXX transition to CLOSED'

Look for the particular case where RS does the close of region w/o
involving Master and log special message in this case. Dodgy. But
until we have Master run shutdown of all regions, better than
the message we currently show.


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

Branch: refs/heads/HBASE-20387
Commit: e705cf1447dd7e44f0cc0297746a0f14135e2f2f
Parents: a2d3f16
Author: Michael Stack <st...@apache.org>
Authored: Mon Aug 13 12:01:54 2018 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Mon Aug 13 15:59:54 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/master/ServerManager.java   |  2 +-
 .../hbase/master/assignment/AssignmentManager.java  | 16 ++++++++++++++--
 2 files changed, 15 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e705cf14/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index 8841808..2cb4d5e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -921,7 +921,7 @@ public class ServerManager {
     }
   }
 
-  boolean isClusterShutdown() {
+  public boolean isClusterShutdown() {
     return this.clusterShutdown.get();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e705cf14/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index 60a2349..70a9680 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -850,8 +850,20 @@ public class AssignmentManager implements ServerListener {
 
     final ServerStateNode serverNode = regionStates.getOrCreateServer(serverName);
     if (!reportTransition(regionNode, serverNode, state, seqId)) {
-      // Don't log if shutting down cluster; during shutdown.
-      LOG.warn("No matching procedure found for {} transition to {}", regionNode, state);
+      // Don't log WARN if shutting down cluster; during shutdown. Avoid the below messages:
+      // 2018-08-13 10:45:10,551 WARN ...AssignmentManager: No matching procedure found for
+      //   rit=OPEN, location=ve0538.halxg.cloudera.com,16020,1533493000958,
+      //   table=IntegrationTestBigLinkedList, region=65ab289e2fc1530df65f6c3d7cde7aa5 transition
+      //   to CLOSED
+      // These happen because on cluster shutdown, we currently let the RegionServers close
+      // regions. This is the only time that region close is not run by the Master (so cluster
+      // goes down fast). Consider changing it so Master runs all shutdowns.
+      if (this.master.getServerManager().isClusterShutdown() &&
+          state.equals(TransitionCode.CLOSED)) {
+        LOG.info("RegionServer {} {}", state, regionNode.getRegionInfo().getEncodedName());
+      } else {
+        LOG.warn("No matching procedure found for {} transition to {}", regionNode, state);
+      }
     }
   }
 


[05/15] hbase git commit: HBASE-20981. Rollback stateCount accounting thrown-off when exception out of rollbackState Signed-off-by: Michael Stack

Posted by bu...@apache.org.
HBASE-20981. Rollback stateCount accounting thrown-off when exception out of rollbackState
Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/HBASE-20387
Commit: 953e5aa88c992cec213bc57c2db12b78908f357f
Parents: 695182b
Author: jackbearden <ja...@jackbearden.com>
Authored: Wed Aug 1 12:50:25 2018 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Sat Aug 11 11:58:38 2018 -0700

----------------------------------------------------------------------
 .../hbase/procedure2/StateMachineProcedure.java |  6 +-
 .../procedure2/TestStateMachineProcedure.java   | 76 ++++++++++++++++++++
 .../hbase/procedure2/TestYieldProcedures.java   |  8 ++-
 3 files changed, 84 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/953e5aa8/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
index 69ecb29..4ed82f2 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
@@ -55,7 +55,7 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
   private final AtomicBoolean aborted = new AtomicBoolean(false);
 
   private Flow stateFlow = Flow.HAS_MORE_STATE;
-  private int stateCount = 0;
+  protected int stateCount = 0;
   private int[] states = null;
 
   private List<Procedure<TEnvironment>> subProcList = null;
@@ -206,13 +206,13 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
     try {
       updateTimestamp();
       rollbackState(env, getCurrentState());
-      stateCount--;
     } finally {
+      stateCount--;
       updateTimestamp();
     }
   }
 
-  private boolean isEofState() {
+  protected boolean isEofState() {
     return stateCount > 0 && states[stateCount-1] == EOF_STATE;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/953e5aa8/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java
index 19ef4bb..b1ed5fb 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java
@@ -142,6 +142,24 @@ public class TestStateMachineProcedure {
   }
 
   @Test
+  public void testChildNormalRollbackStateCount() {
+    procExecutor.getEnvironment().triggerChildRollback = true;
+    TestSMProcedureBadRollback testNormalRollback = new TestSMProcedureBadRollback();
+    long procId = procExecutor.submitProcedure(testNormalRollback);
+    ProcedureTestingUtility.waitProcedure(procExecutor, procId);
+    assertEquals(0, testNormalRollback.stateCount);
+  }
+
+  @Test
+  public void testChildBadRollbackStateCount() {
+    procExecutor.getEnvironment().triggerChildRollback = true;
+    TestSMProcedureBadRollback testBadRollback = new TestSMProcedureBadRollback();
+    long procId = procExecutor.submitProcedure(testBadRollback);
+    ProcedureTestingUtility.waitProcedure(procExecutor, procId);
+    assertEquals(0, testBadRollback.stateCount);
+  }
+
+  @Test
   public void testChildOnLastStepWithRollbackDoubleExecution() throws Exception {
     procExecutor.getEnvironment().triggerChildRollback = true;
     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExecutor, true);
@@ -196,6 +214,64 @@ public class TestStateMachineProcedure {
     }
   }
 
+  public static class TestSMProcedureBadRollback
+          extends StateMachineProcedure<TestProcEnv, TestSMProcedureState> {
+    @Override
+    protected Flow executeFromState(TestProcEnv env, TestSMProcedureState state) {
+      LOG.info("EXEC " + state + " " + this);
+      env.execCount.incrementAndGet();
+      switch (state) {
+        case STEP_1:
+          if (!env.loop) {
+            setNextState(TestSMProcedureState.STEP_2);
+          }
+          break;
+        case STEP_2:
+          addChildProcedure(new SimpleChildProcedure());
+          return Flow.NO_MORE_STATE;
+      }
+      return Flow.HAS_MORE_STATE;
+    }
+    @Override
+    protected void rollbackState(TestProcEnv env, TestSMProcedureState state) {
+      LOG.info("ROLLBACK " + state + " " + this);
+      env.rollbackCount.incrementAndGet();
+    }
+
+    @Override
+    protected TestSMProcedureState getState(int stateId) {
+      return TestSMProcedureState.values()[stateId];
+    }
+
+    @Override
+    protected int getStateId(TestSMProcedureState state) {
+      return state.ordinal();
+    }
+
+    @Override
+    protected TestSMProcedureState getInitialState() {
+      return TestSMProcedureState.STEP_1;
+    }
+
+    @Override
+    protected void rollback(final TestProcEnv env)
+            throws IOException, InterruptedException {
+      if (isEofState()) {
+        stateCount--;
+      }
+      try {
+        updateTimestamp();
+        rollbackState(env, getCurrentState());
+        throw new IOException();
+      } catch(IOException e) {
+        //do nothing for now
+      } finally {
+        stateCount--;
+        updateTimestamp();
+      }
+    }
+  }
+
   public static class SimpleChildProcedure extends NoopProcedure<TestProcEnv> {
     @Override
     protected Procedure[] execute(TestProcEnv env) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/953e5aa8/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestYieldProcedures.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestYieldProcedures.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestYieldProcedures.java
index 8d9b325..18d92ea 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestYieldProcedures.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestYieldProcedures.java
@@ -143,15 +143,17 @@ public class TestYieldProcedures {
       assertEquals(i, info.getStep().ordinal());
     }
 
-    // test rollback (we execute steps twice, one has the IE the other completes)
+    // test rollback (we execute steps twice, rollback counts both IE and completed)
     for (int i = NUM_STATES - 1; i >= 0; --i) {
       TestStateMachineProcedure.ExecutionInfo info = proc.getExecutionInfo().get(count++);
       assertEquals(true, info.isRollback());
       assertEquals(i, info.getStep().ordinal());
+    }
 
-      info = proc.getExecutionInfo().get(count++);
+    for (int i = NUM_STATES - 1; i >= 0; --i) {
+      TestStateMachineProcedure.ExecutionInfo info = proc.getExecutionInfo().get(count++);
       assertEquals(true, info.isRollback());
-      assertEquals(i, info.getStep().ordinal());
+      assertEquals(0, info.getStep().ordinal());
     }
 
     // check runnable queue stats