You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2019/02/06 10:29:41 UTC

[hbase] branch master updated: HBASE-21838 Create a special ReplicationEndpoint just for verifying the WAL entries are fine

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

zhangduo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new 6b60c42  HBASE-21838 Create a special ReplicationEndpoint just for verifying the WAL entries are fine
6b60c42 is described below

commit 6b60c426900bd71b3f4582f3485d1dc6fa724133
Author: zhangduo <zh...@apache.org>
AuthorDate: Wed Feb 6 14:02:52 2019 +0800

    HBASE-21838 Create a special ReplicationEndpoint just for verifying the WAL entries are fine
---
 .../VerifyWALEntriesReplicationEndpoint.java       |  87 ++++++++++++++
 .../TestVerifyCellsReplicationEndpoint.java        | 130 +++++++++++++++++++++
 2 files changed, 217 insertions(+)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/VerifyWALEntriesReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/VerifyWALEntriesReplicationEndpoint.java
new file mode 100644
index 0000000..088827f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/VerifyWALEntriesReplicationEndpoint.java
@@ -0,0 +1,87 @@
+/**
+ * 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.replication;
+
+import java.util.UUID;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A dummy {@link ReplicationEndpoint} that replicates nothing.
+ * <p/>
+ * Mainly used by ITBLL to check whether all the entries in WAL files are fine, since for normal
+ * case, we will only read the WAL files when there are region servers crash and we need to split
+ * the log, but for replication we will read all the entries and pass them to the
+ * {@link ReplicationEndpoint}, so setting up a replication peer can help finding out whether there
+ * are broken entries in WAL files.
+ */
+@InterfaceAudience.Private
+public class VerifyWALEntriesReplicationEndpoint extends BaseReplicationEndpoint {
+
+  @Override
+  public boolean canReplicateToSameCluster() {
+    return true;
+  }
+
+  @Override
+  public UUID getPeerUUID() {
+    return ctx.getClusterId();
+  }
+
+  @Override
+  public WALEntryFilter getWALEntryfilter() {
+    return null;
+  }
+
+  private void checkCell(Cell cell) {
+    // check whether all the fields are fine
+    CellUtil.cloneRow(cell);
+    CellUtil.cloneFamily(cell);
+    CellUtil.cloneQualifier(cell);
+    CellUtil.cloneValue(cell);
+  }
+
+  @Override
+  public boolean replicate(ReplicateContext replicateContext) {
+    replicateContext.entries.stream().map(WAL.Entry::getEdit).flatMap(e -> e.getCells().stream())
+      .forEach(this::checkCell);
+    return true;
+  }
+
+  @Override
+  public void start() {
+    startAsync();
+  }
+
+  @Override
+  public void stop() {
+    stopAsync();
+  }
+
+  @Override
+  protected void doStart() {
+    notifyStarted();
+  }
+
+  @Override
+  protected void doStop() {
+    notifyStopped();
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyCellsReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyCellsReplicationEndpoint.java
new file mode 100644
index 0000000..f1b1004
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyCellsReplicationEndpoint.java
@@ -0,0 +1,130 @@
+/**
+ * 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.replication;
+
+import static org.junit.Assert.assertArrayEquals;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Confirm that the empty replication endpoint can work.
+ */
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestVerifyCellsReplicationEndpoint {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestVerifyCellsReplicationEndpoint.class);
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(TestVerifyCellsReplicationEndpoint.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static final TableName TABLE_NAME = TableName.valueOf("empty");
+
+  private static final byte[] CF = Bytes.toBytes("family");
+
+  private static final byte[] CQ = Bytes.toBytes("qualifier");
+
+  private static final String PEER_ID = "empty";
+
+  private static final BlockingQueue<Cell> CELLS = new LinkedBlockingQueue<>();
+
+  public static final class EndpointForTest extends VerifyWALEntriesReplicationEndpoint {
+
+    @Override
+    public boolean replicate(ReplicateContext replicateContext) {
+      LOG.info(replicateContext.getEntries().toString());
+      replicateContext.entries.stream().map(WAL.Entry::getEdit).map(WALEdit::getCells)
+        .forEachOrdered(CELLS::addAll);
+      return super.replicate(replicateContext);
+    }
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniCluster(3);
+    // notice that we do not need to set replication scope here, EmptyReplicationEndpoint take all
+    // edits no matter what the replications scope is.
+    UTIL.createTable(TABLE_NAME, CF);
+    UTIL.getAdmin().addReplicationPeer(PEER_ID,
+      ReplicationPeerConfig.newBuilder().setClusterKey("zk1:8888:/hbase")
+        .setReplicationEndpointImpl(EndpointForTest.class.getName()).build());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws Exception {
+    try (Table table = UTIL.getConnection().getTable(TABLE_NAME)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    long lastNoCellTime = -1;
+    for (int i = 0; i < 100;) {
+      Cell cell = CELLS.poll();
+      if (cell == null) {
+        if (lastNoCellTime < 0) {
+          lastNoCellTime = System.nanoTime();
+        } else {
+          if (System.nanoTime() - lastNoCellTime >= TimeUnit.SECONDS.toNanos(30)) {
+            throw new TimeoutException("Timeout waiting for wal edit");
+          }
+        }
+        Thread.sleep(1000);
+        continue;
+      }
+      lastNoCellTime = -1;
+      if (!Bytes.equals(CF, CellUtil.cloneFamily(cell))) {
+        // meta edits, such as open/close/flush, etc. skip
+        continue;
+      }
+      assertArrayEquals(Bytes.toBytes(i), CellUtil.cloneRow(cell));
+      assertArrayEquals(CQ, CellUtil.cloneQualifier(cell));
+      assertArrayEquals(Bytes.toBytes(i), CellUtil.cloneValue(cell));
+      i++;
+    }
+  }
+}