You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by wc...@apache.org on 2021/03/10 10:38:53 UTC

[hbase] branch branch-2 updated: HBASE-25548 Optionally allow snapshots to preserve cluster's max file… (#2923)

This is an automated email from the ASF dual-hosted git repository.

wchevreuil pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 7adc000  HBASE-25548 Optionally allow snapshots to preserve cluster's max file… (#2923)
7adc000 is described below

commit 7adc00044c66fbdd69b528194d34303c0518ce83
Author: Wellington Ramos Chevreuil <wc...@apache.org>
AuthorDate: Wed Mar 10 10:20:10 2021 +0000

    HBASE-25548 Optionally allow snapshots to preserve cluster's max file… (#2923)
    
    Signed-off-by: Peter Somogyi <ps...@apache.org>
    (cherry picked from commit 373dc7788d66ba7f7c49d727373c521c639d613c)
---
 .../hadoop/hbase/client/SnapshotDescription.java   |  14 ++-
 .../hadoop/hbase/shaded/protobuf/ProtobufUtil.java |   4 +
 .../src/main/protobuf/Snapshot.proto               |   1 +
 .../hbase/master/snapshot/SnapshotManager.java     |   4 +
 .../hbase/master/snapshot/TakeSnapshotHandler.java |   9 +-
 .../master/snapshot/TestTakeSnapshotHandler.java   | 111 +++++++++++++++++++++
 hbase-shell/src/main/ruby/hbase/admin.rb           |   3 +
 .../src/main/ruby/shell/commands/snapshot.rb       |   2 +-
 src/main/asciidoc/_chapters/ops_mgt.adoc           |  15 +++
 9 files changed, 157 insertions(+), 6 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotDescription.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotDescription.java
index 45b9fdf..154b994 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotDescription.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotDescription.java
@@ -38,6 +38,8 @@ public class SnapshotDescription {
   private final long ttl;
   private final int version;
 
+  private final long maxFileSize;
+
   public SnapshotDescription(String name) {
     this(name, (TableName) null);
   }
@@ -135,14 +137,17 @@ public class SnapshotDescription {
     this.snapShotType = type;
     this.owner = owner;
     this.creationTime = creationTime;
-    this.ttl = getTtlFromSnapshotProps(snapshotProps);
+    this.ttl = getLongFromSnapshotProps(snapshotProps, "TTL");
     this.version = version;
+    this.maxFileSize = getLongFromSnapshotProps(snapshotProps, TableDescriptorBuilder.MAX_FILESIZE);
   }
 
-  private long getTtlFromSnapshotProps(Map<String, Object> snapshotProps) {
-    return MapUtils.getLongValue(snapshotProps, "TTL", -1);
+  private long getLongFromSnapshotProps(Map<String, Object> snapshotProps, String property) {
+    return MapUtils.getLongValue(snapshotProps, property, -1);
   }
 
+
+
   /**
    * SnapshotDescription Parameterized Constructor
    *
@@ -201,6 +206,8 @@ public class SnapshotDescription {
     return this.version;
   }
 
+  public long getMaxFileSize() { return maxFileSize; }
+
   @Override
   public String toString() {
     return new ToStringBuilder(this)
@@ -211,6 +218,7 @@ public class SnapshotDescription {
       .append("creationTime", creationTime)
       .append("ttl", ttl)
       .append("version", version)
+      .append("maxFileSize", maxFileSize)
       .toString();
   }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 274a344..9064ded 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -3038,6 +3038,9 @@ public final class ProtobufUtil {
     if (snapshotDesc.getVersion() != -1) {
       builder.setVersion(snapshotDesc.getVersion());
     }
+    if (snapshotDesc.getMaxFileSize() != -1) {
+      builder.setMaxFileSize(snapshotDesc.getMaxFileSize());
+    }
     builder.setType(ProtobufUtil.createProtosSnapShotDescType(snapshotDesc.getType()));
     return builder.build();
   }
@@ -3053,6 +3056,7 @@ public final class ProtobufUtil {
       createSnapshotDesc(SnapshotProtos.SnapshotDescription snapshotDesc) {
     final Map<String, Object> snapshotProps = new HashMap<>();
     snapshotProps.put("TTL", snapshotDesc.getTtl());
+    snapshotProps.put(TableDescriptorBuilder.MAX_FILESIZE, snapshotDesc.getMaxFileSize());
     return new SnapshotDescription(snapshotDesc.getName(),
             snapshotDesc.hasTable() ? TableName.valueOf(snapshotDesc.getTable()) : null,
             createSnapshotType(snapshotDesc.getType()), snapshotDesc.getOwner(),
diff --git a/hbase-protocol-shaded/src/main/protobuf/Snapshot.proto b/hbase-protocol-shaded/src/main/protobuf/Snapshot.proto
index 5faa7ab..60ed2b4 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Snapshot.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Snapshot.proto
@@ -45,6 +45,7 @@ message SnapshotDescription {
   optional string owner = 6;
   optional UsersAndPermissions users_and_permissions = 7;
   optional int64 ttl = 8 [default = 0];
+  optional int64 max_file_size = 9 [default = 0];
 }
 
 message SnapshotFileInfo {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
index 83025d9..3e8741f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
@@ -150,6 +150,10 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
   /** number of current operations running on the master */
   public static final int SNAPSHOT_POOL_THREADS_DEFAULT = 1;
 
+  /** Conf key for preserving original max file size configs */
+  public static final String SNAPSHOT_MAX_FILE_SIZE_PRESERVE =
+    "hbase.snapshot.max.filesize.preserve";
+
   private boolean stopped;
   private MasterServices master;  // Needed by TableEventHandlers
   private ProcedureCoordinator coordinator;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
index 7c9496b..5ff8a49 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.master.snapshot;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.HashSet;
 import java.util.List;
@@ -31,6 +30,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
@@ -139,12 +139,17 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
   }
 
   private TableDescriptor loadTableDescriptor()
-      throws FileNotFoundException, IOException {
+      throws IOException {
     TableDescriptor htd =
       this.master.getTableDescriptors().get(snapshotTable);
     if (htd == null) {
       throw new IOException("TableDescriptor missing for " + snapshotTable);
     }
+    if (htd.getMaxFileSize()==-1 &&
+        this.snapshot.getMaxFileSize()>0) {
+      htd = TableDescriptorBuilder.newBuilder(htd).setValue(TableDescriptorBuilder.MAX_FILESIZE,
+        Long.toString(this.snapshot.getMaxFileSize())).build();
+    }
     return htd;
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestTakeSnapshotHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestTakeSnapshotHandler.java
new file mode 100644
index 0000000..9000254
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestTakeSnapshotHandler.java
@@ -0,0 +1,111 @@
+/**
+ * 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.master.snapshot;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+
+/**
+ * Unfortunately, couldn't test TakeSnapshotHandler using mocks, because it relies on TableLock,
+ * which is tightly coupled to LockManager and LockProcedure classes, which are both final and
+ * prevents us from mocking its behaviour. Looks like an overkill having to emulate a
+ * whole cluster run for such a small optional property behaviour.
+ */
+@Category({ MediumTests.class})
+public class TestTakeSnapshotHandler {
+
+  private static HBaseTestingUtility UTIL;
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestTakeSnapshotHandler.class);
+
+  @Rule
+  public TestName name = new TestName();
+
+
+  @Before
+  public void setup()  {
+    UTIL = new HBaseTestingUtility();
+  }
+
+  public TableDescriptor createTableInsertDataAndTakeSnapshot(Map<String, Object> snapshotProps)
+      throws Exception {
+    TableDescriptor descriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setColumnFamily(
+          ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("f")).build()).build();
+    UTIL.getConnection().getAdmin().createTable(descriptor);
+    Table table = UTIL.getConnection().getTable(descriptor.getTableName());
+    Put put = new Put(Bytes.toBytes("1"));
+    put.addColumn(Bytes.toBytes("f"), Bytes.toBytes("1"), Bytes.toBytes("v1"));
+    table.put(put);
+    String snapName = "snap"+name.getMethodName();
+    UTIL.getAdmin().snapshot(snapName, descriptor.getTableName(), snapshotProps);
+    TableName cloned = TableName.valueOf(name.getMethodName() + "clone");
+    UTIL.getAdmin().cloneSnapshot(snapName, cloned);
+    return descriptor;
+  }
+
+  @Test
+  public void testPreparePreserveMaxFileSizeEnabled() throws Exception {
+    UTIL.startMiniCluster();
+    Map<String, Object> snapshotProps = new HashMap<>();
+    snapshotProps.put(TableDescriptorBuilder.MAX_FILESIZE, Long.parseLong("21474836480"));
+    TableDescriptor descriptor = createTableInsertDataAndTakeSnapshot(snapshotProps);
+    TableName cloned = TableName.valueOf(name.getMethodName() + "clone");
+    assertEquals(-1,
+      UTIL.getAdmin().getDescriptor(descriptor.getTableName()).getMaxFileSize());
+    assertEquals(21474836480L, UTIL.getAdmin().getDescriptor(cloned).getMaxFileSize());
+  }
+
+  @Test
+  public void testPreparePreserveMaxFileSizeDisabled() throws Exception {
+    UTIL.startMiniCluster();
+    TableDescriptor descriptor = createTableInsertDataAndTakeSnapshot(null);
+    TableName cloned = TableName.valueOf(name.getMethodName() + "clone");
+    assertEquals(-1,
+      UTIL.getAdmin().getDescriptor(descriptor.getTableName()).getMaxFileSize());
+    assertEquals(-1, UTIL.getAdmin().getDescriptor(cloned).getMaxFileSize());
+  }
+
+  @After
+  public void shutdown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+}
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index bf07612..e9a732c 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -1150,6 +1150,9 @@ module Hbase
           ttl = ttl ? ttl.to_java(:long) : -1
           snapshot_props = java.util.HashMap.new
           snapshot_props.put("TTL", ttl)
+          max_filesize = arg[MAX_FILESIZE]
+          max_filesize = max_filesize ? max_filesize.to_java(:long) : -1
+          snapshot_props.put("MAX_FILESIZE", max_filesize)
           if arg[SKIP_FLUSH] == true
             @admin.snapshot(snapshot_name, table_name,
                             org.apache.hadoop.hbase.client.SnapshotType::SKIPFLUSH, snapshot_props)
diff --git a/hbase-shell/src/main/ruby/shell/commands/snapshot.rb b/hbase-shell/src/main/ruby/shell/commands/snapshot.rb
index c591e12..9984494 100644
--- a/hbase-shell/src/main/ruby/shell/commands/snapshot.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/snapshot.rb
@@ -24,7 +24,7 @@ module Shell
 Take a snapshot of specified table. Examples:
 
   hbase> snapshot 'sourceTable', 'snapshotName'
-  hbase> snapshot 'namespace:sourceTable', 'snapshotName', {SKIP_FLUSH => true}
+  hbase> snapshot 'namespace:sourceTable', 'snapshotName', {SKIP_FLUSH => true, MAX_FILESIZE => 21474836480}
 EOF
       end
 
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index a75665c..dab2ed2 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -3167,6 +3167,21 @@ providing default TTL in sec for key: `hbase.master.snapshot.ttl`.
 Value 0 for this config indicates TTL: FOREVER
 
 
+.Take a snapshot with custom MAX_FILESIZE
+
+Optionally, snapshots can be created with a custom max file size configuration that will be
+used by cloned tables, instead of the global `hbase.hregion.max.filesize` configuration property.
+This is mostly useful when exporting snapshots between different clusters. If the HBase cluster where
+the snapshot is originally taken has a much larger value set for `hbase.hregion.max.filesize` than
+one or more clusters where the snapshot is being exported to, a storm of region splits may occur when
+restoring the snapshot on destination clusters. Specifying `MAX_FILESIZE` on properties passed to
+`snapshot` command will save informed value into the table's `MAX_FILESIZE`
+decriptor at snapshot creation time. If the table already defines `MAX_FILESIZE` descriptor,
+this property would be ignored and have no effect.
+
+----
+snapshot 'table01', 'snap01', {MAX_FILESIZE => 21474836480}
+----
 
 .Enable/Disable Snapshot Auto Cleanup on running cluster: