diff --git hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithEncryption.java hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithEncryption.java index 588b0de..7163a6e 100644 --- hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithEncryption.java +++ hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithEncryption.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileReaderV3; import org.apache.hadoop.hbase.io.hfile.HFileWriterV3; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WAL; import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader; import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter; import org.apache.hadoop.hbase.util.Bytes; @@ -61,9 +61,9 @@ public class IntegrationTestIngestWithEncryption extends IntegrationTestIngest { conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName()); conf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase"); conf.setClass("hbase.regionserver.hlog.reader.impl", SecureProtobufLogReader.class, - HLog.Reader.class); + WAL.Reader.class); conf.setClass("hbase.regionserver.hlog.writer.impl", SecureProtobufLogWriter.class, - HLog.Writer.class); + WAL.Writer.class); conf.setBoolean(HConstants.ENABLE_WAL_ENCRYPTION, true); } super.setUpCluster(); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java index 5a99f9c..9e55c73 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java @@ -23,11 +23,11 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) @InterfaceStability.Evolving public interface WALCoprocessorEnvironment extends CoprocessorEnvironment { /** @return reference to the region server services */ - HLog getWAL(); + WALService getWAL(); } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java index 55377ce..0305668 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java @@ -37,7 +37,7 @@ import java.io.IOException; * hooks for adding logic for WALEdits in the region context during reconstruction, * * Defines coprocessor hooks for interacting with operations on the - * {@link org.apache.hadoop.hbase.regionserver.wal.HLog}. + * {@link org.apache.hadoop.hbase.regionserver.wal.WALService}. */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) @InterfaceStability.Evolving diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HLogInputFormat.java hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HLogInputFormat.java index 7fa852e..c191890 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HLogInputFormat.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HLogInputFormat.java @@ -32,7 +32,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WAL; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; @@ -44,7 +44,7 @@ import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; /** - * Simple {@link InputFormat} for {@link HLog} files. + * Simple {@link InputFormat} for {@link WAL} files. */ @InterfaceAudience.Public public class HLogInputFormat extends InputFormat { @@ -54,7 +54,7 @@ public class HLogInputFormat extends InputFormat { public static final String END_TIME_KEY = "hlog.end.time"; /** - * {@link InputSplit} for {@link HLog} files. Each split represent + * {@link InputSplit} for {@link WAL} files. Each split represent * exactly one log file. */ static class HLogSplit extends InputSplit implements Writable { @@ -128,11 +128,11 @@ public class HLogInputFormat extends InputFormat { } /** - * {@link RecordReader} for an {@link HLog} file. + * {@link RecordReader} for an {@link WAL} file. */ static class HLogRecordReader extends RecordReader { - private HLog.Reader reader = null; - private HLog.Entry currentEntry = new HLog.Entry(); + private WAL.Reader reader = null; + private WAL.Entry currentEntry = new WAL.Entry(); private long startTime; private long endTime; @@ -158,7 +158,7 @@ public class HLogInputFormat extends InputFormat { public boolean nextKeyValue() throws IOException, InterruptedException { if (reader == null) return false; - HLog.Entry temp; + WAL.Entry temp; long i = -1; do { // skip older entries diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java index f18e08a..4ae313d 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java @@ -51,9 +51,9 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.regionserver.wal.WAL; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSTableDescriptors; @@ -215,7 +215,7 @@ public class MasterFileSystem { */ Set getFailedServersFromLogFolders() { boolean retrySplitting = !conf.getBoolean("hbase.hlog.split.skip.errors", - HLog.SPLIT_SKIP_ERRORS_DEFAULT); + WAL.SPLIT_SKIP_ERRORS_DEFAULT); Set serverNames = new HashSet(); Path logsDirPath = new Path(this.rootdir, HConstants.HREGION_LOGDIR_NAME); @@ -240,8 +240,8 @@ public class MasterFileSystem { for (FileStatus status : logFolders) { String sn = status.getPath().getName(); // truncate splitting suffix if present (for ServerName parsing) - if (sn.endsWith(HLog.SPLITTING_EXT)) { - sn = sn.substring(0, sn.length() - HLog.SPLITTING_EXT.length()); + if (sn.endsWith(WAL.SPLITTING_EXT)) { + sn = sn.substring(0, sn.length() - WAL.SPLITTING_EXT.length()); } ServerName serverName = ServerName.parseServerName(sn); if (!onlineServers.contains(serverName)) { @@ -312,7 +312,7 @@ public class MasterFileSystem { try { for (ServerName serverName : serverNames) { Path logDir = new Path(this.rootdir, HLogUtil.getHLogDirectoryName(serverName.toString())); - Path splitDir = logDir.suffix(HLog.SPLITTING_EXT); + Path splitDir = logDir.suffix(WAL.SPLITTING_EXT); // Rename the directory so a rogue RS doesn't create more HLogs if (fs.exists(logDir)) { if (!this.fs.rename(logDir, splitDir)) { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java index 616bada..1468988 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java @@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; import org.apache.hadoop.hbase.security.access.AccessControlLists; @@ -380,7 +380,7 @@ public class NamespaceUpgrade implements Tool { ServerName fakeServer = ServerName.valueOf("nsupgrade", 96, 123); String metaLogName = HLogUtil.getHLogDirectoryName(fakeServer.toString()); - HLog metaHLog = HLogFactory.createMetaHLog(fs, rootDir, + WALService metaHLog = HLogFactory.createMetaHLog(fs, rootDir, metaLogName, conf, null, fakeServer.toString()); HRegion meta = HRegion.openHRegion(rootDir, HRegionInfo.FIRST_META_REGIONINFO, diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java index 7a82ed9..2dc523f 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java @@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.WALProtos; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WAL; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Pair; @@ -57,7 +57,7 @@ public class ReplicationProtbufUtil { * @throws java.io.IOException */ public static void replicateWALEntry(final AdminService.BlockingInterface admin, - final HLog.Entry[] entries) throws IOException { + final WAL.Entry[] entries) throws IOException { Pair p = buildReplicateWALEntryRequest(entries); try { @@ -76,7 +76,7 @@ public class ReplicationProtbufUtil { * found. */ public static Pair - buildReplicateWALEntryRequest(final HLog.Entry[] entries) { + buildReplicateWALEntryRequest(final WAL.Entry[] entries) { // Accumulate all the KVs seen in here. List> allkvs = new ArrayList>(entries.length); int size = 0; @@ -85,7 +85,7 @@ public class ReplicationProtbufUtil { AdminProtos.ReplicateWALEntryRequest.Builder builder = AdminProtos.ReplicateWALEntryRequest.newBuilder(); HBaseProtos.UUID.Builder uuidBuilder = HBaseProtos.UUID.newBuilder(); - for (HLog.Entry entry: entries) { + for (WAL.Entry entry: entries) { entryBuilder.clear(); // TODO: this duplicates a lot in HLogKey#getBuilder WALProtos.WALKey.Builder keyBuilder = entryBuilder.getKeyBuilder(); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index fe981e5..89a2a36 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -121,7 +121,8 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescriptio import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor; import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl.WriteEntry; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WAL; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; @@ -268,7 +269,7 @@ public class HRegion implements HeapSize { // , Writable{ final AtomicLong compactionNumBytesCompacted = new AtomicLong(0L); - private final HLog log; + private final WALService log; private final HRegionFileSystem fs; protected final Configuration conf; private final Configuration baseConf; @@ -448,7 +449,7 @@ public class HRegion implements HeapSize { // , Writable{ * @param rsServices reference to {@link RegionServerServices} or null */ @Deprecated - public HRegion(final Path tableDir, final HLog log, final FileSystem fs, + public HRegion(final Path tableDir, final WALService log, final FileSystem fs, final Configuration confParam, final HRegionInfo regionInfo, final HTableDescriptor htd, final RegionServerServices rsServices) { this(new HRegionFileSystem(confParam, fs, tableDir, regionInfo), @@ -472,7 +473,7 @@ public class HRegion implements HeapSize { // , Writable{ * @param htd the table descriptor * @param rsServices reference to {@link RegionServerServices} or null */ - public HRegion(final HRegionFileSystem fs, final HLog log, final Configuration confParam, + public HRegion(final HRegionFileSystem fs, final WALService log, final Configuration confParam, final HTableDescriptor htd, final RegionServerServices rsServices) { if (htd == null) { throw new IllegalArgumentException("Need table descriptor"); @@ -1218,7 +1219,7 @@ public class HRegion implements HeapSize { // , Writable{ } /** @return HLog in use for this region */ - public HLog getLog() { + public WALService getLog() { return this.log; } @@ -1566,7 +1567,7 @@ public class HRegion implements HeapSize { // , Writable{ * @see #internalFlushcache(MonitoredTask) */ protected boolean internalFlushcache( - final HLog wal, final long myseqid, MonitoredTask status) + final WALService wal, final long myseqid, MonitoredTask status) throws IOException { if (this.rsServices != null && this.rsServices.isAborted()) { // Don't flush when server aborting, it's unsafe @@ -3100,7 +3101,7 @@ public class HRegion implements HeapSize { // , Writable{ FileSystem fs = this.fs.getFileSystem(); status.setStatus("Opening logs"); - HLog.Reader reader = null; + WAL.Reader reader = null; try { reader = HLogFactory.createReader(fs, edits, conf); long currentEditSeqId = -1; @@ -3108,7 +3109,7 @@ public class HRegion implements HeapSize { // , Writable{ long skippedEdits = 0; long editsCount = 0; long intervalEdits = 0; - HLog.Entry entry; + WAL.Entry entry; Store store = null; boolean reported_once = false; ServerNonceManager ng = this.rsServices == null ? null : this.rsServices.getNonceManager(); @@ -4020,7 +4021,7 @@ public class HRegion implements HeapSize { // , Writable{ * @param rsServices * @return the new instance */ - static HRegion newHRegion(Path tableDir, HLog log, FileSystem fs, + static HRegion newHRegion(Path tableDir, WALService log, FileSystem fs, Configuration conf, HRegionInfo regionInfo, final HTableDescriptor htd, RegionServerServices rsServices) { try { @@ -4029,7 +4030,7 @@ public class HRegion implements HeapSize { // , Writable{ (Class) conf.getClass(HConstants.REGION_IMPL, HRegion.class); Constructor c = - regionClass.getConstructor(Path.class, HLog.class, FileSystem.class, + regionClass.getConstructor(Path.class, WALService.class, FileSystem.class, Configuration.class, HRegionInfo.class, HTableDescriptor.class, RegionServerServices.class); @@ -4043,10 +4044,10 @@ public class HRegion implements HeapSize { // , Writable{ /** * Convenience method creating new HRegions. Used by createTable and by the * bootstrap code in the HMaster constructor. - * Note, this method creates an {@link HLog} for the created region. It + * Note, this method creates an {@link WALService} for the created region. It * needs to be closed explicitly. Use {@link HRegion#getLog()} to get * access. When done with a region created using this method, you will - * need to explicitly close the {@link HLog} it created too; it will not be + * need to explicitly close the {@link WALService} it created too; it will not be * done for you. Not closing the log will leave at least a daemon thread * running. Call {@link #closeHRegion(HRegion)} and it will do * necessary cleanup for you. @@ -4068,9 +4069,9 @@ public class HRegion implements HeapSize { // , Writable{ * This will do the necessary cleanup a call to * {@link #createHRegion(HRegionInfo, Path, Configuration, HTableDescriptor)} * requires. This method will close the region and then close its - * associated {@link HLog} file. You use it if you call the other createHRegion, - * the one that takes an {@link HLog} instance but don't be surprised by the - * call to the {@link HLog#closeAndDelete()} on the {@link HLog} the + * associated {@link WALService} file. You use it if you call the other createHRegion, + * the one that takes an {@link WALService} instance but don't be surprised by the + * call to the {@link WALService#closeAndDelete()} on the {@link WALService} the * HRegion was carrying. * @param r * @throws IOException @@ -4084,7 +4085,7 @@ public class HRegion implements HeapSize { // , Writable{ /** * Convenience method creating new HRegions. Used by createTable. - * The {@link HLog} for the created region needs to be closed explicitly. + * The {@link WALService} for the created region needs to be closed explicitly. * Use {@link HRegion#getLog()} to get access. * * @param info Info for region to create. @@ -4100,7 +4101,7 @@ public class HRegion implements HeapSize { // , Writable{ public static HRegion createHRegion(final HRegionInfo info, final Path rootDir, final Configuration conf, final HTableDescriptor hTableDescriptor, - final HLog hlog, + final WALService hlog, final boolean initialize) throws IOException { return createHRegion(info, rootDir, conf, hTableDescriptor, @@ -4109,7 +4110,7 @@ public class HRegion implements HeapSize { // , Writable{ /** * Convenience method creating new HRegions. Used by createTable. - * The {@link HLog} for the created region needs to be closed + * The {@link WALService} for the created region needs to be closed * explicitly, if it is not null. * Use {@link HRegion#getLog()} to get access. * @@ -4126,7 +4127,7 @@ public class HRegion implements HeapSize { // , Writable{ public static HRegion createHRegion(final HRegionInfo info, final Path rootDir, final Configuration conf, final HTableDescriptor hTableDescriptor, - final HLog hlog, + final WALService hlog, final boolean initialize, final boolean ignoreHLog) throws IOException { Path tableDir = FSUtils.getTableDir(rootDir, info.getTable()); @@ -4135,7 +4136,7 @@ public class HRegion implements HeapSize { // , Writable{ /** * Convenience method creating new HRegions. Used by createTable. - * The {@link HLog} for the created region needs to be closed + * The {@link WALService} for the created region needs to be closed * explicitly, if it is not null. * Use {@link HRegion#getLog()} to get access. * @@ -4153,7 +4154,7 @@ public class HRegion implements HeapSize { // , Writable{ public static HRegion createHRegion(final HRegionInfo info, final Path rootDir, final Path tableDir, final Configuration conf, final HTableDescriptor hTableDescriptor, - final HLog hlog, + final WALService hlog, final boolean initialize, final boolean ignoreHLog) throws IOException { LOG.info("creating HRegion " + info.getTable().getNameAsString() @@ -4161,7 +4162,7 @@ public class HRegion implements HeapSize { // , Writable{ " Table name == " + info.getTable().getNameAsString()); FileSystem fs = FileSystem.get(conf); HRegionFileSystem rfs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, info); - HLog effectiveHLog = hlog; + WALService effectiveHLog = hlog; if (hlog == null && !ignoreHLog) { effectiveHLog = HLogFactory.createHLog(fs, rfs.getRegionDir(), HConstants.HREGION_LOGDIR_NAME, conf); @@ -4179,7 +4180,7 @@ public class HRegion implements HeapSize { // , Writable{ public static HRegion createHRegion(final HRegionInfo info, final Path rootDir, final Configuration conf, final HTableDescriptor hTableDescriptor, - final HLog hlog) + final WALService hlog) throws IOException { return createHRegion(info, rootDir, conf, hTableDescriptor, hlog, true); } @@ -4198,7 +4199,7 @@ public class HRegion implements HeapSize { // , Writable{ * @throws IOException */ public static HRegion openHRegion(final HRegionInfo info, - final HTableDescriptor htd, final HLog wal, + final HTableDescriptor htd, final WALService wal, final Configuration conf) throws IOException { return openHRegion(info, htd, wal, conf, null, null); @@ -4220,7 +4221,7 @@ public class HRegion implements HeapSize { // , Writable{ * @throws IOException */ public static HRegion openHRegion(final HRegionInfo info, - final HTableDescriptor htd, final HLog wal, final Configuration conf, + final HTableDescriptor htd, final WALService wal, final Configuration conf, final RegionServerServices rsServices, final CancelableProgressable reporter) throws IOException { @@ -4241,7 +4242,7 @@ public class HRegion implements HeapSize { // , Writable{ * @throws IOException */ public static HRegion openHRegion(Path rootDir, final HRegionInfo info, - final HTableDescriptor htd, final HLog wal, final Configuration conf) + final HTableDescriptor htd, final WALService wal, final Configuration conf) throws IOException { return openHRegion(rootDir, info, htd, wal, conf, null, null); } @@ -4262,7 +4263,7 @@ public class HRegion implements HeapSize { // , Writable{ * @throws IOException */ public static HRegion openHRegion(final Path rootDir, final HRegionInfo info, - final HTableDescriptor htd, final HLog wal, final Configuration conf, + final HTableDescriptor htd, final WALService wal, final Configuration conf, final RegionServerServices rsServices, final CancelableProgressable reporter) throws IOException { @@ -4291,7 +4292,7 @@ public class HRegion implements HeapSize { // , Writable{ * @throws IOException */ public static HRegion openHRegion(final Configuration conf, final FileSystem fs, - final Path rootDir, final HRegionInfo info, final HTableDescriptor htd, final HLog wal) + final Path rootDir, final HRegionInfo info, final HTableDescriptor htd, final WALService wal) throws IOException { return openHRegion(conf, fs, rootDir, info, htd, wal, null, null); } @@ -4313,7 +4314,7 @@ public class HRegion implements HeapSize { // , Writable{ * @throws IOException */ public static HRegion openHRegion(final Configuration conf, final FileSystem fs, - final Path rootDir, final HRegionInfo info, final HTableDescriptor htd, final HLog wal, + final Path rootDir, final HRegionInfo info, final HTableDescriptor htd, final WALService wal, final RegionServerServices rsServices, final CancelableProgressable reporter) throws IOException { Path tableDir = FSUtils.getTableDir(rootDir, info.getTable()); @@ -4337,8 +4338,9 @@ public class HRegion implements HeapSize { // , Writable{ * @throws IOException */ public static HRegion openHRegion(final Configuration conf, final FileSystem fs, - final Path rootDir, final Path tableDir, final HRegionInfo info, final HTableDescriptor htd, final HLog wal, - final RegionServerServices rsServices, final CancelableProgressable reporter) + final Path rootDir, final Path tableDir, final HRegionInfo info, final HTableDescriptor htd, + final WALService wal, final RegionServerServices rsServices, + final CancelableProgressable reporter) throws IOException { if (info == null) throw new NullPointerException("Passed region info is null"); if (LOG.isDebugEnabled()) { @@ -5450,7 +5452,7 @@ public class HRegion implements HeapSize { // , Writable{ * @throws IOException */ private static void processTable(final FileSystem fs, final Path p, - final HLog log, final Configuration c, + final WALService log, final Configuration c, final boolean majorCompact) throws IOException { HRegion region = null; @@ -5848,7 +5850,7 @@ public class HRegion implements HeapSize { // , Writable{ final Path logdir = new Path(c.get("hbase.tmp.dir")); final String logname = "hlog" + FSUtils.getTableName(tableDir) + System.currentTimeMillis(); - final HLog log = HLogFactory.createHLog(fs, logdir, logname, c); + final WALService log = HLogFactory.createHLog(fs, logdir, logname, c); try { processTable(fs, tableDir, log, c, majorCompact); } finally { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 1500b74..a159b0d 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -194,7 +194,7 @@ import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler; import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler; import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler; import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; @@ -410,10 +410,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa // HLog and HLog roller. log is protected rather than private to avoid // eclipse warning when accessed by inner classes - protected volatile HLog hlog; + protected volatile WALService hlog; // The meta updates are written to a different hlog. If this // regionserver holds meta regions, then this field will be non-null. - protected volatile HLog hlogForMeta; + protected volatile WALService hlogForMeta; LogRoller hlogRoller; LogRoller metaHLogRoller; @@ -1475,7 +1475,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa * @return A WAL instance. * @throws IOException */ - private HLog setupWALAndReplication() throws IOException { + private WALService setupWALAndReplication() throws IOException { final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME); final String logName = HLogUtil.getHLogDirectoryName(this.serverNameFromMasterPOV.toString()); @@ -1494,7 +1494,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa return instantiateHLog(rootDir, logName); } - private HLog getMetaWAL() throws IOException { + private WALService getMetaWAL() throws IOException { if (this.hlogForMeta != null) return this.hlogForMeta; final String logName = HLogUtil.getHLogDirectoryName(this.serverNameFromMasterPOV.toString()); Path logdir = new Path(rootDir, logName); @@ -1511,7 +1511,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa * @return WAL instance. * @throws IOException */ - protected HLog instantiateHLog(Path rootdir, String logName) throws IOException { + protected WALService instantiateHLog(Path rootdir, String logName) throws IOException { return HLogFactory.createHLog(this.fs.getBackingFs(), rootdir, logName, this.conf, getWALActionListeners(), this.serverNameFromMasterPOV.toString()); } @@ -1520,7 +1520,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa * Called by {@link #instantiateHLog(Path, String)} setting up WAL instance. * Add any {@link WALActionsListener}s you want inserted before WAL startup. * @return List of WALActionsListener that will be passed in to - * {@link org.apache.hadoop.hbase.regionserver.wal.FSHLog} on construction. + * {@link org.apache.hadoop.hbase.regionserver.wal.AbstractWAL} on construction. */ protected List getWALActionListeners() { List listeners = new ArrayList(); @@ -1706,7 +1706,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa return true; } - public HLog getWAL() { + public WALService getWAL() { try { return getWAL(null); } catch (IOException e) { @@ -1716,7 +1716,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa } @Override - public HLog getWAL(HRegionInfo regionInfo) throws IOException { + public WALService getWAL(HRegionInfo regionInfo) throws IOException { //TODO: at some point this should delegate to the HLogFactory //currently, we don't care about the region as much as we care about the //table.. (hence checking the tablename below) @@ -2614,7 +2614,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa HRegion toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName()); if (destination != null) { - HLog wal = getWAL(); + WALService wal = getWAL(); long closeSeqNum = wal.getEarliestMemstoreSeqNum(r.getRegionInfo().getEncodedNameAsBytes()); if (closeSeqNum == HConstants.NO_SEQNUM) { // No edits in WAL for this region; get the sequence number when the region was opened. @@ -3999,7 +3999,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa try { checkOpen(); requestCount.increment(); - HLog wal = this.getWAL(); + WALService wal = this.getWAL(); byte[][] regionsToFlush = wal.rollWriter(true); RollWALWriterResponse.Builder builder = RollWALWriterResponse.newBuilder(); if (regionsToFlush != null) { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java index 9488749..2194a4a 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java @@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; @@ -160,7 +160,7 @@ class LogRoller extends HasThread implements WALActionsListener { } } - protected HLog getWAL() throws IOException { + protected WALService getWAL() throws IOException { return this.services.getWAL(null); } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetaLogRoller.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetaLogRoller.java index d1ebd59..433028d 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetaLogRoller.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetaLogRoller.java @@ -22,7 +22,7 @@ import java.io.IOException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.Server; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; @InterfaceAudience.Private class MetaLogRoller extends LogRoller { @@ -30,7 +30,7 @@ class MetaLogRoller extends LogRoller { super(server, services); } @Override - protected HLog getWAL() throws IOException { + protected WALService getWAL() throws IOException { //The argument to getWAL below could either be HRegionInfo.FIRST_META_REGIONINFO or //HRegionInfo.ROOT_REGIONINFO. Both these share the same WAL. return services.getWAL(HRegionInfo.FIRST_META_REGIONINFO); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java index 9f88e70..976e1db 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheStats; +import org.apache.hadoop.hbase.regionserver.wal.WAL; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.metrics2.MetricsExecutor; @@ -433,16 +434,16 @@ class MetricsRegionServerWrapperImpl //Copy over computed values so that no thread sees half computed values. numStores = tempNumStores; - long tempNumHLogFiles = regionServer.hlog.getNumLogFiles(); + long tempNumHLogFiles = ((WAL)regionServer.hlog).getNumLogFiles(); // meta logs if (regionServer.hlogForMeta != null) { - tempNumHLogFiles += regionServer.hlogForMeta.getNumLogFiles(); + tempNumHLogFiles += ((WAL)regionServer.hlogForMeta).getNumLogFiles(); } numHLogFiles = tempNumHLogFiles; - long tempHlogFileSize = regionServer.hlog.getLogFileSize(); + long tempHlogFileSize = ((WAL)regionServer.hlog).getLogFileSize(); if (regionServer.hlogForMeta != null) { - tempHlogFileSize += regionServer.hlogForMeta.getLogFileSize(); + tempHlogFileSize += ((WAL)regionServer.hlogForMeta).getLogFileSize(); } hlogFileSize = tempHlogFileSize; diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java index 2be10cd..640da2c 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.ipc.PriorityFunction; import org.apache.hadoop.hbase.ipc.RpcServerInterface; import org.apache.hadoop.hbase.master.TableLockManager; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.zookeeper.KeeperException; import java.io.IOException; @@ -46,7 +46,7 @@ public interface RegionServerServices /** @return the HLog for a particular region. Pass null for getting the * default (common) WAL */ - HLog getWAL(HRegionInfo regionInfo) throws IOException; + WALService getWAL(HRegionInfo regionInfo) throws IOException; /** * @return Implementation of {@link CompactionRequestor} or null. diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractWAL.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractWAL.java new file mode 100644 index 0000000..353da8b --- /dev/null +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractWAL.java @@ -0,0 +1,220 @@ +/** + * 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.regionserver.wal; + +import java.io.IOException; +import java.io.OutputStream; +import java.lang.reflect.Method; +import java.net.URLEncoder; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.Syncable; +import org.apache.hadoop.hbase.util.FSUtils; + +import com.google.common.annotations.VisibleForTesting; +/** + * The skeleton WAL implementation, which provides methods and attributes common to all concrete + * implementations. + *

+ */ +@InterfaceAudience.Private +public abstract class AbstractWAL implements WAL, WALService, Syncable { + /** + * file system instance + */ + protected FileSystem fs; + /** + * HBase root directory. + */ + protected Path rootDir; + /** + * WAL directory, where all WAL files would be placed. + */ + protected Path dir; + /** + * conf object + */ + protected Configuration conf; + // Listeners that are called on WAL events. + protected List listeners = + new CopyOnWriteArrayList(); + + /** + * dir path where old logs are kept. + */ + protected Path oldLogDir; + + /** + * block size of the underlying Filesystem. Used when to do the roll. + */ + protected long blocksize; + + /** + * Prefix of a WAL file, usually the region server name it is hosted on. + */ + protected String prefix; + + protected WALCoprocessorHost coprocessorHost; + /** + * Is the WAL is closed + */ + protected volatile boolean closed = false; + /** + * If the WAL is for hbase:meta region + */ + protected boolean forMeta = false; + + /** + * Constructor to instantiate all the required fields of the WAL. + * @param fs + * @param root + * @param logDir + * @param oldLogDir + * @param conf + * @param listeners + * @param failIfLogDirExists + * @param prefix + * @param forMeta + * @throws IOException + */ + protected AbstractWAL(FileSystem fs, Path root, String logDir, String oldLogDir, + Configuration conf, List listeners, boolean failIfLogDirExists, + String prefix, boolean forMeta) throws IOException { + this.fs = fs; + this.rootDir = root; + this.dir = new Path(this.rootDir, logDir); + this.oldLogDir = new Path(this.rootDir, oldLogDir); + this.forMeta = forMeta; + this.conf = conf; + this.blocksize = this.conf.getLong("hbase.regionserver.hlog.blocksize", + FSUtils.getDefaultBlockSize(this.fs, this.dir)); + boolean dirExists = false; + if (failIfLogDirExists && (dirExists = this.fs.exists(dir))) { + throw new IOException("Target HLog directory already exists: " + dir); + } + if (!dirExists && !fs.mkdirs(dir)) { + throw new IOException("Unable to mkdir " + dir); + } + if (!fs.exists(this.oldLogDir)) { + if (!fs.mkdirs(this.oldLogDir)) { + throw new IOException("Unable to mkdir " + this.oldLogDir); + } + } + if (listeners != null) { + for (WALActionsListener i : listeners) { + registerWALActionsListener(i); + } + } + // If prefix is null||empty then just name it hlog + this.prefix = prefix == null || prefix.isEmpty() ? "hlog" : URLEncoder.encode(prefix, "UTF8"); + this.coprocessorHost = new WALCoprocessorHost(this, conf); + } + + @Override + public void registerWALActionsListener(final WALActionsListener listener) { + this.listeners.add(listener); + } + + @Override + public boolean unregisterWALActionsListener(final WALActionsListener listener) { + return this.listeners.remove(listener); + } + /** + * Get the directory we are making logs in. + * @return dir + */ + protected Path getDir() { + return dir; + } + + /** + * @param oldLogDir + * @param p + * @return archive path of a WAL file + */ + static Path getHLogArchivePath(Path oldLogDir, Path p) { + return new Path(oldLogDir, p.getName()); + } + + @Override + public WALCoprocessorHost getCoprocessorHost() { + return coprocessorHost; + } + + /** + * Get the 'getNumCurrentReplicas' on the passed os stream. + * This is used for getting current replicas of a file being written. It is used by all all + * concrete WAL implementations, and by tests (see TestLogRolling). + * @return Method or null. + */ + static Method getGetNumCurrentReplicas(final OutputStream os) { + Method m = null; + if (os != null) { + Class wrappedStreamClass = os.getClass(); + try { + LOG.info("wrappedStreamClass: " + wrappedStreamClass.getCanonicalName()); + m = wrappedStreamClass.getDeclaredMethod("getNumCurrentReplicas", + new Class[] {}); + m.setAccessible(true); + } catch (NoSuchMethodException e) { + LOG.info("FileSystem's output stream doesn't support" + + " getNumCurrentReplicas; --HDFS-826 not available; fsOut=" + + wrappedStreamClass.getName()); + } catch (SecurityException e) { + LOG.info("Doesn't have access to getNumCurrentReplicas on " + + "FileSystems's output stream --HDFS-826 not available; fsOut=" + + wrappedStreamClass.getName(), e); + m = null; // could happen on setAccessible() + } + } + if (m != null) { + if (LOG.isTraceEnabled()) LOG.trace("Using getNumCurrentReplicas--HDFS-826"); + } + return m; + } + + /** + * Currently, we need to expose the writer's OutputStream to tests so that they can manipulate + * the default behavior (such as setting the maxRecoveryErrorCount value for example (see + * {@link TestWALReplay#testReplayEditsWrittenIntoWAL()}). This is done using reflection on the + * underlying HDFS OutputStream. All the implementors need to pass the stream to these tests. + * NOTE: This could be removed once Hadoop1 support is removed. + * @VisibleForTesting + */ + abstract OutputStream getOutputStream(); + + /** + * Cleans up the current writer. This is called by TestHLogSplit + * @return Path to the current WAL file + * @throws IOException + */ + @VisibleForTesting + abstract Path cleanupCurrentWriter() throws IOException; + + /** + * @return if the current WAL has un-synced entries. This is called by TestLogRolledAbort. + */ + @VisibleForTesting + abstract boolean hasUnSyncedEntries(); +} diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/Compressor.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/Compressor.java index 47b55de..8b98003 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/Compressor.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/Compressor.java @@ -68,8 +68,8 @@ public class Compressor { FileSystem inFS = input.getFileSystem(conf); FileSystem outFS = output.getFileSystem(conf); - HLog.Reader in = HLogFactory.createReader(inFS, input, conf, null, false); - HLog.Writer out = null; + WAL.Reader in = HLogFactory.createReader(inFS, input, conf, null, false); + WAL.Writer out = null; try { if (!(in instanceof ReaderBase)) { @@ -80,7 +80,7 @@ public class Compressor { conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, !compress); out = HLogFactory.createWALWriter(outFS, output, conf); - HLog.Entry e = null; + WAL.Entry e = null; while ((e = in.next()) != null) out.append(e); } finally { in.close(); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java index 190bb4e..e917aeb 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java @@ -36,7 +36,6 @@ import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; @@ -49,7 +48,6 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.Syncable; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -108,30 +106,17 @@ import com.google.common.annotations.VisibleForTesting; * */ @InterfaceAudience.Private -class FSHLog implements HLog, Syncable { +class FSHLog extends AbstractWAL { static final Log LOG = LogFactory.getLog(FSHLog.class); - - private final FileSystem fs; - private final Path rootDir; - private final Path dir; - private final Configuration conf; - // Listeners that are called on WAL events. - private List listeners = - new CopyOnWriteArrayList(); - private final long blocksize; - private final String prefix; private final AtomicLong unflushedEntries = new AtomicLong(0); private final AtomicLong syncedTillHere = new AtomicLong(0); private long lastUnSyncedTxid; - private final Path oldLogDir; // all writes pending on AsyncWriter/AsyncSyncer thread with // txid <= failedTxid will fail by throwing asyncIOE private final AtomicLong failedTxid = new AtomicLong(0); private volatile IOException asyncIOE = null; - private WALCoprocessorHost coprocessorHost; - private FSDataOutputStream hdfs_out; // FSDataOutputStream associated with the current SequenceFile.writer // Minimum tolerable replicas, if the actual value is lower than it, // rollWriter will be triggered @@ -173,10 +158,6 @@ class FSHLog implements HLog, Syncable { private final Map oldestFlushingSeqNums = new TreeMap(Bytes.BYTES_COMPARATOR); - private volatile boolean closed = false; - - private boolean forMeta = false; - // The timestamp (in ms) when the log file was created. private volatile long filenum = -1; @@ -357,27 +338,14 @@ class FSHLog implements HLog, Syncable { * @param forMeta if this hlog is meant for meta updates * @throws IOException */ + public FSHLog(final FileSystem fs, final Path root, final String logDir, final String oldLogDir, final Configuration conf, final List listeners, final boolean failIfLogDirExists, final String prefix, boolean forMeta) throws IOException { - super(); - this.fs = fs; - this.rootDir = root; - this.dir = new Path(this.rootDir, logDir); - this.oldLogDir = new Path(this.rootDir, oldLogDir); - this.forMeta = forMeta; - this.conf = conf; - - if (listeners != null) { - for (WALActionsListener i: listeners) { - registerWALActionsListener(i); - } - } + super(fs, root, logDir, oldLogDir, conf, listeners, failIfLogDirExists, prefix, forMeta); - this.blocksize = this.conf.getLong("hbase.regionserver.hlog.blocksize", - FSUtils.getDefaultBlockSize(this.fs, this.dir)); // Roll at 95% of block size. float multi = conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f); this.logrollsize = (long)(this.blocksize * multi); @@ -392,37 +360,20 @@ class FSHLog implements HLog, Syncable { this.closeErrorsTolerated = conf.getInt( "hbase.regionserver.logroll.errors.tolerated", 0); - LOG.info("WAL/HLog configuration: blocksize=" + StringUtils.byteDesc(this.blocksize) + ", rollsize=" + StringUtils.byteDesc(this.logrollsize) + ", enabled=" + this.enabled); - // If prefix is null||empty then just name it hlog - this.prefix = prefix == null || prefix.isEmpty() ? - "hlog" : URLEncoder.encode(prefix, "UTF8"); - - boolean dirExists = false; - if (failIfLogDirExists && (dirExists = this.fs.exists(dir))) { - throw new IOException("Target HLog directory already exists: " + dir); - } - if (!dirExists && !fs.mkdirs(dir)) { - throw new IOException("Unable to mkdir " + dir); - } - if (!fs.exists(this.oldLogDir)) { - if (!fs.mkdirs(this.oldLogDir)) { - throw new IOException("Unable to mkdir " + this.oldLogDir); - } - } // rollWriter sets this.hdfs_out if it can. rollWriter(); // handle the reflection necessary to call getNumCurrentReplicas() - this.getNumCurrentReplicas = getGetNumCurrentReplicas(this.hdfs_out); + if (this.hdfs_out != null) + this.getNumCurrentReplicas = getGetNumCurrentReplicas(this.hdfs_out.getWrappedStream()); final String n = Thread.currentThread().getName(); - asyncWriter = new AsyncWriter(n + "-WAL.AsyncWriter"); asyncWriter.start(); @@ -436,55 +387,9 @@ class FSHLog implements HLog, Syncable { asyncNotifier = new AsyncNotifier(n + "-WAL.AsyncNotifier"); asyncNotifier.start(); - coprocessorHost = new WALCoprocessorHost(this, conf); - this.metrics = new MetricsWAL(); } - /** - * Find the 'getNumCurrentReplicas' on the passed os stream. - * @return Method or null. - */ - private Method getGetNumCurrentReplicas(final FSDataOutputStream os) { - Method m = null; - if (os != null) { - Class wrappedStreamClass = os.getWrappedStream() - .getClass(); - try { - m = wrappedStreamClass.getDeclaredMethod("getNumCurrentReplicas", - new Class[] {}); - m.setAccessible(true); - } catch (NoSuchMethodException e) { - LOG.info("FileSystem's output stream doesn't support" - + " getNumCurrentReplicas; --HDFS-826 not available; fsOut=" - + wrappedStreamClass.getName()); - } catch (SecurityException e) { - LOG.info("Doesn't have access to getNumCurrentReplicas on " - + "FileSystems's output stream --HDFS-826 not available; fsOut=" - + wrappedStreamClass.getName(), e); - m = null; // could happen on setAccessible() - } - } - if (m != null) { - if (LOG.isTraceEnabled()) LOG.trace("Using getNumCurrentReplicas--HDFS-826"); - } - return m; - } - - @Override - public void registerWALActionsListener(final WALActionsListener listener) { - this.listeners.add(listener); - } - - @Override - public boolean unregisterWALActionsListener(final WALActionsListener listener) { - return this.listeners.remove(listener); - } - - @Override - public long getFilenum() { - return this.filenum; - } /** * Method used internal to this class and for tests only. @@ -494,6 +399,7 @@ class FSHLog implements HLog, Syncable { * * usage: see TestLogRolling.java */ + @Override OutputStream getOutputStream() { return this.hdfs_out.getWrappedStream(); } @@ -531,10 +437,10 @@ class FSHLog implements HLog, Syncable { oldPath = computeFilename(currentFilenum); } this.filenum = System.currentTimeMillis(); - Path newPath = computeFilename(); + Path newPath = getCurrentFileName(); while (fs.exists(newPath)) { this.filenum++; - newPath = computeFilename(); + newPath = getCurrentFileName(); } // Tell our listeners that a new log is about to be created @@ -660,33 +566,6 @@ class FSHLog implements HLog, Syncable { } /** - * Takes a region:sequenceId map for a WAL file, and checks whether the file can be archived. - * It compares the region entries present in the passed sequenceNums map with the local copy of - * {@link #oldestUnflushedSeqNums} and {@link #oldestFlushingSeqNums}. If, for all regions, - * the value is lesser than the minimum of values present in the oldestFlushing/UnflushedSeqNums, - * then the wal file is eligible for archiving. - * @param sequenceNums for a HLog, at the time when it was rolled. - * @param oldestFlushingMap - * @param oldestUnflushedMap - * @return true if wal is eligible for archiving, false otherwise. - */ - static boolean areAllRegionsFlushed(Map sequenceNums, - Map oldestFlushingMap, Map oldestUnflushedMap) { - for (Map.Entry regionSeqIdEntry : sequenceNums.entrySet()) { - // find region entries in the flushing/unflushed map. If there is no entry, it means - // a region doesn't have any unflushed entry. - long oldestFlushing = oldestFlushingMap.containsKey(regionSeqIdEntry.getKey()) ? - oldestFlushingMap.get(regionSeqIdEntry.getKey()) : Long.MAX_VALUE; - long oldestUnFlushed = oldestUnflushedMap.containsKey(regionSeqIdEntry.getKey()) ? - oldestUnflushedMap.get(regionSeqIdEntry.getKey()) : Long.MAX_VALUE; - // do a minimum to be sure to contain oldest sequence Id - long minSeqNum = Math.min(oldestFlushing, oldestUnFlushed); - if (minSeqNum <= regionSeqIdEntry.getValue()) return false;// can't archive - } - return true; - } - - /** * Iterates over the given map of regions, and compares their sequence numbers with corresponding * entries in {@link #oldestUnflushedSeqNums}. If the sequence number is greater or equal, the * region is eligible to flush, otherwise, there is no benefit to flush (from the perspective of @@ -713,6 +592,33 @@ class FSHLog implements HLog, Syncable { } /** + * Takes a region:sequenceId map for a WAL file, and checks whether the file can be archived. It + * compares the region entries present in the passed sequenceNums map with the local copy of + * {@link #oldestUnflushedSeqNums} and {@link #oldestFlushingSeqNums}. If, for all regions, the + * value is lesser than the minimum of values present in the oldestFlushing/UnflushedSeqNums, then + * the wal file is eligible for archiving. + * @param sequenceNums for a HLog, at the time when it was rolled. + * @param oldestFlushingMap + * @param oldestUnflushedMap + * @return true if wal is eligible for archiving, false otherwise. + */ + static boolean areAllRegionsFlushed(Map sequenceNums, + Map oldestFlushingMap, Map oldestUnflushedMap) { + for (Map.Entry regionSeqIdEntry : sequenceNums.entrySet()) { + // find region entries in the flushing/unflushed map. If there is no entry, it means + // a region doesn't have any unflushed entry. + long oldestFlushing = oldestFlushingMap.containsKey(regionSeqIdEntry.getKey()) ? + oldestFlushingMap.get(regionSeqIdEntry.getKey()) : Long.MAX_VALUE; + long oldestUnFlushed = oldestUnflushedMap.containsKey(regionSeqIdEntry.getKey()) ? + oldestUnflushedMap.get(regionSeqIdEntry.getKey()) : Long.MAX_VALUE; + // do a minimum to be sure to contain oldest sequence Id + long minSeqNum = Math.min(oldestFlushing, oldestUnFlushed); + if (minSeqNum <= regionSeqIdEntry.getValue()) return false;// can't archive + } + return true; + } + + /** * If the number of un-archived WAL files is greater than maximum allowed, it checks * the first (oldest) WAL file, and returns the regions which should be flushed so that it could * be archived. @@ -740,13 +646,17 @@ class FSHLog implements HLog, Syncable { return regions; } + Path cleanupCurrentWriter() throws IOException { + return cleanupCurrentWriter(this.filenum); + } + /* * Cleans up current writer closing. * Presumes we're operating inside an updateLock scope. * @return Path to current writer or null if none. * @throws IOException */ - Path cleanupCurrentWriter(final long currentfilenum) throws IOException { + private Path cleanupCurrentWriter(final long currentfilenum) throws IOException { Path oldFile = null; if (this.writer != null) { // Close the current writer, get a new one. @@ -812,7 +722,7 @@ class FSHLog implements HLog, Syncable { * using the current HLog file-number * @return Path */ - protected Path computeFilename() { + public Path getCurrentFileName() { return computeFilename(this.filenum); } @@ -826,9 +736,9 @@ class FSHLog implements HLog, Syncable { if (filenum < 0) { throw new RuntimeException("hlog file number can't be < 0"); } - String child = prefix + "." + filenum; + String child = prefix + WAL.WAL_FILE_NAME_DELIMITER + filenum; if (forMeta) { - child += HLog.META_HLOG_FILE_EXTN; + child += WAL.META_HLOG_FILE_EXTN; } return new Path(dir, child); } @@ -844,13 +754,14 @@ class FSHLog implements HLog, Syncable { protected long getFileNumFromFileName(Path fileName) { if (fileName == null) throw new IllegalArgumentException("file name can't be null"); // The path should start with dir/. - String prefixPathStr = new Path(dir, prefix + ".").toString(); + String prefixPathStr = new Path(dir, prefix + WAL.WAL_FILE_NAME_DELIMITER).toString(); if (!fileName.toString().startsWith(prefixPathStr)) { - throw new IllegalArgumentException("The log file " + fileName + " doesn't belong to" + - " this regionserver " + prefixPathStr); + throw new IllegalArgumentException("The log file " + fileName + " doesn't belong to" + + " this regionserver " + prefixPathStr); } String chompedPath = fileName.toString().substring(prefixPathStr.length()); - if (forMeta) chompedPath = chompedPath.substring(0, chompedPath.indexOf(META_HLOG_FILE_EXTN)); + if (forMeta) chompedPath = chompedPath.substring(0, + chompedPath.indexOf(WAL.META_HLOG_FILE_EXTN)); return Long.parseLong(chompedPath); } @@ -1445,10 +1356,6 @@ class FSHLog implements HLog, Syncable { return 0; } - boolean canGetCurReplicas() { - return this.getNumCurrentReplicas != null; - } - @Override public void hsync() throws IOException { syncer(); @@ -1498,7 +1405,7 @@ class FSHLog implements HLog, Syncable { logKey.setScopes(null); } // write to our buffer for the Hlog file. - this.pendingWrites.add(new HLog.Entry(logKey, logEdit)); + this.pendingWrites.add(new WAL.Entry(logKey, logEdit)); } long took = EnvironmentEdgeManager.currentTimeMillis() - now; coprocessorHost.postWALWrite(info, logKey, logEdit); @@ -1516,7 +1423,8 @@ class FSHLog implements HLog, Syncable { /** @return How many items have been added to the log */ - int getNumEntries() { + @Override //TODO is it okay to expose it? + public int getNumEntries() { return numEntries.get(); } @@ -1602,23 +1510,6 @@ class FSHLog implements HLog, Syncable { return lowReplicationRollEnabled; } - /** - * Get the directory we are making logs in. - * - * @return dir - */ - protected Path getDir() { - return dir; - } - - static Path getHLogArchivePath(Path oldLogDir, Path p) { - return new Path(oldLogDir, p.getName()); - } - - static String formatRecoveredEditsFileName(final long seqid) { - return String.format("%019d", seqid); - } - public static final long FIXED_OVERHEAD = ClassSize.align( ClassSize.OBJECT + (5 * ClassSize.REFERENCE) + ClassSize.ATOMIC_INTEGER + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG)); @@ -1647,12 +1538,9 @@ class FSHLog implements HLog, Syncable { HLogSplitter.split(baseDir, p, oldLogDir, fs, conf); } - @Override - public WALCoprocessorHost getCoprocessorHost() { - return coprocessorHost; - } /** Provide access to currently deferred sequence num for tests */ + @Override boolean hasUnSyncedEntries() { return this.lastUnSyncedTxid > this.syncedTillHere.get(); } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java deleted file mode 100644 index 4aa5be4..0000000 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java +++ /dev/null @@ -1,365 +0,0 @@ -/** - * - * 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.regionserver.wal; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicLong; -import java.util.regex.Pattern; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer; -import org.apache.hadoop.io.Writable; - -import com.google.common.annotations.VisibleForTesting; - - -@InterfaceAudience.Private -// TODO: Rename interface to WAL -public interface HLog { - Log LOG = LogFactory.getLog(HLog.class); - - /** File Extension used while splitting an HLog into regions (HBASE-2312) */ - // TODO: this seems like an implementation detail that does not belong here. - String SPLITTING_EXT = "-splitting"; - boolean SPLIT_SKIP_ERRORS_DEFAULT = false; - /** The hbase:meta region's HLog filename extension */ - String META_HLOG_FILE_EXTN = ".meta"; - - /** - * Configuration name of HLog Trailer's warning size. If a waltrailer's size is greater than the - * configured size, a warning is logged. This is used with Protobuf reader/writer. - */ - // TODO: Implementation detail. Why in here? - String WAL_TRAILER_WARN_SIZE = "hbase.regionserver.waltrailer.warn.size"; - int DEFAULT_WAL_TRAILER_WARN_SIZE = 1024 * 1024; // 1MB - - // TODO: Implemenation detail. Why in here? - Pattern EDITFILES_NAME_PATTERN = Pattern.compile("-?[0-9]+"); - String RECOVERED_LOG_TMPFILE_SUFFIX = ".temp"; - - interface Reader { - - /** - * @param fs File system. - * @param path Path. - * @param c Configuration. - * @param s Input stream that may have been pre-opened by the caller; may be null. - */ - void init(FileSystem fs, Path path, Configuration c, FSDataInputStream s) throws IOException; - - void close() throws IOException; - - Entry next() throws IOException; - - Entry next(Entry reuse) throws IOException; - - void seek(long pos) throws IOException; - - long getPosition() throws IOException; - void reset() throws IOException; - - /** - * @return the WALTrailer of the current HLog. It may be null in case of legacy or corrupt WAL - * files. - */ - // TODO: What we need a trailer on WAL for? - WALTrailer getWALTrailer(); - } - - interface Writer { - void init(FileSystem fs, Path path, Configuration c, boolean overwritable) throws IOException; - - void close() throws IOException; - - void sync() throws IOException; - - void append(Entry entry) throws IOException; - - long getLength() throws IOException; - - /** - * Sets HLog's WALTrailer. This trailer is appended at the end of WAL on closing. - * @param walTrailer trailer to append to WAL. - */ - void setWALTrailer(WALTrailer walTrailer); - } - - /** - * Utility class that lets us keep track of the edit with it's key. - * Only used when splitting logs. - */ - // TODO: Remove this Writable. - class Entry implements Writable { - private WALEdit edit; - private HLogKey key; - - public Entry() { - edit = new WALEdit(); - key = new HLogKey(); - } - - /** - * Constructor for both params - * - * @param edit log's edit - * @param key log's key - */ - public Entry(HLogKey key, WALEdit edit) { - super(); - this.key = key; - this.edit = edit; - } - - /** - * Gets the edit - * - * @return edit - */ - public WALEdit getEdit() { - return edit; - } - - /** - * Gets the key - * - * @return key - */ - public HLogKey getKey() { - return key; - } - - /** - * Set compression context for this entry. - * - * @param compressionContext - * Compression context - */ - public void setCompressionContext(CompressionContext compressionContext) { - edit.setCompressionContext(compressionContext); - key.setCompressionContext(compressionContext); - } - - @Override - public String toString() { - return this.key + "=" + this.edit; - } - - @Override - @SuppressWarnings("deprecation") - public void write(DataOutput dataOutput) throws IOException { - this.key.write(dataOutput); - this.edit.write(dataOutput); - } - - @Override - public void readFields(DataInput dataInput) throws IOException { - this.key.readFields(dataInput); - this.edit.readFields(dataInput); - } - } - - /** - * registers WALActionsListener - * - * @param listener - */ - void registerWALActionsListener(final WALActionsListener listener); - - /** - * unregisters WALActionsListener - * - * @param listener - */ - boolean unregisterWALActionsListener(final WALActionsListener listener); - - /** - * @return Current state of the monotonically increasing file id. - */ - // TODO: Remove. Implementation detail. - long getFilenum(); - - /** - * @return the number of HLog files - */ - int getNumLogFiles(); - - /** - * @return the size of HLog files - */ - long getLogFileSize(); - - // TODO: Log rolling should not be in this interface. - /** - * Roll the log writer. That is, start writing log messages to a new file. - * - *

- * The implementation is synchronized in order to make sure there's one rollWriter - * running at any given time. - * - * @return If lots of logs, flush the returned regions so next time through we - * can clean logs. Returns null if nothing to flush. Names are actual - * region names as returned by {@link HRegionInfo#getEncodedName()} - * @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException - * @throws IOException - */ - byte[][] rollWriter() throws FailedLogCloseException, IOException; - - /** - * Roll the log writer. That is, start writing log messages to a new file. - * - *

- * The implementation is synchronized in order to make sure there's one rollWriter - * running at any given time. - * - * @param force - * If true, force creation of a new writer even if no entries have - * been written to the current writer - * @return If lots of logs, flush the returned regions so next time through we - * can clean logs. Returns null if nothing to flush. Names are actual - * region names as returned by {@link HRegionInfo#getEncodedName()} - * @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException - * @throws IOException - */ - byte[][] rollWriter(boolean force) throws FailedLogCloseException, - IOException; - - /** - * Shut down the log. - * - * @throws IOException - */ - void close() throws IOException; - - /** - * Shut down the log and delete the log directory - * - * @throws IOException - */ - void closeAndDelete() throws IOException; - - /** - * Same as appendNoSync(HRegionInfo, TableName, WALEdit, List, long, HTableDescriptor), - * except it causes a sync on the log - * @param sequenceId of the region. - */ - @VisibleForTesting - public void append(HRegionInfo info, TableName tableName, WALEdit edits, - final long now, HTableDescriptor htd, AtomicLong sequenceId) throws IOException; - - /** - * For notification post append to the writer. - * @param entries - */ - void postAppend(final List entries); - - /** - * For notification post writer sync. - */ - void postSync(); - - /** - * Append a set of edits to the log. Log edits are keyed by (encoded) regionName, rowname, and - * log-sequence-id. The HLog is not flushed after this transaction is written to the log. - * @param info - * @param tableName - * @param edits - * @param clusterIds The clusters that have consumed the change (for replication) - * @param now - * @param htd - * @param sequenceId of the region - * @return txid of this transaction - * @throws IOException - */ - long appendNoSync(HRegionInfo info, TableName tableName, WALEdit edits, - List clusterIds, final long now, HTableDescriptor htd, AtomicLong sequenceId, - boolean isInMemstore, long nonceGroup, long nonce) throws IOException; - - // TODO: Do we need all these versions of sync? - void hsync() throws IOException; - - void hflush() throws IOException; - - void sync() throws IOException; - - void sync(long txid) throws IOException; - - /** - * WAL keeps track of the sequence numbers that were not yet flushed from memstores - * in order to be able to do cleanup. This method tells WAL that some region is about - * to flush memstore. - * - * We stash the oldest seqNum for the region, and let the the next edit inserted in this - * region be recorded in {@link #append(HRegionInfo, TableName, WALEdit, long, HTableDescriptor, - * AtomicLong)} as new oldest seqnum. - * In case of flush being aborted, we put the stashed value back; in case of flush succeeding, - * the seqNum of that first edit after start becomes the valid oldest seqNum for this region. - * - * @return true if the flush can proceed, false in case wal is closing (ususally, when server is - * closing) and flush couldn't be started. - */ - boolean startCacheFlush(final byte[] encodedRegionName); - - /** - * Complete the cache flush. - * @param encodedRegionName Encoded region name. - */ - void completeCacheFlush(final byte[] encodedRegionName); - - /** - * Abort a cache flush. Call if the flush fails. Note that the only recovery - * for an aborted flush currently is a restart of the regionserver so the - * snapshot content dropped by the failure gets restored to the memstore.v - * @param encodedRegionName Encoded region name. - */ - void abortCacheFlush(byte[] encodedRegionName); - - /** - * @return Coprocessor host. - */ - WALCoprocessorHost getCoprocessorHost(); - - /** - * Get LowReplication-Roller status - * - * @return lowReplicationRollEnabled - */ - // TODO: This is implementation detail? - boolean isLowReplicationRollEnabled(); - - /** Gets the earliest sequence number in the memstore for this particular region. - * This can serve as best-effort "recent" WAL number for this region. - * @param encodedRegionName The region to get the number for. - * @return The number if present, HConstants.NO_SEQNUM if absent. - */ - long getEarliestMemstoreSeqNum(byte[] encodedRegionName); -} diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java index 2b9130c..c55aa3b 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java @@ -27,41 +27,42 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Writer; +import org.apache.hadoop.hbase.regionserver.wal.WAL.Reader; +import org.apache.hadoop.hbase.regionserver.wal.WAL.Writer; import org.apache.hadoop.hbase.util.CancelableProgressable; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; - +@InterfaceAudience.Private public class HLogFactory { private static final Log LOG = LogFactory.getLog(HLogFactory.class); - public static HLog createHLog(final FileSystem fs, final Path root, final String logName, + public static WALService createHLog(final FileSystem fs, final Path root, final String logName, final Configuration conf) throws IOException { return new FSHLog(fs, root, logName, conf); } - - public static HLog createHLog(final FileSystem fs, final Path root, final String logName, + + public static WALService createHLog(final FileSystem fs, final Path root, final String logName, final String oldLogName, final Configuration conf) throws IOException { return new FSHLog(fs, root, logName, oldLogName, conf); } - - public static HLog createHLog(final FileSystem fs, final Path root, final String logName, + + public static WALService createHLog(final FileSystem fs, final Path root, final String logName, final Configuration conf, final List listeners, final String prefix) throws IOException { return new FSHLog(fs, root, logName, conf, listeners, prefix); } - public static HLog createMetaHLog(final FileSystem fs, final Path root, final String logName, - final Configuration conf, final List listeners, - final String prefix) throws IOException { - return new FSHLog(fs, root, logName, HConstants.HREGION_OLDLOGDIR_NAME, - conf, listeners, false, prefix, true); - } + public static WALService createMetaHLog(final FileSystem fs, final Path root, + final String logName, final Configuration conf, final List listeners, + final String prefix) throws IOException { + return new FSHLog(fs, root, logName, HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners, false, + prefix, true); + } /* * WAL Reader @@ -72,24 +73,24 @@ public class HLogFactory { logReaderClass = null; } - public static HLog.Reader createReader(final FileSystem fs, + public static WAL.Reader createReader(final FileSystem fs, final Path path, Configuration conf) throws IOException { return createReader(fs, path, conf, null); } /** - * Create a reader for the WAL. If you are reading from a file that's being written to - * and need to reopen it multiple times, use {@link HLog.Reader#reset()} instead of this method + * Create a reader for the WAL. If you are reading from a file that's being written to and need + * to reopen it multiple times, use {@link WALService.Reader#reset()} instead of this method * then just seek back to the last known good position. * @return A WAL reader. Close when done with it. * @throws IOException */ - public static HLog.Reader createReader(final FileSystem fs, final Path path, + public static WAL.Reader createReader(final FileSystem fs, final Path path, Configuration conf, CancelableProgressable reporter) throws IOException { return createReader(fs, path, conf, reporter, true); } - public static HLog.Reader createReader(final FileSystem fs, final Path path, + public static WAL.Reader createReader(final FileSystem fs, final Path path, Configuration conf, CancelableProgressable reporter, boolean allowCustom) throws IOException { if (allowCustom && (logReaderClass == null)) { @@ -109,7 +110,7 @@ public class HLogFactory { try { if (lrClass != ProtobufLogReader.class) { // User is overriding the WAL reader, let them. - HLog.Reader reader = lrClass.newInstance(); + WAL.Reader reader = lrClass.newInstance(); reader.init(fs, path, conf, null); return reader; } else { @@ -121,7 +122,7 @@ public class HLogFactory { byte[] magic = new byte[ProtobufLogReader.PB_WAL_MAGIC.length]; boolean isPbWal = (stream.read(magic) == magic.length) && Arrays.equals(magic, ProtobufLogReader.PB_WAL_MAGIC); - HLog.Reader reader = + WAL.Reader reader = isPbWal ? new ProtobufLogReader() : new SequenceFileLogReader(); reader.init(fs, path, conf, stream); return reader; @@ -172,17 +173,17 @@ public class HLogFactory { * @return A WAL writer. Close when done with it. * @throws IOException */ - public static HLog.Writer createWALWriter(final FileSystem fs, + public static WAL.Writer createWALWriter(final FileSystem fs, final Path path, Configuration conf) throws IOException { return createWriter(fs, path, conf, false); } - public static HLog.Writer createRecoveredEditsWriter(final FileSystem fs, + public static WAL.Writer createRecoveredEditsWriter(final FileSystem fs, final Path path, Configuration conf) throws IOException { return createWriter(fs, path, conf, true); } - private static HLog.Writer createWriter(final FileSystem fs, + private static WAL.Writer createWriter(final FileSystem fs, final Path path, Configuration conf, boolean overwritable) throws IOException { try { @@ -190,7 +191,7 @@ public class HLogFactory { logWriterClass = conf.getClass("hbase.regionserver.hlog.writer.impl", ProtobufLogWriter.class, Writer.class); } - HLog.Writer writer = (HLog.Writer)logWriterClass.newInstance(); + WAL.Writer writer = (WAL.Writer)logWriterClass.newInstance(); writer.init(fs, path, conf, overwritable); return writer; } catch (Exception e) { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogPrettyPrinter.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogPrettyPrinter.java index 0aa9b2f..4927767 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogPrettyPrinter.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogPrettyPrinter.java @@ -39,7 +39,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader; +import org.apache.hadoop.hbase.regionserver.wal.WAL.Reader; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.codehaus.jackson.map.ObjectMapper; @@ -242,7 +242,7 @@ public class HLogPrettyPrinter { } Reader log = HLogFactory.createReader(fs, p, conf); try { - FSHLog.Entry entry; + AbstractWAL.Entry entry; while ((entry = log.next()) != null) { HLogKey key = entry.getKey(); WALEdit edit = entry.getEdit(); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java index 1dbd3d6..dbc1a66 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java @@ -93,9 +93,9 @@ import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionStoreSeq import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.StoreSequenceId; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.LastSequenceId; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Writer; +import org.apache.hadoop.hbase.regionserver.wal.WAL.Entry; +import org.apache.hadoop.hbase.regionserver.wal.WAL.Reader; +import org.apache.hadoop.hbase.regionserver.wal.WAL.Writer; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CancelableProgressable; import org.apache.hadoop.hbase.util.ClassSize; @@ -252,7 +252,7 @@ public class HLogSplitter { boolean isCorrupted = false; Preconditions.checkState(status == null); boolean skipErrors = conf.getBoolean("hbase.hlog.split.skip.errors", - HLog.SPLIT_SKIP_ERRORS_DEFAULT); + WAL.SPLIT_SKIP_ERRORS_DEFAULT); int interval = conf.getInt("hbase.splitlog.report.interval.loglines", 1024); Path logPath = logfile.getPath(); boolean outputSinkStarted = false; @@ -507,7 +507,7 @@ public class HLogSplitter { } static String getTmpRecoveredEditsFileName(String fileName) { - return fileName + HLog.RECOVERED_LOG_TMPFILE_SUFFIX; + return fileName + WAL.RECOVERED_LOG_TMPFILE_SUFFIX; } /** @@ -1330,8 +1330,8 @@ public class HLogSplitter { * Map key -> value layout * : -> Queue */ - private Map>> serverToBufferQueueMap = - new ConcurrentHashMap>>(); + private Map>> serverToBufferQueueMap = + new ConcurrentHashMap>>(); private List thrown = new ArrayList(); // The following sink is used in distrubitedLogReplay mode for entries of regions in a disabling @@ -1372,10 +1372,10 @@ public class HLogSplitter { // process workitems String maxLocKey = null; int maxSize = 0; - List> maxQueue = null; + List> maxQueue = null; synchronized (this.serverToBufferQueueMap) { for (String key : this.serverToBufferQueueMap.keySet()) { - List> curQueue = this.serverToBufferQueueMap.get(key); + List> curQueue = this.serverToBufferQueueMap.get(key); if (curQueue.size() > maxSize) { maxSize = curQueue.size(); maxQueue = curQueue; @@ -1409,7 +1409,7 @@ public class HLogSplitter { private void groupEditsByServer(List entries) throws IOException { Set nonExistentTables = null; Long cachedLastFlushedSequenceId = -1l; - for (HLog.Entry entry : entries) { + for (WAL.Entry entry : entries) { WALEdit edit = entry.getEdit(); TableName table = entry.getKey().getTablename(); // clear scopes which isn't needed for recovery @@ -1490,13 +1490,13 @@ public class HLogSplitter { synchronized (serverToBufferQueueMap) { locKey = loc.getHostnamePort() + KEY_DELIMITER + table; - List> queue = serverToBufferQueueMap.get(locKey); + List> queue = serverToBufferQueueMap.get(locKey); if (queue == null) { queue = - Collections.synchronizedList(new ArrayList>()); + Collections.synchronizedList(new ArrayList>()); serverToBufferQueueMap.put(locKey, queue); } - queue.add(new Pair(loc, entry)); + queue.add(new Pair(loc, entry)); } // store regions we have recovered so far addToRecoveredRegions(loc.getRegionInfo().getEncodedName()); @@ -1565,7 +1565,7 @@ public class HLogSplitter { return loc; } - private void processWorkItems(String key, List> actions) + private void processWorkItems(String key, List> actions) throws IOException { RegionServerWriter rsw = null; @@ -1648,7 +1648,7 @@ public class HLogSplitter { protected boolean flush() throws IOException { String curLoc = null; int curSize = 0; - List> curQueue = null; + List> curQueue = null; synchronized (this.serverToBufferQueueMap) { for (String locationKey : this.serverToBufferQueueMap.keySet()) { curQueue = this.serverToBufferQueueMap.get(locationKey); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java index 684f78c..c417f99 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java @@ -53,7 +53,7 @@ public class HLogUtil { * Pattern used to validate a HLog file name */ private static final Pattern pattern = - Pattern.compile(".*\\.\\d*("+HLog.META_HLOG_FILE_EXTN+")*"); + Pattern.compile(".*\\.\\d*("+WAL.META_HLOG_FILE_EXTN+")*"); /** * @param filename @@ -182,8 +182,9 @@ public class HLogUtil { logDirName = logDir.getName(); } ServerName serverName = null; - if (logDirName.endsWith(HLog.SPLITTING_EXT)) { - logDirName = logDirName.substring(0, logDirName.length() - HLog.SPLITTING_EXT.length()); + if (logDirName.endsWith(WAL.SPLITTING_EXT)) { + logDirName = logDirName + .substring(0, logDirName.length() - WAL.SPLITTING_EXT.length()); } try { serverName = ServerName.parseServerName(logDirName); @@ -222,11 +223,11 @@ public class HLogUtil { // There can be other files in this directory other than edit files. // In particular, on error, we'll move aside the bad edit file giving // it a timestamp suffix. See moveAsideBadEditsFile. - Matcher m = HLog.EDITFILES_NAME_PATTERN.matcher(p.getName()); + Matcher m = WAL.EDITFILES_NAME_PATTERN.matcher(p.getName()); result = fs.isFile(p) && m.matches(); // Skip the file whose name ends with RECOVERED_LOG_TMPFILE_SUFFIX, // because it means splithlog thread is writting this file. - if (p.getName().endsWith(HLog.RECOVERED_LOG_TMPFILE_SUFFIX)) { + if (p.getName().endsWith(WAL.RECOVERED_LOG_TMPFILE_SUFFIX)) { result = false; } } catch (IOException e) { @@ -248,7 +249,7 @@ public class HLogUtil { } public static boolean isMetaFile(String p) { - if (p != null && p.endsWith(HLog.META_HLOG_FILE_EXTN)) { + if (p != null && p.endsWith(WAL.META_HLOG_FILE_EXTN)) { return true; } return false; @@ -261,7 +262,7 @@ public class HLogUtil { * the compaction from finishing if this regionserver has already lost its lease on the log. * @param sequenceId Used by HLog to get sequence Id for the waledit. */ - public static void writeCompactionMarker(HLog log, HTableDescriptor htd, HRegionInfo info, + public static void writeCompactionMarker(WALService log, HTableDescriptor htd, HRegionInfo info, final CompactionDescriptor c, AtomicLong sequenceId) throws IOException { WALEdit e = WALEdit.createCompaction(c); long now = EnvironmentEdgeManager.currentTimeMillis(); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java index 1f808eb..4bed041 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java @@ -202,7 +202,7 @@ public class ProtobufLogReader extends ReaderBase { } @Override - protected boolean readNext(HLog.Entry entry) throws IOException { + protected boolean readNext(WAL.Entry entry) throws IOException { while (true) { // OriginalPosition might be < 0 on local fs; if so, it is useless to us. long originalPosition = this.inputStream.getPos(); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java index 6d10b26..66c4edb 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java @@ -68,8 +68,8 @@ public class ProtobufLogWriter extends WriterBase { super.init(fs, path, conf, overwritable); assert this.output == null; boolean doCompress = initializeCompressionContext(conf, path); - this.trailerWarnSize = conf.getInt(HLog.WAL_TRAILER_WARN_SIZE, - HLog.DEFAULT_WAL_TRAILER_WARN_SIZE); + this.trailerWarnSize = conf.getInt(WAL.WAL_TRAILER_WARN_SIZE, + WAL.DEFAULT_WAL_TRAILER_WARN_SIZE); int bufferSize = FSUtils.getDefaultBufferSize(fs); short replication = (short)conf.getInt( "hbase.regionserver.hlog.replication", FSUtils.getDefaultReplication(fs, path)); @@ -97,7 +97,7 @@ public class ProtobufLogWriter extends WriterBase { } @Override - public void append(HLog.Entry entry) throws IOException { + public void append(WAL.Entry entry) throws IOException { entry.setCompressionContext(compressionContext); entry.getKey().getBuilder(compressor).setFollowingKvCount(entry.getEdit().size()) .build().writeDelimitedTo(output); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java index 4f7204e..9c01314 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer; import org.apache.hadoop.hbase.util.FSUtils; @InterfaceAudience.Private -public abstract class ReaderBase implements HLog.Reader { +public abstract class ReaderBase implements WAL.Reader { private static final Log LOG = LogFactory.getLog(ReaderBase.class); protected Configuration conf; protected FileSystem fs; @@ -64,8 +64,8 @@ public abstract class ReaderBase implements HLog.Reader { this.path = path; this.fs = fs; this.fileLength = this.fs.getFileStatus(path).getLen(); - this.trailerWarnSize = conf.getInt(HLog.WAL_TRAILER_WARN_SIZE, - HLog.DEFAULT_WAL_TRAILER_WARN_SIZE); + this.trailerWarnSize = conf.getInt(WAL.WAL_TRAILER_WARN_SIZE, + WAL.DEFAULT_WAL_TRAILER_WARN_SIZE); initReader(stream); boolean compression = hasCompression(); @@ -86,15 +86,15 @@ public abstract class ReaderBase implements HLog.Reader { } @Override - public HLog.Entry next() throws IOException { + public WAL.Entry next() throws IOException { return next(null); } @Override - public HLog.Entry next(HLog.Entry reuse) throws IOException { - HLog.Entry e = reuse; + public WAL.Entry next(WAL.Entry reuse) throws IOException { + WAL.Entry e = reuse; if (e == null) { - e = new HLog.Entry(new HLogKey(), new WALEdit()); + e = new WAL.Entry(new HLogKey(), new WALEdit()); } if (compressionContext != null) { e.setCompressionContext(compressionContext); @@ -152,7 +152,7 @@ public abstract class ReaderBase implements HLog.Reader { * @param e The entry to read into. * @return Whether there was anything to read. */ - protected abstract boolean readNext(HLog.Entry e) throws IOException; + protected abstract boolean readNext(WAL.Entry e) throws IOException; /** * Performs a filesystem-level seek to a certain position in an underlying file. diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java index 9dc0719..31ca8b8 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java @@ -32,7 +32,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry; +import org.apache.hadoop.hbase.regionserver.wal.WAL.Entry; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile.Metadata; import org.apache.hadoop.io.Text; diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WAL.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WAL.java new file mode 100644 index 0000000..901da61 --- /dev/null +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WAL.java @@ -0,0 +1,227 @@ +/** + * + * 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.regionserver.wal; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.List; +import java.util.regex.Pattern; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer; +import org.apache.hadoop.io.Writable; + +/** + * The Write Ahead Log (WAL) stores all the edits to the HStore. + * This interface provides APIs, reader and writer abstractions for all WAL implementors. + *

+ * A WAL file is rolled once it reaches a HDFS block size. + * See {@link FSHLog} for an example implementation. + */ +@InterfaceAudience.Private +public interface WAL { + + interface Reader{ + /** + * @param fs File system. + * @param path Path. + * @param c Configuration. + * @param s Input stream that may have been pre-opened by the caller; may be null. + */ + void init(FileSystem fs, Path path, Configuration c, FSDataInputStream s) throws IOException; + + void close() throws IOException; + + Entry next() throws IOException; + + Entry next(Entry reuse) throws IOException; + + void seek(long pos) throws IOException; + + long getPosition() throws IOException; + void reset() throws IOException; + + /** + * @return the WALTrailer of the current HLog. It may be null in case of legacy or corrupt WAL + * files. + */ + // TODO: What we need a trailer on WAL for? + WALTrailer getWALTrailer(); + } + + interface Writer { + void init(FileSystem fs, Path path, Configuration c, boolean overwritable) throws IOException; + + void close() throws IOException; + + void sync() throws IOException; + + void append(Entry entry) throws IOException; + + long getLength() throws IOException; + + /** + * Sets HLog's WALTrailer. This trailer is appended at the end of WAL on closing. + * @param walTrailer trailer to append to WAL. + */ + void setWALTrailer(WALTrailer walTrailer); + } + + /** + * Configuration name of HLog Trailer's warning size. If a waltrailer's size is greater than the + * configured size, a warning is logged. This is used with Protobuf reader/writer. + */ + public static final String WAL_TRAILER_WARN_SIZE = "hbase.regionserver.waltrailer.warn.size"; + public static final int DEFAULT_WAL_TRAILER_WARN_SIZE = 1024 * 1024; // 1MB + public static final Pattern EDITFILES_NAME_PATTERN = Pattern.compile("-?[0-9]+"); + public static final String RECOVERED_LOG_TMPFILE_SUFFIX = ".temp"; + public static final String WAL_FILE_NAME_DELIMITER = "."; + /** File Extension used while splitting an HLog into regions (HBASE-2312) */ + public static final String SPLITTING_EXT = "-splitting"; + public static final boolean SPLIT_SKIP_ERRORS_DEFAULT = false; + /** The hbase:meta region's HLog filename extension */ + public static final String META_HLOG_FILE_EXTN = ".meta"; + + /** + * @return the total number of WAL files (including rolled WALs). + */ + int getNumLogFiles(); + + /** + * returns the number of rolled WAL files. + */ + int getNumRolledLogFiles(); + + /** + * @return the path of the current WAL file. + */ + Path getCurrentFileName(); + + /** + * @return the current size of all HLog files (including rolled files). + */ + long getLogFileSize(); + + /** + * For notification post append to the writer. + *

+ * Note: This should be called after invoking append() on the underlying hdfs stream. + * @param entries + */ + void postAppend(final List entries); + + /** + * For notification post writer sync. + *

+ * NOTE: This should be called after invoking the sync() on the + * underlying hdfs stream. + */ + void postSync(); + + /** + * @return the number of entries in the current WAL file + */ + int getNumEntries(); + + /** + * Get LowReplication-Roller status + * @return lowReplicationRollEnabled + */ + boolean isLowReplicationRollEnabled(); + + /** + * Utility class that lets us keep track of the edit with it's key. + * Only used when splitting logs. + */ + // TODO: Remove this Writable. + class Entry implements Writable { + private WALEdit edit; + private HLogKey key; + + public Entry() { + edit = new WALEdit(); + key = new HLogKey(); + } + + /** + * Constructor for both params + * + * @param edit log's edit + * @param key log's key + */ + public Entry(HLogKey key, WALEdit edit) { + super(); + this.key = key; + this.edit = edit; + } + + /** + * Gets the edit + * + * @return edit + */ + public WALEdit getEdit() { + return edit; + } + + /** + * Gets the key + * + * @return key + */ + public HLogKey getKey() { + return key; + } + + /** + * Set compression context for this entry. + * + * @param compressionContext + * Compression context + */ + public void setCompressionContext(CompressionContext compressionContext) { + edit.setCompressionContext(compressionContext); + key.setCompressionContext(compressionContext); + } + + @Override + public String toString() { + return this.key + "=" + this.edit; + } + + @Override + @SuppressWarnings("deprecation") + public void write(DataOutput dataOutput) throws IOException { + this.key.write(dataOutput); + this.edit.write(dataOutput); + } + + @Override + public void readFields(DataInput dataInput) throws IOException { + this.key.readFields(dataInput); + this.edit.readFields(dataInput); + } + } + +} diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java index c0675cb..9d4f12c 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java @@ -30,7 +30,8 @@ import org.apache.hadoop.conf.Configuration; /** * Implements the coprocessor environment and runtime support for coprocessors - * loaded within a {@link FSHLog}. + * loaded within a {@link AbstractWAL}. + * TODO: There was FSHLog instance here? Shouldn't be it a WALService? */ @InterfaceAudience.Private public class WALCoprocessorHost @@ -42,10 +43,10 @@ public class WALCoprocessorHost static class WALEnvironment extends CoprocessorHost.Environment implements WALCoprocessorEnvironment { - private FSHLog wal; + private AbstractWAL wal; @Override - public FSHLog getWAL() { + public AbstractWAL getWAL() { return wal; } @@ -60,19 +61,19 @@ public class WALCoprocessorHost */ public WALEnvironment(Class implClass, final Coprocessor impl, final int priority, final int seq, final Configuration conf, - final FSHLog hlog) { + final AbstractWAL hlog) { super(impl, priority, seq, conf); this.wal = hlog; } } - FSHLog wal; + AbstractWAL wal; /** * Constructor * @param log the write ahead log * @param conf the configuration */ - public WALCoprocessorHost(final FSHLog log, final Configuration conf) { + public WALCoprocessorHost(final AbstractWAL log, final Configuration conf) { // We don't want to require an Abortable passed down through (FS)HLog, so // this means that a failure to load of a WAL coprocessor won't abort the // server. This isn't ideal, and means that security components that diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java index 917a7c9..8f483bd 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java @@ -93,17 +93,17 @@ public class WALEditsReplaySink { * @param entries * @throws IOException */ - public void replayEntries(List> entries) throws IOException { + public void replayEntries(List> entries) throws IOException { if (entries.size() == 0) { return; } int batchSize = entries.size(); - Map> entriesByRegion = - new HashMap>(); + Map> entriesByRegion = + new HashMap>(); HRegionLocation loc = null; - HLog.Entry entry = null; - List regionEntries = null; + WAL.Entry entry = null; + List regionEntries = null; // Build the action list. for (int i = 0; i < batchSize; i++) { loc = entries.get(i).getFirst(); @@ -111,7 +111,7 @@ public class WALEditsReplaySink { if (entriesByRegion.containsKey(loc.getRegionInfo())) { regionEntries = entriesByRegion.get(loc.getRegionInfo()); } else { - regionEntries = new ArrayList(); + regionEntries = new ArrayList(); entriesByRegion.put(loc.getRegionInfo(), regionEntries); } regionEntries.add(entry); @@ -120,9 +120,9 @@ public class WALEditsReplaySink { long startTime = EnvironmentEdgeManager.currentTimeMillis(); // replaying edits by region - for (Map.Entry> _entry : entriesByRegion.entrySet()) { + for (Map.Entry> _entry : entriesByRegion.entrySet()) { HRegionInfo curRegion = _entry.getKey(); - List allActions = _entry.getValue(); + List allActions = _entry.getValue(); // send edits in chunks int totalActions = allActions.size(); int replayedActions = 0; @@ -156,7 +156,7 @@ public class WALEditsReplaySink { } private void replayEdits(final HRegionLocation regionLoc, final HRegionInfo regionInfo, - final List entries) throws IOException { + final List entries) throws IOException { try { RpcRetryingCallerFactory factory = RpcRetryingCallerFactory.instantiate(conf); ReplayServerCallable callable = @@ -179,11 +179,11 @@ public class WALEditsReplaySink { */ class ReplayServerCallable extends RegionServerCallable { private HRegionInfo regionInfo; - private List entries; + private List entries; ReplayServerCallable(final HConnection connection, final TableName tableName, final HRegionLocation regionLoc, final HRegionInfo regionInfo, - final List entries) { + final List entries) { super(connection, tableName, null); this.entries = entries; this.regionInfo = regionInfo; @@ -200,11 +200,11 @@ public class WALEditsReplaySink { return null; } - private void replayToServer(HRegionInfo regionInfo, List entries) + private void replayToServer(HRegionInfo regionInfo, List entries) throws IOException, ServiceException { if (entries.isEmpty()) return; - HLog.Entry[] entriesArray = new HLog.Entry[entries.size()]; + WAL.Entry[] entriesArray = new WAL.Entry[entries.size()]; entriesArray = entries.toArray(entriesArray); AdminService.BlockingInterface remoteSvr = conn.getAdmin(getLocation().getServerName()); @@ -225,7 +225,7 @@ public class WALEditsReplaySink { // if not due to connection issue, the following code should run fast because it uses // cached location boolean skip = false; - for (HLog.Entry entry : this.entries) { + for (WAL.Entry entry : this.entries) { WALEdit edit = entry.getEdit(); List kvs = edit.getKeyValues(); for (KeyValue kv : kvs) { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALService.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALService.java new file mode 100644 index 0000000..49bf2a3 --- /dev/null +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALService.java @@ -0,0 +1,184 @@ +/** + * + * 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.regionserver.wal; + +import java.io.IOException; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; + +import com.google.common.annotations.VisibleForTesting; + +/** + * A Write Ahead Log (WAL) provides service for reading, writing waledits. This interface provides + * APIs for WAL users (such as RegionServer) to use the WAL (do append, sync, etc). + */ + +@InterfaceAudience.Private +public interface WALService { + Log LOG = LogFactory.getLog(WALService.class); + + /** + * registers WALActionsListener + * + * @param listener + */ + void registerWALActionsListener(final WALActionsListener listener); + + /** + * unregisters WALActionsListener + * + * @param listener + */ + boolean unregisterWALActionsListener(final WALActionsListener listener); + + /** + * Roll the log writer. That is, start writing log messages to a new file. + * + *

+ * The implementation is synchronized in order to make sure there's one rollWriter + * running at any given time. + * + * @return If lots of logs, flush the returned regions so next time through we + * can clean logs. Returns null if nothing to flush. Names are actual + * region names as returned by {@link HRegionInfo#getEncodedName()} + * @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException + * @throws IOException + */ + byte[][] rollWriter() throws FailedLogCloseException, IOException; + + /** + * Roll the log writer. That is, start writing log messages to a new file. + * + *

+ * The implementation is synchronized in order to make sure there's one rollWriter + * running at any given time. + * + * @param force + * If true, force creation of a new writer even if no entries have + * been written to the current writer + * @return If lots of logs, flush the returned regions so next time through we + * can clean logs. Returns null if nothing to flush. Names are actual + * region names as returned by {@link HRegionInfo#getEncodedName()} + * @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException + * @throws IOException + */ + byte[][] rollWriter(boolean force) throws FailedLogCloseException, + IOException; + + /** + * Shut down the log. + * + * @throws IOException + */ + void close() throws IOException; + + /** + * Shut down the log and delete the log directory + * + * @throws IOException + */ + void closeAndDelete() throws IOException; + + /** + * Same as appendNoSync(HRegionInfo, TableName, WALEdit, List, long, HTableDescriptor), + * except it causes a sync on the log + * @param sequenceId of the region. + */ + @VisibleForTesting + public void append(HRegionInfo info, TableName tableName, WALEdit edits, + final long now, HTableDescriptor htd, AtomicLong sequenceId) throws IOException; + + /** + * Append a set of edits to the log. Log edits are keyed by (encoded) regionName, rowname, and + * log-sequence-id. The HLog is not flushed after this transaction is written to the log. + * @param info + * @param tableName + * @param edits + * @param clusterIds The clusters that have consumed the change (for replication) + * @param now + * @param htd + * @param sequenceId of the region + * @return txid of this transaction + * @throws IOException + */ + long appendNoSync(HRegionInfo info, TableName tableName, WALEdit edits, + List clusterIds, final long now, HTableDescriptor htd, AtomicLong sequenceId, + boolean isInMemstore, long nonceGroup, long nonce) throws IOException; + + // TODO: Do we need all these versions of sync? + void hsync() throws IOException; + + void hflush() throws IOException; + + void sync() throws IOException; + + void sync(long txid) throws IOException; + + /** + * WAL keeps track of the sequence numbers that were not yet flushed from memstores + * in order to be able to do cleanup. This method tells WAL that some region is about + * to flush memstore. + * + * We stash the oldest seqNum for the region, and let the the next edit inserted in this + * region be recorded in {@link #append(HRegionInfo, TableName, WALEdit, long, HTableDescriptor, + * AtomicLong)} as new oldest seqnum. + * In case of flush being aborted, we put the stashed value back; in case of flush succeeding, + * the seqNum of that first edit after start becomes the valid oldest seqNum for this region. + * + * @return true if the flush can proceed, false in case wal is closing (ususally, when server is + * closing) and flush couldn't be started. + */ + boolean startCacheFlush(final byte[] encodedRegionName); + + /** + * Complete the cache flush. + * @param encodedRegionName Encoded region name. + */ + void completeCacheFlush(final byte[] encodedRegionName); + + /** + * Abort a cache flush. Call if the flush fails. Note that the only recovery + * for an aborted flush currently is a restart of the regionserver so the + * snapshot content dropped by the failure gets restored to the memstore.v + * @param encodedRegionName Encoded region name. + */ + void abortCacheFlush(byte[] encodedRegionName); + + /** + * @return Coprocessor host. + */ + WALCoprocessorHost getCoprocessorHost(); + + + /** Gets the earliest sequence number in the memstore for this particular region. + * This can serve as best-effort "recent" WAL number for this region. + * @param encodedRegionName The region to get the number for. + * @return The number if present, HConstants.NO_SEQNUM if absent. + */ + long getEarliestMemstoreSeqNum(byte[] encodedRegionName); +} diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WriterBase.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WriterBase.java index b6e02b3..0b7e822 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WriterBase.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WriterBase.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.util.FSUtils; * custom dictionary compression. */ @InterfaceAudience.Private -public abstract class WriterBase implements HLog.Writer { +public abstract class WriterBase implements WAL.Writer { protected CompressionContext compressionContext; protected Configuration conf; diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationHLogReaderManager.java hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationHLogReaderManager.java index 6f8b7f5..ed3dcc6 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationHLogReaderManager.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationHLogReaderManager.java @@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WAL; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import java.io.IOException; @@ -40,7 +40,7 @@ public class ReplicationHLogReaderManager { private final FileSystem fs; private final Configuration conf; private long position = 0; - private HLog.Reader reader; + private WAL.Reader reader; private Path lastPath; /** @@ -60,7 +60,7 @@ public class ReplicationHLogReaderManager { * @return an HLog reader. * @throws IOException */ - public HLog.Reader openReader(Path path) throws IOException { + public WAL.Reader openReader(Path path) throws IOException { // Detect if this is a new file, if so get a new reader else // reset the current reader so that we see the new data if (this.reader == null || !this.lastPath.equals(path)) { @@ -82,8 +82,8 @@ public class ReplicationHLogReaderManager { * @return a new entry or null * @throws IOException */ - public HLog.Entry readNextAndSetPosition() throws IOException { - HLog.Entry entry = this.reader.next(); + public WAL.Entry readNextAndSetPosition() throws IOException { + WAL.Entry entry = this.reader.next(); // Store the position so that in the future the reader can start // reading from here. If the above call to next() throws an // exception, the position won't be changed and retry will happen diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java index 242d9d4..c5b6278 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java @@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WAL; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.replication.ReplicationException; @@ -94,7 +94,7 @@ public class ReplicationSource extends Thread // Max number of entries in entriesArray private int replicationQueueNbCapacity; // Our reader for the current log - private HLog.Reader reader; + private WAL.Reader reader; // Last position in the log that we sent to ZooKeeper private long lastLoggedPosition = -1; // Path of the current log @@ -287,7 +287,7 @@ public class ReplicationSource extends Thread boolean gotIOE = false; currentNbOperations = 0; - List entries = new ArrayList(1); + List entries = new ArrayList(1); currentSize = 0; try { if (readAllEntriesToReplicateOrNextFile(currentWALisBeingWrittenTo, entries)) { @@ -369,8 +369,8 @@ public class ReplicationSource extends Thread * entries * @throws IOException */ - protected boolean readAllEntriesToReplicateOrNextFile(boolean currentWALisBeingWrittenTo, List entries) - throws IOException{ + protected boolean readAllEntriesToReplicateOrNextFile(boolean currentWALisBeingWrittenTo, + List entries) throws IOException { long seenEntries = 0; if (LOG.isTraceEnabled()) { LOG.trace("Seeking in " + this.currentPath + " at position " @@ -378,7 +378,7 @@ public class ReplicationSource extends Thread } this.repLogReader.seek(); long positionBeforeRead = this.repLogReader.getPosition(); - HLog.Entry entry = + WAL.Entry entry = this.repLogReader.readNextAndSetPosition(); while (entry != null) { WALEdit edit = entry.getEdit(); @@ -486,7 +486,7 @@ public class ReplicationSource extends Thread new Path(manager.getLogDir().getParent(), curDeadServerName); Path[] locs = new Path[] { new Path(deadRsDirectory, currentPath.getName()), - new Path(deadRsDirectory.suffix(HLog.SPLITTING_EXT), + new Path(deadRsDirectory.suffix(WAL.SPLITTING_EXT), currentPath.getName()), }; for (Path possibleLogLocation : locs) { @@ -596,7 +596,7 @@ public class ReplicationSource extends Thread * We only want KVs that are scoped other than local * @param entry The entry to check for replication */ - protected void removeNonReplicableEdits(HLog.Entry entry) { + protected void removeNonReplicableEdits(WAL.Entry entry) { NavigableMap scopes = entry.getKey().getScopes(); ArrayList kvs = entry.getEdit().getKeyValues(); int size = kvs.size(); @@ -636,7 +636,7 @@ public class ReplicationSource extends Thread * @param currentWALisBeingWrittenTo was the current WAL being (seemingly) * written to when this method was called */ - protected void shipEdits(boolean currentWALisBeingWrittenTo, List entries) { + protected void shipEdits(boolean currentWALisBeingWrittenTo, List entries) { int sleepMultiplier = 1; if (entries.isEmpty()) { LOG.warn("Was given 0 edits to ship"); @@ -658,7 +658,7 @@ public class ReplicationSource extends Thread " entries of total size " + currentSize); } ReplicationProtbufUtil.replicateWALEntry(rrs, - entries.toArray(new HLog.Entry[entries.size()])); + entries.toArray(new WAL.Entry[entries.size()])); if (this.lastLoggedPosition != this.repLogReader.getPosition()) { this.manager.logPositionAndCleanOldLogs(this.currentPath, this.peerClusterZnode, this.repLogReader.getPosition(), diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/HbaseObjectWritableFor96Migration.java hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/HbaseObjectWritableFor96Migration.java index 5a8ca69..af4f525 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/HbaseObjectWritableFor96Migration.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/HbaseObjectWritableFor96Migration.java @@ -87,7 +87,7 @@ import org.apache.hadoop.hbase.io.WritableWithSize; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.regionserver.RegionOpeningState; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WAL; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ProtoUtil; @@ -229,8 +229,8 @@ class HbaseObjectWritableFor96Migration implements Writable, WritableWithSize, C addToMap(Delete [].class, code++); - addToMap(HLog.Entry.class, code++); - addToMap(HLog.Entry[].class, code++); + addToMap(WAL.Entry.class, code++); + addToMap(WAL.Entry[].class, code++); addToMap(HLogKey.class, code++); addToMap(List.class, code++); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotLogSplitter.java hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotLogSplitter.java index 3443c5f..c895e11 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotLogSplitter.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotLogSplitter.java @@ -32,7 +32,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.io.HLogLink; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WAL; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; @@ -59,7 +59,7 @@ class SnapshotLogSplitter implements Closeable { static final Log LOG = LogFactory.getLog(SnapshotLogSplitter.class); private final class LogWriter implements Closeable { - private HLog.Writer writer; + private WAL.Writer writer; private Path logFile; private long seqId; @@ -78,7 +78,7 @@ class SnapshotLogSplitter implements Closeable { fs.rename(logFile, finalFile); } - public void append(final HLog.Entry entry) throws IOException { + public void append(final WAL.Entry entry) throws IOException { writer.append(entry); if (seqId < entry.getKey().getLogSeqNum()) { seqId = entry.getKey().getLogSeqNum(); @@ -87,7 +87,7 @@ class SnapshotLogSplitter implements Closeable { private String logFileName(long seqId, boolean temp) { String fileName = String.format("%019d", seqId); - if (temp) fileName += HLog.RECOVERED_LOG_TMPFILE_SUFFIX; + if (temp) fileName += WAL.RECOVERED_LOG_TMPFILE_SUFFIX; return fileName; } } @@ -146,9 +146,9 @@ class SnapshotLogSplitter implements Closeable { * @param logPath Snapshot HLog reference path */ public void splitLog(final Path logPath) throws IOException { - HLog.Reader log = HLogFactory.createReader(fs, logPath, conf); + WAL.Reader log = HLogFactory.createReader(fs, logPath, conf); try { - HLog.Entry entry; + WAL.Entry entry; LogWriter writer = null; byte[] regionName = null; byte[] newRegionName = null; @@ -173,7 +173,7 @@ class SnapshotLogSplitter implements Closeable { // Append Entry key = new HLogKey(newRegionName, tableName, key.getLogSeqNum(), key.getWriteTime(), key.getClusterIds(), key.getNonceGroup(), key.getNonce()); - writer.append(new HLog.Entry(key, entry.getEdit())); + writer.append(new WAL.Entry(key, entry.getEdit())); } } catch (IOException e) { LOG.warn("Something wrong during the log split", e); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java index 1b97d69..e8cd837 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java @@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.zookeeper.KeeperException; /** @@ -188,7 +188,7 @@ public class HBaseFsckRepair { // Create HRegion Path root = FSUtils.getRootDir(conf); HRegion region = HRegion.createHRegion(hri, root, conf, htd); - HLog hlog = region.getLog(); + WALService hlog = region.getLog(); // Close the new region to flush to disk. Close log file too. region.close(); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java index 00c1c2b..9a569d9 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java @@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; /** @@ -137,7 +137,7 @@ class HMerge { protected final FileSystem fs; protected final Path rootDir; protected final HTableDescriptor htd; - protected final HLog hlog; + protected final WALService hlog; private final long maxFilesize; diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java index 1f3f737..023cef9 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java @@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.io.WritableComparator; import org.apache.hadoop.util.GenericOptionsParser; import org.apache.hadoop.util.Tool; @@ -180,7 +180,7 @@ public class Merge extends Configured implements Tool { Bytes.toStringBinary(meta.getRegionName())); } HRegion merged = null; - HLog log = utils.getLog(); + WALService log = utils.getLog(); HRegion r1 = HRegion.openHRegion(info1, htd, log, getConf()); try { HRegion r2 = HRegion.openHRegion(info2, htd, log, getConf()); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java index 57dcf54..abe71a6 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; /** @@ -49,7 +49,7 @@ public class MetaUtils { private static final Log LOG = LogFactory.getLog(MetaUtils.class); private final Configuration conf; private FileSystem fs; - private HLog log; + private WALService log; private HRegion metaRegion; private Map metaRegions = Collections.synchronizedSortedMap( new TreeMap(Bytes.BYTES_COMPARATOR)); @@ -84,7 +84,7 @@ public class MetaUtils { * @return the HLog * @throws IOException e */ - public synchronized HLog getLog() throws IOException { + public synchronized WALService getLog() throws IOException { if (this.log == null) { String logName = HConstants.HREGION_LOGDIR_NAME + "_" + System.currentTimeMillis(); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index be16e30..1930294 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -81,7 +81,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.RegionServerServices; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.tool.Canary; import org.apache.hadoop.hbase.util.Bytes; @@ -1554,11 +1554,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * @return created hregion * @throws IOException */ - public HRegion createLocalHRegion(HRegionInfo info, HTableDescriptor desc, HLog hlog) throws IOException { + public HRegion createLocalHRegion(HRegionInfo info, HTableDescriptor desc, WALService hlog) + throws IOException { return HRegion.createHRegion(info, getDataTestDir(), getConfiguration(), desc, hlog); } - /** * @param tableName * @param startKey @@ -1573,7 +1573,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { */ public HRegion createLocalHRegion(byte[] tableName, byte[] startKey, byte[] stopKey, String callingMethod, Configuration conf, boolean isReadOnly, Durability durability, - HLog hlog, byte[]... families) throws IOException { + WALService hlog, byte[]... families) throws IOException { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); htd.setReadOnly(isReadOnly); for (byte[] family : families) { diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java index 50ad030..ce04d73 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java @@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.regionserver.Leases; import org.apache.hadoop.hbase.regionserver.RegionServerAccounting; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.ServerNonceManager; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -192,7 +192,7 @@ class MockRegionServerServices implements RegionServerServices { } @Override - public HLog getWAL(HRegionInfo regionInfo) throws IOException { + public WALService getWAL(HRegionInfo regionInfo) throws IOException { return null; } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java index cc84600..5db6a2b 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java @@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.apache.hadoop.hdfs.DFSClient; @@ -80,7 +80,7 @@ public class TestIOFencing { ((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL); ((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hdfs.server.namenode.FSNamesystem")).getLogger().setLevel(Level.ALL); ((Log4JLogger)DFSClient.LOG).getLogger().setLevel(Level.ALL); - ((Log4JLogger)HLog.LOG).getLogger().setLevel(Level.ALL); + ((Log4JLogger)WALService.LOG).getLogger().setLevel(Level.ALL); } public abstract static class CompactionBlockerRegion extends HRegion { @@ -89,7 +89,7 @@ public class TestIOFencing { volatile CountDownLatch compactionsWaiting = new CountDownLatch(0); @SuppressWarnings("deprecation") - public CompactionBlockerRegion(Path tableDir, HLog log, + public CompactionBlockerRegion(Path tableDir, WALService log, FileSystem fs, Configuration confParam, HRegionInfo info, HTableDescriptor htd, RegionServerServices rsServices) { super(tableDir, log, fs, confParam, info, htd, rsServices); @@ -136,7 +136,7 @@ public class TestIOFencing { */ public static class BlockCompactionsInPrepRegion extends CompactionBlockerRegion { - public BlockCompactionsInPrepRegion(Path tableDir, HLog log, + public BlockCompactionsInPrepRegion(Path tableDir, WALService log, FileSystem fs, Configuration confParam, HRegionInfo info, HTableDescriptor htd, RegionServerServices rsServices) { super(tableDir, log, fs, confParam, info, htd, rsServices); @@ -159,7 +159,7 @@ public class TestIOFencing { * entry to go the WAL before blocking, but blocks afterwards */ public static class BlockCompactionsInCompletionRegion extends CompactionBlockerRegion { - public BlockCompactionsInCompletionRegion(Path tableDir, HLog log, + public BlockCompactionsInCompletionRegion(Path tableDir, WALService log, FileSystem fs, Configuration confParam, HRegionInfo info, HTableDescriptor htd, RegionServerServices rsServices) { super(tableDir, log, fs, confParam, info, htd, rsServices); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java index 376c38b..5dccf24 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java @@ -60,7 +60,7 @@ import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreScanner; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -215,7 +215,7 @@ public class TestRegionObserverScannerOpenHook { private static volatile CountDownLatch compactionStateChangeLatch = null; @SuppressWarnings("deprecation") - public CompactionCompletionNotifyingRegion(Path tableDir, HLog log, + public CompactionCompletionNotifyingRegion(Path tableDir, WALService log, FileSystem fs, Configuration confParam, HRegionInfo info, HTableDescriptor htd, RegionServerServices rsServices) { super(tableDir, log, fs, confParam, info, htd, rsServices); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java index 0264d76..3909430 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost; @@ -141,7 +141,7 @@ public class TestWALObserver { fs.mkdirs(new Path(basedir, hri.getEncodedName())); final AtomicLong sequenceId = new AtomicLong(0); - HLog log = HLogFactory.createHLog(this.fs, hbaseRootDir, + WALService log = HLogFactory.createHLog(this.fs, hbaseRootDir, TestWALObserver.class.getName(), this.conf); SampleRegionWALObserver cp = getCoprocessor(log); @@ -239,7 +239,7 @@ public class TestWALObserver { final Configuration newConf = HBaseConfiguration.create(this.conf); // HLog wal = new HLog(this.fs, this.dir, this.oldLogDir, this.conf); - HLog wal = createWAL(this.conf); + WALService wal = createWAL(this.conf); // Put p = creatPutWith2Families(TEST_ROW); WALEdit edit = new WALEdit(); long now = EnvironmentEdgeManager.currentTimeMillis(); @@ -264,7 +264,7 @@ public class TestWALObserver { LOG.info("WALSplit path == " + p); FileSystem newFS = FileSystem.get(newConf); // Make a new wal for new region open. - HLog wal2 = createWAL(newConf); + WALService wal2 = createWAL(newConf); HRegion region = HRegion.openHRegion(newConf, FileSystem.get(newConf), hbaseRootDir, hri, htd, wal2, TEST_UTIL.getHBaseCluster().getRegionServer(0), null); long seqid2 = region.getOpenSeqNum(); @@ -290,12 +290,12 @@ public class TestWALObserver { */ @Test public void testWALObserverLoaded() throws Exception { - HLog log = HLogFactory.createHLog(fs, hbaseRootDir, + WALService log = HLogFactory.createHLog(fs, hbaseRootDir, TestWALObserver.class.getName(), conf); assertNotNull(getCoprocessor(log)); } - private SampleRegionWALObserver getCoprocessor(HLog wal) throws Exception { + private SampleRegionWALObserver getCoprocessor(WALService wal) throws Exception { WALCoprocessorHost host = wal.getCoprocessorHost(); Coprocessor c = host.findCoprocessor(SampleRegionWALObserver.class .getName()); @@ -366,14 +366,13 @@ public class TestWALObserver { return splits.get(0); } - private HLog createWAL(final Configuration c) throws IOException { + private WALService createWAL(final Configuration c) throws IOException { return HLogFactory.createHLog(FileSystem.get(c), hbaseRootDir, logName, c); } - private void addWALEdits(final TableName tableName, final HRegionInfo hri, - final byte[] rowName, final byte[] family, final int count, - EnvironmentEdge ee, final HLog wal, final HTableDescriptor htd, final AtomicLong sequenceId) - throws IOException { + private void addWALEdits(final TableName tableName, final HRegionInfo hri, final byte[] rowName, + final byte[] family, final int count, EnvironmentEdge ee, final WALService wal, + final HTableDescriptor htd, final AtomicLong sequenceId) throws IOException { String familyStr = Bytes.toString(family); for (int j = 0; j < count; j++) { byte[] qualifierBytes = Bytes.toBytes(Integer.toString(j)); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java index ca5098d..ec961f9 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java @@ -49,7 +49,7 @@ import org.apache.hadoop.hbase.filter.FilterList.Operator; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.RegionScanner; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Assert; @@ -217,7 +217,7 @@ public class TestFilter { @After public void tearDown() throws Exception { - HLog hlog = region.getLog(); + WALService hlog = region.getLog(); region.close(); hlog.closeAndDelete(); } @@ -1442,7 +1442,7 @@ public class TestFilter { assertEquals(2, resultCount); scanner.close(); - HLog hlog = testRegion.getLog(); + WALService hlog = testRegion.getLog(); testRegion.close(); hlog.closeAndDelete(); } @@ -2045,7 +2045,7 @@ public class TestFilter { results.clear(); } assertFalse(scanner.next(results)); - HLog hlog = testRegion.getLog(); + WALService hlog = testRegion.getLog(); testRegion.close(); hlog.closeAndDelete(); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java index 718f2fd..f8f7d16 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java @@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Assert; @@ -149,7 +149,7 @@ public class TestInvocationRecordFilter { @After public void tearDown() throws Exception { - HLog hlog = region.getLog(); + WALService hlog = region.getLog(); region.close(); hlog.closeAndDelete(); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java index ed8551c..e680966 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.mapreduce.HLogInputFormat.HLogRecordReader; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; @@ -112,7 +112,7 @@ public class TestHLogRecordReader { */ @Test public void testPartialRead() throws Exception { - HLog log = HLogFactory.createHLog(fs, hbaseDir, logName, conf); + WALService log = HLogFactory.createHLog(fs, hbaseDir, logName, conf); long ts = System.currentTimeMillis(); WALEdit edit = new WALEdit(); final AtomicLong sequenceId = new AtomicLong(0); @@ -161,7 +161,7 @@ public class TestHLogRecordReader { */ @Test public void testHLogRecordReader() throws Exception { - HLog log = HLogFactory.createHLog(fs, hbaseDir, logName, conf); + WALService log = HLogFactory.createHLog(fs, hbaseDir, logName, conf); byte [] value = Bytes.toBytes("value"); final AtomicLong sequenceId = new AtomicLong(0); WALEdit edit = new WALEdit(); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java index 309af73..0f29ca9 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java @@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.WALPlayer.HLogKeyValueMapper; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; @@ -105,7 +105,7 @@ public class TestWALPlayer { t1.delete(d); // replay the WAL, map table 1 to table 2 - HLog log = cluster.getRegionServer(0).getWAL(); + WALService log = cluster.getRegionServer(0).getWAL(); log.rollWriter(); String walInputDir = new Path(cluster.getMaster().getMasterFileSystem() .getRootDir(), HConstants.HREGION_LOGDIR_NAME).toString(); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java index 45b9885..49c0d20 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java @@ -92,7 +92,7 @@ import org.apache.hadoop.hbase.regionserver.Leases; import org.apache.hadoop.hbase.regionserver.RegionServerAccounting; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.ServerNonceManager; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -511,7 +511,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { } @Override - public HLog getWAL(HRegionInfo regionInfo) throws IOException { + public WALService getWAL(HRegionInfo regionInfo) throws IOException { // TODO Auto-generated method stub return null; } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java index e81da00..2beb6a0 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java @@ -86,7 +86,8 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WAL; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; @@ -1414,12 +1415,12 @@ public class TestDistributedLogSplitting { } } - public void makeHLog(HLog log, List regions, String tname, String fname, + public void makeHLog(WALService log, List regions, String tname, String fname, int num_edits, int edit_size) throws IOException { makeHLog(log, regions, tname, fname, num_edits, edit_size, true); } - public void makeHLog(HLog log, List regions, String tname, String fname, + public void makeHLog(WALService log, List regions, String tname, String fname, int num_edits, int edit_size, boolean closeLog) throws IOException { TableName fullTName = TableName.valueOf(tname); // remove root and meta region @@ -1483,7 +1484,7 @@ public class TestDistributedLogSplitting { private int countHLog(Path log, FileSystem fs, Configuration conf) throws IOException { int count = 0; - HLog.Reader in = HLogFactory.createReader(fs, log, conf); + WAL.Reader in = HLogFactory.createReader(fs, log, conf); while (in.next() != null) { count++; } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java index 9811fd7..5c6f78b 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java @@ -61,7 +61,7 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.io.HeapSize; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; import org.junit.Rule; @@ -589,7 +589,7 @@ public class TestAtomicOperation { public static class MockHRegion extends HRegion { - public MockHRegion(Path tableDir, HLog log, FileSystem fs, Configuration conf, + public MockHRegion(Path tableDir, WALService log, FileSystem fs, Configuration conf, final HRegionInfo regionInfo, final HTableDescriptor htd, RegionServerServices rsServices) { super(tableDir, log, fs, conf, regionInfo, htd, rsServices); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java index bf47874..dd45cf1 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java @@ -49,7 +49,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileBlock; import org.apache.hadoop.hbase.io.hfile.HFileReaderV2; import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.io.hfile.TestHFileWriterV2; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; @@ -124,7 +124,7 @@ public class TestCacheOnWriteInSchema { private final String testDescription; private HRegion region; private HStore store; - private HLog hlog; + private WALService hlog; private FileSystem fs; public TestCacheOnWriteInSchema(CacheOnWriteType cowType) { diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java index 5b72623..416bcd5 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java @@ -74,7 +74,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor; import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Threads; @@ -139,7 +139,7 @@ public class TestCompaction { @After public void tearDown() throws Exception { - HLog hlog = r.getLog(); + WALService hlog = r.getLog(); this.r.close(); hlog.closeAndDelete(); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java index 9a19524..31d972a 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java @@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.SmallTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; @@ -64,7 +64,7 @@ public class TestDefaultCompactSelection extends TestCase { protected static final long minSize = 10; protected static final long maxSize = 2100; - private HLog hlog; + private WALService hlog; private HRegion region; @Override diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index e518074..e1cca4f 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -110,7 +110,8 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor; import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl; import org.apache.hadoop.hbase.regionserver.HRegion.RowLock; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WAL; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; @@ -295,13 +296,13 @@ public class TestHRegion { for (long i = minSeqId; i <= maxSeqId; i += 10) { Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i)); fs.create(recoveredEdits); - HLog.Writer writer = HLogFactory.createRecoveredEditsWriter(fs, recoveredEdits, conf); + WAL.Writer writer = HLogFactory.createRecoveredEditsWriter(fs, recoveredEdits, conf); long time = System.nanoTime(); WALEdit edit = new WALEdit(); edit.add(new KeyValue(row, family, Bytes.toBytes(i), time, KeyValue.Type.Put, Bytes .toBytes(i))); - writer.append(new HLog.Entry(new HLogKey(regionName, tableName, i, time, + writer.append(new WAL.Entry(new HLogKey(regionName, tableName, i, time, HConstants.DEFAULT_CLUSTER_ID), edit)); writer.close(); @@ -345,13 +346,13 @@ public class TestHRegion { for (long i = minSeqId; i <= maxSeqId; i += 10) { Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i)); fs.create(recoveredEdits); - HLog.Writer writer = HLogFactory.createRecoveredEditsWriter(fs, recoveredEdits, conf); + WAL.Writer writer = HLogFactory.createRecoveredEditsWriter(fs, recoveredEdits, conf); long time = System.nanoTime(); WALEdit edit = new WALEdit(); edit.add(new KeyValue(row, family, Bytes.toBytes(i), time, KeyValue.Type.Put, Bytes .toBytes(i))); - writer.append(new HLog.Entry(new HLogKey(regionName, tableName, i, time, + writer.append(new WAL.Entry(new HLogKey(regionName, tableName, i, time, HConstants.DEFAULT_CLUSTER_ID), edit)); writer.close(); @@ -470,11 +471,11 @@ public class TestHRegion { Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", 1000)); fs.create(recoveredEdits); - HLog.Writer writer = HLogFactory.createRecoveredEditsWriter(fs, recoveredEdits, conf); + WAL.Writer writer = HLogFactory.createRecoveredEditsWriter(fs, recoveredEdits, conf); long time = System.nanoTime(); - writer.append(new HLog.Entry(new HLogKey(regionName, tableName, 10, time, + writer.append(new WAL.Entry(new HLogKey(regionName, tableName, 10, time, HConstants.DEFAULT_CLUSTER_ID), WALEdit.createCompaction(compactionDescriptor))); writer.close(); @@ -3882,8 +3883,8 @@ public class TestHRegion { TableName tableName = TableName.valueOf(method); byte[] family = Bytes.toBytes("family"); Path logDir = new Path(new Path(DIR + method), "log"); - HLog hlog = HLogFactory.createHLog(fs, logDir, UUID.randomUUID().toString(), conf); - final HLog log = spy(hlog); + WALService hlog = HLogFactory.createHLog(fs, logDir, UUID.randomUUID().toString(), conf); + final WALService log = spy(hlog); this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, method, conf, false, tableDurability, log, new byte[][] { family }); @@ -4085,7 +4086,7 @@ public class TestHRegion { */ private static HRegion initHRegion(byte[] tableName, byte[] startKey, byte[] stopKey, String callingMethod, Configuration conf, boolean isReadOnly, Durability durability, - HLog hlog, byte[]... families) throws IOException { + WALService hlog, byte[]... families) throws IOException { return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, callingMethod, conf, isReadOnly, durability, hlog, families); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java index 89bcac3..74186d5 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java @@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.util.Bytes; import org.apache.zookeeper.KeeperException; @@ -66,7 +66,7 @@ public class TestRegionMergeTransaction { private HRegion region_a; private HRegion region_b; private HRegion region_c; - private HLog wal; + private WALService wal; private FileSystem fs; // Start rows of region_a,region_b,region_c private static final byte[] STARTROW_A = new byte[] { 'a', 'a', 'a' }; @@ -392,7 +392,7 @@ public class TestRegionMergeTransaction { private class MockedFailedMergedRegionOpen extends IOException { } - private HRegion createRegion(final Path testdir, final HLog wal, + private HRegion createRegion(final Path testdir, final WALService wal, final byte[] startrow, final byte[] endrow) throws IOException { // Make a region with start and end keys. diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java index 9333039..380147e 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java @@ -47,7 +47,7 @@ import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.PairOfSameType; @@ -70,7 +70,7 @@ public class TestSplitTransaction { private final Path testdir = TEST_UTIL.getDataTestDir(this.getClass().getName()); private HRegion parent; - private HLog wal; + private WALService wal; private FileSystem fs; private static final byte [] STARTROW = new byte [] {'a', 'a', 'a'}; // '{' is next ascii after 'z'. @@ -327,7 +327,7 @@ public class TestSplitTransaction { return rowcount; } - HRegion createRegion(final Path testdir, final HLog wal) + HRegion createRegion(final Path testdir, final WALService wal) throws IOException { // Make a region with start and end keys. Use 'aaa', to 'AAA'. The load // region utility will add rows between 'aaa' and 'zzz'. diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java index 793b839..13ed155 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java @@ -65,7 +65,7 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; @@ -163,7 +163,7 @@ public class TestStore extends TestCase { htd.addFamily(hcd); HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); - HLog hlog = HLogFactory.createHLog(fs, basedir, logName, conf); + WALService hlog = HLogFactory.createHLog(fs, basedir, logName, conf); HRegion region = new HRegion(tableDir, hlog, fs, conf, info, htd, null); store = new HStore(region, hcd, conf); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java index 2164a43..82d6e66 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java @@ -23,8 +23,7 @@ import java.io.IOException; import java.util.LinkedList; import java.util.Queue; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.regionserver.wal.WAL.Entry; public class FaultySequenceFileLogReader extends SequenceFileLogReader { @@ -40,13 +39,13 @@ public class FaultySequenceFileLogReader extends SequenceFileLogReader { } @Override - public HLog.Entry next(HLog.Entry reuse) throws IOException { + public Entry next(Entry reuse) throws IOException { this.entryStart = this.getPosition(); boolean b = true; if (nextQueue.isEmpty()) { // Read the whole thing at once and fake reading while (b == true) { - HLog.Entry e = new HLog.Entry(new HLogKey(), new WALEdit()); + Entry e = new Entry(new HLogKey(), new WALEdit()); if (compressionContext != null) { e.setCompressionContext(compressionContext); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java index 4eec8d1..492bb6e 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java @@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry; +import org.apache.hadoop.hbase.regionserver.wal.WAL.Entry; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.util.Tool; @@ -60,7 +60,7 @@ import com.yammer.metrics.core.MetricsRegistry; import com.yammer.metrics.reporting.ConsoleReporter; /** - * This class runs performance benchmarks for {@link HLog}. + * This class runs performance benchmarks for {@link WALService}. * See usage for this tool by running: * $ hbase org.apache.hadoop.hbase.regionserver.wal.HLogPerformanceEvaluation -h */ @@ -119,7 +119,7 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool byte[] key = new byte[keySize]; byte[] value = new byte[valueSize]; Random rand = new Random(Thread.currentThread().getId()); - HLog hlog = region.getLog(); + WALService hlog = region.getLog(); ArrayList clusters = new ArrayList(); long nonce = HConstants.NO_NONCE; @@ -224,9 +224,9 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName()); conf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase"); conf.setClass("hbase.regionserver.hlog.reader.impl", SecureProtobufLogReader.class, - HLog.Reader.class); + WAL.Reader.class); conf.setClass("hbase.regionserver.hlog.writer.impl", SecureProtobufLogWriter.class, - HLog.Writer.class); + WAL.Writer.class); conf.setBoolean(HConstants.ENABLE_WAL_ENCRYPTION, true); conf.set(HConstants.CRYPTO_WAL_ALGORITHM_CONF_KEY, cipher); } @@ -245,42 +245,42 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool // Initialize Table Descriptor HTableDescriptor htd = createHTableDescriptor(numFamilies); final long whenToRoll = roll; - HLog hlog = new FSHLog(fs, rootRegionDir, "wals", getConf()) { + WALService hlog = new FSHLog(fs, rootRegionDir, "wals", getConf()) { int appends = 0; - long lastSync = 0; - - @Override - protected void doWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit, - HTableDescriptor htd) - throws IOException { - this.appends++; - if (this.appends % whenToRoll == 0) { - LOG.info("Rolling after " + appends + " edits"); - rollWriter(); - } - super.doWrite(info, logKey, logEdit, htd); - }; - - @Override - public void postSync() { - super.postSync(); - syncMeter.mark(); - long now = System.nanoTime(); - if (lastSync > 0) { - long diff = now - lastSync; - syncHistogram.update(diff); - } - this.lastSync = now; - } - - @Override - public void postAppend(List entries) { - super.postAppend(entries); - int size = 0; - for (Entry e: entries) size += e.getEdit().heapSize(); - appendMeter.mark(size); - } - }; + long lastSync = 0; + + @Override + protected void doWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit, + HTableDescriptor htd) + throws IOException { + this.appends++; + if (this.appends % whenToRoll == 0) { + LOG.info("Rolling after " + appends + " edits"); + rollWriter(); + } + super.doWrite(info, logKey, logEdit, htd); + }; + + @Override + public void postSync() { + super.postSync(); + syncMeter.mark(); + long now = System.nanoTime(); + if (lastSync > 0) { + long diff = now - lastSync; + syncHistogram.update(diff); + } + this.lastSync = now; + } + + @Override + public void postAppend(List entries) { + super.postAppend(entries); + int size = 0; + for (Entry e: entries) size += e.getEdit().heapSize(); + appendMeter.mark(size); + } + }; hlog.rollWriter(); HRegion region = null; try { @@ -341,7 +341,7 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool * @throws IOException */ private long verify(final Path wal, final boolean verbose) throws IOException { - HLog.Reader reader = HLogFactory.createReader(wal.getFileSystem(getConf()), wal, getConf()); + WAL.Reader reader = HLogFactory.createReader(wal.getFileSystem(getConf()), wal, getConf()); long count = 0; Map sequenceIds = new HashMap(); try { @@ -405,8 +405,8 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool System.exit(1); } - private HRegion openRegion(final FileSystem fs, final Path dir, final HTableDescriptor htd, final HLog hlog) - throws IOException { + private HRegion openRegion(final FileSystem fs, final Path dir, final HTableDescriptor htd, + final WALService hlog) throws IOException { // Initialize HRegion HRegionInfo regionInfo = new HRegionInfo(htd.getTableName()); return HRegion.createHRegion(regionInfo, dir, getConf(), htd, hlog); @@ -415,7 +415,7 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool private void closeRegion(final HRegion region) throws IOException { if (region != null) { region.close(); - HLog wal = region.getLog(); + WALService wal = region.getLog(); if (wal != null) wal.close(); } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtilsForTests.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtilsForTests.java index f2fcf80..7181736 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtilsForTests.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtilsForTests.java @@ -18,6 +18,7 @@ */ package org.apache.hadoop.hbase.regionserver.wal; +import org.apache.hadoop.fs.Path; /** * An Utility testcase that returns the number of log files that @@ -27,17 +28,34 @@ package org.apache.hadoop.hbase.regionserver.wal; * This is interesting for test only. */ public class HLogUtilsForTests { - + /** * * @param log * @return */ - public static int getNumRolledLogFiles(HLog log) { - return ((FSHLog) log).getNumRolledLogFiles(); + public static int getNumRolledLogFiles(WALService log) { + return ((AbstractWAL) log).getNumRolledLogFiles(); + } + + public static int getNumEntries(WALService log) { + return ((AbstractWAL) log).getNumEntries(); } - public static int getNumEntries(HLog log) { - return ((FSHLog) log).getNumEntries(); + /** + * A WAL file name is of the format: + * {@link WAL#WAL_FILE_NAME_DELIMITER}[.meta]. + * It returns the file create timestamp from the file name. + * @return the file number that is part of the WAL file name + */ + public static long extractFileNumFromPath(Path hlogName) { + if (hlogName == null) throw new IllegalArgumentException("The HLog path couldn't be null"); + String[] walPathStrs = null; + String hlogPath = hlogName.toString(); + // if it is a meta wal file, it would have a -meta prefix at the end. + boolean metaWAL = (hlogPath.endsWith(WAL.META_HLOG_FILE_EXTN)) ? true : false; + walPathStrs = hlogPath.split("\\" + WAL.WAL_FILE_NAME_DELIMITER); + if (metaWAL) return Long.parseLong(walPathStrs[walPathStrs.length - 2]); + return Long.parseLong(walPathStrs[walPathStrs.length - 1]); } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/InstrumentedSequenceFileLogWriter.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/InstrumentedSequenceFileLogWriter.java index d240e66..ee5a164 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/InstrumentedSequenceFileLogWriter.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/InstrumentedSequenceFileLogWriter.java @@ -30,7 +30,7 @@ public class InstrumentedSequenceFileLogWriter extends ProtobufLogWriter { public static boolean activateFailure = false; @Override - public void append(HLog.Entry entry) throws IOException { + public void append(WAL.Entry entry) throws IOException { super.append(entry); if (activateFailure && Bytes.equals(entry.getKey().getEncodedRegionName(), "break".getBytes())) { System.out.println(getClass().getName() + ": I will throw an exception now..."); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java index 206034e..78ef7db 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java @@ -41,7 +41,7 @@ import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; /** - * Implementation of {@link HLog.Writer} that delegates to + * Implementation of {@link WALService.Writer} that delegates to * SequenceFile.Writer. Legacy implementation only used for compat tests. */ @InterfaceAudience.Private @@ -162,7 +162,7 @@ public class SequenceFileLogWriter extends WriterBase { } @Override - public void append(HLog.Entry entry) throws IOException { + public void append(WAL.Entry entry) throws IOException { entry.setCompressionContext(compressionContext); try { this.writer.append(entry.getKey(), entry.getEdit()); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java index fd3c7d6..a46d7d9 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java @@ -76,7 +76,7 @@ public class TestDurability { @Test public void testDurability() throws Exception { - HLog wal = HLogFactory.createHLog(FS, DIR, "hlogdir", + WALService wal = HLogFactory.createHLog(FS, DIR, "hlogdir", "hlogdir_archive", CONF); byte[] tableName = Bytes.toBytes("TestDurability"); HRegion region = createHRegion(tableName, "region", wal, false); @@ -139,7 +139,7 @@ public class TestDurability { byte[] col3 = Bytes.toBytes("col3"); // Setting up region - HLog wal = HLogFactory.createHLog(FS, DIR, "myhlogdir", + WALService wal = HLogFactory.createHLog(FS, DIR, "myhlogdir", "myhlogdir_archive", CONF); byte[] tableName = Bytes.toBytes("TestIncrement"); HRegion region = createHRegion(tableName, "increment", wal, false); @@ -196,11 +196,11 @@ public class TestDurability { return p; } - private void verifyHLogCount(HLog log, int expected) throws Exception { - Path walPath = ((FSHLog) log).computeFilename(); - HLog.Reader reader = HLogFactory.createReader(FS, walPath, CONF); + private void verifyHLogCount(WALService log, int expected) throws Exception { + Path walPath = ((AbstractWAL) log).getCurrentFileName(); + WAL.Reader reader = HLogFactory.createReader(FS, walPath, CONF); int count = 0; - HLog.Entry entry = new HLog.Entry(); + WAL.Entry entry = new WAL.Entry(); while (reader.next(entry) != null) count++; reader.close(); assertEquals(expected, count); @@ -208,7 +208,7 @@ public class TestDurability { // lifted from TestAtomicOperation private HRegion createHRegion (byte [] tableName, String callingMethod, - HLog log, boolean isAsyncLogFlush) + WALService log, boolean isAsyncLogFlush) throws IOException { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); htd.setAsyncLogFlush(isAsyncLogFlush); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java new file mode 100644 index 0000000..1d35b40 --- /dev/null +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java @@ -0,0 +1,381 @@ +/** + * + * 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.regionserver.wal; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.LargeTests; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.util.FSUtils; +import org.junit.Test; +import org.junit.experimental.categories.Category; +/** + * Provides FSHLog test cases. Since FSHLog is the default WAL implementation, we can use the + * default configuration. + */ +@Category(LargeTests.class) +public class TestFSHLog extends TestHLog { + + @Test @Override + public void testLogCleaning() throws Exception { + LOG.info("testLogCleaning"); + final TableName tableName = + TableName.valueOf("testLogCleaning"); + final TableName tableName2 = + TableName.valueOf("testLogCleaning2"); + + FSHLog log = (FSHLog)HLogFactory.createHLog(fs, hbaseDir, + getName(), conf); + final AtomicLong sequenceId = new AtomicLong(1); + try { + HRegionInfo hri = new HRegionInfo(tableName, + HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); + HRegionInfo hri2 = new HRegionInfo(tableName2, + HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); + + // Add a single edit and make sure that rolling won't remove the file + // Before HBASE-3198 it used to delete it + addEdits(log, hri, tableName, 1, sequenceId); + log.rollWriter(); + assertEquals(1, log.getNumRolledLogFiles()); + + // See if there's anything wrong with more than 1 edit + addEdits(log, hri, tableName, 2, sequenceId); + log.rollWriter(); + assertEquals(2, log.getNumRolledLogFiles()); + + // Now mix edits from 2 regions, still no flushing + addEdits(log, hri, tableName, 1, sequenceId); + addEdits(log, hri2, tableName2, 1, sequenceId); + addEdits(log, hri, tableName, 1, sequenceId); + addEdits(log, hri2, tableName2, 1, sequenceId); + log.rollWriter(); + assertEquals(3, log.getNumRolledLogFiles()); + + // Flush the first region, we expect to see the first two files getting + // archived. We need to append something or writer won't be rolled. + addEdits(log, hri2, tableName2, 1, sequenceId); + log.startCacheFlush(hri.getEncodedNameAsBytes()); + log.completeCacheFlush(hri.getEncodedNameAsBytes()); + log.rollWriter(); + assertEquals(2, log.getNumRolledLogFiles()); + + // Flush the second region, which removes all the remaining output files + // since the oldest was completely flushed and the two others only contain + // flush information + addEdits(log, hri2, tableName2, 1, sequenceId); + log.startCacheFlush(hri2.getEncodedNameAsBytes()); + log.completeCacheFlush(hri2.getEncodedNameAsBytes()); + log.rollWriter(); + assertEquals(0, log.getNumRolledLogFiles()); + } finally { + if (log != null) log.closeAndDelete(); + } + } + + + /** + * tests the log comparator. Ensure that we are not mixing meta logs with non-meta logs (throws + * exception if we do). Comparison is based on the timestamp present in the wal name. + * @throws Exception + */ + @Test @Override + public void testHLogComparator() throws Exception { + FSHLog hlog1 = null; + FSHLog hlogMeta = null; + try { + hlog1 = (FSHLog) HLogFactory.createHLog(fs, FSUtils.getRootDir(conf), dir.toString(), conf); + LOG.debug("Log obtained is: " + hlog1); + Comparator comp = hlog1.LOG_NAME_COMPARATOR; + Path p1 = hlog1.computeFilename(11); + Path p2 = hlog1.computeFilename(12); + // comparing with itself returns 0 + assertTrue(comp.compare(p1, p1) == 0); + // comparing with different filenum. + assertTrue(comp.compare(p1, p2) < 0); + hlogMeta = (FSHLog) HLogFactory.createMetaHLog(fs, FSUtils.getRootDir(conf), dir.toString(), + conf, null, null); + Comparator compMeta = hlogMeta.LOG_NAME_COMPARATOR; + + Path p1WithMeta = hlogMeta.computeFilename(11); + Path p2WithMeta = hlogMeta.computeFilename(12); + assertTrue(compMeta.compare(p1WithMeta, p1WithMeta) == 0); + assertTrue(compMeta.compare(p1WithMeta, p2WithMeta) < 0); + // mixing meta and non-meta logs gives error + boolean ex = false; + try { + comp.compare(p1WithMeta, p2); + } catch (Exception e) { + ex = true; + } + assertTrue("Comparator doesn't complain while checking meta log files", ex); + boolean exMeta = false; + try { + compMeta.compare(p1WithMeta, p2); + } catch (Exception e) { + exMeta = true; + } + assertTrue("Meta comparator doesn't complain while checking log files", exMeta); + } finally { + if (hlog1 != null) hlog1.close(); + if (hlogMeta != null) hlogMeta.close(); + } + } + + /** + * Tests wal archiving by adding data, doing flushing/rolling and checking we archive old logs + * and also don't archive "live logs" (that is, a log with un-flushed entries). + *

+ * This is what it does: + * It creates two regions, and does a series of inserts along with log rolling. + * Whenever a WAL is rolled, HLogBase checks previous wals for archiving. A wal is eligible for + * archiving if for all the regions which have entries in that wal file, have flushed - past + * their maximum sequence id in that wal file. + *

+ * @throws IOException + */ + @Test @Override + public void testWALArchiving() throws IOException { + LOG.debug("testWALArchiving"); + TableName table1 = TableName.valueOf("t1"); + TableName table2 = TableName.valueOf("t2"); + FSHLog hlog = (FSHLog) HLogFactory.createHLog(fs, FSUtils.getRootDir(conf), dir.toString(), + conf); + try { + assertEquals(0, hlog.getNumRolledLogFiles()); + HRegionInfo hri1 = new HRegionInfo(table1, HConstants.EMPTY_START_ROW, + HConstants.EMPTY_END_ROW); + HRegionInfo hri2 = new HRegionInfo(table2, HConstants.EMPTY_START_ROW, + HConstants.EMPTY_END_ROW); + // ensure that we don't split the regions. + hri1.setSplit(false); + hri2.setSplit(false); + // variables to mock region sequenceIds. + final AtomicLong sequenceId1 = new AtomicLong(1); + final AtomicLong sequenceId2 = new AtomicLong(1); + // start with the testing logic: insert a waledit, and roll writer + addEdits(hlog, hri1, table1, 1, sequenceId1); + hlog.rollWriter(); + // assert that the wal is rolled + assertEquals(1, hlog.getNumRolledLogFiles()); + // add edits in the second wal file, and roll writer. + addEdits(hlog, hri1, table1, 1, sequenceId1); + hlog.rollWriter(); + // assert that the wal is rolled + assertEquals(2, hlog.getNumRolledLogFiles()); + // add a waledit to table1, and flush the region. + addEdits(hlog, hri1, table1, 3, sequenceId1); + flushRegion(hlog, hri1.getEncodedNameAsBytes()); + // roll log; all old logs should be archived. + hlog.rollWriter(); + assertEquals(0, hlog.getNumRolledLogFiles()); + // add an edit to table2, and roll writer + addEdits(hlog, hri2, table2, 1, sequenceId2); + hlog.rollWriter(); + assertEquals(1, hlog.getNumRolledLogFiles()); + // add edits for table1, and roll writer + addEdits(hlog, hri1, table1, 2, sequenceId1); + hlog.rollWriter(); + assertEquals(2, hlog.getNumRolledLogFiles()); + // add edits for table2, and flush hri1. + addEdits(hlog, hri2, table2, 2, sequenceId2); + flushRegion(hlog, hri1.getEncodedNameAsBytes()); + // the log : region-sequenceId map is + // log1: region2 (unflushed) + // log2: region1 (flushed) + // log3: region2 (unflushed) + // roll the writer; log2 should be archived. + hlog.rollWriter(); + assertEquals(2, hlog.getNumRolledLogFiles()); + // flush region2, and all logs should be archived. + addEdits(hlog, hri2, table2, 2, sequenceId2); + flushRegion(hlog, hri2.getEncodedNameAsBytes()); + hlog.rollWriter(); + assertEquals(0, hlog.getNumRolledLogFiles()); + } finally { + if (hlog != null) hlog.close(); + } + } + + /** + * On rolling a wal after reaching the threshold, {@link WALService#rollWriter()} returns the + * list of regions which should be flushed in order to archive the oldest wal file. + *

+ * This method tests this behavior by inserting edits and rolling the wal enough times to reach + * the max number of logs threshold. It checks whether we get the "right regions" for flush on + * rolling the wal. + * @throws Exception + */ + @Test @Override + public void testFindMemStoresEligibleForFlush() throws Exception { + LOG.debug("testFindMemStoresEligibleForFlush"); + Configuration conf1 = HBaseConfiguration.create(conf); + conf1.setInt("hbase.regionserver.maxlogs", 1); + FSHLog hlog = (FSHLog) HLogFactory.createHLog(fs, FSUtils.getRootDir(conf1), dir.toString(), + conf1); + TableName t1 = TableName.valueOf("t1"); + TableName t2 = TableName.valueOf("t2"); + HRegionInfo hri1 = new HRegionInfo(t1, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); + HRegionInfo hri2 = new HRegionInfo(t2, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); + // variables to mock region sequenceIds + final AtomicLong sequenceId1 = new AtomicLong(1); + final AtomicLong sequenceId2 = new AtomicLong(1); + // add edits and roll the wal + try { + addEdits(hlog, hri1, t1, 2, sequenceId1); + hlog.rollWriter(); + // add some more edits and roll the wal. This would reach the log number threshold + addEdits(hlog, hri1, t1, 2, sequenceId1); + hlog.rollWriter(); + // with above rollWriter call, the max logs limit is reached. + assertTrue(hlog.getNumRolledLogFiles() == 2); + + // get the regions to flush; since there is only one region in the oldest wal, it should + // return only one region. + byte[][] regionsToFlush = hlog.findRegionsToForceFlush(); + assertEquals(1, regionsToFlush.length); + assertEquals(hri1.getEncodedNameAsBytes(), regionsToFlush[0]); + // insert edits in second region + addEdits(hlog, hri2, t2, 2, sequenceId2); + // get the regions to flush, it should still read region1. + regionsToFlush = hlog.findRegionsToForceFlush(); + assertEquals(regionsToFlush.length, 1); + assertEquals(hri1.getEncodedNameAsBytes(), regionsToFlush[0]); + // flush region 1, and roll the wal file. Only last wal which has entries for region1 should + // remain. + flushRegion(hlog, hri1.getEncodedNameAsBytes()); + hlog.rollWriter(); + // only one wal should remain now (that is for the second region). + assertEquals(1, hlog.getNumRolledLogFiles()); + // flush the second region + flushRegion(hlog, hri2.getEncodedNameAsBytes()); + hlog.rollWriter(true); + // no wal should remain now. + assertEquals(0, hlog.getNumRolledLogFiles()); + // add edits both to region 1 and region 2, and roll. + addEdits(hlog, hri1, t1, 2, sequenceId1); + addEdits(hlog, hri2, t2, 2, sequenceId2); + hlog.rollWriter(); + // add edits and roll the writer, to reach the max logs limit. + assertEquals(1, hlog.getNumRolledLogFiles()); + addEdits(hlog, hri1, t1, 2, sequenceId1); + hlog.rollWriter(); + // it should return two regions to flush, as the oldest wal file has entries + // for both regions. + regionsToFlush = hlog.findRegionsToForceFlush(); + assertEquals(2, regionsToFlush.length); + // flush both regions + flushRegion(hlog, hri1.getEncodedNameAsBytes()); + flushRegion(hlog, hri2.getEncodedNameAsBytes()); + hlog.rollWriter(true); + assertEquals(0, hlog.getNumRolledLogFiles()); + // Add an edit to region1, and roll the wal. + addEdits(hlog, hri1, t1, 2, sequenceId1); + // tests partial flush: roll on a partial flush, and ensure that wal is not archived. + hlog.startCacheFlush(hri1.getEncodedNameAsBytes()); + hlog.rollWriter(); + hlog.completeCacheFlush(hri1.getEncodedNameAsBytes()); + assertEquals(1, hlog.getNumRolledLogFiles()); + } finally { + if (hlog != null) hlog.close(); + } + } + + /** + * Simulates HLog append ops for a region and tests + * {@link FSHLog#areAllRegionsFlushed(Map, Map, Map)} API. + * It compares the region sequenceIds with oldestFlushing and oldestUnFlushed entries. + * If a region's entries are larger than min of (oldestFlushing, oldestUnFlushed), then the + * region should be flushed before archiving this WAL. + */ + @Test + public void testAllRegionsFlushed() { + LOG.debug("testAllRegionsFlushed"); + Map oldestFlushingSeqNo = new HashMap(); + Map oldestUnFlushedSeqNo = new HashMap(); + Map seqNo = new HashMap(); + // create a table + TableName t1 = TableName.valueOf("t1"); + // create a region + HRegionInfo hri1 = new HRegionInfo(t1, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); + // variables to mock region sequenceIds + final AtomicLong sequenceId1 = new AtomicLong(1); + // test empty map + assertTrue(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo)); + // add entries in the region + seqNo.put(hri1.getEncodedNameAsBytes(), sequenceId1.incrementAndGet()); + oldestUnFlushedSeqNo.put(hri1.getEncodedNameAsBytes(), sequenceId1.get()); + // should say region1 is not flushed. + assertFalse(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo)); + // test with entries in oldestFlushing map. + oldestUnFlushedSeqNo.clear(); + oldestFlushingSeqNo.put(hri1.getEncodedNameAsBytes(), sequenceId1.get()); + assertFalse(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo)); + // simulate region flush, i.e., clear oldestFlushing and oldestUnflushed maps + oldestFlushingSeqNo.clear(); + oldestUnFlushedSeqNo.clear(); + assertTrue(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo)); + // insert some large values for region1 + oldestUnFlushedSeqNo.put(hri1.getEncodedNameAsBytes(), 1000l); + seqNo.put(hri1.getEncodedNameAsBytes(), 1500l); + assertFalse(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo)); + + // tests when oldestUnFlushed/oldestFlushing contains larger value. + // It means region is flushed. + oldestFlushingSeqNo.put(hri1.getEncodedNameAsBytes(), 1200l); + oldestUnFlushedSeqNo.clear(); + seqNo.put(hri1.getEncodedNameAsBytes(), 1199l); + assertTrue(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo)); + } + + @Test + public void testFailedToCreateHLogIfParentRenamed() throws IOException { + FSHLog log = (FSHLog)HLogFactory.createHLog( + fs, hbaseDir, "testFailedToCreateHLogIfParentRenamed", conf); + long filenum = System.currentTimeMillis(); + Path path = log.computeFilename(filenum); + HLogFactory.createWALWriter(fs, path, conf); + Path parent = path.getParent(); + path = log.computeFilename(filenum + 1); + Path newPath = new Path(parent.getParent(), parent.getName() + "-splitting"); + fs.rename(parent, newPath); + try { + HLogFactory.createWALWriter(fs, path, conf); + fail("It should fail to create the new WAL"); + } catch (IOException ioe) { + // expected, good. + } + } + + +} diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java index 94cfe69..8d08992 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java @@ -23,10 +23,7 @@ import static org.junit.Assert.*; import java.io.IOException; import java.lang.reflect.Method; import java.net.BindException; -import java.util.Comparator; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; @@ -39,7 +36,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader; +import org.apache.hadoop.hbase.regionserver.wal.WAL.Reader; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Threads; @@ -61,26 +58,32 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -/** JUnit test case for HLog */ +/** + * The base class to test WAL implementations. It has concrete test methods for basic WAL + * functionalities (such as append/sync data) which all WAL implementations should provide. + * It also provides abstract methods which are specific to a implementation (such as number of WAL + * files rolled, etc). All concrete WAL implementation should provide a test class with TestHLog + * as the base class. See {@link TestFSHLog} for example. + */ @Category(LargeTests.class) @SuppressWarnings("deprecation") -public class TestHLog { - private static final Log LOG = LogFactory.getLog(TestHLog.class); +public abstract class TestHLog { + protected static final Log LOG = LogFactory.getLog(TestHLog.class); { ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL); ((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL); ((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hdfs.server.namenode.FSNamesystem")) .getLogger().setLevel(Level.ALL); ((Log4JLogger)DFSClient.LOG).getLogger().setLevel(Level.ALL); - ((Log4JLogger)HLog.LOG).getLogger().setLevel(Level.ALL); + ((Log4JLogger)WALService.LOG).getLogger().setLevel(Level.ALL); } - private static Configuration conf; - private static FileSystem fs; - private static Path dir; + protected static Configuration conf; + protected static FileSystem fs; + protected static Path dir; private static MiniDFSCluster cluster; private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static Path hbaseDir; + protected static Path hbaseDir; private static Path oldLogDir; @Before @@ -132,7 +135,7 @@ public class TestHLog { TEST_UTIL.shutdownMiniCluster(); } - private static String getName() { + static String getName() { // TODO Auto-generated method stub return "TestHLog"; } @@ -163,7 +166,7 @@ public class TestHLog { TableName.valueOf(getName()); final byte [] rowName = tableName.getName(); Path logdir = new Path(hbaseDir, HConstants.HREGION_LOGDIR_NAME); - HLog log = HLogFactory.createHLog(fs, hbaseDir, + WALService log = HLogFactory.createHLog(fs, hbaseDir, HConstants.HREGION_LOGDIR_NAME, conf); final int howmany = 3; HRegionInfo[] infos = new HRegionInfo[3]; @@ -242,10 +245,10 @@ public class TestHLog { out.close(); in.close(); - HLog wal = HLogFactory.createHLog(fs, dir, "hlogdir", conf); + WALService wal = HLogFactory.createHLog(fs, dir, "hlogdir", conf); final AtomicLong sequenceId = new AtomicLong(1); final int total = 20; - HLog.Reader reader = null; + WAL.Reader reader = null; try { HRegionInfo info = new HRegionInfo(tableName, @@ -262,10 +265,10 @@ public class TestHLog { // gives you EOFE. wal.sync(); // Open a Reader. - Path walPath = ((FSHLog) wal).computeFilename(); + Path walPath = ((AbstractWAL) wal).getCurrentFileName(); reader = HLogFactory.createReader(fs, walPath, conf); int count = 0; - HLog.Entry entry = new HLog.Entry(); + WAL.Entry entry = new WAL.Entry(); while ((entry = reader.next(entry)) != null) count++; assertEquals(total, count); reader.close(); @@ -320,12 +323,12 @@ public class TestHLog { assertEquals(howmany * howmany, splits.size()); for (int i = 0; i < splits.size(); i++) { LOG.info("Verifying=" + splits.get(i)); - HLog.Reader reader = HLogFactory.createReader(fs, splits.get(i), conf); + WAL.Reader reader = HLogFactory.createReader(fs, splits.get(i), conf); try { int count = 0; String previousRegion = null; long seqno = -1; - HLog.Entry entry = new HLog.Entry(); + WAL.Entry entry = new WAL.Entry(); while((entry = reader.next(entry)) != null) { HLogKey key = entry.getKey(); String region = Bytes.toString(key.getEncodedRegionName()); @@ -362,7 +365,7 @@ public class TestHLog { HRegionInfo regioninfo = new HRegionInfo(tableName, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false); - HLog wal = HLogFactory.createHLog(fs, dir, "hlogdir", + WALService wal = HLogFactory.createHLog(fs, dir, "hlogdir", "hlogdir_archive", conf); final AtomicLong sequenceId = new AtomicLong(1); final int total = 20; @@ -378,7 +381,7 @@ public class TestHLog { // Now call sync to send the data to HDFS datanodes wal.sync(); int namenodePort = cluster.getNameNodePort(); - final Path walPath = ((FSHLog) wal).computeFilename(); + final Path walPath = ((AbstractWAL) wal).getCurrentFileName(); // Stop the cluster. (ensure restart since we're sharing MiniDFSCluster) @@ -458,9 +461,9 @@ public class TestHLog { throw t.exception; // Make sure you can read all the content - HLog.Reader reader = HLogFactory.createReader(fs, walPath, conf); + WAL.Reader reader = HLogFactory.createReader(fs, walPath, conf); int count = 0; - HLog.Entry entry = new HLog.Entry(); + WAL.Entry entry = new WAL.Entry(); while (reader.next(entry) != null) { count++; assertTrue("Should be one KeyValue per WALEdit", @@ -483,8 +486,8 @@ public class TestHLog { final TableName tableName = TableName.valueOf("tablename"); final byte [] row = Bytes.toBytes("row"); - HLog.Reader reader = null; - HLog log = null; + WAL.Reader reader = null; + WALService log = null; try { log = HLogFactory.createHLog(fs, hbaseDir, getName(), conf); final AtomicLong sequenceId = new AtomicLong(1); @@ -507,14 +510,14 @@ public class TestHLog { log.startCacheFlush(info.getEncodedNameAsBytes()); log.completeCacheFlush(info.getEncodedNameAsBytes()); log.close(); - Path filename = ((FSHLog) log).computeFilename(); + Path filename = ((AbstractWAL) log).getCurrentFileName(); log = null; // Now open a reader on the log and assert append worked. reader = HLogFactory.createReader(fs, filename, conf); // Above we added all columns on a single row so we only read one // entry in the below... thats why we have '1'. for (int i = 0; i < 1; i++) { - HLog.Entry entry = reader.next(null); + WAL.Entry entry = reader.next(null); if (entry == null) break; HLogKey key = entry.getKey(); WALEdit val = entry.getEdit(); @@ -545,7 +548,7 @@ public class TestHLog { TableName.valueOf("tablename"); final byte [] row = Bytes.toBytes("row"); Reader reader = null; - HLog log = HLogFactory.createHLog(fs, hbaseDir, getName(), conf); + WALService log = HLogFactory.createHLog(fs, hbaseDir, getName(), conf); final AtomicLong sequenceId = new AtomicLong(1); try { // Write columns named 1, 2, 3, etc. and then values of single byte @@ -565,11 +568,11 @@ public class TestHLog { log.startCacheFlush(hri.getEncodedNameAsBytes()); log.completeCacheFlush(hri.getEncodedNameAsBytes()); log.close(); - Path filename = ((FSHLog) log).computeFilename(); + Path filename = ((AbstractWAL) log).getCurrentFileName(); log = null; // Now open a reader on the log and assert append worked. reader = HLogFactory.createReader(fs, filename, conf); - HLog.Entry entry = reader.next(); + WAL.Entry entry = reader.next(); assertEquals(COL_COUNT, entry.getEdit().size()); int idx = 0; for (KeyValue val : entry.getEdit().getKeyValues()) { @@ -601,7 +604,7 @@ public class TestHLog { final TableName tableName = TableName.valueOf("tablename"); final byte [] row = Bytes.toBytes("row"); - HLog log = HLogFactory.createHLog(fs, hbaseDir, getName(), conf); + WALService log = HLogFactory.createHLog(fs, hbaseDir, getName(), conf); final AtomicLong sequenceId = new AtomicLong(1); try { DumbWALActionsListener visitor = new DumbWALActionsListener(); @@ -633,82 +636,6 @@ public class TestHLog { } @Test - public void testLogCleaning() throws Exception { - LOG.info("testLogCleaning"); - final TableName tableName = - TableName.valueOf("testLogCleaning"); - final TableName tableName2 = - TableName.valueOf("testLogCleaning2"); - - HLog log = HLogFactory.createHLog(fs, hbaseDir, - getName(), conf); - final AtomicLong sequenceId = new AtomicLong(1); - try { - HRegionInfo hri = new HRegionInfo(tableName, - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - HRegionInfo hri2 = new HRegionInfo(tableName2, - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - - // Add a single edit and make sure that rolling won't remove the file - // Before HBASE-3198 it used to delete it - addEdits(log, hri, tableName, 1, sequenceId); - log.rollWriter(); - assertEquals(1, ((FSHLog) log).getNumRolledLogFiles()); - - // See if there's anything wrong with more than 1 edit - addEdits(log, hri, tableName, 2, sequenceId); - log.rollWriter(); - assertEquals(2, ((FSHLog) log).getNumRolledLogFiles()); - - // Now mix edits from 2 regions, still no flushing - addEdits(log, hri, tableName, 1, sequenceId); - addEdits(log, hri2, tableName2, 1, sequenceId); - addEdits(log, hri, tableName, 1, sequenceId); - addEdits(log, hri2, tableName2, 1, sequenceId); - log.rollWriter(); - assertEquals(3, ((FSHLog) log).getNumRolledLogFiles()); - - // Flush the first region, we expect to see the first two files getting - // archived. We need to append something or writer won't be rolled. - addEdits(log, hri2, tableName2, 1, sequenceId); - log.startCacheFlush(hri.getEncodedNameAsBytes()); - log.completeCacheFlush(hri.getEncodedNameAsBytes()); - log.rollWriter(); - assertEquals(2, ((FSHLog) log).getNumRolledLogFiles()); - - // Flush the second region, which removes all the remaining output files - // since the oldest was completely flushed and the two others only contain - // flush information - addEdits(log, hri2, tableName2, 1, sequenceId); - log.startCacheFlush(hri2.getEncodedNameAsBytes()); - log.completeCacheFlush(hri2.getEncodedNameAsBytes()); - log.rollWriter(); - assertEquals(0, ((FSHLog) log).getNumRolledLogFiles()); - } finally { - if (log != null) log.closeAndDelete(); - } - } - - @Test - public void testFailedToCreateHLogIfParentRenamed() throws IOException { - FSHLog log = (FSHLog)HLogFactory.createHLog( - fs, hbaseDir, "testFailedToCreateHLogIfParentRenamed", conf); - long filenum = System.currentTimeMillis(); - Path path = log.computeFilename(filenum); - HLogFactory.createWALWriter(fs, path, conf); - Path parent = path.getParent(); - path = log.computeFilename(filenum + 1); - Path newPath = new Path(parent.getParent(), parent.getName() + "-splitting"); - fs.rename(parent, newPath); - try { - HLogFactory.createWALWriter(fs, path, conf); - fail("It should fail to create the new WAL"); - } catch (IOException ioe) { - // expected, good. - } - } - - @Test public void testGetServerNameFromHLogDirectoryName() throws IOException { ServerName sn = ServerName.valueOf("hn", 450, 1398); String hl = FSUtils.getRootDir(conf) + "/" + HLogUtil.getHLogDirectoryName(sn.toString()); @@ -744,7 +671,7 @@ public class TestHLog { @Test public void testWALCoprocessorLoaded() throws Exception { // test to see whether the coprocessor is loaded or not. - HLog log = HLogFactory.createHLog(fs, hbaseDir, + WALService log = HLogFactory.createHLog(fs, hbaseDir, getName(), conf); try { WALCoprocessorHost host = log.getCoprocessorHost(); @@ -755,7 +682,7 @@ public class TestHLog { } } - private void addEdits(HLog log, HRegionInfo hri, TableName tableName, + protected void addEdits(WALService log, HRegionInfo hri, TableName tableName, int times, AtomicLong sequenceId) throws IOException { HTableDescriptor htd = new HTableDescriptor(); htd.addFamily(new HColumnDescriptor("row")); @@ -783,7 +710,7 @@ public class TestHLog { long timestamp = System.currentTimeMillis(); Path path = new Path(dir, "temphlog"); SequenceFileLogWriter sflw = null; - HLog.Reader reader = null; + WAL.Reader reader = null; try { HRegionInfo hri = new HRegionInfo(tableName, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); @@ -803,7 +730,7 @@ public class TestHLog { String value = i + "" + j; edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value))); } - sflw.append(new HLog.Entry(key, edit)); + sflw.append(new WAL.Entry(key, edit)); } sflw.sync(); sflw.close(); @@ -812,7 +739,7 @@ public class TestHLog { reader = HLogFactory.createReader(fs, path, conf); assertTrue(reader instanceof SequenceFileLogReader); for (int i = 0; i < recordCount; ++i) { - HLog.Entry entry = reader.next(); + WAL.Entry entry = reader.next(); assertNotNull(entry); assertEquals(columnCount, entry.getEdit().size()); assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName()); @@ -825,7 +752,7 @@ public class TestHLog { idx++; } } - HLog.Entry entry = reader.next(); + WAL.Entry entry = reader.next(); assertNull(entry); } finally { if (sflw != null) { @@ -868,8 +795,8 @@ public class TestHLog { Path path = new Path(dir, "temphlog"); // delete the log if already exists, for test only fs.delete(path, true); - HLog.Writer writer = null; - HLog.Reader reader = null; + WAL.Writer writer = null; + WAL.Reader reader = null; try { HRegionInfo hri = new HRegionInfo(tableName, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); @@ -888,7 +815,7 @@ public class TestHLog { String value = i + "" + j; edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value))); } - writer.append(new HLog.Entry(key, edit)); + writer.append(new WAL.Entry(key, edit)); } writer.sync(); if (withTrailer) writer.close(); @@ -902,7 +829,7 @@ public class TestHLog { assertNull(reader.getWALTrailer()); } for (int i = 0; i < recordCount; ++i) { - HLog.Entry entry = reader.next(); + WAL.Entry entry = reader.next(); assertNotNull(entry); assertEquals(columnCount, entry.getEdit().size()); assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName()); @@ -915,7 +842,7 @@ public class TestHLog { idx++; } } - HLog.Entry entry = reader.next(); + WAL.Entry entry = reader.next(); assertNull(entry); } finally { if (writer != null) { @@ -927,264 +854,46 @@ public class TestHLog { } } + // Impl specific test methods /** - * tests the log comparator. Ensure that we are not mixing meta logs with non-meta logs (throws - * exception if we do). Comparison is based on the timestamp present in the wal name. + * Tests how the logs are cleaned with rolling, etc. * @throws Exception */ - @Test - public void testHLogComparator() throws Exception { - HLog hlog1 = null; - HLog hlogMeta = null; - try { - hlog1 = HLogFactory.createHLog(fs, FSUtils.getRootDir(conf), dir.toString(), conf); - LOG.debug("Log obtained is: " + hlog1); - Comparator comp = ((FSHLog) hlog1).LOG_NAME_COMPARATOR; - Path p1 = ((FSHLog) hlog1).computeFilename(11); - Path p2 = ((FSHLog) hlog1).computeFilename(12); - // comparing with itself returns 0 - assertTrue(comp.compare(p1, p1) == 0); - // comparing with different filenum. - assertTrue(comp.compare(p1, p2) < 0); - hlogMeta = HLogFactory.createMetaHLog(fs, FSUtils.getRootDir(conf), dir.toString(), conf, - null, null); - Comparator compMeta = ((FSHLog) hlogMeta).LOG_NAME_COMPARATOR; - - Path p1WithMeta = ((FSHLog) hlogMeta).computeFilename(11); - Path p2WithMeta = ((FSHLog) hlogMeta).computeFilename(12); - assertTrue(compMeta.compare(p1WithMeta, p1WithMeta) == 0); - assertTrue(compMeta.compare(p1WithMeta, p2WithMeta) < 0); - // mixing meta and non-meta logs gives error - boolean ex = false; - try { - comp.compare(p1WithMeta, p2); - } catch (Exception e) { - ex = true; - } - assertTrue("Comparator doesn't complain while checking meta log files", ex); - boolean exMeta = false; - try { - compMeta.compare(p1WithMeta, p2); - } catch (Exception e) { - exMeta = true; - } - assertTrue("Meta comparator doesn't complain while checking log files", exMeta); - } finally { - if (hlog1 != null) hlog1.close(); - if (hlogMeta != null) hlogMeta.close(); - } - } + abstract void testLogCleaning() throws Exception; /** - * Tests wal archiving by adding data, doing flushing/rolling and checking we archive old logs - * and also don't archive "live logs" (that is, a log with un-flushed entries). - *

- * This is what it does: - * It creates two regions, and does a series of inserts along with log rolling. - * Whenever a WAL is rolled, FSHLog checks previous wals for archiving. A wal is eligible for - * archiving if for all the regions which have entries in that wal file, have flushed - past - * their maximum sequence id in that wal file. - *

+ * Tests the log file comparator, used to sort the log files. + * @throws Exception + */ + abstract void testHLogComparator() throws Exception; + + /** + * Tests how the WAL archiving scheme is working. * @throws IOException */ - @Test - public void testWALArchiving() throws IOException { - LOG.debug("testWALArchiving"); - TableName table1 = TableName.valueOf("t1"); - TableName table2 = TableName.valueOf("t2"); - HLog hlog = HLogFactory.createHLog(fs, FSUtils.getRootDir(conf), dir.toString(), conf); - try { - assertEquals(0, ((FSHLog) hlog).getNumRolledLogFiles()); - HRegionInfo hri1 = new HRegionInfo(table1, HConstants.EMPTY_START_ROW, - HConstants.EMPTY_END_ROW); - HRegionInfo hri2 = new HRegionInfo(table2, HConstants.EMPTY_START_ROW, - HConstants.EMPTY_END_ROW); - // ensure that we don't split the regions. - hri1.setSplit(false); - hri2.setSplit(false); - // variables to mock region sequenceIds. - final AtomicLong sequenceId1 = new AtomicLong(1); - final AtomicLong sequenceId2 = new AtomicLong(1); - // start with the testing logic: insert a waledit, and roll writer - addEdits(hlog, hri1, table1, 1, sequenceId1); - hlog.rollWriter(); - // assert that the wal is rolled - assertEquals(1, ((FSHLog) hlog).getNumRolledLogFiles()); - // add edits in the second wal file, and roll writer. - addEdits(hlog, hri1, table1, 1, sequenceId1); - hlog.rollWriter(); - // assert that the wal is rolled - assertEquals(2, ((FSHLog) hlog).getNumRolledLogFiles()); - // add a waledit to table1, and flush the region. - addEdits(hlog, hri1, table1, 3, sequenceId1); - flushRegion(hlog, hri1.getEncodedNameAsBytes()); - // roll log; all old logs should be archived. - hlog.rollWriter(); - assertEquals(0, ((FSHLog) hlog).getNumRolledLogFiles()); - // add an edit to table2, and roll writer - addEdits(hlog, hri2, table2, 1, sequenceId2); - hlog.rollWriter(); - assertEquals(1, ((FSHLog) hlog).getNumRolledLogFiles()); - // add edits for table1, and roll writer - addEdits(hlog, hri1, table1, 2, sequenceId1); - hlog.rollWriter(); - assertEquals(2, ((FSHLog) hlog).getNumRolledLogFiles()); - // add edits for table2, and flush hri1. - addEdits(hlog, hri2, table2, 2, sequenceId2); - flushRegion(hlog, hri1.getEncodedNameAsBytes()); - // the log : region-sequenceId map is - // log1: region2 (unflushed) - // log2: region1 (flushed) - // log3: region2 (unflushed) - // roll the writer; log2 should be archived. - hlog.rollWriter(); - assertEquals(2, ((FSHLog) hlog).getNumRolledLogFiles()); - // flush region2, and all logs should be archived. - addEdits(hlog, hri2, table2, 2, sequenceId2); - flushRegion(hlog, hri2.getEncodedNameAsBytes()); - hlog.rollWriter(); - assertEquals(0, ((FSHLog) hlog).getNumRolledLogFiles()); - } finally { - if (hlog != null) hlog.close(); - } - } + abstract void testWALArchiving() throws IOException; /** - * On rolling a wal after reaching the threshold, {@link HLog#rollWriter()} returns the list of - * regions which should be flushed in order to archive the oldest wal file. - *

- * This method tests this behavior by inserting edits and rolling the wal enough times to reach - * the max number of logs threshold. It checks whether we get the "right regions" for flush on - * rolling the wal. - * @throws Exception + * Tests that the impl returns correct memstores to flush when looking at the old wal. + * When the number of WALs reached value defined by hbase.regionserver.maxlogs, the log + * should try to flush regions so that older WAL files become eligible for archiving. Tests + * whether the implementation is returning the right regions to flush. */ - @Test - public void testFindMemStoresEligibleForFlush() throws Exception { - LOG.debug("testFindMemStoresEligibleForFlush"); - Configuration conf1 = HBaseConfiguration.create(conf); - conf1.setInt("hbase.regionserver.maxlogs", 1); - HLog hlog = HLogFactory.createHLog(fs, FSUtils.getRootDir(conf1), dir.toString(), conf1); - TableName t1 = TableName.valueOf("t1"); - TableName t2 = TableName.valueOf("t2"); - HRegionInfo hri1 = new HRegionInfo(t1, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - HRegionInfo hri2 = new HRegionInfo(t2, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - // variables to mock region sequenceIds - final AtomicLong sequenceId1 = new AtomicLong(1); - final AtomicLong sequenceId2 = new AtomicLong(1); - // add edits and roll the wal - try { - addEdits(hlog, hri1, t1, 2, sequenceId1); - hlog.rollWriter(); - // add some more edits and roll the wal. This would reach the log number threshold - addEdits(hlog, hri1, t1, 2, sequenceId1); - hlog.rollWriter(); - // with above rollWriter call, the max logs limit is reached. - assertTrue(((FSHLog) hlog).getNumRolledLogFiles() == 2); - - // get the regions to flush; since there is only one region in the oldest wal, it should - // return only one region. - byte[][] regionsToFlush = ((FSHLog) hlog).findRegionsToForceFlush(); - assertEquals(1, regionsToFlush.length); - assertEquals(hri1.getEncodedNameAsBytes(), regionsToFlush[0]); - // insert edits in second region - addEdits(hlog, hri2, t2, 2, sequenceId2); - // get the regions to flush, it should still read region1. - regionsToFlush = ((FSHLog) hlog).findRegionsToForceFlush(); - assertEquals(regionsToFlush.length, 1); - assertEquals(hri1.getEncodedNameAsBytes(), regionsToFlush[0]); - // flush region 1, and roll the wal file. Only last wal which has entries for region1 should - // remain. - flushRegion(hlog, hri1.getEncodedNameAsBytes()); - hlog.rollWriter(); - // only one wal should remain now (that is for the second region). - assertEquals(1, ((FSHLog) hlog).getNumRolledLogFiles()); - // flush the second region - flushRegion(hlog, hri2.getEncodedNameAsBytes()); - hlog.rollWriter(true); - // no wal should remain now. - assertEquals(0, ((FSHLog) hlog).getNumRolledLogFiles()); - // add edits both to region 1 and region 2, and roll. - addEdits(hlog, hri1, t1, 2, sequenceId1); - addEdits(hlog, hri2, t2, 2, sequenceId2); - hlog.rollWriter(); - // add edits and roll the writer, to reach the max logs limit. - assertEquals(1, ((FSHLog) hlog).getNumRolledLogFiles()); - addEdits(hlog, hri1, t1, 2, sequenceId1); - hlog.rollWriter(); - // it should return two regions to flush, as the oldest wal file has entries - // for both regions. - regionsToFlush = ((FSHLog) hlog).findRegionsToForceFlush(); - assertEquals(2, regionsToFlush.length); - // flush both regions - flushRegion(hlog, hri1.getEncodedNameAsBytes()); - flushRegion(hlog, hri2.getEncodedNameAsBytes()); - hlog.rollWriter(true); - assertEquals(0, ((FSHLog) hlog).getNumRolledLogFiles()); - // Add an edit to region1, and roll the wal. - addEdits(hlog, hri1, t1, 2, sequenceId1); - // tests partial flush: roll on a partial flush, and ensure that wal is not archived. - hlog.startCacheFlush(hri1.getEncodedNameAsBytes()); - hlog.rollWriter(); - hlog.completeCacheFlush(hri1.getEncodedNameAsBytes()); - assertEquals(1, ((FSHLog) hlog).getNumRolledLogFiles()); - } finally { - if (hlog != null) hlog.close(); - } - } + abstract void testFindMemStoresEligibleForFlush() throws Exception; /** - * Simulates HLog append ops for a region and tests - * {@link FSHLog#areAllRegionsFlushed(Map, Map, Map)} API. - * It compares the region sequenceIds with oldestFlushing and oldestUnFlushed entries. - * If a region's entries are larger than min of (oldestFlushing, oldestUnFlushed), then the - * region should be flushed before archiving this WAL. - */ + * Tests IO fencing. + * @throws IOException + */ @Test - public void testAllRegionsFlushed() { - LOG.debug("testAllRegionsFlushed"); - Map oldestFlushingSeqNo = new HashMap(); - Map oldestUnFlushedSeqNo = new HashMap(); - Map seqNo = new HashMap(); - // create a table - TableName t1 = TableName.valueOf("t1"); - // create a region - HRegionInfo hri1 = new HRegionInfo(t1, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - // variables to mock region sequenceIds - final AtomicLong sequenceId1 = new AtomicLong(1); - // test empty map - assertTrue(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo)); - // add entries in the region - seqNo.put(hri1.getEncodedNameAsBytes(), sequenceId1.incrementAndGet()); - oldestUnFlushedSeqNo.put(hri1.getEncodedNameAsBytes(), sequenceId1.get()); - // should say region1 is not flushed. - assertFalse(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo)); - // test with entries in oldestFlushing map. - oldestUnFlushedSeqNo.clear(); - oldestFlushingSeqNo.put(hri1.getEncodedNameAsBytes(), sequenceId1.get()); - assertFalse(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo)); - // simulate region flush, i.e., clear oldestFlushing and oldestUnflushed maps - oldestFlushingSeqNo.clear(); - oldestUnFlushedSeqNo.clear(); - assertTrue(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo)); - // insert some large values for region1 - oldestUnFlushedSeqNo.put(hri1.getEncodedNameAsBytes(), 1000l); - seqNo.put(hri1.getEncodedNameAsBytes(), 1500l); - assertFalse(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo)); - - // tests when oldestUnFlushed/oldestFlushing contains larger value. - // It means region is flushed. - oldestFlushingSeqNo.put(hri1.getEncodedNameAsBytes(), 1200l); - oldestUnFlushedSeqNo.clear(); - seqNo.put(hri1.getEncodedNameAsBytes(), 1199l); - assertTrue(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo)); - } + public abstract void testFailedToCreateHLogIfParentRenamed() throws IOException; /** * helper method to simulate region flush for a WAL. * @param hlog * @param regionEncodedName */ - private void flushRegion(HLog hlog, byte[] regionEncodedName) { + protected void flushRegion(WALService hlog, byte[] regionEncodedName) { hlog.startCacheFlush(regionEncodedName); hlog.completeCacheFlush(regionEncodedName); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogMethods.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogMethods.java index 5b68f9f..68f4ff6 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogMethods.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogMethods.java @@ -116,7 +116,7 @@ public class TestHLogMethods { EntryBuffers sink = splitter.new EntryBuffers(1*1024*1024); for (int i = 0; i < 1000; i++) { - HLog.Entry entry = createTestLogEntry(i); + WAL.Entry entry = createTestLogEntry(i); sink.appendEntry(entry); } @@ -131,7 +131,7 @@ public class TestHLogMethods { // Insert some more entries for (int i = 0; i < 500; i++) { - HLog.Entry entry = createTestLogEntry(i); + WAL.Entry entry = createTestLogEntry(i); sink.appendEntry(entry); } // Asking for another chunk shouldn't work since the first one @@ -153,7 +153,7 @@ public class TestHLogMethods { assertEquals(0, sink.totalBuffered); } - private HLog.Entry createTestLogEntry(int i) { + private WAL.Entry createTestLogEntry(int i) { long seq = i; long now = i * 1000; @@ -161,7 +161,7 @@ public class TestHLogMethods { edit.add(KeyValueTestUtil.create("row", "fam", "qual", 1234, "val")); HLogKey key = new HLogKey(TEST_REGION, TEST_TABLE, seq, now, HConstants.DEFAULT_CLUSTER_ID); - HLog.Entry entry = new HLog.Entry(key, edit); + WAL.Entry entry = new WAL.Entry(key, edit); return entry; } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java index 57a2549..1113ad1 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java @@ -63,8 +63,8 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.LargeTests; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader; +import org.apache.hadoop.hbase.regionserver.wal.WAL.Entry; +import org.apache.hadoop.hbase.regionserver.wal.WAL.Reader; import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.CorruptedLogFileException; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; @@ -91,7 +91,7 @@ import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; /** - * Testing {@link HLog} splitting code. + * Testing {@link WALService} splitting code. */ @Category(LargeTests.class) public class TestHLogSplit { @@ -140,7 +140,7 @@ public class TestHLogSplit { public static void setUpBeforeClass() throws Exception { FSUtils.setRootDir(TEST_UTIL.getConfiguration(), HBASEDIR); TEST_UTIL.getConfiguration().setClass("hbase.regionserver.hlog.writer.impl", - InstrumentedSequenceFileLogWriter.class, HLog.Writer.class); + InstrumentedSequenceFileLogWriter.class, WAL.Writer.class); TEST_UTIL.getConfiguration().setBoolean("dfs.support.broken.append", true); TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true); // This is how you turn off shortcircuit read currently. TODO: Fix. Should read config. @@ -280,7 +280,7 @@ public class TestHLogSplit { final int numOfWriters = walToKeepOpen + 1; // The below method writes numOfWriters files each with ENTRIES entries for a total of numOfWriters * ENTRIES // added per column family in the region. - HLog.Writer[] writers = null; + WAL.Writer[] writers = null; try { DistributedFileSystem dfs = (DistributedFileSystem)FileSystem.get(conf); writers = generateHLogs(dfs, numOfWriters, ENTRIES, walToKeepOpen); @@ -290,14 +290,14 @@ public class TestHLogSplit { // Update counter so has all edits written so far. editsCount.addAndGet(numOfWriters * NUM_WRITERS); // This WAL should be open still after our call to generateHLogs -- we asked it leave it open. - HLog.Writer writer = writers[walToKeepOpen]; + WAL.Writer writer = writers[walToKeepOpen]; loop(writer); return null; } }); } - private void loop(final HLog.Writer writer) { + private void loop(final WAL.Writer writer) { byte [] regionBytes = Bytes.toBytes(this.region); while (true) { try { @@ -343,8 +343,8 @@ public class TestHLogSplit { HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); fs.mkdirs(regiondir); long now = System.currentTimeMillis(); - HLog.Entry entry = - new HLog.Entry(new HLogKey(encoded, + WAL.Entry entry = + new WAL.Entry(new HLogKey(encoded, TableName.META_TABLE_NAME, 1, now, HConstants.DEFAULT_CLUSTER_ID), new WALEdit()); Path p = HLogSplitter.getRegionSplitEditsPath(fs, entry, HBASEDIR, true); @@ -365,8 +365,8 @@ public class TestHLogSplit { HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); fs.mkdirs(regiondir); long now = System.currentTimeMillis(); - HLog.Entry entry = - new HLog.Entry(new HLogKey(encoded, + WAL.Entry entry = + new WAL.Entry(new HLogKey(encoded, TableName.META_TABLE_NAME, 1, now, HConstants.DEFAULT_CLUSTER_ID), new WALEdit()); Path parent = HLogUtil.getRegionDirRecoveredEditsDir(regiondir); @@ -545,7 +545,7 @@ public class TestHLogSplit { try { Path c1 = new Path(HLOGDIR, HLOG_FILE_PREFIX + "0"); conf.setClass("hbase.regionserver.hlog.reader.impl", - FaultySequenceFileLogReader.class, HLog.Reader.class); + FaultySequenceFileLogReader.class, WAL.Reader.class); for (FaultySequenceFileLogReader.FailureType failureType : FaultySequenceFileLogReader.FailureType.values()) { conf.set("faultysequencefilelogreader.failuretype", failureType.name()); generateHLogs(1, ENTRIES, -1); @@ -575,7 +575,7 @@ public class TestHLogSplit { try { conf.setClass("hbase.regionserver.hlog.reader.impl", - FaultySequenceFileLogReader.class, HLog.Reader.class); + FaultySequenceFileLogReader.class, WAL.Reader.class); conf.set("faultysequencefilelogreader.failuretype", FaultySequenceFileLogReader.FailureType.BEGINNING.name()); generateHLogs(Integer.MAX_VALUE); fs.initialize(fs.getUri(), conf); @@ -598,7 +598,7 @@ public class TestHLogSplit { try { conf.setClass("hbase.regionserver.hlog.reader.impl", - FaultySequenceFileLogReader.class, HLog.Reader.class); + FaultySequenceFileLogReader.class, WAL.Reader.class); conf.set("faultysequencefilelogreader.failuretype", FaultySequenceFileLogReader.FailureType.BEGINNING.name()); generateHLogs(-1); fs.initialize(fs.getUri(), conf); @@ -636,9 +636,9 @@ public class TestHLogSplit { assertEquals(1, splitLog.length); int actualCount = 0; - HLog.Reader in = HLogFactory.createReader(fs, splitLog[0], conf); + WAL.Reader in = HLogFactory.createReader(fs, splitLog[0], conf); @SuppressWarnings("unused") - HLog.Entry entry; + WAL.Entry entry; while ((entry = in.next()) != null) ++actualCount; assertEquals(entryCount-1, actualCount); @@ -667,9 +667,9 @@ public class TestHLogSplit { assertEquals(1, splitLog.length); int actualCount = 0; - HLog.Reader in = HLogFactory.createReader(fs, splitLog[0], conf); + WAL.Reader in = HLogFactory.createReader(fs, splitLog[0], conf); @SuppressWarnings("unused") - HLog.Entry entry; + WAL.Entry entry; while ((entry = in.next()) != null) ++actualCount; assertEquals(entryCount, actualCount); @@ -727,7 +727,7 @@ public class TestHLogSplit { @Test(timeout=300000, expected = IOException.class) public void testSplitWillFailIfWritingToRegionFails() throws Exception { //leave 5th log open so we could append the "trap" - HLog.Writer [] writer = generateHLogs(4); + WAL.Writer [] writer = generateHLogs(4); fs.initialize(fs.getUri(), conf); @@ -806,11 +806,11 @@ public class TestHLogSplit { // Set up a splitter that will throw an IOE on the output side HLogSplitter logSplitter = new HLogSplitter( conf, HBASEDIR, fs, null, null) { - protected HLog.Writer createWriter(FileSystem fs, + protected WAL.Writer createWriter(FileSystem fs, Path logfile, Configuration conf) throws IOException { - HLog.Writer mockWriter = Mockito.mock(HLog.Writer.class); + WAL.Writer mockWriter = Mockito.mock(WAL.Writer.class); Mockito.doThrow(new IOException("Injected")).when( - mockWriter).append(Mockito.any()); + mockWriter).append(Mockito.any()); return mockWriter; } }; @@ -1000,9 +1000,9 @@ public class TestHLogSplit { localConf, HBASEDIR, fs, null, null) { /* Produce a mock writer that doesn't write anywhere */ - protected HLog.Writer createWriter(FileSystem fs, Path logfile, Configuration conf) + protected WAL.Writer createWriter(FileSystem fs, Path logfile, Configuration conf) throws IOException { - HLog.Writer mockWriter = Mockito.mock(HLog.Writer.class); + WAL.Writer mockWriter = Mockito.mock(WAL.Writer.class); Mockito.doAnswer(new Answer() { int expectedIndex = 0; @@ -1015,7 +1015,7 @@ public class TestHLogSplit { Thread.currentThread().interrupt(); } } - HLog.Entry entry = (Entry) invocation.getArguments()[0]; + WAL.Entry entry = (Entry) invocation.getArguments()[0]; WALEdit edit = entry.getEdit(); List keyValues = edit.getKeyValues(); assertEquals(1, keyValues.size()); @@ -1026,7 +1026,7 @@ public class TestHLogSplit { expectedIndex++; return null; } - }).when(mockWriter).append(Mockito.any()); + }).when(mockWriter).append(Mockito.any()); return mockWriter; } @@ -1034,18 +1034,18 @@ public class TestHLogSplit { protected Reader getReader(FileSystem fs, Path curLogFile, Configuration conf, CancelableProgressable reporter) throws IOException { Reader mockReader = Mockito.mock(Reader.class); - Mockito.doAnswer(new Answer() { + Mockito.doAnswer(new Answer() { int index = 0; @Override - public HLog.Entry answer(InvocationOnMock invocation) throws Throwable { + public WAL.Entry answer(InvocationOnMock invocation) throws Throwable { if (index >= numFakeEdits) return null; // Generate r0 through r4 in round robin fashion int regionIdx = index % regions.size(); byte region[] = new byte[] {(byte)'r', (byte) (0x30 + regionIdx)}; - HLog.Entry ret = createTestEntry(TABLE_NAME, region, + WAL.Entry ret = createTestEntry(TABLE_NAME, region, Bytes.toBytes((int)(index / regions.size())), FAMILY, QUALIFIER, VALUE, index); index++; @@ -1074,7 +1074,7 @@ public class TestHLogSplit { @Test (timeout=300000) @Ignore("Need HADOOP-6886, HADOOP-6840, & HDFS-617 for this. HDFS 0.20.205.1+ should have this") public void testLogRollAfterSplitStart() throws IOException { - HLog log = null; + AbstractWAL log = null; String logName = "testLogRollAfterSplitStart"; Path thisTestsDir = new Path(HBASEDIR, logName); @@ -1084,7 +1084,7 @@ public class TestHLogSplit { TableName.valueOf(this.getClass().getName()); HRegionInfo regioninfo = new HRegionInfo(tableName, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - log = HLogFactory.createHLog(fs, HBASEDIR, logName, conf); + log = (AbstractWAL) HLogFactory.createHLog(fs, HBASEDIR, logName, conf); final AtomicLong sequenceId = new AtomicLong(1); final int total = 20; @@ -1097,7 +1097,7 @@ public class TestHLogSplit { } // Send the data to HDFS datanodes and close the HDFS writer log.sync(); - ((FSHLog) log).cleanupCurrentWriter(log.getFilenum()); + log.cleanupCurrentWriter(); /* code taken from ProcessServerShutdown.process() * handles RS shutdowns (as observed by the Master) @@ -1164,7 +1164,7 @@ public class TestHLogSplit { } fs.mkdirs(new Path(tableDir, region)); - HLog.Writer writer = HLogFactory.createWALWriter(fs, + WAL.Writer writer = HLogFactory.createWALWriter(fs, julietLog, conf); appendEntry(writer, TableName.valueOf("juliet"), ("juliet").getBytes(), ("r").getBytes(), FAMILY, QUALIFIER, VALUE, 0); @@ -1283,9 +1283,9 @@ public class TestHLogSplit { HLogSplitter logSplitter = new HLogSplitter( conf, HBASEDIR, fs, null, null) { - protected HLog.Writer createWriter(FileSystem fs, Path logfile, Configuration conf) + protected WAL.Writer createWriter(FileSystem fs, Path logfile, Configuration conf) throws IOException { - HLog.Writer writer = HLogFactory.createRecoveredEditsWriter(fs, logfile, conf); + WAL.Writer writer = HLogFactory.createRecoveredEditsWriter(fs, logfile, conf); // After creating writer, simulate region's // replayRecoveredEditsIfAny() which gets SplitEditFiles of this // region and delete them, excluding files with '.temp' suffix. @@ -1324,12 +1324,13 @@ public class TestHLogSplit { } - private HLog.Writer [] generateHLogs(int leaveOpen) throws IOException { + private WAL.Writer [] generateHLogs(int leaveOpen) throws IOException { return generateHLogs(NUM_WRITERS, ENTRIES, leaveOpen); } - private HLog.Writer [] generateHLogs(final int writers, final int entries, final int leaveOpen) throws IOException { - return generateHLogs((DistributedFileSystem)this.fs, writers, entries, leaveOpen); + private WAL.Writer[] generateHLogs(final int writers, final int entries, final int leaveOpen) + throws IOException { + return generateHLogs((DistributedFileSystem) this.fs, writers, entries, leaveOpen); } private static void makeRegionDirs(FileSystem fs, List regions) throws IOException { @@ -1339,11 +1340,11 @@ public class TestHLogSplit { } } - private static HLog.Writer [] generateHLogs(final DistributedFileSystem dfs, int writers, int entries, int leaveOpen) - throws IOException { + private static WAL.Writer[] generateHLogs(final DistributedFileSystem dfs, int writers, + int entries, int leaveOpen) throws IOException { makeRegionDirs(dfs, REGIONS); dfs.mkdirs(HLOGDIR); - HLog.Writer [] ws = new HLog.Writer[writers]; + WAL.Writer [] ws = new WAL.Writer[writers]; int seq = 0; for (int i = 0; i < writers; i++) { ws[i] = HLogFactory.createWALWriter(dfs, new Path(HLOGDIR, HLOG_FILE_PREFIX + i), dfs.getConf()); @@ -1458,8 +1459,8 @@ public class TestHLogSplit { @SuppressWarnings("unused") private void dumpHLog(Path log, FileSystem fs, Configuration conf) throws IOException { - HLog.Entry entry; - HLog.Reader in = HLogFactory.createReader(fs, log, conf); + WAL.Entry entry; + WAL.Reader in = HLogFactory.createReader(fs, log, conf); while ((entry = in.next()) != null) { System.out.println(entry); } @@ -1467,7 +1468,7 @@ public class TestHLogSplit { private int countHLog(Path log, FileSystem fs, Configuration conf) throws IOException { int count = 0; - HLog.Reader in = HLogFactory.createReader(fs, log, conf); + WAL.Reader in = HLogFactory.createReader(fs, log, conf); while (in.next() != null) { count++; } @@ -1475,7 +1476,7 @@ public class TestHLogSplit { } - public static long appendEntry(HLog.Writer writer, TableName table, byte[] region, + public static long appendEntry(WAL.Writer writer, TableName table, byte[] region, byte[] row, byte[] family, byte[] qualifier, byte[] value, long seq) throws IOException { @@ -1486,7 +1487,7 @@ public class TestHLogSplit { return seq; } - private static HLog.Entry createTestEntry( + private static WAL.Entry createTestEntry( TableName table, byte[] region, byte[] row, byte[] family, byte[] qualifier, byte[] value, long seq) { @@ -1494,14 +1495,14 @@ public class TestHLogSplit { WALEdit edit = new WALEdit(); seq++; edit.add(new KeyValue(row, family, qualifier, time, KeyValue.Type.Put, value)); - return new HLog.Entry(new HLogKey(region, table, seq, time, + return new WAL.Entry(new HLogKey(region, table, seq, time, HConstants.DEFAULT_CLUSTER_ID), edit); } private void injectEmptyFile(String suffix, boolean closeFile) throws IOException { - HLog.Writer writer = HLogFactory.createWALWriter( + WAL.Writer writer = HLogFactory.createWALWriter( fs, new Path(HLOGDIR, HLOG_FILE_PREFIX + suffix), conf); if (closeFile) writer.close(); } @@ -1543,11 +1544,11 @@ public class TestHLogSplit { } private boolean logsAreEqual(Path p1, Path p2) throws IOException { - HLog.Reader in1, in2; + WAL.Reader in1, in2; in1 = HLogFactory.createReader(fs, p1, conf); in2 = HLogFactory.createReader(fs, p2, conf); - HLog.Entry entry1; - HLog.Entry entry2; + WAL.Entry entry1; + WAL.Entry entry2; while ((entry1 = in1.next()) != null) { entry2 = in2.next(); if ((entry1.getKey().compareTo(entry2.getKey()) != 0) || diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java index 9381454..0423fc7 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java @@ -61,7 +61,7 @@ public class TestLogRollAbort { .getLogger().setLevel(Level.ALL); ((Log4JLogger)HRegionServer.LOG).getLogger().setLevel(Level.ALL); ((Log4JLogger)HRegion.LOG).getLogger().setLevel(Level.ALL); - ((Log4JLogger)HLog.LOG).getLogger().setLevel(Level.ALL); + ((Log4JLogger)WALService.LOG).getLogger().setLevel(Level.ALL); } // Need to override this setup so we can edit the config before it gets sent @@ -128,9 +128,8 @@ public class TestLogRollAbort { HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); - HLog log = server.getWAL(); + WALService log = server.getWAL(); - assertTrue("Need HDFS-826 for this test", ((FSHLog) log).canGetCurReplicas()); // don't run this test without append support (HDFS-200 & HDFS-142) assertTrue("Need append support for this test", FSUtils.isAppendSupported(TEST_UTIL.getConfiguration())); @@ -152,9 +151,8 @@ public class TestLogRollAbort { log.rollWriter(true); } catch (FailedLogCloseException flce) { assertTrue("Should have deferred flush log edits outstanding", - ((FSHLog) log).hasUnSyncedEntries()); + ((AbstractWAL) log).hasUnSyncedEntries()); } } - } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java index 557212e..7e50d05 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java @@ -82,7 +82,12 @@ import org.junit.experimental.categories.Category; public class TestLogRolling { private static final Log LOG = LogFactory.getLog(TestLogRolling.class); private HRegionServer server; - private HLog log; + /** + * This class invokes APIs that are present both in WALService and WAL. Having a AbstractWAL + * reference helps in avoiding type casting each time when a WAL obtained from a RegionServer + * invokes any WAL API. + */ + private AbstractWAL log; private String tableName; private byte[] value; private FileSystem fs; @@ -100,7 +105,7 @@ public class TestLogRolling { ((Log4JLogger)DFSClient.LOG).getLogger().setLevel(Level.ALL); ((Log4JLogger)HRegionServer.LOG).getLogger().setLevel(Level.ALL); ((Log4JLogger)HRegion.LOG).getLogger().setLevel(Level.ALL); - ((Log4JLogger)HLog.LOG).getLogger().setLevel(Level.ALL); + ((Log4JLogger)WALService.LOG).getLogger().setLevel(Level.ALL); } /** @@ -193,12 +198,12 @@ public class TestLogRolling { // When the hbase:meta table can be opened, the region servers are running new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME); this.server = cluster.getRegionServerThreads().get(0).getRegionServer(); - this.log = server.getWAL(); + this.log = (AbstractWAL) server.getWAL(); HTable table = createTestTable(this.tableName); server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); - this.log = server.getWAL(); + this.log = (AbstractWAL) server.getWAL(); for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls doPut(table, i); if (i % 32 == 0) { @@ -221,7 +226,7 @@ public class TestLogRolling { public void testLogRolling() throws Exception { this.tableName = getName(); startAndWriteData(); - LOG.info("after writing there are " + ((FSHLog) log).getNumRolledLogFiles() + " log files"); + LOG.info("after writing there are " + log.getNumRolledLogFiles() + " log files"); // flush all regions @@ -234,9 +239,9 @@ public class TestLogRolling { // Now roll the log log.rollWriter(); - int count = ((FSHLog) log).getNumRolledLogFiles(); + int count = log.getNumRolledLogFiles(); LOG.info("after flushing all regions and rolling logs there are " + - ((FSHLog) log).getNumRolledLogFiles() + " log files"); + log.getNumRolledLogFiles() + " log files"); assertTrue(("actual count: " + count), count <= 2); } @@ -297,9 +302,9 @@ public class TestLogRolling { /** * Give me the HDFS pipeline for this log file */ - DatanodeInfo[] getPipeline(HLog log) throws IllegalArgumentException, + DatanodeInfo[] getPipeline(WALService log) throws IllegalArgumentException, IllegalAccessException, InvocationTargetException { - OutputStream stm = ((FSHLog) log).getOutputStream(); + OutputStream stm = ((AbstractWAL) log).getOutputStream(); Method getPipeline = null; for (Method m : stm.getClass().getDeclaredMethods()) { if (m.getName().endsWith("getPipeline")) { @@ -327,7 +332,7 @@ public class TestLogRolling { LOG.info("Replication=" + fs.getDefaultReplication()); this.server = cluster.getRegionServer(0); - this.log = server.getWAL(); + this.log = (AbstractWAL) server.getWAL(); // Create the test table and open it String tableName = getName(); @@ -339,9 +344,8 @@ public class TestLogRolling { assertTrue(table.isAutoFlush()); server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); - this.log = server.getWAL(); + this.log = (AbstractWAL) server.getWAL(); - assertTrue("Need HDFS-826 for this test", ((FSHLog) log).canGetCurReplicas()); // don't run this test without append support (HDFS-200 & HDFS-142) assertTrue("Need append support for this test", FSUtils .isAppendSupported(TEST_UTIL.getConfiguration())); @@ -368,12 +372,13 @@ public class TestLogRolling { writeData(table, 2); long curTime = System.currentTimeMillis(); - long oldFilenum = ((FSHLog) log).getFilenum(); + LOG.info("log.getCurrentFileName(): " + log.getCurrentFileName()); + long oldFilenum = HLogUtilsForTests.extractFileNumFromPath(log.getCurrentFileName()); assertTrue("Log should have a timestamp older than now", curTime > oldFilenum && oldFilenum != -1); assertTrue("The log shouldn't have rolled yet", - oldFilenum == ((FSHLog) log).getFilenum()); + oldFilenum == HLogUtilsForTests.extractFileNumFromPath(log.getCurrentFileName())); final DatanodeInfo[] pipeline = getPipeline(log); assertTrue(pipeline.length == fs.getDefaultReplication()); @@ -383,7 +388,7 @@ public class TestLogRolling { // this write should succeed, but trigger a log roll writeData(table, 2); - long newFilenum = ((FSHLog) log).getFilenum(); + long newFilenum = HLogUtilsForTests.extractFileNumFromPath(log.getCurrentFileName()); assertTrue("Missing datanode should've triggered a log roll", newFilenum > oldFilenum && newFilenum > curTime); @@ -391,15 +396,17 @@ public class TestLogRolling { // write some more log data (this should use a new hdfs_out) writeData(table, 3); assertTrue("The log should not roll again.", - ((FSHLog) log).getFilenum() == newFilenum); + HLogUtilsForTests.extractFileNumFromPath(log.getCurrentFileName()) == newFilenum); // kill another datanode in the pipeline, so the replicas will be lower than // the configured value 2. assertTrue(dfsCluster.stopDataNode(pipeline[1].getName()) != null); + Method getNumCurrentReplicas = AbstractWAL.getGetNumCurrentReplicas(log.getOutputStream()); batchWriteAndWait(table, 3, false, 14000); + int replication = getLogReplication(getNumCurrentReplicas, log.getOutputStream()); assertTrue("LowReplication Roller should've been disabled, current replication=" - + ((FSHLog) log).getLogReplication(), - !log.isLowReplicationRollEnabled()); + + replication, + !((AbstractWAL)log).isLowReplicationRollEnabled()); dfsCluster .startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null); @@ -408,14 +415,36 @@ public class TestLogRolling { // and the LowReplication Roller will be enabled. log.rollWriter(true); batchWriteAndWait(table, 13, true, 10000); + replication = getLogReplication(getNumCurrentReplicas, log.getOutputStream()); assertTrue("New log file should have the default replication instead of " + - ((FSHLog) log).getLogReplication(), - ((FSHLog) log).getLogReplication() == fs.getDefaultReplication()); + replication, replication == fs.getDefaultReplication()); assertTrue("LowReplication Roller should've been enabled", log.isLowReplicationRollEnabled()); } /** + * @param getNumCurrentReplicas + * @param fStream + * @return the replication of the file being written via the passed stream + * @throws IOException + */ + private int getLogReplication(Method getNumCurrentReplicas, OutputStream fStream) + throws IOException { + if (getNumCurrentReplicas != null && fStream != null) { + Object repl; + try { + repl = getNumCurrentReplicas.invoke(fStream, new Object[] {}); + if (repl instanceof Integer) { + return ((Integer) repl).intValue(); + } + } catch (Exception e) { + LOG.warn("Unable to get the replication of the log ", e); + return 0; + } + } + return 0; + } + /** * Test that HLog is rolled when all data nodes in the pipeline have been * restarted. * @throws Exception @@ -430,7 +459,7 @@ public class TestLogRolling { new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME); this.server = cluster.getRegionServer(0); - this.log = server.getWAL(); + this.log = (AbstractWAL) server.getWAL(); // Create the test table and open it String tableName = getName(); @@ -441,10 +470,10 @@ public class TestLogRolling { HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); - this.log = server.getWAL(); + this.log = (AbstractWAL) server.getWAL(); final List paths = new ArrayList(); final List preLogRolledCalled = new ArrayList(); - paths.add(((FSHLog) log).computeFilename()); + paths.add(log.getCurrentFileName()); log.registerWALActionsListener(new WALActionsListener() { @Override public void preLogRoll(Path oldFile, Path newFile) { @@ -471,7 +500,6 @@ public class TestLogRolling { WALEdit logEdit) {} }); - assertTrue("Need HDFS-826 for this test", ((FSHLog) log).canGetCurReplicas()); // don't run this test without append support (HDFS-200 & HDFS-142) assertTrue("Need append support for this test", FSUtils .isAppendSupported(TEST_UTIL.getConfiguration())); @@ -481,11 +509,13 @@ public class TestLogRolling { table.setAutoFlush(true, true); long curTime = System.currentTimeMillis(); - long oldFilenum = log.getFilenum(); + LOG.info("og.getCurrentFileName()): " + log.getCurrentFileName()); + long oldFilenum = HLogUtilsForTests.extractFileNumFromPath(log.getCurrentFileName()); assertTrue("Log should have a timestamp older than now", curTime > oldFilenum && oldFilenum != -1); - assertTrue("The log shouldn't have rolled yet", oldFilenum == log.getFilenum()); + assertTrue("The log shouldn't have rolled yet", oldFilenum == + HLogUtilsForTests.extractFileNumFromPath(log.getCurrentFileName())); // roll all datanodes in the pipeline dfsCluster.restartDataNodes(); @@ -496,7 +526,7 @@ public class TestLogRolling { // this write should succeed, but trigger a log roll writeData(table, 1003); - long newFilenum = log.getFilenum(); + long newFilenum = HLogUtilsForTests.extractFileNumFromPath(log.getCurrentFileName()); assertTrue("Missing datanode should've triggered a log roll", newFilenum > oldFilenum && newFilenum > curTime); @@ -524,14 +554,15 @@ public class TestLogRolling { FSUtils fsUtils = FSUtils.getInstance(fs, TEST_UTIL.getConfiguration()); for (Path p : paths) { LOG.debug("recovering lease for " + p); - fsUtils.recoverFileLease(((HFileSystem)fs).getBackingFs(), p, TEST_UTIL.getConfiguration(), null); + fsUtils.recoverFileLease(((HFileSystem) fs).getBackingFs(), p, TEST_UTIL.getConfiguration(), + null); LOG.debug("Reading HLog "+FSUtils.getPath(p)); - HLog.Reader reader = null; + WAL.Reader reader = null; try { reader = HLogFactory.createReader(fs, p, TEST_UTIL.getConfiguration()); - HLog.Entry entry; + WAL.Entry entry; while ((entry = reader.next()) != null) { LOG.debug("#"+entry.getKey().getLogSeqNum()+": "+entry.getEdit().getKeyValues()); for (KeyValue kv : entry.getEdit().getKeyValues()) { @@ -592,8 +623,7 @@ public class TestLogRolling { HTable table2 = createTestTable(tableName2); server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); - this.log = server.getWAL(); - FSHLog fshLog = (FSHLog)log; + this.log = (AbstractWAL) server.getWAL(); HRegion region = server.getOnlineRegions(table2.getName()).get(0); Store s = region.getStore(HConstants.CATALOG_FAMILY); @@ -606,12 +636,12 @@ public class TestLogRolling { admin.flush(table2.getTableName()); } doPut(table2, 3); // don't flush yet, or compaction might trigger before we roll WAL - assertEquals("Should have no WAL after initial writes", 0, fshLog.getNumRolledLogFiles()); + assertEquals("Should have no WAL after initial writes", 0, log.getNumRolledLogFiles()); assertEquals(2, s.getStorefilesCount()); // Roll the log and compact table2, to have compaction record in the 2nd WAL. - fshLog.rollWriter(); - assertEquals("Should have WAL; one table is not flushed", 1, fshLog.getNumRolledLogFiles()); + log.rollWriter(); + assertEquals("Should have WAL; one table is not flushed", 1, log.getNumRolledLogFiles()); admin.flush(table2.getTableName()); region.compactStores(); // Wait for compaction in case if flush triggered it before us. @@ -623,14 +653,14 @@ public class TestLogRolling { // Write some value to the table so the WAL cannot be deleted until table is flushed. doPut(table, 0); // Now 2nd WAL will have compaction record for table2 and put for table. - fshLog.rollWriter(); // 1st WAL deleted, 2nd not deleted yet. - assertEquals("Should have WAL; one table is not flushed", 1, fshLog.getNumRolledLogFiles()); + log.rollWriter(); // 1st WAL deleted, 2nd not deleted yet. + assertEquals("Should have WAL; one table is not flushed", 1, log.getNumRolledLogFiles()); // Flush table to make latest WAL obsolete; write another record, and roll again. admin.flush(table.getTableName()); doPut(table, 1); - fshLog.rollWriter(); // Now 2nd WAL is deleted and 3rd is added. - assertEquals("Should have 1 WALs at the end", 1, fshLog.getNumRolledLogFiles()); + log.rollWriter(); // Now 2nd WAL is deleted and 3rd is added. + assertEquals("Should have 1 WALs at the end", 1, log.getNumRolledLogFiles()); table.close(); table2.close(); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java index 6831869..1dcbc1a 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java @@ -54,7 +54,7 @@ public class TestLogRollingNoCluster { public void testContendedLogRolling() throws IOException, InterruptedException { FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration()); Path dir = TEST_UTIL.getDataTestDir(); - HLog wal = HLogFactory.createHLog(fs, dir, "logs", + WALService wal = HLogFactory.createHLog(fs, dir, "logs", TEST_UTIL.getConfiguration()); Appender [] appenders = null; @@ -86,11 +86,11 @@ public class TestLogRollingNoCluster { */ static class Appender extends Thread { private final Log log; - private final HLog wal; + private final WALService wal; private final int count; private Exception e = null; - Appender(final HLog wal, final int index, final int count) { + Appender(final WALService wal, final int index, final int count) { super("" + index); this.wal = wal; this.count = count; @@ -116,7 +116,7 @@ public class TestLogRollingNoCluster { for (int i = 0; i < this.count; i++) { long now = System.currentTimeMillis(); // Roll every ten edits if the log has anything in it. - if (i % 10 == 0 && ((FSHLog) this.wal).getNumEntries() > 0) { + if (i % 10 == 0 && ((AbstractWAL) this.wal).getNumEntries() > 0) { this.wal.rollWriter(); } WALEdit edit = new WALEdit(); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadOldRootAndMetaEdits.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadOldRootAndMetaEdits.java index 8e88f7b..b61fbf6 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadOldRootAndMetaEdits.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadOldRootAndMetaEdits.java @@ -60,7 +60,7 @@ public class TestReadOldRootAndMetaEdits { @BeforeClass public static void setupBeforeClass() throws Exception { TEST_UTIL.getConfiguration().setClass("hbase.regionserver.hlog.writer.impl", - SequenceFileLogWriter.class, HLog.Writer.class); + SequenceFileLogWriter.class, WAL.Writer.class); fs = TEST_UTIL.getTestFileSystem(); dir = new Path(TEST_UTIL.createRootDir(), "testReadOldRootAndMetaEdits"); fs.mkdirs(dir); @@ -81,15 +81,15 @@ public class TestReadOldRootAndMetaEdits { LOG.debug("testReadOldRootAndMetaEdits"); Configuration conf = HBaseConfiguration.create(); conf.setClass("hbase.regionserver.hlog.writer.impl", SequenceFileLogWriter.class, - HLog.Writer.class); + WAL.Writer.class); // kv list to be used for all WALEdits. byte[] row = Bytes.toBytes("row"); KeyValue kv = new KeyValue(row, row, row, row); List kvs = new ArrayList(); kvs.add(kv); - HLog.Writer writer = null; - HLog.Reader reader = null; + WAL.Writer writer = null; + WAL.Reader reader = null; // a regular table TableName t = TableName.valueOf("t"); HRegionInfo tRegionInfo = null; @@ -98,16 +98,16 @@ public class TestReadOldRootAndMetaEdits { Path path = new Path(dir, "t"); try { tRegionInfo = new HRegionInfo(t, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - HLog.Entry tEntry = createAEntry(new HLogKey(tRegionInfo.getEncodedNameAsBytes(), t, + WAL.Entry tEntry = createAEntry(new HLogKey(tRegionInfo.getEncodedNameAsBytes(), t, ++logCount, timestamp, HConstants.DEFAULT_CLUSTER_ID), kvs); // create a old root edit (-ROOT-). - HLog.Entry rootEntry = createAEntry(new HLogKey(Bytes.toBytes(TableName.OLD_ROOT_STR), + WAL.Entry rootEntry = createAEntry(new HLogKey(Bytes.toBytes(TableName.OLD_ROOT_STR), TableName.OLD_ROOT_TABLE_NAME, ++logCount, timestamp, HConstants.DEFAULT_CLUSTER_ID), kvs); // create a old meta edit (hbase:meta). - HLog.Entry oldMetaEntry = createAEntry(new HLogKey(Bytes.toBytes(TableName.OLD_META_STR), + WAL.Entry oldMetaEntry = createAEntry(new HLogKey(Bytes.toBytes(TableName.OLD_META_STR), TableName.OLD_META_TABLE_NAME, ++logCount, timestamp, HConstants.DEFAULT_CLUSTER_ID), kvs); @@ -123,7 +123,7 @@ public class TestReadOldRootAndMetaEdits { // read the log and see things are okay. reader = HLogFactory.createReader(fs, path, conf); - HLog.Entry entry = reader.next(); + WAL.Entry entry = reader.next(); assertNotNull(entry); assertTrue(entry.getKey().getTablename().equals(t)); assertEquals(Bytes.toString(entry.getKey().getEncodedRegionName()), @@ -148,11 +148,11 @@ public class TestReadOldRootAndMetaEdits { * the WALEdit and passed HLogKey. * @return HLog.Entry instance for the passed HLogKey and KeyValues */ - private HLog.Entry createAEntry(HLogKey hlogKey, List kvs) { + private WAL.Entry createAEntry(HLogKey hlogKey, List kvs) { WALEdit edit = new WALEdit(); for (KeyValue kv : kvs ) edit.add(kv); - return new HLog.Entry(hlogKey, edit); + return new WAL.Entry(hlogKey, edit); } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureHLog.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureHLog.java index 6b4304d..523840d 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureHLog.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureHLog.java @@ -63,9 +63,9 @@ public class TestSecureHLog { conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName()); conf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase"); conf.setClass("hbase.regionserver.hlog.reader.impl", SecureProtobufLogReader.class, - HLog.Reader.class); + WAL.Reader.class); conf.setClass("hbase.regionserver.hlog.writer.impl", SecureProtobufLogWriter.class, - HLog.Writer.class); + WAL.Writer.class); conf.setBoolean(HConstants.ENABLE_WAL_ENCRYPTION, true); } @@ -85,14 +85,15 @@ public class TestSecureHLog { final AtomicLong sequenceId = new AtomicLong(1); // Write the WAL - HLog wal = new FSHLog(fs, TEST_UTIL.getDataTestDir(), logDir.toString(), + WALService wal = HLogFactory.createHLog(fs, TEST_UTIL.getDataTestDir(), logDir.toString(), TEST_UTIL.getConfiguration()); + for (int i = 0; i < total; i++) { WALEdit kvs = new WALEdit(); kvs.add(new KeyValue(row, family, Bytes.toBytes(i), value)); wal.append(regioninfo, tableName, kvs, System.currentTimeMillis(), htd, sequenceId); } - final Path walPath = ((FSHLog) wal).computeFilename(); + final Path walPath = ((AbstractWAL) wal).getCurrentFileName(); wal.close(); // Insure edits are not plaintext @@ -104,10 +105,10 @@ public class TestSecureHLog { assertFalse("Cells appear to be plaintext", Bytes.contains(fileData, value)); // Confirm the WAL can be read back - HLog.Reader reader = HLogFactory.createReader(TEST_UTIL.getTestFileSystem(), walPath, + WAL.Reader reader = HLogFactory.createReader(TEST_UTIL.getTestFileSystem(), walPath, TEST_UTIL.getConfiguration()); int count = 0; - HLog.Entry entry = new HLog.Entry(); + WAL.Entry entry = new WAL.Entry(); while (reader.next(entry) != null) { count++; List kvs = entry.getEdit().getKeyValues(); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java index 9dc6e2b..d41d5ec 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java @@ -34,9 +34,9 @@ public class TestSecureWALReplay extends TestWALReplay { conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName()); conf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase"); conf.setClass("hbase.regionserver.hlog.reader.impl", SecureProtobufLogReader.class, - HLog.Reader.class); + WAL.Reader.class); conf.setClass("hbase.regionserver.hlog.writer.impl", SecureProtobufLogWriter.class, - HLog.Writer.class); + WAL.Writer.class); conf.setBoolean(HConstants.ENABLE_WAL_ENCRYPTION, true); TestWALReplay.setUpBeforeClass(); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java index 0a31d9f..207361a 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java @@ -88,7 +88,7 @@ public class TestWALActionsListener { List list = new ArrayList(); list.add(observer); DummyWALActionsListener laterobserver = new DummyWALActionsListener(); - HLog hlog = HLogFactory.createHLog(fs, TEST_UTIL.getDataTestDir(), logName, + WALService hlog = HLogFactory.createHLog(fs, TEST_UTIL.getDataTestDir(), logName, conf, list, null); final AtomicLong sequenceId = new AtomicLong(1); HRegionInfo hri = new HRegionInfo(TableName.valueOf(SOME_BYTES), diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java index 5aeb31b..e089415 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java @@ -268,7 +268,7 @@ public class TestWALReplay { HRegion.closeHRegion(region2); final byte [] rowName = tableName.getName(); - HLog wal1 = createWAL(this.conf); + WALService wal1 = createWAL(this.conf); // Add 1k to each family. final int countPerFamily = 1000; final AtomicLong sequenceId = new AtomicLong(1); @@ -279,7 +279,7 @@ public class TestWALReplay { wal1.close(); runWALSplit(this.conf); - HLog wal2 = createWAL(this.conf); + WALService wal2 = createWAL(this.conf); // Add 1k to each family. for (HColumnDescriptor hcd: htd.getFamilies()) { addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, @@ -288,7 +288,7 @@ public class TestWALReplay { wal2.close(); runWALSplit(this.conf); - HLog wal3 = createWAL(this.conf); + WALService wal3 = createWAL(this.conf); try { HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal3); long seqid = region.getOpenSeqNum(); @@ -328,7 +328,7 @@ public class TestWALReplay { HRegion region2 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd); HRegion.closeHRegion(region2); - HLog wal = createWAL(this.conf); + WALService wal = createWAL(this.conf); HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf); Path f = new Path(basedir, "hfile"); HFileContext context = new HFileContextBuilder().build(); @@ -353,7 +353,7 @@ public class TestWALReplay { user.runAs(new PrivilegedExceptionAction() { public Object run() throws Exception { runWALSplit(newConf); - HLog wal2 = createWAL(newConf); + WALService wal2 = createWAL(newConf); HRegion region2 = HRegion.openHRegion(newConf, FileSystem.get(newConf), hbaseRootDir, hri, htd, wal2); @@ -395,7 +395,7 @@ public class TestWALReplay { // Write countPerFamily edits into the three families. Do a flush on one // of the families during the load of edits so its seqid is not same as // others to test we do right thing when different seqids. - HLog wal = createWAL(this.conf); + WALService wal = createWAL(this.conf); HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal); long seqid = region.getOpenSeqNum(); boolean first = true; @@ -418,7 +418,7 @@ public class TestWALReplay { region.close(true); wal.close(); runWALSplit(this.conf); - HLog wal2 = createWAL(this.conf); + WALService wal2 = createWAL(this.conf); HRegion region2 = HRegion.openHRegion(conf, this.fs, hbaseRootDir, hri, htd, wal2); long seqid2 = region2.getOpenSeqNum(); assertTrue(seqid + result.size() < seqid2); @@ -437,7 +437,7 @@ public class TestWALReplay { wal2.sync(); // Set down maximum recovery so we dfsclient doesn't linger retrying something // long gone. - HBaseTestingUtility.setMaxRecoveryErrorCount(((FSHLog) wal2).getOutputStream(), 1); + HBaseTestingUtility.setMaxRecoveryErrorCount(((AbstractWAL) wal2).getOutputStream(), 1); final Configuration newConf = HBaseConfiguration.create(this.conf); User user = HBaseTestingUtility.getDifferentUser(newConf, tableName.getNameAsString()); @@ -446,7 +446,7 @@ public class TestWALReplay { runWALSplit(newConf); FileSystem newFS = FileSystem.get(newConf); // Make a new wal for new region open. - HLog wal3 = createWAL(newConf); + WALService wal3 = createWAL(newConf); final AtomicInteger countOfRestoredEdits = new AtomicInteger(0); HRegion region3 = new HRegion(basedir, wal3, newFS, newConf, hri, htd, null) { @Override @@ -507,7 +507,7 @@ public class TestWALReplay { // Write countPerFamily edits into the three families. Do a flush on one // of the families during the load of edits so its seqid is not same as // others to test we do right thing when different seqids. - HLog wal = createWAL(this.conf); + WALService wal = createWAL(this.conf); HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal); long seqid = region.getOpenSeqNum(); for (HColumnDescriptor hcd: htd.getFamilies()) { @@ -540,7 +540,7 @@ public class TestWALReplay { // Let us try to split and recover runWALSplit(this.conf); - HLog wal2 = createWAL(this.conf); + WALService wal2 = createWAL(this.conf); HRegion region2 = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal2); long seqid2 = region2.getOpenSeqNum(); assertTrue(seqid + result.size() < seqid2); @@ -592,7 +592,7 @@ public class TestWALReplay { // Write countPerFamily edits into the three families. Do a flush on one // of the families during the load of edits so its seqid is not same as // others to test we do right thing when different seqids. - HLog wal = createWAL(this.conf); + WALService wal = createWAL(this.conf); RegionServerServices rsServices = Mockito.mock(RegionServerServices.class); Mockito.doReturn(false).when(rsServices).isAborted(); Configuration customConf = new Configuration(this.conf); @@ -648,7 +648,7 @@ public class TestWALReplay { // Let us try to split and recover runWALSplit(this.conf); - HLog wal2 = createWAL(this.conf); + WALService wal2 = createWAL(this.conf); Mockito.doReturn(false).when(rsServices).isAborted(); HRegion region2 = HRegion.openHRegion(this.hbaseRootDir, hri, htd, wal2, this.conf, rsServices, null); @@ -687,7 +687,7 @@ public class TestWALReplay { HRegion region2 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd); HRegion.closeHRegion(region2); - final HLog wal = createWAL(this.conf); + final WALService wal = createWAL(this.conf); final byte[] rowName = tableName.getName(); final byte[] regionName = hri.getEncodedNameAsBytes(); final AtomicLong sequenceId = new AtomicLong(1); @@ -721,7 +721,7 @@ public class TestWALReplay { wal.sync(); // Set down maximum recovery so we dfsclient doesn't linger retrying something // long gone. - HBaseTestingUtility.setMaxRecoveryErrorCount(((FSHLog) wal).getOutputStream(), 1); + HBaseTestingUtility.setMaxRecoveryErrorCount(((AbstractWAL) wal).getOutputStream(), 1); // Make a new conf and a new fs for the splitter to run on so we can take // over old wal. final Configuration newConf = HBaseConfiguration.create(this.conf); @@ -734,13 +734,13 @@ public class TestWALReplay { // 100k seems to make for about 4 flushes during HRegion#initialize. newConf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 1024 * 100); // Make a new wal for new region. - HLog newWal = createWAL(newConf); + WALService newWal = createWAL(newConf); final AtomicInteger flushcount = new AtomicInteger(0); try { final HRegion region = new HRegion(basedir, newWal, newFS, newConf, hri, htd, null) { protected boolean internalFlushcache( - final HLog wal, final long myseqid, MonitoredTask status) + final WALService wal, final long myseqid, MonitoredTask status) throws IOException { LOG.info("InternalFlushCache Invoked"); boolean b = super.internalFlushcache(wal, myseqid, @@ -822,7 +822,7 @@ public class TestWALReplay { boolean doCompleteCacheFlush = false; public MockHLog(FileSystem fs, Path rootDir, String logName, Configuration conf) throws IOException { - super(fs, rootDir, logName, conf); + super(fs, rootDir, logName, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, false); } @Override @@ -840,12 +840,12 @@ public class TestWALReplay { htd.addFamily(a); return htd; } - + private MockHLog createMockWAL(Configuration conf) throws IOException { MockHLog wal = new MockHLog(FileSystem.get(conf), hbaseRootDir, logName, conf); // Set down maximum recovery so we dfsclient doesn't linger retrying something // long gone. - HBaseTestingUtility.setMaxRecoveryErrorCount(((FSHLog) wal).getOutputStream(), 1); + HBaseTestingUtility.setMaxRecoveryErrorCount(((AbstractWAL) wal).getOutputStream(), 1); return wal; } @@ -886,7 +886,7 @@ public class TestWALReplay { } private void addWALEdits(final TableName tableName, final HRegionInfo hri, final byte[] rowName, - final byte[] family, final int count, EnvironmentEdge ee, final HLog wal, + final byte[] family, final int count, EnvironmentEdge ee, final WALService wal, final HTableDescriptor htd, final AtomicLong sequenceId) throws IOException { String familyStr = Bytes.toString(family); @@ -943,12 +943,12 @@ public class TestWALReplay { * @return WAL with retries set down from 5 to 1 only. * @throws IOException */ - private HLog createWAL(final Configuration c) throws IOException { - HLog wal = HLogFactory.createHLog(FileSystem.get(c), + private WALService createWAL(final Configuration c) throws IOException { + WALService wal = HLogFactory.createHLog(FileSystem.get(c), hbaseRootDir, logName, c); // Set down maximum recovery so we dfsclient doesn't linger retrying something // long gone. - HBaseTestingUtility.setMaxRecoveryErrorCount(((FSHLog) wal).getOutputStream(), 1); + HBaseTestingUtility.setMaxRecoveryErrorCount(((AbstractWAL) wal).getOutputStream(), 1); return wal; } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java index 4c5ece5..793bd24 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java @@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MediumTests; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WAL; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; @@ -79,7 +79,7 @@ public class TestReplicationSource { Path logPath = new Path(logDir, "log"); if (!FS.exists(logDir)) FS.mkdirs(logDir); if (!FS.exists(oldLogDir)) FS.mkdirs(oldLogDir); - HLog.Writer writer = HLogFactory.createWALWriter(FS, + WAL.Writer writer = HLogFactory.createWALWriter(FS, logPath, conf); for(int i = 0; i < 3; i++) { byte[] b = Bytes.toBytes(Integer.toString(i)); @@ -88,14 +88,14 @@ public class TestReplicationSource { edit.add(kv); HLogKey key = new HLogKey(b, TableName.valueOf(b), 0, 0, HConstants.DEFAULT_CLUSTER_ID); - writer.append(new HLog.Entry(key, edit)); + writer.append(new WAL.Entry(key, edit)); writer.sync(); } writer.close(); - HLog.Reader reader = HLogFactory.createReader(FS, + WAL.Reader reader = HLogFactory.createReader(FS, logPath, conf); - HLog.Entry entry = reader.next(); + WAL.Entry entry = reader.next(); assertNotNull(entry); Path oldLogPath = new Path(oldLogDir, "log"); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationHLogReaderManager.java hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationHLogReaderManager.java index dfc89bc..67e5735 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationHLogReaderManager.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationHLogReaderManager.java @@ -28,7 +28,8 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.LargeTests; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WAL; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; @@ -47,7 +48,6 @@ import org.junit.runners.Parameterized.Parameters; import static org.junit.Assert.*; -import java.io.EOFException; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -70,7 +70,7 @@ public class TestReplicationHLogReaderManager { HConstants.EMPTY_START_ROW, HConstants.LAST_ROW, false); private static final HTableDescriptor htd = new HTableDescriptor(tableName); - private HLog log; + private WALService log; private ReplicationHLogReaderManager logManager; private PathWatcher pathWatcher; private int nbRows; @@ -148,7 +148,7 @@ public class TestReplicationHLogReaderManager { // There's one edit in the log, read it. Reading past it needs to return nulls assertNotNull(logManager.openReader(path)); logManager.seek(); - HLog.Entry entry = logManager.readNextAndSetPosition(); + WAL.Entry entry = logManager.readNextAndSetPosition(); assertNotNull(entry); entry = logManager.readNextAndSetPosition(); assertNull(entry); @@ -183,7 +183,7 @@ public class TestReplicationHLogReaderManager { logManager.openReader(path); logManager.seek(); for (int i = 0; i < nbRows; i++) { - HLog.Entry e = logManager.readNextAndSetPosition(); + WAL.Entry e = logManager.readNextAndSetPosition(); if (e == null) { fail("Should have enough entries"); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java index b9e4d8f..db50f53 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java @@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.catalog.CatalogTracker; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; @@ -187,7 +187,7 @@ public class TestReplicationSourceManager { List listeners = new ArrayList(); listeners.add(replication); - HLog hlog = HLogFactory.createHLog(fs, utility.getDataTestDir(), logName, + WALService hlog = HLogFactory.createHLog(fs, utility.getDataTestDir(), logName, conf, listeners, URLEncoder.encode("regionserver:60020", "UTF8")); final AtomicLong sequenceId = new AtomicLong(1); manager.init(); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotLogSplitter.java hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotLogSplitter.java index d66623c..fbb5637 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotLogSplitter.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotLogSplitter.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.SmallTests; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WAL; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; @@ -121,9 +121,9 @@ public class TestSnapshotLogSplitter { byte[] regionName = Bytes.toBytes(regionStatus.getPath().getName()); assertFalse(regionsMap.containsKey(regionName)); for (FileStatus logStatus: FSUtils.listStatus(fs, regionEdits)) { - HLog.Reader reader = HLogFactory.createReader(fs, logStatus.getPath(), conf); + WAL.Reader reader = HLogFactory.createReader(fs, logStatus.getPath(), conf); try { - HLog.Entry entry; + WAL.Entry entry; while ((entry = reader.next()) != null) { HLogKey key = entry.getKey(); assertEquals(tableName, key.getTablename()); @@ -144,7 +144,7 @@ public class TestSnapshotLogSplitter { */ private void writeTestLog(final Path logFile) throws IOException { fs.mkdirs(logFile.getParent()); - HLog.Writer writer = HLogFactory.createWALWriter(fs, logFile, conf); + WAL.Writer writer = HLogFactory.createWALWriter(fs, logFile, conf); try { for (int i = 0; i < 7; ++i) { TableName tableName = getTableName(i); @@ -156,7 +156,7 @@ public class TestSnapshotLogSplitter { System.currentTimeMillis(), HConstants.DEFAULT_CLUSTER_ID); WALEdit edit = new WALEdit(); edit.add(new KeyValue(rowkey, TEST_FAMILY, TEST_QUALIFIER, rowkey)); - writer.append(new HLog.Entry(key, edit)); + writer.append(new WAL.Entry(key, edit)); } } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java index 563d51d..13942e6 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java @@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.WALService; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.util.ToolRunner; @@ -196,7 +196,7 @@ public class TestMergeTool extends HBaseTestCase { * @throws Exception */ private HRegion mergeAndVerify(final String msg, final String regionName1, - final String regionName2, final HLog log, final int upperbound) + final String regionName2, final WALService log, final int upperbound) throws Exception { Merge merger = new Merge(this.conf); LOG.info(msg); @@ -276,7 +276,7 @@ public class TestMergeTool extends HBaseTestCase { + System.currentTimeMillis(); LOG.info("Creating log " + logPath.toString() + "/" + logName); - HLog log = HLogFactory.createHLog(this.fs, logPath, + WALService log = HLogFactory.createHLog(this.fs, logPath, logName, this.conf); try {