001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.regionserver.wal; 019 020import static org.junit.jupiter.api.Assertions.assertEquals; 021 022import java.io.IOException; 023import org.apache.hadoop.fs.FileStatus; 024import org.apache.hadoop.fs.FileSystem; 025import org.apache.hadoop.fs.Path; 026import org.apache.hadoop.hbase.HBaseTestingUtil; 027import org.apache.hadoop.hbase.TableName; 028import org.apache.hadoop.hbase.util.Bytes; 029import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 030import org.apache.hadoop.hbase.wal.WALFactory; 031import org.apache.hadoop.hbase.wal.WALProvider; 032import org.junit.jupiter.api.AfterAll; 033import org.junit.jupiter.api.AfterEach; 034import org.junit.jupiter.api.BeforeAll; 035import org.junit.jupiter.api.BeforeEach; 036import org.junit.jupiter.api.Test; 037import org.junit.jupiter.api.TestInfo; 038 039import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer; 040 041/** 042 * WAL tests that can be reused across providers. 043 */ 044public abstract class AbstractTestProtobufLog { 045 protected static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); 046 047 protected FileSystem fs; 048 protected Path dir; 049 protected WALFactory wals; 050 051 private String currentTest; 052 053 @BeforeEach 054 public void setUp(TestInfo testInfo) throws Exception { 055 currentTest = testInfo.getTestMethod().get().getName(); 056 fs = TEST_UTIL.getDFSCluster().getFileSystem(); 057 dir = new Path(TEST_UTIL.createRootDir(), currentTest); 058 wals = new WALFactory(TEST_UTIL.getConfiguration(), currentTest); 059 } 060 061 @AfterEach 062 public void tearDown() throws Exception { 063 wals.close(); 064 FileStatus[] entries = fs.listStatus(new Path("/")); 065 for (FileStatus dir : entries) { 066 fs.delete(dir.getPath(), true); 067 } 068 } 069 070 @BeforeAll 071 public static void setUpBeforeClass() throws Exception { 072 // Make block sizes small. 073 TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024); 074 // needed for testAppendClose() 075 // quicker heartbeat interval for faster DN death notification 076 TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000); 077 TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1); 078 TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000); 079 080 // faster failover with cluster.shutdown();fs.close() idiom 081 TEST_UTIL.getConfiguration().setInt("dfs.client.block.recovery.retries", 1); 082 TEST_UTIL.startMiniDFSCluster(3); 083 } 084 085 @AfterAll 086 public static void tearDownAfterClass() throws Exception { 087 TEST_UTIL.shutdownMiniCluster(); 088 } 089 090 /** 091 * Reads the WAL with and without WALTrailer. 092 */ 093 @Test 094 public void testWALTrailer() throws IOException { 095 // make sure that the size for WALTrailer is 0, we need this assumption when reading partial 096 // WALTrailer 097 assertEquals(0, WALTrailer.newBuilder().build().getSerializedSize()); 098 // read With trailer. 099 doRead(true); 100 // read without trailer 101 doRead(false); 102 } 103 104 /** 105 * Appends entries in the WAL and reads it. 106 * @param withTrailer If 'withTrailer' is true, it calls a close on the WALwriter before reading 107 * so that a trailer is appended to the WAL. Otherwise, it starts reading after 108 * the sync call. This means that reader is not aware of the trailer. In this 109 * scenario, if the reader tries to read the trailer in its next() call, it 110 * returns false from ProtoBufLogReader. 111 */ 112 private void doRead(boolean withTrailer) throws IOException { 113 int columnCount = 5; 114 int recordCount = 5; 115 TableName tableName = TableName.valueOf("tablename"); 116 byte[] row = Bytes.toBytes("row"); 117 long timestamp = EnvironmentEdgeManager.currentTime(); 118 Path path = new Path(dir, "tempwal"); 119 // delete the log if already exists, for test only 120 fs.delete(path, true); 121 fs.mkdirs(dir); 122 try (WALProvider.Writer writer = createWriter(path)) { 123 ProtobufLogTestHelper.doWrite(writer, withTrailer, tableName, columnCount, recordCount, row, 124 timestamp); 125 try (ProtobufWALStreamReader reader = 126 (ProtobufWALStreamReader) wals.createStreamReader(fs, path)) { 127 ProtobufLogTestHelper.doRead(reader, withTrailer, tableName, columnCount, recordCount, row, 128 timestamp); 129 } 130 } 131 } 132 133 protected abstract WALProvider.Writer createWriter(Path path) throws IOException; 134}