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

[GitHub] [hbase] mygood opened a new pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern

mygood opened a new pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern
URL: https://github.com/apache/hbase/pull/1016
 
 
   Design Objective:
    1. Merge empty region
    2. Neat region
    3. merge expired region
   Usage: bin/hbase onlinemerge [--tableName=] [--startRegion=] [--stopRegion=] [--maxRegionSize=] [--maxRegionCreateTime=] [--numMaxMergePlans=] [--targetRegionCount=] [--printExecutionPlan=] [--configMergePauseTime=]
   Options:
   --h or --h print help
   --tableName table name must be not null
   --startRegion start region
   --stopRegion stop region
   --maxRegionSize max region size Unit GB
   --maxRegionCreateTime max Region Create Time yyyy/MM/dd HH:mm:ss
   --numMaxMergePlans num MaxMerge Plans
   --targetRegionCount target Region Count
   --configMergePauseTime config Merge Pause Time In milliseconds
   --printExecutionPlan Value default is true print execution plans false is execution merge
   
   Examples:
   bin/hbase onlinemerge --tableName=test:test1 --startRegion=test:test1,,1576835912332.01d0d6c2b41e204104524d9aec6074fb. --stopRegion=test:test1,bbbbbbbb,1573044786980.0c9b5bd93f3b19eb9bd1a1011ddff66f. --maxRegionSize=0 --maxRegionCreateTime=yyyy/MM/dd HH:mm:ss --numMaxMergePlans=2 --targetRegionCount=4 --printExecutionPlan=false
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern
URL: https://github.com/apache/hbase/pull/1016#discussion_r368176192
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/util/OnlineMergeTool.java
 ##########
 @@ -0,0 +1,528 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ * 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.
+ */
+
+package org.apache.hadoop.hbase.util;
+
+import java.io.IOException;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility that can merge any two regions in the same table: adjacent,
+ * overlapping or disjoint. It can also merge every regions, two by two.
+ */
+@InterfaceAudience.Private
+public class OnlineMergeTool extends Configured implements Tool {
+  static final Logger LOG = LoggerFactory.getLogger(OnlineMergeTool.class);
+  private final int COMPACTPAUSETIME = 180 * 1000;
+  private final int DEFAULTMERGEPAUSETIME = 120 * 1000;
+  private final String COMPACTIONATTRIBUTE = "MAJOR";
+  private final long GB = 1024L * 1024L * 1024L;
+  private final SimpleDateFormat DATE_FORMAT
+      = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss");
+  private final HBaseConfiguration conf;
+  private volatile MetaUtils utils;
+  private volatile boolean isMetaTable;
+  private volatile Connection connection;
+  private volatile Admin admin;
+  // Name of table
+  private String tableName = null;
+  // Name of region 1
+  private String startRegion = null;
+  // Name of region 2
+  private String stopRegion = null;
+  // Name of maxRegionSize
+  private Long maxRegionSize = 0L;
+  // Name of maxRegionCreateTime
+  private String maxRegionCreateTime = null;
+  // Name of numMaxMergePlans
+  private String numMaxMergePlans = null;
+  // Name of targetRegionCount
+  private Long targetRegionCount = 0L;
+  /**
+   * print Execution Plan information
+   */
+  private boolean printExecutionPlan = true;
+  /**
+   * config merge pause time
+   */
+  private int mergePauseTime = 0;
+
+  /**
+   * default constructor
+   */
+  public OnlineMergeTool() throws IOException {
+    this(new HBaseConfiguration());
+  }
+
+  /**
+   * @param conf The current configuration.
+   * @throws IOException If IO problem encountered
+   */
+  public OnlineMergeTool(HBaseConfiguration conf) throws IOException {
+    super(conf);
+    this.conf = conf;
+    this.conf.setInt("hbase.client.retries.number", 3);
+    this.conf.setInt("hbase.client.pause", 1000);
+    this.connection = HConnectionManager.createConnection(this.conf);
+    this.admin = connection.getAdmin();
+  }
+
+  /**
+   * Main program
+   *
+   * @param args The command line parameters.
+   */
+  public static void main(String[] args) {
+    int status = 0;
+    try {
+      status = ToolRunner.run(new OnlineMergeTool(), args);
+    } catch (Exception e) {
+      LOG.error("exiting due to error", e);
+      status = -1;
+    }
+    System.exit(status);
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+
+    if (!doCommandLine(args)) {
+      return -1;
+    }
+
+    isMetaTable = Bytes.compareTo(Bytes.toBytes(tableName), HConstants.META_TABLE_NAME) == 0;
+    // Verify file system is up.
+    FileSystem fs = FileSystem.get(this.conf);              // get DFS handle
+    LOG.info("Verifying that file system is available...");
+    try {
+      FSUtils.checkFileSystemAvailable(fs);
+    } catch (IOException e) {
+      LOG.error("File system is not available", e);
+      return -1;
+    }
+
+    // Verify HBase is up
+    LOG.info("Verifying that HBase is running...");
+    try {
+      HBaseAdmin.checkHBaseAvailable(conf);
+    } catch (MasterNotRunningException e) {
+      LOG.error("HBase cluster must be on-line.");
+      return -1;
+    }
+
+    // Initialize MetaUtils and and get the root of the HBase installation
+
+    this.utils = new MetaUtils(conf);
+    List<HRegionInfo> hRegionInfoList = admin.getTableRegions(TableName.valueOf(tableName));
+    try {
+      if (isMetaTable) {
+        throw new Exception("Can't merge meta tables online");
+      } else if (hRegionInfoList.size() <= targetRegionCount) {
+        throw new Exception("Can't merge tables because regionCount="
+            + hRegionInfoList.size() + " less than targetRegionCount=" + targetRegionCount);
+      } else if (printExecutionPlan) {
+        executionPlan();
+      } else {
+        mergeRegions();
+      }
+      return 0;
+    } catch (Exception e) {
+      LOG.error("Merge failed", e);
+      return -1;
+
+    } finally {
+      if (this.utils != null) {
+        this.utils.shutdown();
+      }
+    }
+  }
+
+  private boolean doCommandLine(final String[] args) {
+    if (args.length < 1) {
+      printUsage(null);
+      return false;
+    }
+    try {
+      for (int i = 0; i < args.length; i++) {
+        String cmd = args[i];
+        if (cmd.equals("-h") || cmd.startsWith("--h")) {
+          printUsage(null);
+          return false;
+        }
+
+        final String tableNameKey = "--tableName=";
+        if (cmd.startsWith(tableNameKey)) {
+          tableName = cmd.substring(tableNameKey.length());
+          continue;
+        }
+
+        final String startRegionKey = "--startRegion=";
+        if (cmd.startsWith(startRegionKey)) {
+          startRegion = cmd.substring(startRegionKey.length());
+          continue;
+        }
+
+        final String stopRegionKey = "--stopRegion=";
+        if (cmd.startsWith(stopRegionKey)) {
+          stopRegion = cmd.substring(stopRegionKey.length());
+          continue;
+        }
+
+        final String maxRegionSizeKey = "--maxRegionSize=";
+        if (cmd.startsWith(maxRegionSizeKey)) {
+          maxRegionSize = Long.parseLong(cmd.substring(maxRegionSizeKey.length())) * GB;
+          continue;
+        }
+
+        final String maxRegionCreateTimeKey = "--maxRegionCreateTime=";
+        if (cmd.startsWith(maxRegionCreateTimeKey)) {
+          maxRegionCreateTime = cmd.substring(maxRegionCreateTimeKey.length());
+          continue;
+        }
+
+        final String numMaxMergePlansKey = "--numMaxMergePlans=";
+        if (cmd.startsWith(numMaxMergePlansKey)) {
+          numMaxMergePlans = cmd.substring(numMaxMergePlansKey.length());
+          continue;
+        }
+
+        final String targetRegionCountKey = "--targetRegionCount=";
+        if (cmd.startsWith(targetRegionCountKey)) {
+          targetRegionCount = Long.parseLong(cmd.substring(targetRegionCountKey.length()));
+          continue;
+        }
+
+        final String printExecutionPlanKey = "--printExecutionPlan=";
+        if (cmd.startsWith(printExecutionPlanKey)) {
+          printExecutionPlan = Boolean.parseBoolean(cmd.substring(printExecutionPlanKey.length()));
+          continue;
+        }
+
+        final String mergePauseTimekey = "--configMergePauseTime=";
+        if (cmd.startsWith(mergePauseTimekey)) {
+          mergePauseTime = Integer.parseInt(cmd.substring(mergePauseTimekey.length()));
+          continue;
+        }
+      }
+
+      if (null == tableName || tableName.isEmpty()) {
+        printUsage("table name must be not null");
+        return false;
+      }
+
+      if (null == maxRegionSize || tableName.isEmpty()) {
+        printUsage("table name must be not null");
+        return false;
+      }
+
+
+      if (startRegion != null && stopRegion != null) {
+        if (notInTable(Bytes.toBytes(tableName), Bytes.toBytes(startRegion))
+            || notInTable(Bytes.toBytes(tableName), Bytes.toBytes(stopRegion))) {
+          LOG.error("Can't merge region not in table or region is null startRegion is "
+              + startRegion + " stopRegion is " + stopRegion);
+          return false;
+        } else if (startRegion.equals(stopRegion)) {
+          LOG.error("Can't merge a region with itself");
+          return false;
+        }
+      }
+      if (startRegion != null) {
+        if (null == stopRegion) {
+          printUsage("The startRegion and the stopRegion must be used in pairs stopRegion="
+              + stopRegion);
+          return false;
+        }
+      }
+      if (stopRegion != null) {
+        if (null == startRegion) {
+          printUsage("The startRegion and the stopRegion must be used in pairs startRegion="
+              + startRegion);
+          return false;
+        }
+      }
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      printUsage("Can't start because " + e.getMessage());
+      return false;
+    }
+    return true;
+  }
+
+  private List<Pair<byte[], byte[]>> executionPlan() throws IOException, ParseException {
+    List<HRegionInfo> hris = getListRegionInfo(tableName, startRegion, stopRegion);
+    if (hris.size() < 2) {
+      throw new IOException("The table doesn't have 2 or more regions region count=" + hris.size());
+    }
+    RegionLocator regionLocator = connection.getRegionLocator(TableName.valueOf(tableName));
+    RegionSizeCalculator regionSizeCalculator = new RegionSizeCalculator(regionLocator, admin);
+    List<Pair<byte[], byte[]>> mergePlans = new ArrayList<Pair<byte[], byte[]>>();
+    for (int i = 0; i < hris.size() - 1; i += 2) {
+      HRegionInfo regionInfo1 = hris.get(i);
+      HRegionInfo regionInfo2 = hris.get(i + 1);
+      if (regionInfo1.isOffline() || regionInfo1.isSplit()
+          || regionInfo2.isOffline() || regionInfo2.isSplit()) {
+        LOG.info("Skip Region split or offline region1=" + regionInfo1.getRegionNameAsString()
+            + " region2=" + regionInfo2.getRegionNameAsString());
+        continue;
+      }
+      if (null != maxRegionCreateTime) {
+        long time2Timestamp = DATE_FORMAT.parse(maxRegionCreateTime).getTime();
+        if (regionInfo1.getRegionId() > time2Timestamp
+            || regionInfo2.getRegionId() > time2Timestamp) {
+          StringBuffer mesg = new StringBuffer();
+          mesg.append("Skip Region timestamp region1=");
+          mesg.append(Bytes.toString(regionInfo1.getEncodedNameAsBytes()));
+          mesg.append(" region1Timestamp=").append(regionInfo1.getRegionId());
+          mesg.append(" > maxRegionCreateTime=").append(time2Timestamp);
+          mesg.append(" or region2=").append(Bytes.toString(regionInfo2.getEncodedNameAsBytes()));
+          mesg.append(" region1Timestamp=").append(regionInfo2.getRegionId());
+          mesg.append(" > maxRegionCreateTime=").append(time2Timestamp);
+          LOG.info(mesg.toString());
+          continue;
+        }
+      }
+      long regionSize = regionSizeCalculator.getRegionSize(regionInfo1.getRegionName());
+      long regionSize_next = regionSizeCalculator.getRegionSize(regionInfo2.getRegionName());
+      if (regionSize > maxRegionSize || regionSize_next > maxRegionSize) {
+        StringBuilder mesg = new StringBuilder();
+        mesg.append("Skip Region size region1=");
+        mesg.append(Bytes.toString(regionInfo1.getEncodedNameAsBytes()));
+        mesg.append(" region1Size=").append(regionSize);
+        mesg.append(" > maxRegionSize=").append(maxRegionSize);
+        mesg.append(" or region2=").append(Bytes.toString(regionInfo2.getEncodedNameAsBytes()));
+        mesg.append(" region2Size=").append(regionSize_next);
+        mesg.append(" > maxRegionSize=").append(maxRegionSize);
+
+        LOG.info(mesg.toString());
+        continue;
+      }
+      Pair<byte[], byte[]> pair = new Pair<byte[], byte[]>();
+      pair.setFirst(regionInfo1.getEncodedNameAsBytes());
+      pair.setSecond(regionInfo2.getEncodedNameAsBytes());
+      mergePlans.add(pair);
+      StringBuilder mesg = new StringBuilder();
+      mesg.append("Print merge plans region1=");
+      mesg.append(Bytes.toString(regionInfo1.getEncodedNameAsBytes()));
+      mesg.append(" region1=").append(regionSize / GB);
+      mesg.append("G region2=");
+      mesg.append(Bytes.toString(regionInfo2.getEncodedNameAsBytes()));
+      mesg.append(" region2=").append(regionSize_next / GB).append("G");
+      LOG.info(mesg.toString());
+    }
+    return mergePlans;
+  }
+
+  /*
+   * Merges two regions from a user table.
+   */
+  private void mergeRegions()
+      throws IOException, InterruptedException, ParseException {
+    List<Pair<byte[], byte[]>> mergePlans = executionPlan();
+    if (mergePlans.size() < 1) {
+      printCompletedMesg();
+      return;
+    }
+    if (null == numMaxMergePlans) {
+      for (Pair<byte[], byte[]> region : mergePlans) {
+        admin.mergeRegions(region.getFirst(), region.getSecond(), false);
+        LOG.info("Merging regions " + Bytes.toString(region.getFirst()) + " and " +
+            Bytes.toString(region.getSecond()) + " in table " + tableName);
+      }
+      Thread.sleep(mergePauseTime > 0 ? mergePauseTime : DEFAULTMERGEPAUSETIME);
+      if (maxRegionSize != 0) {
+        admin.compact(TableName.valueOf(tableName));
+        LOG.info("Table=" + tableName + " is runing compact");
+        runCompaction(tableName);
+      }
+    } else {
+      long numMaxMergePlans2Long = Long.parseLong(numMaxMergePlans);
+      for (int i = 0; i < mergePlans.size(); i++) {
+        admin.mergeRegions(mergePlans.get(i).getFirst(), mergePlans.get(i).getSecond(), false);
+        LOG.info("Merging regions " + Bytes.toString(mergePlans.get(i).getFirst()) + " and " +
+            Bytes.toString(mergePlans.get(i).getSecond())
+            + " in table " + tableName);
+        if (i + 1 % numMaxMergePlans2Long == 0) {
+          Thread.sleep(mergePauseTime > 0 ? mergePauseTime : DEFAULTMERGEPAUSETIME);
+          if (maxRegionSize != 0) {
+            admin.compact(TableName.valueOf(tableName));
+            LOG.info("Table=" + tableName + " is runing compact");
+            runCompaction(tableName);
+          }
+        }
+      }
+      Thread.sleep(mergePauseTime > 0 ? mergePauseTime : DEFAULTMERGEPAUSETIME);
+      if (maxRegionSize != 0) {
+        admin.compact(TableName.valueOf(tableName));
+        LOG.info("Table=" + tableName + " is runing compact");
+        runCompaction(tableName);
+      }
+    }
+    List<HRegionInfo> hRegionInfoList = admin.getTableRegions(TableName.valueOf(tableName));
+    if (hRegionInfoList.size() <= targetRegionCount
+        || getListRegionInfo(tableName, startRegion, stopRegion).size() < 2) {
+      printCompletedMesg();
+    } else {
+      mergeRegions();
+    }
+  }
+
+  /**
+   * print merge completed Mesg
+   */
+  private void printCompletedMesg() {
+    StringBuilder mesg = new StringBuilder();
+    mesg.append("Merge completed table=");
+    mesg.append(tableName);
+    mesg.append(" startRegion=");
+    mesg.append(startRegion);
+    mesg.append(" stopRegion=");
+    mesg.append(stopRegion);
+    mesg.append(" maxRegionSize=");
+    mesg.append(maxRegionSize / GB).append("G");
+    mesg.append(" maxRegionCreateTime=").append(maxRegionCreateTime);
+    mesg.append(" numMaxMergePlans=");
+    mesg.append(numMaxMergePlans);
+    mesg.append(" targetRegionCount=");
+    mesg.append(targetRegionCount);
+    LOG.info(mesg.toString());
+  }
+
+  /**
+   * Get the list of a HRIs in a table
+   *
+   * @return list of hris
+   * @throws IOException If IO problem encountered
+   */
+  List<HRegionInfo> getListRegionInfo(String tableName, String startRegion, String stopRegion)
+      throws IOException {
+    boolean isAdd = false;
+    List<HRegionInfo> hris = new ArrayList<HRegionInfo>();
+    List<HRegionInfo> tableRegions = this.admin.getTableRegions(TableName.valueOf(tableName));
+    for (HRegionInfo hri : tableRegions) {
+      if (null == startRegion && null == stopRegion) {
+        hris.add(hri);
+        LOG.info("Add legitimate range resgion=" + hri.getRegionNameAsString());
+
+      } else if (null != startRegion && null != stopRegion) {
+        if (hri.getRegionNameAsString().equals(startRegion)) {
+          LOG.info("Open interval startRegion=" + hri.getRegionNameAsString());
+          isAdd = true;
+          continue;
+        }
+        if (hri.getRegionNameAsString().equals(stopRegion)) {
+          LOG.info("Open interval stopRegion=" + hri.getRegionNameAsString());
+          isAdd = false;
+          break;
+        }
+        if (isAdd) {
+          hris.add(hri);
+          LOG.info("Add legitimate range resgion=" + hri.getRegionNameAsString());
+        }
+      }
+    }
+    return hris;
+  }
+
+  /**
+   * Waiting for compaction complete
+   *
+   * @param tableName table name
+   * @throws IOException          If IO problem encountered
+   * @throws InterruptedException If Interrupted problem encountered
+   */
+  private void runCompaction(String tableName) throws IOException, InterruptedException {
+    while (true) {
+      long startTime = System.currentTimeMillis();
+      String compactionState = this.admin.getCompactionState(
+          TableName.valueOf(tableName)).toString();
+      if (!COMPACTIONATTRIBUTE.equals(compactionState)) {
+        LOG.info("Table=" + tableName + " compationState="
+            + compactionState + " compact complete");
+        break;
+      }
+      Thread.sleep(COMPACTPAUSETIME);
+      long waitTime = (System.currentTimeMillis() - startTime) / 1000;
+      LOG.info("Table=" + tableName + " compationState="
+          + compactionState + " the waiting time " + waitTime + "seconds");
+    }
+  }
+
+  private boolean notInTable(final byte[] tn, final byte[] rn) {
+    if (WritableComparator.compareBytes(tn, 0, tn.length, rn, 0, tn.length) != 0) {
+      LOG.error("Region " + Bytes.toString(rn) + " does not belong to table " +
+          Bytes.toString(tn));
+      return true;
+    }
+    return false;
+  }
+
+  /*
+   * @param errorMsg Error message.  Can be null.
+   */
+  private void printUsage(final String errorMsg) {
+    if (errorMsg != null && errorMsg.length() > 0) {
+      System.err.println("ERROR: " + errorMsg);
+    }
+    System.err.println(
+        "Usage: bin/hbase onlinemerge [--tableName=] " +
+            "[--startRegion=] [--stopRegion=] [--maxRegionSize=] " +
+            "[--maxRegionCreateTime=] [--numMaxMergePlans=] " +
+            "[--targetRegionCount=] [--printExecutionPlan=] [--configMergePauseTime=]\n");
+    System.err.println("Options:");
+    System.err.println("--h or --h              print help");
+    System.err.println("--tableName             table name must be not null");
+    System.err.println("--startRegion           start region");
 
 Review comment:
   Say if name is encoded name or full region name.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern
URL: https://github.com/apache/hbase/pull/1016#discussion_r368175342
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/util/OnlineMergeTool.java
 ##########
 @@ -0,0 +1,528 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ * 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.
+ */
+
+package org.apache.hadoop.hbase.util;
+
+import java.io.IOException;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility that can merge any two regions in the same table: adjacent,
+ * overlapping or disjoint. It can also merge every regions, two by two.
+ */
+@InterfaceAudience.Private
+public class OnlineMergeTool extends Configured implements Tool {
+  static final Logger LOG = LoggerFactory.getLogger(OnlineMergeTool.class);
+  private final int COMPACTPAUSETIME = 180 * 1000;
+  private final int DEFAULTMERGEPAUSETIME = 120 * 1000;
+  private final String COMPACTIONATTRIBUTE = "MAJOR";
+  private final long GB = 1024L * 1024L * 1024L;
+  private final SimpleDateFormat DATE_FORMAT
+      = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss");
 
 Review comment:
   Do ISO8601?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern
URL: https://github.com/apache/hbase/pull/1016#discussion_r368176220
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/util/OnlineMergeTool.java
 ##########
 @@ -0,0 +1,528 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ * 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.
+ */
+
+package org.apache.hadoop.hbase.util;
+
+import java.io.IOException;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility that can merge any two regions in the same table: adjacent,
+ * overlapping or disjoint. It can also merge every regions, two by two.
+ */
+@InterfaceAudience.Private
+public class OnlineMergeTool extends Configured implements Tool {
+  static final Logger LOG = LoggerFactory.getLogger(OnlineMergeTool.class);
+  private final int COMPACTPAUSETIME = 180 * 1000;
+  private final int DEFAULTMERGEPAUSETIME = 120 * 1000;
+  private final String COMPACTIONATTRIBUTE = "MAJOR";
+  private final long GB = 1024L * 1024L * 1024L;
+  private final SimpleDateFormat DATE_FORMAT
+      = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss");
+  private final HBaseConfiguration conf;
+  private volatile MetaUtils utils;
+  private volatile boolean isMetaTable;
+  private volatile Connection connection;
+  private volatile Admin admin;
+  // Name of table
+  private String tableName = null;
+  // Name of region 1
+  private String startRegion = null;
+  // Name of region 2
+  private String stopRegion = null;
+  // Name of maxRegionSize
+  private Long maxRegionSize = 0L;
+  // Name of maxRegionCreateTime
+  private String maxRegionCreateTime = null;
+  // Name of numMaxMergePlans
+  private String numMaxMergePlans = null;
+  // Name of targetRegionCount
+  private Long targetRegionCount = 0L;
+  /**
+   * print Execution Plan information
+   */
+  private boolean printExecutionPlan = true;
+  /**
+   * config merge pause time
+   */
+  private int mergePauseTime = 0;
+
+  /**
+   * default constructor
+   */
+  public OnlineMergeTool() throws IOException {
+    this(new HBaseConfiguration());
+  }
+
+  /**
+   * @param conf The current configuration.
+   * @throws IOException If IO problem encountered
+   */
+  public OnlineMergeTool(HBaseConfiguration conf) throws IOException {
+    super(conf);
+    this.conf = conf;
+    this.conf.setInt("hbase.client.retries.number", 3);
+    this.conf.setInt("hbase.client.pause", 1000);
+    this.connection = HConnectionManager.createConnection(this.conf);
+    this.admin = connection.getAdmin();
+  }
+
+  /**
+   * Main program
+   *
+   * @param args The command line parameters.
+   */
+  public static void main(String[] args) {
+    int status = 0;
+    try {
+      status = ToolRunner.run(new OnlineMergeTool(), args);
+    } catch (Exception e) {
+      LOG.error("exiting due to error", e);
+      status = -1;
+    }
+    System.exit(status);
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+
+    if (!doCommandLine(args)) {
+      return -1;
+    }
+
+    isMetaTable = Bytes.compareTo(Bytes.toBytes(tableName), HConstants.META_TABLE_NAME) == 0;
+    // Verify file system is up.
+    FileSystem fs = FileSystem.get(this.conf);              // get DFS handle
+    LOG.info("Verifying that file system is available...");
+    try {
+      FSUtils.checkFileSystemAvailable(fs);
+    } catch (IOException e) {
+      LOG.error("File system is not available", e);
+      return -1;
+    }
+
+    // Verify HBase is up
+    LOG.info("Verifying that HBase is running...");
+    try {
+      HBaseAdmin.checkHBaseAvailable(conf);
+    } catch (MasterNotRunningException e) {
+      LOG.error("HBase cluster must be on-line.");
+      return -1;
+    }
+
+    // Initialize MetaUtils and and get the root of the HBase installation
+
+    this.utils = new MetaUtils(conf);
+    List<HRegionInfo> hRegionInfoList = admin.getTableRegions(TableName.valueOf(tableName));
+    try {
+      if (isMetaTable) {
+        throw new Exception("Can't merge meta tables online");
+      } else if (hRegionInfoList.size() <= targetRegionCount) {
+        throw new Exception("Can't merge tables because regionCount="
+            + hRegionInfoList.size() + " less than targetRegionCount=" + targetRegionCount);
+      } else if (printExecutionPlan) {
+        executionPlan();
+      } else {
+        mergeRegions();
+      }
+      return 0;
+    } catch (Exception e) {
+      LOG.error("Merge failed", e);
+      return -1;
+
+    } finally {
+      if (this.utils != null) {
+        this.utils.shutdown();
+      }
+    }
+  }
+
+  private boolean doCommandLine(final String[] args) {
+    if (args.length < 1) {
+      printUsage(null);
+      return false;
+    }
+    try {
+      for (int i = 0; i < args.length; i++) {
+        String cmd = args[i];
+        if (cmd.equals("-h") || cmd.startsWith("--h")) {
+          printUsage(null);
+          return false;
+        }
+
+        final String tableNameKey = "--tableName=";
+        if (cmd.startsWith(tableNameKey)) {
+          tableName = cmd.substring(tableNameKey.length());
+          continue;
+        }
+
+        final String startRegionKey = "--startRegion=";
+        if (cmd.startsWith(startRegionKey)) {
+          startRegion = cmd.substring(startRegionKey.length());
+          continue;
+        }
+
+        final String stopRegionKey = "--stopRegion=";
+        if (cmd.startsWith(stopRegionKey)) {
+          stopRegion = cmd.substring(stopRegionKey.length());
+          continue;
+        }
+
+        final String maxRegionSizeKey = "--maxRegionSize=";
+        if (cmd.startsWith(maxRegionSizeKey)) {
+          maxRegionSize = Long.parseLong(cmd.substring(maxRegionSizeKey.length())) * GB;
+          continue;
+        }
+
+        final String maxRegionCreateTimeKey = "--maxRegionCreateTime=";
+        if (cmd.startsWith(maxRegionCreateTimeKey)) {
+          maxRegionCreateTime = cmd.substring(maxRegionCreateTimeKey.length());
+          continue;
+        }
+
+        final String numMaxMergePlansKey = "--numMaxMergePlans=";
+        if (cmd.startsWith(numMaxMergePlansKey)) {
+          numMaxMergePlans = cmd.substring(numMaxMergePlansKey.length());
+          continue;
+        }
+
+        final String targetRegionCountKey = "--targetRegionCount=";
+        if (cmd.startsWith(targetRegionCountKey)) {
+          targetRegionCount = Long.parseLong(cmd.substring(targetRegionCountKey.length()));
+          continue;
+        }
+
+        final String printExecutionPlanKey = "--printExecutionPlan=";
+        if (cmd.startsWith(printExecutionPlanKey)) {
+          printExecutionPlan = Boolean.parseBoolean(cmd.substring(printExecutionPlanKey.length()));
+          continue;
+        }
+
+        final String mergePauseTimekey = "--configMergePauseTime=";
+        if (cmd.startsWith(mergePauseTimekey)) {
+          mergePauseTime = Integer.parseInt(cmd.substring(mergePauseTimekey.length()));
+          continue;
+        }
+      }
+
+      if (null == tableName || tableName.isEmpty()) {
+        printUsage("table name must be not null");
+        return false;
+      }
+
+      if (null == maxRegionSize || tableName.isEmpty()) {
+        printUsage("table name must be not null");
+        return false;
+      }
+
+
+      if (startRegion != null && stopRegion != null) {
+        if (notInTable(Bytes.toBytes(tableName), Bytes.toBytes(startRegion))
+            || notInTable(Bytes.toBytes(tableName), Bytes.toBytes(stopRegion))) {
+          LOG.error("Can't merge region not in table or region is null startRegion is "
+              + startRegion + " stopRegion is " + stopRegion);
+          return false;
+        } else if (startRegion.equals(stopRegion)) {
+          LOG.error("Can't merge a region with itself");
+          return false;
+        }
+      }
+      if (startRegion != null) {
+        if (null == stopRegion) {
+          printUsage("The startRegion and the stopRegion must be used in pairs stopRegion="
+              + stopRegion);
+          return false;
+        }
+      }
+      if (stopRegion != null) {
+        if (null == startRegion) {
+          printUsage("The startRegion and the stopRegion must be used in pairs startRegion="
+              + startRegion);
+          return false;
+        }
+      }
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      printUsage("Can't start because " + e.getMessage());
+      return false;
+    }
+    return true;
+  }
+
+  private List<Pair<byte[], byte[]>> executionPlan() throws IOException, ParseException {
+    List<HRegionInfo> hris = getListRegionInfo(tableName, startRegion, stopRegion);
+    if (hris.size() < 2) {
+      throw new IOException("The table doesn't have 2 or more regions region count=" + hris.size());
+    }
+    RegionLocator regionLocator = connection.getRegionLocator(TableName.valueOf(tableName));
+    RegionSizeCalculator regionSizeCalculator = new RegionSizeCalculator(regionLocator, admin);
+    List<Pair<byte[], byte[]>> mergePlans = new ArrayList<Pair<byte[], byte[]>>();
+    for (int i = 0; i < hris.size() - 1; i += 2) {
+      HRegionInfo regionInfo1 = hris.get(i);
+      HRegionInfo regionInfo2 = hris.get(i + 1);
+      if (regionInfo1.isOffline() || regionInfo1.isSplit()
+          || regionInfo2.isOffline() || regionInfo2.isSplit()) {
+        LOG.info("Skip Region split or offline region1=" + regionInfo1.getRegionNameAsString()
+            + " region2=" + regionInfo2.getRegionNameAsString());
+        continue;
+      }
+      if (null != maxRegionCreateTime) {
+        long time2Timestamp = DATE_FORMAT.parse(maxRegionCreateTime).getTime();
+        if (regionInfo1.getRegionId() > time2Timestamp
+            || regionInfo2.getRegionId() > time2Timestamp) {
+          StringBuffer mesg = new StringBuffer();
+          mesg.append("Skip Region timestamp region1=");
+          mesg.append(Bytes.toString(regionInfo1.getEncodedNameAsBytes()));
+          mesg.append(" region1Timestamp=").append(regionInfo1.getRegionId());
+          mesg.append(" > maxRegionCreateTime=").append(time2Timestamp);
+          mesg.append(" or region2=").append(Bytes.toString(regionInfo2.getEncodedNameAsBytes()));
+          mesg.append(" region1Timestamp=").append(regionInfo2.getRegionId());
+          mesg.append(" > maxRegionCreateTime=").append(time2Timestamp);
+          LOG.info(mesg.toString());
+          continue;
+        }
+      }
+      long regionSize = regionSizeCalculator.getRegionSize(regionInfo1.getRegionName());
+      long regionSize_next = regionSizeCalculator.getRegionSize(regionInfo2.getRegionName());
+      if (regionSize > maxRegionSize || regionSize_next > maxRegionSize) {
+        StringBuilder mesg = new StringBuilder();
+        mesg.append("Skip Region size region1=");
+        mesg.append(Bytes.toString(regionInfo1.getEncodedNameAsBytes()));
+        mesg.append(" region1Size=").append(regionSize);
+        mesg.append(" > maxRegionSize=").append(maxRegionSize);
+        mesg.append(" or region2=").append(Bytes.toString(regionInfo2.getEncodedNameAsBytes()));
+        mesg.append(" region2Size=").append(regionSize_next);
+        mesg.append(" > maxRegionSize=").append(maxRegionSize);
+
+        LOG.info(mesg.toString());
+        continue;
+      }
+      Pair<byte[], byte[]> pair = new Pair<byte[], byte[]>();
+      pair.setFirst(regionInfo1.getEncodedNameAsBytes());
+      pair.setSecond(regionInfo2.getEncodedNameAsBytes());
+      mergePlans.add(pair);
+      StringBuilder mesg = new StringBuilder();
+      mesg.append("Print merge plans region1=");
+      mesg.append(Bytes.toString(regionInfo1.getEncodedNameAsBytes()));
+      mesg.append(" region1=").append(regionSize / GB);
+      mesg.append("G region2=");
+      mesg.append(Bytes.toString(regionInfo2.getEncodedNameAsBytes()));
+      mesg.append(" region2=").append(regionSize_next / GB).append("G");
+      LOG.info(mesg.toString());
+    }
+    return mergePlans;
+  }
+
+  /*
+   * Merges two regions from a user table.
+   */
+  private void mergeRegions()
+      throws IOException, InterruptedException, ParseException {
+    List<Pair<byte[], byte[]>> mergePlans = executionPlan();
+    if (mergePlans.size() < 1) {
+      printCompletedMesg();
+      return;
+    }
+    if (null == numMaxMergePlans) {
+      for (Pair<byte[], byte[]> region : mergePlans) {
+        admin.mergeRegions(region.getFirst(), region.getSecond(), false);
+        LOG.info("Merging regions " + Bytes.toString(region.getFirst()) + " and " +
+            Bytes.toString(region.getSecond()) + " in table " + tableName);
+      }
+      Thread.sleep(mergePauseTime > 0 ? mergePauseTime : DEFAULTMERGEPAUSETIME);
+      if (maxRegionSize != 0) {
+        admin.compact(TableName.valueOf(tableName));
+        LOG.info("Table=" + tableName + " is runing compact");
+        runCompaction(tableName);
+      }
+    } else {
+      long numMaxMergePlans2Long = Long.parseLong(numMaxMergePlans);
+      for (int i = 0; i < mergePlans.size(); i++) {
+        admin.mergeRegions(mergePlans.get(i).getFirst(), mergePlans.get(i).getSecond(), false);
+        LOG.info("Merging regions " + Bytes.toString(mergePlans.get(i).getFirst()) + " and " +
+            Bytes.toString(mergePlans.get(i).getSecond())
+            + " in table " + tableName);
+        if (i + 1 % numMaxMergePlans2Long == 0) {
+          Thread.sleep(mergePauseTime > 0 ? mergePauseTime : DEFAULTMERGEPAUSETIME);
+          if (maxRegionSize != 0) {
+            admin.compact(TableName.valueOf(tableName));
+            LOG.info("Table=" + tableName + " is runing compact");
+            runCompaction(tableName);
+          }
+        }
+      }
+      Thread.sleep(mergePauseTime > 0 ? mergePauseTime : DEFAULTMERGEPAUSETIME);
+      if (maxRegionSize != 0) {
+        admin.compact(TableName.valueOf(tableName));
+        LOG.info("Table=" + tableName + " is runing compact");
+        runCompaction(tableName);
+      }
+    }
+    List<HRegionInfo> hRegionInfoList = admin.getTableRegions(TableName.valueOf(tableName));
+    if (hRegionInfoList.size() <= targetRegionCount
+        || getListRegionInfo(tableName, startRegion, stopRegion).size() < 2) {
+      printCompletedMesg();
+    } else {
+      mergeRegions();
+    }
+  }
+
+  /**
+   * print merge completed Mesg
+   */
+  private void printCompletedMesg() {
+    StringBuilder mesg = new StringBuilder();
+    mesg.append("Merge completed table=");
+    mesg.append(tableName);
+    mesg.append(" startRegion=");
+    mesg.append(startRegion);
+    mesg.append(" stopRegion=");
+    mesg.append(stopRegion);
+    mesg.append(" maxRegionSize=");
+    mesg.append(maxRegionSize / GB).append("G");
+    mesg.append(" maxRegionCreateTime=").append(maxRegionCreateTime);
+    mesg.append(" numMaxMergePlans=");
+    mesg.append(numMaxMergePlans);
+    mesg.append(" targetRegionCount=");
+    mesg.append(targetRegionCount);
+    LOG.info(mesg.toString());
+  }
+
+  /**
+   * Get the list of a HRIs in a table
+   *
+   * @return list of hris
+   * @throws IOException If IO problem encountered
+   */
+  List<HRegionInfo> getListRegionInfo(String tableName, String startRegion, String stopRegion)
+      throws IOException {
+    boolean isAdd = false;
+    List<HRegionInfo> hris = new ArrayList<HRegionInfo>();
+    List<HRegionInfo> tableRegions = this.admin.getTableRegions(TableName.valueOf(tableName));
+    for (HRegionInfo hri : tableRegions) {
+      if (null == startRegion && null == stopRegion) {
+        hris.add(hri);
+        LOG.info("Add legitimate range resgion=" + hri.getRegionNameAsString());
+
+      } else if (null != startRegion && null != stopRegion) {
+        if (hri.getRegionNameAsString().equals(startRegion)) {
+          LOG.info("Open interval startRegion=" + hri.getRegionNameAsString());
+          isAdd = true;
+          continue;
+        }
+        if (hri.getRegionNameAsString().equals(stopRegion)) {
+          LOG.info("Open interval stopRegion=" + hri.getRegionNameAsString());
+          isAdd = false;
+          break;
+        }
+        if (isAdd) {
+          hris.add(hri);
+          LOG.info("Add legitimate range resgion=" + hri.getRegionNameAsString());
+        }
+      }
+    }
+    return hris;
+  }
+
+  /**
+   * Waiting for compaction complete
+   *
+   * @param tableName table name
+   * @throws IOException          If IO problem encountered
+   * @throws InterruptedException If Interrupted problem encountered
+   */
+  private void runCompaction(String tableName) throws IOException, InterruptedException {
+    while (true) {
+      long startTime = System.currentTimeMillis();
+      String compactionState = this.admin.getCompactionState(
+          TableName.valueOf(tableName)).toString();
+      if (!COMPACTIONATTRIBUTE.equals(compactionState)) {
+        LOG.info("Table=" + tableName + " compationState="
+            + compactionState + " compact complete");
+        break;
+      }
+      Thread.sleep(COMPACTPAUSETIME);
+      long waitTime = (System.currentTimeMillis() - startTime) / 1000;
+      LOG.info("Table=" + tableName + " compationState="
+          + compactionState + " the waiting time " + waitTime + "seconds");
+    }
+  }
+
+  private boolean notInTable(final byte[] tn, final byte[] rn) {
+    if (WritableComparator.compareBytes(tn, 0, tn.length, rn, 0, tn.length) != 0) {
+      LOG.error("Region " + Bytes.toString(rn) + " does not belong to table " +
+          Bytes.toString(tn));
+      return true;
+    }
+    return false;
+  }
+
+  /*
+   * @param errorMsg Error message.  Can be null.
+   */
+  private void printUsage(final String errorMsg) {
+    if (errorMsg != null && errorMsg.length() > 0) {
+      System.err.println("ERROR: " + errorMsg);
+    }
+    System.err.println(
+        "Usage: bin/hbase onlinemerge [--tableName=] " +
+            "[--startRegion=] [--stopRegion=] [--maxRegionSize=] " +
+            "[--maxRegionCreateTime=] [--numMaxMergePlans=] " +
+            "[--targetRegionCount=] [--printExecutionPlan=] [--configMergePauseTime=]\n");
+    System.err.println("Options:");
+    System.err.println("--h or --h              print help");
+    System.err.println("--tableName             table name must be not null");
+    System.err.println("--startRegion           start region");
+    System.err.println("--stopRegion            stop region");
 
 Review comment:
   ditto

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern
URL: https://github.com/apache/hbase/pull/1016#discussion_r368175421
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/util/OnlineMergeTool.java
 ##########
 @@ -0,0 +1,528 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ * 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.
+ */
+
+package org.apache.hadoop.hbase.util;
+
+import java.io.IOException;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility that can merge any two regions in the same table: adjacent,
+ * overlapping or disjoint. It can also merge every regions, two by two.
+ */
+@InterfaceAudience.Private
+public class OnlineMergeTool extends Configured implements Tool {
+  static final Logger LOG = LoggerFactory.getLogger(OnlineMergeTool.class);
+  private final int COMPACTPAUSETIME = 180 * 1000;
+  private final int DEFAULTMERGEPAUSETIME = 120 * 1000;
+  private final String COMPACTIONATTRIBUTE = "MAJOR";
+  private final long GB = 1024L * 1024L * 1024L;
+  private final SimpleDateFormat DATE_FORMAT
+      = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss");
+  private final HBaseConfiguration conf;
+  private volatile MetaUtils utils;
+  private volatile boolean isMetaTable;
+  private volatile Connection connection;
+  private volatile Admin admin;
+  // Name of table
+  private String tableName = null;
+  // Name of region 1
+  private String startRegion = null;
+  // Name of region 2
+  private String stopRegion = null;
+  // Name of maxRegionSize
+  private Long maxRegionSize = 0L;
+  // Name of maxRegionCreateTime
+  private String maxRegionCreateTime = null;
+  // Name of numMaxMergePlans
+  private String numMaxMergePlans = null;
+  // Name of targetRegionCount
+  private Long targetRegionCount = 0L;
+  /**
+   * print Execution Plan information
+   */
+  private boolean printExecutionPlan = true;
+  /**
+   * config merge pause time
+   */
+  private int mergePauseTime = 0;
+
+  /**
+   * default constructor
+   */
+  public OnlineMergeTool() throws IOException {
+    this(new HBaseConfiguration());
+  }
+
+  /**
+   * @param conf The current configuration.
+   * @throws IOException If IO problem encountered
+   */
+  public OnlineMergeTool(HBaseConfiguration conf) throws IOException {
 
 Review comment:
   Yeah, arent' all merges online? Why not just MergeTool?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] mygood commented on issue #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern

Posted by GitBox <gi...@apache.org>.
mygood commented on issue #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern
URL: https://github.com/apache/hbase/pull/1016#issuecomment-574135797
 
 
   @saintstack Thank you review code

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern
URL: https://github.com/apache/hbase/pull/1016#issuecomment-591225685
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  11m 50s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  shelldocs  |   0m  0s |  Shelldocs was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ branch-1.4 Compile Tests _ |
   | +0 :ok: |  mvndep  |   1m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   7m 22s |  branch-1.4 passed  |
   | +1 :green_heart: |  compile  |   1m 50s |  branch-1.4 passed with JDK v1.8.0_242  |
   | +1 :green_heart: |  compile  |   1m 53s |  branch-1.4 passed with JDK v1.7.0_252  |
   | +1 :green_heart: |  checkstyle  |   7m 57s |  branch-1.4 passed  |
   | +1 :green_heart: |  shadedjars  |   3m  2s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 39s |  branch-1.4 passed with JDK v1.8.0_242  |
   | +1 :green_heart: |  javadoc  |   3m 50s |  branch-1.4 passed with JDK v1.7.0_252  |
   | +0 :ok: |  spotbugs  |   2m 42s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |  14m 16s |  branch-1.4 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m  7s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 13s |  the patch passed with JDK v1.8.0_242  |
   | +1 :green_heart: |  javac  |   2m 13s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 12s |  the patch passed with JDK v1.7.0_252  |
   | +1 :green_heart: |  javac  |   3m 12s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   9m 34s |  the patch passed  |
   | +1 :green_heart: |  shellcheck  |   0m  2s |  There were no new shellcheck issues.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   3m 39s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |   2m 59s |  Patch does not cause any errors with Hadoop 2.7.7.  |
   | +1 :green_heart: |  javadoc  |   2m 43s |  the patch passed with JDK v1.8.0_242  |
   | +1 :green_heart: |  javadoc  |   3m 47s |  the patch passed with JDK v1.7.0_252  |
   | +1 :green_heart: |  findbugs  |  15m  8s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 166m 31s |  root in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 59s |  The patch does not generate ASF License warnings.  |
   |  |   | 273m 23s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.mapreduce.TestLoadIncrementalHFilesUseSecurityEndPoint |
   |   | hadoop.hbase.client.TestAdmin2 |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.6 Server=19.03.6 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1016 |
   | Optional Tests | dupname asflicense shellcheck shelldocs javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux e6f933d0c6ca 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1016/out/precommit/personality/provided.sh |
   | git revision | branch-1.4 / 38bf65a |
   | Default Java | 1.7.0_252 |
   | Multi-JDK versions | /usr/lib/jvm/zulu-8-amd64:1.8.0_242 /usr/lib/jvm/zulu-7-amd64:1.7.0_252 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/1/artifact/out/patch-unit-root.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/1/testReport/ |
   | Max. process+thread count | 4095 (vs. ulimit of 10000) |
   | modules | C: hbase-server . U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/1/console |
   | versions | git=1.9.1 maven=3.0.5 shellcheck=0.7.0 findbugs=3.0.1 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern
URL: https://github.com/apache/hbase/pull/1016#discussion_r368175257
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/util/OnlineMergeTool.java
 ##########
 @@ -0,0 +1,528 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ * 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.
+ */
+
+package org.apache.hadoop.hbase.util;
+
+import java.io.IOException;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility that can merge any two regions in the same table: adjacent,
+ * overlapping or disjoint. It can also merge every regions, two by two.
 
 Review comment:
   This is for branch-1? In branch-2, we have a multimerge procedure that can take many (adjacent) regions and squash them up together.
   
   We could have this in branch-1. What would you do for branch-2?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern
URL: https://github.com/apache/hbase/pull/1016#discussion_r368174999
 
 

 ##########
 File path: bin/hbase.cmd
 ##########
 @@ -436,4 +436,5 @@ goto :eof
   echo   mapredcp        Dump CLASSPATH entries required by mapreduce
   echo   version         Print the version
   echo   CLASSNAME       Run the class named CLASSNAME
+  echo   onlinemerge     Run the merge tool
 
 Review comment:
   What is difference between a merge and an online merge? Why does this have to be here and not in the shell? Shell already has a merge region facility. Add this as an option to the shell command?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] mygood commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern

Posted by GitBox <gi...@apache.org>.
mygood commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern
URL: https://github.com/apache/hbase/pull/1016#discussion_r368393637
 
 

 ##########
 File path: bin/hbase.cmd
 ##########
 @@ -436,4 +436,5 @@ goto :eof
   echo   mapredcp        Dump CLASSPATH entries required by mapreduce
   echo   version         Print the version
   echo   CLASSNAME       Run the class named CLASSNAME
+  echo   onlinemerge     Run the merge tool
 
 Review comment:
   When a large table has tens of thousands of region to set a TTL for this table, there will be many empty region. I want to merge the empty region, but I don't want to merge them one by one through HBase shell. I want to have a merge task to execute.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] mygood commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern

Posted by GitBox <gi...@apache.org>.
mygood commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern
URL: https://github.com/apache/hbase/pull/1016#discussion_r368393575
 
 

 ##########
 File path: bin/hbase.cmd
 ##########
 @@ -436,4 +436,5 @@ goto :eof
   echo   mapredcp        Dump CLASSPATH entries required by mapreduce
   echo   version         Print the version
   echo   CLASSNAME       Run the class named CLASSNAME
+  echo   onlinemerge     Run the merge tool
 
 Review comment:
   当一个大表会有上万的region 为这个表设置一个TTL 会有很多空region 我希望去合并空但是我不希望通过hbase  shell 一个个合并,希望有一个合并的任务去执行。
   
   When a large table has tens of thousands of region to set a TTL for this table, there will be many empty region. I want to merge the empty region, but I don't want to merge them one by one through HBase shell. I want to have a merge task to execute.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern
URL: https://github.com/apache/hbase/pull/1016#discussion_r368174853
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/util/OnlineMergeTool.java
 ##########
 @@ -0,0 +1,528 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
 
 Review comment:
   Fix year.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] mygood commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern

Posted by GitBox <gi...@apache.org>.
mygood commented on a change in pull request #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern
URL: https://github.com/apache/hbase/pull/1016#discussion_r368393575
 
 

 ##########
 File path: bin/hbase.cmd
 ##########
 @@ -436,4 +436,5 @@ goto :eof
   echo   mapredcp        Dump CLASSPATH entries required by mapreduce
   echo   version         Print the version
   echo   CLASSNAME       Run the class named CLASSNAME
+  echo   onlinemerge     Run the merge tool
 
 Review comment:
   当一个大表会有上万的region 为这个表设置一个TTL 会有很多空region 我希望去合并空但是我不希望通过hbase  shell 一个个合并,希望有一个合并的任务去执行。
   
   When a large table has tens of thousands of region to set a TTL for this table, there will be many empty region. I want to merge the empty region, but I don't want to merge them one by one through HBase shell. I want to have a merge task to execute.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern
URL: https://github.com/apache/hbase/pull/1016#issuecomment-576370260
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 38s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  shelldocs  |   0m  0s |  Shelldocs was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ branch-1.4 Compile Tests _ |
   | +0 :ok: |  mvndep  |   1m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   7m 11s |  branch-1.4 passed  |
   | +1 :green_heart: |  compile  |   1m 41s |  branch-1.4 passed with JDK v1.8.0_242  |
   | +1 :green_heart: |  compile  |   1m 46s |  branch-1.4 passed with JDK v1.7.0_252  |
   | +1 :green_heart: |  checkstyle  |   6m 46s |  branch-1.4 passed  |
   | +1 :green_heart: |  shadedjars  |   2m 47s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 37s |  branch-1.4 passed with JDK v1.8.0_242  |
   | +1 :green_heart: |  javadoc  |   3m 38s |  branch-1.4 passed with JDK v1.7.0_252  |
   | +0 :ok: |  spotbugs  |   2m 30s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |  12m 59s |  branch-1.4 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 18s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 57s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 43s |  the patch passed with JDK v1.8.0_242  |
   | +1 :green_heart: |  javac  |   1m 43s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 46s |  the patch passed with JDK v1.7.0_252  |
   | +1 :green_heart: |  javac  |   1m 46s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   6m 40s |  the patch passed  |
   | +1 :green_heart: |  shellcheck  |   0m  1s |  There were no new shellcheck issues.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   2m 43s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |   2m 34s |  Patch does not cause any errors with Hadoop 2.7.7.  |
   | +1 :green_heart: |  javadoc  |   2m 35s |  the patch passed with JDK v1.8.0_242  |
   | +1 :green_heart: |  javadoc  |   3m 37s |  the patch passed with JDK v1.7.0_252  |
   | +1 :green_heart: |  findbugs  |  13m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 173m 41s |  root in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m 14s |  The patch does not generate ASF License warnings.  |
   |  |   | 257m 45s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1016 |
   | Optional Tests | dupname asflicense shellcheck shelldocs javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 2c331e3ca2f0 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1016/out/precommit/personality/provided.sh |
   | git revision | branch-1.4 / 8cb278e |
   | Default Java | 1.7.0_252 |
   | Multi-JDK versions | /usr/lib/jvm/zulu-8-amd64:1.8.0_242 /usr/lib/jvm/zulu-7-amd64:1.7.0_252 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/4/testReport/ |
   | Max. process+thread count | 4241 (vs. ulimit of 10000) |
   | modules | C: hbase-server . U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/4/console |
   | versions | git=1.9.1 maven=3.0.5 shellcheck=0.7.0 findbugs=3.0.1 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern
URL: https://github.com/apache/hbase/pull/1016#issuecomment-573138221
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 17s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  shelldocs  |   0m  0s |  Shelldocs was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ branch-1.4 Compile Tests _ |
   | +0 :ok: |  mvndep  |   1m 30s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   9m  0s |  branch-1.4 passed  |
   | +1 :green_heart: |  compile  |   3m  4s |  branch-1.4 passed with JDK v1.8.0_232  |
   | +1 :green_heart: |  compile  |   2m 48s |  branch-1.4 passed with JDK v1.7.0_242  |
   | +1 :green_heart: |  checkstyle  |   8m 22s |  branch-1.4 passed  |
   | +1 :green_heart: |  shadedjars  |   2m 45s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 35s |  branch-1.4 passed with JDK v1.8.0_232  |
   | +1 :green_heart: |  javadoc  |   3m 37s |  branch-1.4 passed with JDK v1.7.0_242  |
   | +0 :ok: |  spotbugs  |   2m 29s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |  13m  5s |  branch-1.4 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 57s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 43s |  the patch passed with JDK v1.8.0_232  |
   | +1 :green_heart: |  javac  |   1m 43s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 46s |  the patch passed with JDK v1.7.0_242  |
   | +1 :green_heart: |  javac  |   1m 46s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   6m 39s |  the patch passed  |
   | +1 :green_heart: |  shellcheck  |   0m  2s |  There were no new shellcheck issues.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   2m 44s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |   2m 32s |  Patch does not cause any errors with Hadoop 2.7.7.  |
   | +1 :green_heart: |  javadoc  |   2m 34s |  the patch passed with JDK v1.8.0_232  |
   | +1 :green_heart: |  javadoc  |   3m 38s |  the patch passed with JDK v1.7.0_242  |
   | +1 :green_heart: |  findbugs  |  13m 21s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 208m 46s |  root in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m  9s |  The patch does not generate ASF License warnings.  |
   |  |   | 300m 47s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1016 |
   | Optional Tests | dupname asflicense shellcheck shelldocs javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 709cf1c4a6d2 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1016/out/precommit/personality/provided.sh |
   | git revision | branch-1.4 / 879b7ea |
   | Default Java | 1.7.0_242 |
   | Multi-JDK versions | /usr/lib/jvm/zulu-8-amd64:1.8.0_232 /usr/lib/jvm/zulu-7-amd64:1.7.0_242 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/1/testReport/ |
   | Max. process+thread count | 4130 (vs. ulimit of 10000) |
   | modules | C: hbase-server . U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/1/console |
   | versions | git=1.9.1 maven=3.0.5 shellcheck=0.7.0 findbugs=3.0.1 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1016: HBASE-23656 [MERGETOOL] HBASE Support Merge region by pattern
URL: https://github.com/apache/hbase/pull/1016#issuecomment-576267509
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 40s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  shelldocs  |   0m  0s |  Shelldocs was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ branch-1.4 Compile Tests _ |
   | +0 :ok: |  mvndep  |   1m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   7m  1s |  branch-1.4 passed  |
   | +1 :green_heart: |  compile  |   1m 44s |  branch-1.4 passed with JDK v1.8.0_242  |
   | +1 :green_heart: |  compile  |   1m 45s |  branch-1.4 passed with JDK v1.7.0_252  |
   | +1 :green_heart: |  checkstyle  |   6m 35s |  branch-1.4 passed  |
   | +1 :green_heart: |  shadedjars  |   2m 43s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 35s |  branch-1.4 passed with JDK v1.8.0_242  |
   | +1 :green_heart: |  javadoc  |   3m 35s |  branch-1.4 passed with JDK v1.7.0_252  |
   | +0 :ok: |  spotbugs  |   2m 32s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |  13m  2s |  branch-1.4 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 18s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   1m  5s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m 52s |  root in the patch failed with JDK v1.8.0_242.  |
   | -1 :x: |  javac  |   0m 52s |  root in the patch failed with JDK v1.8.0_242.  |
   | -1 :x: |  compile  |   0m 57s |  root in the patch failed with JDK v1.7.0_252.  |
   | -1 :x: |  javac  |   0m 57s |  root in the patch failed with JDK v1.7.0_252.  |
   | -1 :x: |  checkstyle  |   1m 30s |  hbase-server: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -1 :x: |  checkstyle  |   5m  4s |  root: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | +1 :green_heart: |  shellcheck  |   0m  2s |  There were no new shellcheck issues.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | -1 :x: |  shadedjars  |   1m 41s |  patch has 114 errors when building our shaded downstream artifacts.  |
   | -1 :x: |  hadoopcheck  |   1m 35s |  The patch causes 114 errors with Hadoop v2.7.7.  |
   | -1 :x: |  javadoc  |   0m 29s |  hbase-server-jdk1.8.0_242 with JDK v1.8.0_242 generated 10 new + 4 unchanged - 0 fixed = 14 total (was 4)  |
   | -1 :x: |  javadoc  |   2m  2s |  root-jdk1.8.0_242 with JDK v1.8.0_242 generated 10 new + 7 unchanged - 0 fixed = 17 total (was 7)  |
   | -1 :x: |  javadoc  |   0m 39s |  hbase-server-jdk1.7.0_252 with JDK v1.7.0_252 generated 10 new + 4 unchanged - 0 fixed = 14 total (was 4)  |
   | -1 :x: |  javadoc  |   2m 58s |  root-jdk1.7.0_252 with JDK v1.7.0_252 generated 10 new + 10 unchanged - 0 fixed = 20 total (was 10)  |
   | -1 :x: |  findbugs  |   0m 33s |  hbase-server in the patch failed.  |
   | -1 :x: |  findbugs  |   5m 19s |  root in the patch failed.  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  11m 14s |  root in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 35s |  The patch does not generate ASF License warnings.  |
   |  |   |  79m 41s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1016 |
   | Optional Tests | dupname asflicense shellcheck shelldocs javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux a07b2c81261b 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1016/out/precommit/personality/provided.sh |
   | git revision | branch-1.4 / 8cb278e |
   | Default Java | 1.7.0_252 |
   | Multi-JDK versions | /usr/lib/jvm/zulu-8-amd64:1.8.0_242 /usr/lib/jvm/zulu-7-amd64:1.7.0_252 |
   | mvninstall | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/patch-mvninstall-root.txt |
   | compile | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/patch-compile-root-jdk1.8.0_242.txt |
   | javac | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/patch-compile-root-jdk1.8.0_242.txt |
   | compile | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/patch-compile-root-jdk1.7.0_252.txt |
   | javac | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/patch-compile-root-jdk1.7.0_252.txt |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/diff-checkstyle-hbase-server.txt |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/diff-checkstyle-root.txt |
   | shadedjars | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/patch-shadedjars.txt |
   | hadoopcheck | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/patch-javac-2.7.7.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/diff-javadoc-javadoc-hbase-server-jdk1.8.0_242.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/diff-javadoc-javadoc-root-jdk1.8.0_242.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/diff-javadoc-javadoc-hbase-server-jdk1.7.0_252.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/diff-javadoc-javadoc-root-jdk1.7.0_252.txt |
   | findbugs | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/patch-findbugs-hbase-server.txt |
   | findbugs | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/patch-findbugs-root.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/artifact/out/patch-unit-root.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/testReport/ |
   | Max. process+thread count | 230 (vs. ulimit of 10000) |
   | modules | C: hbase-server . U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1016/3/console |
   | versions | git=1.9.1 maven=3.0.5 shellcheck=0.7.0 findbugs=3.0.1 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services