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 java.io.IOException;
021import java.util.concurrent.CompletableFuture;
022import org.apache.hadoop.conf.Configuration;
023import org.apache.hadoop.fs.FileSystem;
024import org.apache.hadoop.fs.Path;
025import org.apache.hadoop.hbase.HBaseClassTestRule;
026import org.apache.hadoop.hbase.HConstants;
027import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor;
028import org.apache.hadoop.hbase.regionserver.RegionServerServices;
029import org.apache.hadoop.hbase.testclassification.SmallTests;
030import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
031import org.junit.AfterClass;
032import org.junit.BeforeClass;
033import org.junit.ClassRule;
034import org.junit.experimental.categories.Category;
035
036import org.apache.hbase.thirdparty.io.netty.channel.Channel;
037import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
038import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
039import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
040
041@Category({ RegionServerServices.class, SmallTests.class })
042public class TestAsyncFSWALDurability extends WALDurabilityTestBase<CustomAsyncFSWAL> {
043
044  @ClassRule
045  public static final HBaseClassTestRule CLASS_RULE =
046    HBaseClassTestRule.forClass(TestAsyncFSWALDurability.class);
047
048  private static NioEventLoopGroup GROUP;
049
050  @BeforeClass
051  public static void setUpBeforeClass() {
052    GROUP = new NioEventLoopGroup();
053  }
054
055  @AfterClass
056  public static void tearDownAfterClass() throws Exception {
057    GROUP.shutdownGracefully().get();
058  }
059
060  @Override
061  protected CustomAsyncFSWAL getWAL(FileSystem fs, Path root, String logDir, Configuration conf)
062    throws IOException {
063    CustomAsyncFSWAL wal =
064      new CustomAsyncFSWAL(fs, root, logDir, conf, GROUP, NioSocketChannel.class);
065    wal.init();
066    return wal;
067  }
068
069  @Override
070  protected void resetSyncFlag(CustomAsyncFSWAL wal) {
071    wal.resetSyncFlag();
072  }
073
074  @Override
075  protected Boolean getSyncFlag(CustomAsyncFSWAL wal) {
076    return wal.getSyncFlag();
077  }
078
079  @Override
080  protected Boolean getWriterSyncFlag(CustomAsyncFSWAL wal) {
081    return wal.getWriterSyncFlag();
082  }
083}
084
085class CustomAsyncFSWAL extends AsyncFSWAL {
086
087  private Boolean syncFlag;
088
089  private Boolean writerSyncFlag;
090
091  public CustomAsyncFSWAL(FileSystem fs, Path rootDir, String logDir, Configuration conf,
092    EventLoopGroup eventLoopGroup, Class<? extends Channel> channelClass)
093    throws FailedLogCloseException, IOException {
094    super(fs, null, rootDir, logDir, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null,
095      null, null, null, eventLoopGroup, channelClass,
096      StreamSlowMonitor.create(conf, "monitorForSuffix"));
097  }
098
099  @Override
100  protected AsyncWriter createWriterInstance(FileSystem fs, Path path) throws IOException {
101    AsyncWriter writer = super.createWriterInstance(fs, path);
102    return new AsyncWriter() {
103
104      @Override
105      public void close() throws IOException {
106        writer.close();
107      }
108
109      @Override
110      public long getLength() {
111        return writer.getLength();
112      }
113
114      @Override
115      public long getSyncedLength() {
116        return writer.getSyncedLength();
117      }
118
119      @Override
120      public CompletableFuture<Long> sync(boolean forceSync) {
121        writerSyncFlag = forceSync;
122        return writer.sync(forceSync);
123      }
124
125      @Override
126      public void append(Entry entry) {
127        writer.append(entry);
128      }
129    };
130  }
131
132  @Override
133  protected void doSync(boolean forceSync) throws IOException {
134    syncFlag = forceSync;
135    super.doSync(forceSync);
136  }
137
138  @Override
139  protected void doSync(long txid, boolean forceSync) throws IOException {
140    syncFlag = forceSync;
141    super.doSync(txid, forceSync);
142  }
143
144  void resetSyncFlag() {
145    this.syncFlag = null;
146    this.writerSyncFlag = null;
147  }
148
149  Boolean getSyncFlag() {
150    return syncFlag;
151  }
152
153  Boolean getWriterSyncFlag() {
154    return writerSyncFlag;
155  }
156}