From b94e6738d920cb5bf297214883e0cf29522597b9 Mon Sep 17 00:00:00 2001 From: stack Date: Tue, 18 Nov 2014 12:12:35 -0800 Subject: [PATCH] HBASE-10378 Refactor write-ahead-log implementation -- ADDEDNUM --- bin/hbase | 7 +- bin/hbase.cmd | 11 +- .../hbase/DroppedSnapshotException.java | 2 +- .../org/apache/hadoop/hbase/client/Admin.java | 15 +- .../client/replication/ReplicationAdmin.java | 2 +- .../RegionServerCoprocessorRpcChannel.java | 2 +- .../hbase/protobuf/ResponseConverter.java | 16 - .../hbase/replication/ReplicationQueues.java | 24 +- .../replication/ReplicationQueuesClient.java | 6 +- .../ReplicationQueuesClientZKImpl.java | 2 +- .../replication/ReplicationQueuesZKImpl.java | 70 +- .../apache/hadoop/hbase/zookeeper/ZKUtil.java | 12 +- .../org/apache/hadoop/hbase/HConstants.java | 4 +- .../org/apache/hadoop/hbase/KeyValue.java | 2 +- .../src/main/resources/hbase-default.xml | 10 +- .../master/MetricsMasterFileSystemSource.java | 6 +- .../MetricsRegionServerSource.java | 8 +- .../MetricsRegionServerWrapper.java | 8 +- .../wal/MetricsEditsReplaySource.java | 2 +- .../regionserver/wal/MetricsWALSource.java | 12 +- .../wal/TestMetricsHLogSource.java | 32 - .../MetricsRegionServerSourceImpl.java | 4 +- .../wal/MetricsWALSourceImpl.java | 3 +- .../IntegrationTestIngestWithEncryption.java | 7 +- .../hbase/mttr/IntegrationTestMTTR.java | 4 +- .../hbase/protobuf/generated/AdminProtos.java | 12 + .../generated/RegionServerStatusProtos.java | 16 +- .../hbase/protobuf/generated/WALProtos.java | 16 +- .../protobuf/generated/ZooKeeperProtos.java | 4 +- hbase-protocol/src/main/protobuf/Admin.proto | 4 + .../main/protobuf/RegionServerStatus.proto | 2 +- hbase-protocol/src/main/protobuf/WAL.proto | 9 +- .../src/main/protobuf/ZooKeeper.proto | 2 +- .../tmpl/regionserver/ServerMetricsTmpl.jamon | 16 +- .../apache/hadoop/hbase/SplitLogCounters.java | 2 +- .../org/apache/hadoop/hbase/SplitLogTask.java | 1 - .../SplitLogWorkerCoordination.java | 8 +- .../ZKSplitLogManagerCoordination.java | 8 +- .../ZkSplitLogWorkerCoordination.java | 12 +- .../hbase/coprocessor/BaseRegionObserver.java | 19 + .../hbase/coprocessor/BaseWALObserver.java | 25 +- .../hbase/coprocessor/CoprocessorHost.java | 74 + .../hbase/coprocessor/RegionObserver.java | 64 +- .../WALCoprocessorEnvironment.java | 6 +- .../hadoop/hbase/coprocessor/WALObserver.java | 63 +- .../apache/hadoop/hbase/fs/HFileSystem.java | 16 +- .../org/apache/hadoop/hbase/io/HLogLink.java | 69 - .../hbase/mapreduce/HFileOutputFormat2.java | 4 +- .../hbase/mapreduce/HLogInputFormat.java | 235 +-- .../mapreduce/MultiTableOutputFormat.java | 4 +- .../mapreduce/TableSnapshotInputFormat.java | 2 +- .../hadoop/hbase/mapreduce/WALPlayer.java | 52 +- .../hbase/master/AssignmentManager.java | 14 +- .../hadoop/hbase/master/MasterFileSystem.java | 45 +- .../hbase/master/MetricsMasterFileSystem.java | 4 +- .../hadoop/hbase/master/RegionStates.java | 4 +- .../hadoop/hbase/master/ServerManager.java | 8 +- .../hadoop/hbase/master/SplitLogManager.java | 40 +- .../hbase/master/cleaner/LogCleaner.java | 6 +- .../master/cleaner/TimeToLiveLogCleaner.java | 2 +- .../handler/MetaServerShutdownHandler.java | 4 +- .../master/handler/ServerShutdownHandler.java | 20 +- .../master/snapshot/SnapshotLogCleaner.java | 8 +- .../hbase/migration/NamespaceUpgrade.java | 17 +- .../protobuf/ReplicationProtbufUtil.java | 26 +- .../hbase/regionserver/HRegionServer.java | 234 ++- .../hadoop/hbase/regionserver/HStore.java | 6 +- .../hadoop/hbase/regionserver/LogRoller.java | 109 +- .../hbase/regionserver/MemStoreFlusher.java | 4 +- .../hbase/regionserver/MetaLogRoller.java | 38 - .../MetricsRegionServerWrapperImpl.java | 29 +- .../hbase/regionserver/RSRpcServices.java | 37 +- .../regionserver/RegionCoprocessorHost.java | 60 +- .../regionserver/RegionServerAccounting.java | 2 +- .../regionserver/RegionServerServices.java | 6 +- .../hbase/regionserver/SplitLogWorker.java | 12 +- .../handler/HLogSplitterHandler.java | 106 -- .../regionserver/wal/CompressionContext.java | 11 +- .../hbase/regionserver/wal/Compressor.java | 18 +- .../hadoop/hbase/regionserver/wal/FSHLog.java | 681 ++++---- .../hbase/regionserver/wal/FSWALEntry.java | 11 +- .../hadoop/hbase/regionserver/wal/HLog.java | 445 ----- .../hbase/regionserver/wal/HLogFactory.java | 207 --- .../hbase/regionserver/wal/HLogKey.java | 430 +---- .../regionserver/wal/HLogPrettyPrinter.java | 315 +--- .../hbase/regionserver/wal/HLogUtil.java | 367 ----- .../hbase/regionserver/wal/MetricsWAL.java | 13 +- .../regionserver/wal/ProtobufLogReader.java | 40 +- .../regionserver/wal/ProtobufLogWriter.java | 14 +- .../hbase/regionserver/wal/ReaderBase.java | 29 +- .../wal/SecureProtobufLogWriter.java | 6 +- .../wal/SequenceFileLogReader.java | 21 +- .../regionserver/wal/WALActionsListener.java | 76 +- .../hbase/regionserver/wal/WALCellCodec.java | 2 +- .../regionserver/wal/WALCoprocessorHost.java | 67 +- .../hbase/regionserver/wal/WALEdit.java | 8 +- .../regionserver/wal/WALEditsReplaySink.java | 31 +- .../hbase/regionserver/wal/WriterBase.java | 4 +- .../replication/ChainWALEntryFilter.java | 2 +- .../replication/ReplicationEndpoint.java | 8 +- .../replication/ScopeWALEntryFilter.java | 2 +- .../SystemTableWALEntryFilter.java | 2 +- .../replication/TableCfWALEntryFilter.java | 2 +- .../hbase/replication/WALEntryFilter.java | 10 +- .../master/ReplicationLogCleaner.java | 26 +- .../HBaseInterClusterReplicationEndpoint.java | 6 +- .../RegionReplicaReplicationEndpoint.java | 26 +- .../replication/regionserver/Replication.java | 36 +- .../ReplicationHLogReaderManager.java | 144 -- .../regionserver/ReplicationSink.java | 2 +- .../regionserver/ReplicationSource.java | 38 +- .../ReplicationSourceManager.java | 104 +- .../security/access/AccessController.java | 2 +- .../HbaseObjectWritableFor96Migration.java | 38 +- .../hadoop/hbase/snapshot/ExportSnapshot.java | 14 +- .../hadoop/hbase/snapshot/SnapshotInfo.java | 6 +- .../hbase/snapshot/SnapshotReferenceUtil.java | 8 +- .../apache/hadoop/hbase/util/FSHDFSUtils.java | 2 +- .../apache/hadoop/hbase/util/FSVisitor.java | 14 +- .../apache/hadoop/hbase/util/HBaseFsck.java | 6 +- .../org/apache/hadoop/hbase/util/HMerge.java | 18 +- .../org/apache/hadoop/hbase/util/Merge.java | 7 +- .../apache/hadoop/hbase/util/MetaUtils.java | 35 +- .../hadoop/hbase/util/RegionSplitter.java | 2 +- .../hadoop/hbase/zookeeper/ZKSplitLog.java | 2 +- .../apache/hadoop/hbase/HBaseTestCase.java | 2 +- .../hadoop/hbase/HBaseTestingUtility.java | 22 +- .../hbase/MockRegionServerServices.java | 4 +- .../apache/hadoop/hbase/TestIOFencing.java | 13 +- .../hbase/backup/TestHFileArchiving.java | 2 +- .../hadoop/hbase/client/TestAdmin2.java | 10 +- .../coprocessor/SampleRegionWALObserver.java | 77 +- .../coprocessor/SimpleRegionObserver.java | 49 +- .../TestRegionObserverInterface.java | 69 +- .../TestRegionObserverScannerOpenHook.java | 4 +- .../hbase/coprocessor/TestWALObserver.java | 199 ++- .../hadoop/hbase/filter/TestFilter.java | 14 +- .../filter/TestInvocationRecordFilter.java | 8 +- .../hadoop/hbase/fs/TestBlockReorder.java | 108 +- .../apache/hadoop/hbase/io/TestHeapSize.java | 2 +- .../hbase/mapreduce/TestHLogRecordReader.java | 230 +-- .../hbase/mapreduce/TestImportExport.java | 55 +- .../mapreduce/TestTableMapReduceUtil.java | 8 +- .../hadoop/hbase/mapreduce/TestWALPlayer.java | 31 +- .../hadoop/hbase/master/MockRegionServer.java | 4 +- .../master/TestDistributedLogSplitting.java | 131 +- .../hbase/master/TestMasterFailover.java | 2 +- .../snapshot/TestSnapshotFileCache.java | 17 +- .../MetricsRegionServerWrapperStub.java | 6 +- .../regionserver/TestAtomicOperation.java | 4 +- .../TestCacheOnWriteInSchema.java | 20 +- .../hbase/regionserver/TestCompaction.java | 6 +- .../TestDefaultCompactSelection.java | 19 +- .../TestGetClosestAtOrBefore.java | 6 +- .../hbase/regionserver/TestHRegion.java | 233 +-- .../regionserver/TestMajorCompaction.java | 6 +- .../regionserver/TestMinorCompaction.java | 6 +- .../TestRegionMergeTransaction.java | 29 +- .../regionserver/TestSplitTransaction.java | 22 +- .../hadoop/hbase/regionserver/TestStore.java | 16 +- .../TestStoreFileRefresherChore.java | 12 +- .../hbase/regionserver/wal/FaultyHLog.java | 70 - .../wal/FaultySequenceFileLogReader.java | 12 +- .../wal/HLogPerformanceEvaluation.java | 566 ------- .../regionserver/wal/HLogUtilsForTests.java | 43 - .../InstrumentedSequenceFileLogWriter.java | 40 - .../wal/SequenceFileLogWriter.java | 19 +- .../regionserver/wal/TestDurability.java | 55 +- .../hbase/regionserver/wal/TestHLog.java | 1343 --------------- .../regionserver/wal/TestHLogFiltering.java | 154 -- .../regionserver/wal/TestHLogMethods.java | 171 -- .../wal/TestHLogReaderOnSecureHLog.java | 198 --- .../hbase/regionserver/wal/TestHLogSplit.java | 1454 ----------------- .../wal/TestHLogSplitCompressed.java | 36 - .../regionserver/wal/TestLogRollAbort.java | 37 +- .../regionserver/wal/TestLogRollPeriod.java | 23 +- .../regionserver/wal/TestLogRolling.java | 151 +- .../wal/TestLogRollingNoCluster.java | 30 +- .../wal/TestReadOldRootAndMetaEdits.java | 39 +- .../regionserver/wal/TestSecureHLog.java | 130 -- .../regionserver/wal/TestSecureWALReplay.java | 6 +- .../wal/TestWALActionsListener.java | 63 +- .../hbase/regionserver/wal/TestWALReplay.java | 133 +- .../TestMultiSlaveReplication.java | 63 +- ...tReplicationChangingPeerRegionservers.java | 2 +- .../replication/TestReplicationEndpoint.java | 7 +- ...TestReplicationKillMasterRSCompressed.java | 2 +- .../TestReplicationSmallTests.java | 8 +- .../replication/TestReplicationSource.java | 24 +- .../TestReplicationWALEntryFilters.java | 39 +- .../TestRegionReplicaReplicationEndpoint.java | 8 +- ...ionReplicaReplicationEndpointNoMaster.java | 16 +- .../TestReplicationHLogReaderManager.java | 239 --- .../TestReplicationSourceManager.java | 47 +- .../hadoop/hbase/util/TestFSVisitor.java | 19 +- .../hadoop/hbase/util/TestHBaseFsck.java | 16 +- .../hadoop/hbase/util/TestMergeTool.java | 56 +- hbase-shell/src/main/ruby/hbase/admin.rb | 8 +- hbase-shell/src/main/ruby/shell.rb | 8 +- .../src/main/ruby/shell/commands/hlog_roll.rb | 39 - src/main/docbkx/book.xml | 29 +- src/main/docbkx/ops_mgt.xml | 19 +- src/main/docbkx/performance.xml | 4 +- src/main/docbkx/troubleshooting.xml | 10 +- 204 files changed, 2936 insertions(+), 9161 deletions(-) delete mode 100644 hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestMetricsHLogSource.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/io/HLogLink.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetaLogRoller.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/HLogSplitterHandler.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationHLogReaderManager.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultyHLog.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtilsForTests.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/InstrumentedSequenceFileLogWriter.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogFiltering.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogMethods.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogReaderOnSecureHLog.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplitCompressed.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureHLog.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationHLogReaderManager.java delete mode 100644 hbase-shell/src/main/ruby/shell/commands/hlog_roll.rb diff --git a/bin/hbase b/bin/hbase index 8678eb8fcc9..4573c570b9e 100755 --- a/bin/hbase +++ b/bin/hbase @@ -83,7 +83,7 @@ if [ $# = 0 ]; then echo "Some commands take arguments. Pass no args or -h for usage." echo " shell Run the HBase shell" echo " hbck Run the hbase 'fsck' tool" - echo " hlog Write-ahead-log analyzer" + echo " wal Write-ahead-log analyzer" echo " hfile Store file analyzer" echo " zkcli Run the ZooKeeper shell" echo " upgrade Upgrade hbase" @@ -288,8 +288,9 @@ if [ "$COMMAND" = "shell" ] ; then CLASS="org.jruby.Main -X+O ${JRUBY_OPTS} ${HBASE_HOME}/bin/hirb.rb" elif [ "$COMMAND" = "hbck" ] ; then CLASS='org.apache.hadoop.hbase.util.HBaseFsck' -elif [ "$COMMAND" = "hlog" ] ; then - CLASS='org.apache.hadoop.hbase.regionserver.wal.HLogPrettyPrinter' +# TODO remove old 'hlog' version +elif [ "$COMMAND" = "hlog" -o "$COMMAND" = "wal" ] ; then + CLASS='org.apache.hadoop.hbase.wal.WALPrettyPrinter' elif [ "$COMMAND" = "hfile" ] ; then CLASS='org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter' elif [ "$COMMAND" = "zkcli" ] ; then diff --git a/bin/hbase.cmd b/bin/hbase.cmd index 1b2227b9644..68f97c32f48 100644 --- a/bin/hbase.cmd +++ b/bin/hbase.cmd @@ -200,7 +200,7 @@ goto :MakeCmdArgsLoop set hbase-command-arguments=%_hbasearguments% @rem figure out which class to run -set corecommands=shell master regionserver thrift thrift2 rest avro hlog hbck hfile zookeeper zkcli upgrade mapredcp +set corecommands=shell master regionserver thrift thrift2 rest avro hlog wal hbck hfile zookeeper zkcli upgrade mapredcp for %%i in ( %corecommands% ) do ( if "%hbase-command%"=="%%i" set corecommand=true ) @@ -364,8 +364,13 @@ goto :eof set CLASS=org.apache.hadoop.hbase.util.HBaseFsck goto :eof +@rem TODO remove older 'hlog' command :hlog - set CLASS=org.apache.hadoop.hbase.regionserver.wal.HLogPrettyPrinter + set CLASS=org.apache.hadoop.hbase.wal.WALPrettyPrinter + goto :eof + +:wal + set CLASS=org.apache.hadoop.hbase.wal.WALPrettyPrinter goto :eof :hfile @@ -405,7 +410,7 @@ goto :eof echo Some commands take arguments. Pass no args or -h for usage." echo shell Run the HBase shell echo hbck Run the hbase 'fsck' tool - echo hlog Write-ahead-log analyzer + echo wal Write-ahead-log analyzer echo hfile Store file analyzer echo zkcli Run the ZooKeeper shell echo upgrade Upgrade hbase diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java index 6ff1b1f7ca1..bdb7f531dd8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java @@ -22,7 +22,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; /** * Thrown during flush if the possibility snapshot content was not properly - * persisted into store files. Response should include replay of hlog content. + * persisted into store files. Response should include replay of wal content. */ @InterfaceAudience.Public @InterfaceStability.Stable diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index 69b33d092ac..a0ce1795b58 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -890,18 +890,17 @@ public interface Admin extends Abortable, Closeable { HTableDescriptor[] getTableDescriptors(List names) throws IOException; /** - * Roll the log writer. That is, start writing log messages to a new file. + * Roll the log writer. I.e. for filesystem based write ahead logs, start writing to a new file. * - * @param serverName The servername of the regionserver. A server name is made of host, port and - * startcode. This is mandatory. Here is an example: - * host187.example.com,60020,1289493121758 - * @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()} + * Note that the actual rolling of the log writer is asynchronous and may not be complete when + * this method returns. As a side effect of this call, the named region server may schedule + * store flushes at the request of the wal. + * + * @param serverName The servername of the regionserver. * @throws IOException if a remote or network exception occurs * @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException */ - byte[][] rollHLogWriter(String serverName) throws IOException, FailedLogCloseException; + void rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException; /** * Helper delegage to getClusterStatus().getMasterCoprocessors(). diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java index 3a337602a12..73551af8b8d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java @@ -64,7 +64,7 @@ import com.google.common.collect.Lists; * Adding a new peer results in creating new outbound connections from every * region server to a subset of region servers on the slave cluster. Each * new stream of replication will start replicating from the beginning of the - * current HLog, meaning that edits from that past will be replicated. + * current WAL, meaning that edits from that past will be replicated. *

*

* Removing a peer is a destructive and irreversible operation that stops diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java index 122bfdc03c4..027fde4a3af 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java @@ -15,7 +15,7 @@ import java.io.IOException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.ClusterConnection; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java index ced458a504b..70da40c0389 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRespons import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse; -import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest; @@ -183,21 +182,6 @@ public final class ResponseConverter { // End utilities for Client // Start utilities for Admin - /** - * Get the list of regions to flush from a RollLogWriterResponse - * - * @param proto the RollLogWriterResponse - * @return the the list of regions to flush - */ - public static byte[][] getRegions(final RollWALWriterResponse proto) { - if (proto == null || proto.getRegionToFlushCount() == 0) return null; - List regions = new ArrayList(); - for (ByteString region: proto.getRegionToFlushList()) { - regions.add(region.toByteArray()); - } - return (byte[][])regions.toArray(); - } - /** * Get the list of region info from a GetOnlineRegionResponse * diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java index 611f6631c8a..3dbbc336540 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; /** * This provides an interface for maintaining a region server's replication queues. These queues - * keep track of the HLogs that still need to be replicated to remote clusters. + * keep track of the WALs that still need to be replicated to remote clusters. */ @InterfaceAudience.Private public interface ReplicationQueues { @@ -45,31 +45,31 @@ public interface ReplicationQueues { void removeQueue(String queueId); /** - * Add a new HLog file to the given queue. If the queue does not exist it is created. + * Add a new WAL file to the given queue. If the queue does not exist it is created. * @param queueId a String that identifies the queue. - * @param filename name of the HLog + * @param filename name of the WAL */ void addLog(String queueId, String filename) throws ReplicationException; /** - * Remove an HLog file from the given queue. + * Remove an WAL file from the given queue. * @param queueId a String that identifies the queue. - * @param filename name of the HLog + * @param filename name of the WAL */ void removeLog(String queueId, String filename); /** - * Set the current position for a specific HLog in a given queue. + * Set the current position for a specific WAL in a given queue. * @param queueId a String that identifies the queue - * @param filename name of the HLog + * @param filename name of the WAL * @param position the current position in the file */ void setLogPosition(String queueId, String filename, long position); /** - * Get the current position for a specific HLog in a given queue. + * Get the current position for a specific WAL in a given queue. * @param queueId a String that identifies the queue - * @param filename name of the HLog + * @param filename name of the WAL * @return the current position in the file */ long getLogPosition(String queueId, String filename) throws ReplicationException; @@ -80,9 +80,9 @@ public interface ReplicationQueues { void removeAllQueues(); /** - * Get a list of all HLogs in the given queue. + * Get a list of all WALs in the given queue. * @param queueId a String that identifies the queue - * @return a list of HLogs, null if this region server is dead and has no outstanding queues + * @return a list of WALs, null if this region server is dead and has no outstanding queues */ List getLogsInQueue(String queueId); @@ -95,7 +95,7 @@ public interface ReplicationQueues { /** * Take ownership for the set of queues belonging to a dead region server. * @param regionserver the id of the dead region server - * @return A SortedMap of the queues that have been claimed, including a SortedSet of HLogs in + * @return A SortedMap of the queues that have been claimed, including a SortedSet of WALs in * each queue. Returns an empty map if no queues were failed-over. */ SortedMap> claimQueues(String regionserver); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java index 689afba98be..e8fa4df24ba 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java @@ -23,7 +23,7 @@ import java.util.List; /** * This provides an interface for clients of replication to view replication queues. These queues - * keep track of the HLogs that still need to be replicated to remote clusters. + * keep track of the WALs that still need to be replicated to remote clusters. */ public interface ReplicationQueuesClient { @@ -40,10 +40,10 @@ public interface ReplicationQueuesClient { List getListOfReplicators(); /** - * Get a list of all HLogs in the given queue on the given region server. + * Get a list of all WALs in the given queue on the given region server. * @param serverName the server name of the region server that owns the queue * @param queueId a String that identifies the queue - * @return a list of HLogs, null if this region server is dead and has no outstanding queues + * @return a list of WALs, null if this region server is dead and has no outstanding queues */ List getLogsInQueue(String serverName, String queueId); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java index a7d36c1903b..fba1fef018a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java @@ -51,7 +51,7 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem try { result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode); } catch (KeeperException e) { - this.abortable.abort("Failed to get list of hlogs for queueId=" + queueId + this.abortable.abort("Failed to get list of wals for queueId=" + queueId + " and serverName=" + serverName, e); } return result; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java index 28285467387..3ed51c73a71 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java @@ -42,21 +42,21 @@ import org.apache.zookeeper.KeeperException; /** * This class provides an implementation of the ReplicationQueues interface using Zookeeper. The * base znode that this class works at is the myQueuesZnode. The myQueuesZnode contains a list of - * all outstanding HLog files on this region server that need to be replicated. The myQueuesZnode is + * all outstanding WAL files on this region server that need to be replicated. The myQueuesZnode is * the regionserver name (a concatenation of the region server’s hostname, client port and start * code). For example: * * /hbase/replication/rs/hostname.example.org,6020,1234 * - * Within this znode, the region server maintains a set of HLog replication queues. These queues are + * Within this znode, the region server maintains a set of WAL replication queues. These queues are * represented by child znodes named using there give queue id. For example: * * /hbase/replication/rs/hostname.example.org,6020,1234/1 * /hbase/replication/rs/hostname.example.org,6020,1234/2 * - * Each queue has one child znode for every HLog that still needs to be replicated. The value of - * these HLog child znodes is the latest position that has been replicated. This position is updated - * every time a HLog entry is replicated. For example: + * Each queue has one child znode for every WAL that still needs to be replicated. The value of + * these WAL child znodes is the latest position that has been replicated. This position is updated + * every time a WAL entry is replicated. For example: * * /hbase/replication/rs/hostname.example.org,6020,1234/1/23522342.23422 [VALUE: 254] */ @@ -113,7 +113,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R znode = ZKUtil.joinZNode(znode, filename); ZKUtil.deleteNode(this.zookeeper, znode); } catch (KeeperException e) { - this.abortable.abort("Failed to remove hlog from queue (queueId=" + queueId + ", filename=" + this.abortable.abort("Failed to remove wal from queue (queueId=" + queueId + ", filename=" + filename + ")", e); } } @@ -126,7 +126,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R // Why serialize String of Long and not Long as bytes? ZKUtil.setData(this.zookeeper, znode, ZKUtil.positionToByteArray(position)); } catch (KeeperException e) { - this.abortable.abort("Failed to write replication hlog position (filename=" + filename + this.abortable.abort("Failed to write replication wal position (filename=" + filename + ", position=" + position + ")", e); } } @@ -146,12 +146,12 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R return 0; } try { - return ZKUtil.parseHLogPositionFrom(bytes); + return ZKUtil.parseWALPositionFrom(bytes); } catch (DeserializationException de) { - LOG.warn("Failed to parse HLogPosition for queueId=" + queueId + " and hlog=" + filename + LOG.warn("Failed to parse WALPosition for queueId=" + queueId + " and wal=" + filename + "znode content, continuing."); } - // if we can not parse the position, start at the beginning of the hlog file + // if we can not parse the position, start at the beginning of the wal file // again return 0; } @@ -166,10 +166,10 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R SortedMap> newQueues = new TreeMap>(); // check whether there is multi support. If yes, use it. if (conf.getBoolean(HConstants.ZOOKEEPER_USEMULTI, true)) { - LOG.info("Atomically moving " + regionserverZnode + "'s hlogs to my queue"); + LOG.info("Atomically moving " + regionserverZnode + "'s wals to my queue"); newQueues = copyQueuesFromRSUsingMulti(regionserverZnode); } else { - LOG.info("Moving " + regionserverZnode + "'s hlogs to my queue"); + LOG.info("Moving " + regionserverZnode + "'s wals to my queue"); if (!lockOtherRS(regionserverZnode)) { return newQueues; } @@ -200,7 +200,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R try { result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode); } catch (KeeperException e) { - this.abortable.abort("Failed to get list of hlogs for queueId=" + queueId, e); + this.abortable.abort("Failed to get list of wals for queueId=" + queueId, e); } return result; } @@ -283,10 +283,10 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R } /** - * It "atomically" copies all the hlogs queues from another region server and returns them all + * It "atomically" copies all the wals queues from another region server and returns them all * sorted per peer cluster (appended with the dead server's znode). * @param znode pertaining to the region server to copy the queues from - * @return HLog queues sorted per peer cluster + * @return WAL queues sorted per peer cluster */ private SortedMap> copyQueuesFromRSUsingMulti(String znode) { SortedMap> queues = new TreeMap>(); @@ -308,8 +308,8 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R String newPeerZnode = ZKUtil.joinZNode(this.myQueuesZnode, newPeerId); // check the logs queue for the old peer cluster String oldClusterZnode = ZKUtil.joinZNode(deadRSZnodePath, peerId); - List hlogs = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode); - if (hlogs == null || hlogs.size() == 0) { + List wals = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode); + if (wals == null || wals.size() == 0) { listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode)); continue; // empty log queue. } @@ -319,15 +319,15 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R ZKUtilOp op = ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY); listOfOps.add(op); // get the offset of the logs and set it to new znodes - for (String hlog : hlogs) { - String oldHlogZnode = ZKUtil.joinZNode(oldClusterZnode, hlog); - byte[] logOffset = ZKUtil.getData(this.zookeeper, oldHlogZnode); - LOG.debug("Creating " + hlog + " with data " + Bytes.toString(logOffset)); - String newLogZnode = ZKUtil.joinZNode(newPeerZnode, hlog); + for (String wal : wals) { + String oldWalZnode = ZKUtil.joinZNode(oldClusterZnode, wal); + byte[] logOffset = ZKUtil.getData(this.zookeeper, oldWalZnode); + LOG.debug("Creating " + wal + " with data " + Bytes.toString(logOffset)); + String newLogZnode = ZKUtil.joinZNode(newPeerZnode, wal); listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset)); // add ops for deleting - listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldHlogZnode)); - logQueue.add(hlog); + listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode)); + logQueue.add(wal); } // add delete op for peer listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode)); @@ -350,10 +350,10 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R } /** - * This methods copies all the hlogs queues from another region server and returns them all sorted + * This methods copies all the wals queues from another region server and returns them all sorted * per peer cluster (appended with the dead server's znode) * @param znode server names to copy - * @return all hlogs for all peers of that cluster, null if an error occurred + * @return all wals for all peers of that cluster, null if an error occurred */ private SortedMap> copyQueuesFromRS(String znode) { // TODO this method isn't atomic enough, we could start copying and then @@ -381,31 +381,31 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R String newCluster = cluster + "-" + znode; String newClusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, newCluster); String clusterPath = ZKUtil.joinZNode(nodePath, cluster); - List hlogs = ZKUtil.listChildrenNoWatch(this.zookeeper, clusterPath); + List wals = ZKUtil.listChildrenNoWatch(this.zookeeper, clusterPath); // That region server didn't have anything to replicate for this cluster - if (hlogs == null || hlogs.size() == 0) { + if (wals == null || wals.size() == 0) { continue; } ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, newClusterZnode, HConstants.EMPTY_BYTE_ARRAY); SortedSet logQueue = new TreeSet(); queues.put(newCluster, logQueue); - for (String hlog : hlogs) { - String z = ZKUtil.joinZNode(clusterPath, hlog); + for (String wal : wals) { + String z = ZKUtil.joinZNode(clusterPath, wal); byte[] positionBytes = ZKUtil.getData(this.zookeeper, z); long position = 0; try { - position = ZKUtil.parseHLogPositionFrom(positionBytes); + position = ZKUtil.parseWALPositionFrom(positionBytes); } catch (DeserializationException e) { - LOG.warn("Failed parse of hlog position from the following znode: " + z + LOG.warn("Failed parse of wal position from the following znode: " + z + ", Exception: " + e); } - LOG.debug("Creating " + hlog + " with data " + position); - String child = ZKUtil.joinZNode(newClusterZnode, hlog); + LOG.debug("Creating " + wal + " with data " + position); + String child = ZKUtil.joinZNode(newClusterZnode, wal); // Position doesn't actually change, we are just deserializing it for // logging, so just use the already serialized version ZKUtil.createAndWatch(this.zookeeper, child, positionBytes); - logQueue.add(hlog); + logQueue.add(wal); } } } catch (KeeperException e) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java index 31f273e59c9..da0d8b28af1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java @@ -1670,7 +1670,7 @@ public class ZKUtil { if (data != null && data.length > 0) { // log position long position = 0; try { - position = ZKUtil.parseHLogPositionFrom(ZKUtil.getData(zkw, znodeToProcess)); + position = ZKUtil.parseWALPositionFrom(ZKUtil.getData(zkw, znodeToProcess)); sb.append(position); } catch (DeserializationException ignored) { } catch (InterruptedException e) { @@ -1884,7 +1884,7 @@ public class ZKUtil { /** * @param position * @return Serialized protobuf of position with pb magic prefix prepended suitable - * for use as content of an hlog position in a replication queue. + * for use as content of an wal position in a replication queue. */ public static byte[] positionToByteArray(final long position) { byte[] bytes = ZooKeeperProtos.ReplicationHLogPosition.newBuilder().setPosition(position) @@ -1893,13 +1893,13 @@ public class ZKUtil { } /** - * @param bytes - Content of a HLog position znode. - * @return long - The current HLog position. + * @param bytes - Content of a WAL position znode. + * @return long - The current WAL position. * @throws DeserializationException */ - public static long parseHLogPositionFrom(final byte[] bytes) throws DeserializationException { + public static long parseWALPositionFrom(final byte[] bytes) throws DeserializationException { if (bytes == null) { - throw new DeserializationException("Unable to parse null HLog position."); + throw new DeserializationException("Unable to parse null WAL position."); } if (ProtobufUtil.isPBMagicPrefix(bytes)) { int pblen = ProtobufUtil.lengthOfPBMagic(); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index d4d17c9690b..595325acc2e 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -899,7 +899,7 @@ public final class HConstants { /** File permission umask to use when creating hbase data files */ public static final String DATA_FILE_UMASK_KEY = "hbase.data.umask"; - /** Configuration name of HLog Compression */ + /** Configuration name of WAL Compression */ public static final String ENABLE_WAL_COMPRESSION = "hbase.regionserver.wal.enablecompression"; @@ -1031,7 +1031,7 @@ public final class HConstants { /** Configuration key for the name of the master WAL encryption key for the cluster, a string */ public static final String CRYPTO_WAL_KEY_NAME_CONF_KEY = "hbase.crypto.wal.key.name"; - /** Configuration key for enabling HLog encryption, a boolean */ + /** Configuration key for enabling WAL encryption, a boolean */ public static final String ENABLE_WAL_ENCRYPTION = "hbase.regionserver.wal.encryption"; /** Configuration key for setting RPC codec class name */ diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java index 22b40ec223c..516fd811c7e 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java @@ -1135,7 +1135,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId, /** * Produces a string map for this key/value pair. Useful for programmatic use - * and manipulation of the data stored in an HLogKey, for example, printing + * and manipulation of the data stored in an WALKey, for example, printing * as JSON. Values are left out due to their tendency to be large. If needed, * they can be added manually. * diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml index 90b2c0fae31..b9e96db8b54 100644 --- a/hbase-common/src/main/resources/hbase-default.xml +++ b/hbase-common/src/main/resources/hbase-default.xml @@ -112,8 +112,8 @@ possible configurations would overwhelm and obscure the important. hbase.master.logcleaner.plugins org.apache.hadoop.hbase.master.cleaner.TimeToLiveLogCleaner A comma-separated list of BaseLogCleanerDelegate invoked by - the LogsCleaner service. These WAL/HLog cleaners are called in order, - so put the HLog cleaner that prunes the most HLog files in front. To + the LogsCleaner service. These WAL cleaners are called in order, + so put the cleaner that prunes the most files in front. To implement your own BaseLogCleanerDelegate, just put it in HBase's classpath and add the fully qualified class name here. Always add the above default log cleaners in the list. @@ -121,7 +121,7 @@ possible configurations would overwhelm and obscure the important. hbase.master.logcleaner.ttl 600000 - Maximum time a HLog can stay in the .oldlogdir directory, + Maximum time a WAL can stay in the .oldlogdir directory, after which it will be cleaned by a Master thread. @@ -265,12 +265,12 @@ possible configurations would overwhelm and obscure the important. hbase.regionserver.hlog.reader.impl org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader - The HLog file reader implementation. + The WAL file reader implementation. hbase.regionserver.hlog.writer.impl org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter - The HLog file writer implementation. + The WAL file writer implementation. hbase.master.distributed.log.replay diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java index 2307599c22d..6cf942b2bd1 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java @@ -48,9 +48,9 @@ public interface MetricsMasterFileSystemSource extends BaseSource { String SPLIT_SIZE_NAME = "hlogSplitSize"; String META_SPLIT_TIME_DESC = "Time it takes to finish splitMetaLog()"; - String META_SPLIT_SIZE_DESC = "Size of hbase:meta HLog files being split"; - String SPLIT_TIME_DESC = "Time it takes to finish HLog.splitLog()"; - String SPLIT_SIZE_DESC = "Size of HLog files being split"; + String META_SPLIT_SIZE_DESC = "Size of hbase:meta WAL files being split"; + String SPLIT_TIME_DESC = "Time it takes to finish WAL.splitLog()"; + String SPLIT_SIZE_DESC = "Size of WAL files being split"; void updateMetaWALSplitTime(long time); diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java index 458ed01910e..7ac2501607f 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java @@ -129,10 +129,10 @@ public interface MetricsRegionServerSource extends BaseSource { String REGION_COUNT_DESC = "Number of regions"; String STORE_COUNT = "storeCount"; String STORE_COUNT_DESC = "Number of Stores"; - String HLOGFILE_COUNT = "hlogFileCount"; - String HLOGFILE_COUNT_DESC = "Number of HLog Files"; - String HLOGFILE_SIZE = "hlogFileSize"; - String HLOGFILE_SIZE_DESC = "Size of all HLog Files"; + String WALFILE_COUNT = "hlogFileCount"; + String WALFILE_COUNT_DESC = "Number of WAL Files"; + String WALFILE_SIZE = "hlogFileSize"; + String WALFILE_SIZE_DESC = "Size of all WAL Files"; String STOREFILE_COUNT = "storeFileCount"; String STOREFILE_COUNT_DESC = "Number of Store Files"; String MEMSTORE_SIZE = "memStoreSize"; diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java index 513a0db8351..e0b5e1b0d24 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java @@ -68,14 +68,14 @@ public interface MetricsRegionServerWrapper { long getNumStores(); /** - * Get the number of HLog files of this region server. + * Get the number of WAL files of this region server. */ - public long getNumHLogFiles(); + public long getNumWALFiles(); /** - * Get the size of HLog files of this region server. + * Get the size of WAL files of this region server. */ - public long getHLogFileSize(); + public long getWALFileSize(); /** * Get the number of store files hosted on this region server. diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySource.java index 793429d7a42..4f8cb36d3d1 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySource.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySource.java @@ -39,7 +39,7 @@ public interface MetricsEditsReplaySource extends BaseSource { /** * Description */ - String METRICS_DESCRIPTION = "Metrics about HBase RegionServer HLog Edits Replay"; + String METRICS_DESCRIPTION = "Metrics about HBase RegionServer WAL Edits Replay"; /** * The name of the metrics context that metrics will be under in jmx diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSource.java index 1c59f657139..ba0df80e1d0 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSource.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSource.java @@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.regionserver.wal; import org.apache.hadoop.hbase.metrics.BaseSource; /** - * Interface of the source that will export metrics about the region server's HLog. + * Interface of the source that will export metrics about the region server's WAL. */ public interface MetricsWALSource extends BaseSource { @@ -39,7 +39,7 @@ public interface MetricsWALSource extends BaseSource { /** * Description */ - String METRICS_DESCRIPTION = "Metrics about HBase RegionServer HLog"; + String METRICS_DESCRIPTION = "Metrics about HBase RegionServer WAL"; /** * The name of the metrics context that metrics will be under in jmx @@ -52,11 +52,11 @@ public interface MetricsWALSource extends BaseSource { String APPEND_COUNT = "appendCount"; String APPEND_COUNT_DESC = "Number of appends to the write ahead log."; String APPEND_SIZE = "appendSize"; - String APPEND_SIZE_DESC = "Size (in bytes) of the data appended to the HLog."; + String APPEND_SIZE_DESC = "Size (in bytes) of the data appended to the WAL."; String SLOW_APPEND_COUNT = "slowAppendCount"; String SLOW_APPEND_COUNT_DESC = "Number of appends that were slow."; String SYNC_TIME = "syncTime"; - String SYNC_TIME_DESC = "The time it took to sync the HLog to HDFS."; + String SYNC_TIME_DESC = "The time it took to sync the WAL to HDFS."; /** * Add the append size. @@ -69,7 +69,7 @@ public interface MetricsWALSource extends BaseSource { void incrementAppendTime(long time); /** - * Increment the count of hlog appends + * Increment the count of wal appends */ void incrementAppendCount(); @@ -79,7 +79,7 @@ public interface MetricsWALSource extends BaseSource { void incrementSlowAppendCount(); /** - * Add the time it took to sync the hlog. + * Add the time it took to sync the wal. */ void incrementSyncTime(long time); diff --git a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestMetricsHLogSource.java b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestMetricsHLogSource.java deleted file mode 100644 index b2bf1f208f2..00000000000 --- a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestMetricsHLogSource.java +++ /dev/null @@ -1,32 +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 org.apache.hadoop.hbase.CompatibilitySingletonFactory; -import org.junit.Test; - -public class TestMetricsHLogSource { - - @Test(expected=RuntimeException.class) - public void testGetInstanceNoHadoopCompat() throws Exception { - //This should throw an exception because there is no compat lib on the class path. - CompatibilitySingletonFactory.getInstance(MetricsWALSource.class); - - } -} diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java index bc1aa079f26..d3414e20cb8 100644 --- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java +++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java @@ -168,8 +168,8 @@ public class MetricsRegionServerSourceImpl if (rsWrap != null) { mrb.addGauge(Interns.info(REGION_COUNT, REGION_COUNT_DESC), rsWrap.getNumOnlineRegions()) .addGauge(Interns.info(STORE_COUNT, STORE_COUNT_DESC), rsWrap.getNumStores()) - .addGauge(Interns.info(HLOGFILE_COUNT, HLOGFILE_COUNT_DESC), rsWrap.getNumHLogFiles()) - .addGauge(Interns.info(HLOGFILE_SIZE, HLOGFILE_SIZE_DESC), rsWrap.getHLogFileSize()) + .addGauge(Interns.info(WALFILE_COUNT, WALFILE_COUNT_DESC), rsWrap.getNumWALFiles()) + .addGauge(Interns.info(WALFILE_SIZE, WALFILE_SIZE_DESC), rsWrap.getWALFileSize()) .addGauge(Interns.info(STOREFILE_COUNT, STOREFILE_COUNT_DESC), rsWrap.getNumStoreFiles()) .addGauge(Interns.info(MEMSTORE_SIZE, MEMSTORE_SIZE_DESC), rsWrap.getMemstoreSize()) .addGauge(Interns.info(STOREFILE_SIZE, STOREFILE_SIZE_DESC), rsWrap.getStoreFileSize()) diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSourceImpl.java index ad8f24cf263..d602d2f917e 100644 --- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSourceImpl.java +++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSourceImpl.java @@ -25,9 +25,10 @@ import org.apache.hadoop.metrics2.lib.MutableCounterLong; /** - * Class that transitions metrics from HLog's MetricsWAL into the metrics subsystem. + * Class that transitions metrics from MetricsWAL into the metrics subsystem. * * Implements BaseSource through BaseSourceImpl, following the pattern. + * @see org.apache.hadoop.hbase.regionserver.wal.MetricsWAL */ @InterfaceAudience.Private public class MetricsWALSourceImpl extends BaseSourceImpl implements MetricsWALSource { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithEncryption.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithEncryption.java index 9212095b882..66ac62fcc94 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithEncryption.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithEncryption.java @@ -26,7 +26,8 @@ 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.wal.WAL.Reader; +import org.apache.hadoop.hbase.wal.WALProvider.Writer; import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader; import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter; import org.apache.hadoop.hbase.testclassification.IntegrationTests; @@ -61,9 +62,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); + Reader.class); conf.setClass("hbase.regionserver.hlog.writer.impl", SecureProtobufLogWriter.class, - HLog.Writer.class); + Writer.class); conf.setBoolean(HConstants.ENABLE_WAL_ENCRYPTION, true); } super.setUpCluster(); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java index 12adc80dc04..f64528b02d8 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java @@ -82,7 +82,7 @@ import com.google.common.base.Objects; *

    *
  1. * Load Test Tool.
    - * This runs so that all RegionServers will have some load and HLogs will be full. + * This runs so that all RegionServers will have some load and WALs will be full. *
  2. *
  3. * Scan thread.
    @@ -151,7 +151,7 @@ public class IntegrationTestMTTR { private static Action restartMasterAction; /** - * The load test tool used to create load and make sure that HLogs aren't empty. + * The load test tool used to create load and make sure that WALs aren't empty. */ private static LoadTestTool loadTool; diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java index 9ffe0fdce66..382874289df 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java @@ -16956,6 +16956,12 @@ public final class AdminProtos { } /** * Protobuf type {@code RollWALWriterResponse} + * + *
    +   *
    +   * Roll request responses no longer include regions to flush
    +   * this list will always be empty when talking to a 1.0 server
    +   * 
    */ public static final class RollWALWriterResponse extends com.google.protobuf.GeneratedMessage @@ -17241,6 +17247,12 @@ public final class AdminProtos { } /** * Protobuf type {@code RollWALWriterResponse} + * + *
    +     *
    +     * Roll request responses no longer include regions to flush
    +     * this list will always be empty when talking to a 1.0 server
    +     * 
    */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RegionServerStatusProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RegionServerStatusProtos.java index 19a8d74a0bb..ec169d54f2f 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RegionServerStatusProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RegionServerStatusProtos.java @@ -4496,7 +4496,7 @@ public final class RegionServerStatusProtos { * required uint64 last_flushed_sequence_id = 1; * *
    -     ** the last HLog sequence id flushed from MemStore to HFile for the region 
    +     * the last WAL sequence id flushed from MemStore to HFile for the region 
          * 
    */ boolean hasLastFlushedSequenceId(); @@ -4504,7 +4504,7 @@ public final class RegionServerStatusProtos { * required uint64 last_flushed_sequence_id = 1; * *
    -     ** the last HLog sequence id flushed from MemStore to HFile for the region 
    +     * the last WAL sequence id flushed from MemStore to HFile for the region 
          * 
    */ long getLastFlushedSequenceId(); @@ -4612,7 +4612,7 @@ public final class RegionServerStatusProtos { * required uint64 last_flushed_sequence_id = 1; * *
    -     ** the last HLog sequence id flushed from MemStore to HFile for the region 
    +     * the last WAL sequence id flushed from MemStore to HFile for the region 
          * 
    */ public boolean hasLastFlushedSequenceId() { @@ -4622,7 +4622,7 @@ public final class RegionServerStatusProtos { * required uint64 last_flushed_sequence_id = 1; * *
    -     ** the last HLog sequence id flushed from MemStore to HFile for the region 
    +     * the last WAL sequence id flushed from MemStore to HFile for the region 
          * 
    */ public long getLastFlushedSequenceId() { @@ -4908,7 +4908,7 @@ public final class RegionServerStatusProtos { * required uint64 last_flushed_sequence_id = 1; * *
    -       ** the last HLog sequence id flushed from MemStore to HFile for the region 
    +       * the last WAL sequence id flushed from MemStore to HFile for the region 
            * 
    */ public boolean hasLastFlushedSequenceId() { @@ -4918,7 +4918,7 @@ public final class RegionServerStatusProtos { * required uint64 last_flushed_sequence_id = 1; * *
    -       ** the last HLog sequence id flushed from MemStore to HFile for the region 
    +       * the last WAL sequence id flushed from MemStore to HFile for the region 
            * 
    */ public long getLastFlushedSequenceId() { @@ -4928,7 +4928,7 @@ public final class RegionServerStatusProtos { * required uint64 last_flushed_sequence_id = 1; * *
    -       ** the last HLog sequence id flushed from MemStore to HFile for the region 
    +       * the last WAL sequence id flushed from MemStore to HFile for the region 
            * 
    */ public Builder setLastFlushedSequenceId(long value) { @@ -4941,7 +4941,7 @@ public final class RegionServerStatusProtos { * required uint64 last_flushed_sequence_id = 1; * *
    -       ** the last HLog sequence id flushed from MemStore to HFile for the region 
    +       * the last WAL sequence id flushed from MemStore to HFile for the region 
            * 
    */ public Builder clearLastFlushedSequenceId() { diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java index af61d47d6b5..977db42bbd0 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java @@ -1242,7 +1242,9 @@ public final class WALProtos { * Protobuf type {@code WALKey} * *
    -   * Protocol buffer version of HLogKey; see HLogKey comment, not really a key but WALEdit header for some KVs
    +   *
    +   * Protocol buffer version of WALKey; see WALKey comment, not really a key but WALEdit header
    +   * for some KVs
        * 
    */ public static final class WALKey extends @@ -2033,7 +2035,9 @@ public final class WALProtos { * Protobuf type {@code WALKey} * *
    -     * Protocol buffer version of HLogKey; see HLogKey comment, not really a key but WALEdit header for some KVs
    +     *
    +     * Protocol buffer version of WALKey; see WALKey comment, not really a key but WALEdit header
    +     * for some KVs
          * 
    */ public static final class Builder extends @@ -10021,8 +10025,10 @@ public final class WALProtos { * *
        **
    -   * A trailer that is appended to the end of a properly closed HLog WAL file.
    +   * A trailer that is appended to the end of a properly closed WAL file.
        * If missing, this is either a legacy or a corrupted WAL file.
    +   * N.B. This trailer currently doesn't contain any information and we
    +   * purposefully don't expose it in the WAL APIs. It's for future growth.
        * 
    */ public static final class WALTrailer extends @@ -10246,8 +10252,10 @@ public final class WALProtos { * *
          **
    -     * A trailer that is appended to the end of a properly closed HLog WAL file.
    +     * A trailer that is appended to the end of a properly closed WAL file.
          * If missing, this is either a legacy or a corrupted WAL file.
    +     * N.B. This trailer currently doesn't contain any information and we
    +     * purposefully don't expose it in the WAL APIs. It's for future growth.
          * 
    */ public static final class Builder extends diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java index d46bc1b09c0..5a1fbf13bed 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java @@ -6020,7 +6020,7 @@ public final class ZooKeeperProtos { * *
        **
    -   * Used by replication. Holds the current position in an HLog file.
    +   * Used by replication. Holds the current position in an WAL file.
        * 
    */ public static final class ReplicationHLogPosition extends @@ -6288,7 +6288,7 @@ public final class ZooKeeperProtos { * *
          **
    -     * Used by replication. Holds the current position in an HLog file.
    +     * Used by replication. Holds the current position in an WAL file.
          * 
    */ public static final class Builder extends diff --git a/hbase-protocol/src/main/protobuf/Admin.proto b/hbase-protocol/src/main/protobuf/Admin.proto index ec34e72546b..fcc4e1d0248 100644 --- a/hbase-protocol/src/main/protobuf/Admin.proto +++ b/hbase-protocol/src/main/protobuf/Admin.proto @@ -204,6 +204,10 @@ message ReplicateWALEntryResponse { message RollWALWriterRequest { } +/* + * Roll request responses no longer include regions to flush + * this list will always be empty when talking to a 1.0 server + */ message RollWALWriterResponse { // A list of encoded name of regions to flush repeated bytes region_to_flush = 1; diff --git a/hbase-protocol/src/main/protobuf/RegionServerStatus.proto b/hbase-protocol/src/main/protobuf/RegionServerStatus.proto index df9a521121c..75e5ae4903b 100644 --- a/hbase-protocol/src/main/protobuf/RegionServerStatus.proto +++ b/hbase-protocol/src/main/protobuf/RegionServerStatus.proto @@ -74,7 +74,7 @@ message GetLastFlushedSequenceIdRequest { } message GetLastFlushedSequenceIdResponse { - /** the last HLog sequence id flushed from MemStore to HFile for the region */ + /* the last WAL sequence id flushed from MemStore to HFile for the region */ required uint64 last_flushed_sequence_id = 1; } diff --git a/hbase-protocol/src/main/protobuf/WAL.proto b/hbase-protocol/src/main/protobuf/WAL.proto index dae92d2eabd..f8a15344dca 100644 --- a/hbase-protocol/src/main/protobuf/WAL.proto +++ b/hbase-protocol/src/main/protobuf/WAL.proto @@ -31,7 +31,10 @@ message WALHeader { optional string cell_codec_cls_name = 5; } -// Protocol buffer version of HLogKey; see HLogKey comment, not really a key but WALEdit header for some KVs +/* + * Protocol buffer version of WALKey; see WALKey comment, not really a key but WALEdit header + * for some KVs + */ message WALKey { required bytes encoded_region_name = 1; required bytes table_name = 2; @@ -144,8 +147,10 @@ message RegionEventDescriptor { } /** - * A trailer that is appended to the end of a properly closed HLog WAL file. + * A trailer that is appended to the end of a properly closed WAL file. * If missing, this is either a legacy or a corrupted WAL file. + * N.B. This trailer currently doesn't contain any information and we + * purposefully don't expose it in the WAL APIs. It's for future growth. */ message WALTrailer { } diff --git a/hbase-protocol/src/main/protobuf/ZooKeeper.proto b/hbase-protocol/src/main/protobuf/ZooKeeper.proto index 9512bd69a28..bac881bc1f2 100644 --- a/hbase-protocol/src/main/protobuf/ZooKeeper.proto +++ b/hbase-protocol/src/main/protobuf/ZooKeeper.proto @@ -128,7 +128,7 @@ message ReplicationState { } /** - * Used by replication. Holds the current position in an HLog file. + * Used by replication. Holds the current position in an WAL file. */ message ReplicationHLogPosition { required int64 position = 1; diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon index d09c4bf4b64..c496bf66584 100644 --- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon +++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon @@ -40,7 +40,7 @@ java.lang.management.ManagementFactory;
  4. Base Stats
  5. Memory
  6. Requests
  7. -
  8. hlogs
  9. +
  10. wals
  11. Storefiles
  12. Queues
  13. @@ -55,7 +55,7 @@ java.lang.management.ManagementFactory; <& requestStats; mWrap = mWrap &>
    - <& hlogStats; mWrap = mWrap &> + <& walStats; mWrap = mWrap &>
    <& storeStats; mWrap = mWrap &> @@ -76,7 +76,7 @@ java.lang.management.ManagementFactory; Requests Per Second Num. Regions Block locality - Slow HLog Append Count + Slow WAL Append Count <% String.format("%.0f", mWrap.getRequestsPerSecond()) %> @@ -119,20 +119,20 @@ MetricsRegionServerWrapper mWrap; -<%def hlogStats> +<%def walStats> <%args> MetricsRegionServerWrapper mWrap; - - + + - - + +
    Num. HLog FilesSize. HLog Files (bytes)Num. WAL FilesSize. WAL Files (bytes)
    <% mWrap.getNumHLogFiles() %><% mWrap.getHLogFileSize() %><% mWrap.getNumWALFiles() %><% mWrap.getWALFileSize() %>
    diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java index b9614b4ef4c..f1a8c59d36c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java @@ -32,7 +32,7 @@ public class SplitLogCounters { public final static AtomicLong tot_mgr_log_split_batch_start = new AtomicLong(0); public final static AtomicLong tot_mgr_log_split_batch_success = new AtomicLong(0); public final static AtomicLong tot_mgr_log_split_batch_err = new AtomicLong(0); - public final static AtomicLong tot_mgr_new_unexpected_hlogs = new AtomicLong(0); + public final static AtomicLong tot_mgr_new_unexpected_wals = new AtomicLong(0); public final static AtomicLong tot_mgr_log_split_start = new AtomicLong(0); public final static AtomicLong tot_mgr_log_split_success = new AtomicLong(0); public final static AtomicLong tot_mgr_log_split_err = new AtomicLong(0); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogTask.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogTask.java index bb7a14e8e2d..e0caf321029 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogTask.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogTask.java @@ -24,7 +24,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; -import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; import org.apache.hadoop.hbase.util.Bytes; import com.google.protobuf.InvalidProtocolBufferException; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogWorkerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogWorkerCoordination.java index ed3b2320607..63990ef4a00 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogWorkerCoordination.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogWorkerCoordination.java @@ -29,12 +29,12 @@ import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionStoreSeq import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.SplitLogWorker; import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor; -import org.apache.hadoop.hbase.regionserver.handler.HLogSplitterHandler; +import org.apache.hadoop.hbase.regionserver.handler.WALSplitterHandler; import com.google.common.annotations.VisibleForTesting; /** - * Coordinated operations for {@link SplitLogWorker} and {@link HLogSplitterHandler} Important + * Coordinated operations for {@link SplitLogWorker} and {@link WALSplitterHandler} Important * methods for SplitLogWorker:
    * {@link #isReady()} called from {@link SplitLogWorker#run()} to check whether the coordination is * ready to supply the tasks
    @@ -44,7 +44,7 @@ import com.google.common.annotations.VisibleForTesting; * for external changes in coordination (if required)
    * {@link #endTask(SplitLogTask, AtomicLong, SplitTaskDetails)} notify coordination engine that *

    - * Important methods for HLogSplitterHandler:
    + * Important methods for WALSplitterHandler:
    * splitting task has completed. */ @InterfaceAudience.Private @@ -112,7 +112,7 @@ public interface SplitLogWorkerCoordination { */ void removeListener(); - /* HLogSplitterHandler part */ + /* WALSplitterHandler part */ /** * Notify coordination engine that splitting task has completed. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java index 0f8baa3691f..b67a1c49ee9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java @@ -54,8 +54,8 @@ import org.apache.hadoop.hbase.master.SplitLogManager.Task; import org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; import org.apache.hadoop.hbase.regionserver.SplitLogWorker; -import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; import org.apache.hadoop.hbase.zookeeper.ZKUtil; @@ -115,7 +115,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements @Override public Status finish(ServerName workerName, String logfile) { try { - HLogSplitter.finishSplitLogFile(logfile, manager.getServer().getConfiguration()); + WALSplitter.finishSplitLogFile(logfile, manager.getServer().getConfiguration()); } catch (IOException e) { LOG.warn("Could not finish splitting of log file " + logfile, e); return Status.ERR; @@ -716,7 +716,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements } // decode the file name t = ZKSplitLog.getFileName(t); - ServerName serverName = HLogUtil.getServerNameFromHLogDirectoryName(new Path(t)); + ServerName serverName = DefaultWALProvider.getServerNameFromWALDirectoryName(new Path(t)); if (serverName != null) { knownFailedServers.add(serverName.getServerName()); } else { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java index 2945565a586..0e14618d9a5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java @@ -46,8 +46,8 @@ import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.SplitLogWorker; import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor; -import org.apache.hadoop.hbase.regionserver.handler.HLogSplitterHandler; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.regionserver.handler.WALSplitterHandler; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; import org.apache.hadoop.hbase.util.CancelableProgressable; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; @@ -318,8 +318,8 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements splitTaskDetails.setTaskNode(curTask); splitTaskDetails.setCurTaskZKVersion(zkVersion); - HLogSplitterHandler hsh = - new HLogSplitterHandler(server, this, splitTaskDetails, reporter, + WALSplitterHandler hsh = + new WALSplitterHandler(server, this, splitTaskDetails, reporter, this.tasksInProgress, splitTaskExecutor, mode); server.getExecutorService().submit(hsh); } @@ -418,7 +418,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements // pick meta wal firstly int offset = (int) (Math.random() * paths.size()); for (int i = 0; i < paths.size(); i++) { - if (HLogUtil.isMetaFile(paths.get(i))) { + if (DefaultWALProvider.isMetaFile(paths.get(i))) { offset = i; break; } @@ -581,7 +581,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements } /* - * Next part is related to HLogSplitterHandler + * Next part is related to WALSplitterHandler */ /** * endTask() can fail and the only way to recover out of it is for the {@link SplitLogManager} to diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java index 8b26eea71ea..215ff16696b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java @@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFile.Reader; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Pair; @@ -444,14 +445,32 @@ public abstract class BaseRegionObserver implements RegionObserver { final InternalScanner s) throws IOException { } + /** + * Implementers should override this version of the method and leave the deprecated one as-is. + */ + @Override + public void preWALRestore(ObserverContext env, + HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException { + } + @Override public void preWALRestore(ObserverContext env, HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException { + preWALRestore(env, info, (WALKey)logKey, logEdit); + } + + /** + * Implementers should override this version of the method and leave the deprecated one as-is. + */ + @Override + public void postWALRestore(ObserverContext env, + HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException { } @Override public void postWALRestore(ObserverContext env, HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException { + postWALRestore(env, info, (WALKey)logKey, logEdit); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseWALObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseWALObserver.java index 0836da93747..cfddcd4d4d3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseWALObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseWALObserver.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; /** @@ -42,13 +43,31 @@ public class BaseWALObserver implements WALObserver { @Override public void stop(CoprocessorEnvironment e) throws IOException { } + /** + * Implementers should override this method and leave the deprecated version as-is. + */ @Override - public boolean preWALWrite(ObserverContext ctx, HRegionInfo info, - HLogKey logKey, WALEdit logEdit) throws IOException { + public boolean preWALWrite(ObserverContext ctx, + HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException { return false; } + @Override + public boolean preWALWrite(ObserverContext ctx, HRegionInfo info, + HLogKey logKey, WALEdit logEdit) throws IOException { + return preWALWrite(ctx, info, (WALKey)logKey, logEdit); + } + + /** + * Implementers should override this method and leave the deprecated version as-is. + */ + @Override + public void postWALWrite(ObserverContext ctx, + HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException { } + @Override public void postWALWrite(ObserverContext ctx, HRegionInfo info, - HLogKey logKey, WALEdit logEdit) throws IOException { } + HLogKey logKey, WALEdit logEdit) throws IOException { + postWALWrite(ctx, info, (WALKey)logKey, logEdit); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java index 17fcabc58b7..f819fbc8ece 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java @@ -29,6 +29,7 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; import java.util.UUID; +import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; @@ -557,4 +558,77 @@ public abstract class CoprocessorHost { "coprocessor set.", e); } } + + /** + * Used to gracefully handle fallback to deprecated methods when we + * evolve coprocessor APIs. + * + * When a particular Coprocessor API is updated to change methods, hosts can support fallback + * to the deprecated API by using this method to determine if an instance implements the new API. + * In the event that said support is partial, then in the face of a runtime issue that prevents + * proper operation {@link #legacyWarning(Class, String)} should be used to let operators know. + * + * For examples of this in action, see the implementation of + *

      + *
    • {@link org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost} + *
    • {@link org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost} + *
    + * + * @param clazz Coprocessor you wish to evaluate + * @param methodName the name of the non-deprecated method version + * @param parameterTypes the Class of the non-deprecated method's arguments in the order they are + * declared. + */ + @InterfaceAudience.Private + protected static boolean useLegacyMethod(final Class clazz, + final String methodName, final Class... parameterTypes) { + boolean useLegacy; + // Use reflection to see if they implement the non-deprecated version + try { + clazz.getDeclaredMethod(methodName, parameterTypes); + LOG.debug("Found an implementation of '" + methodName + "' that uses updated method " + + "signature. Skipping legacy support for invocations in '" + clazz +"'."); + useLegacy = false; + } catch (NoSuchMethodException exception) { + useLegacy = true; + } catch (SecurityException exception) { + LOG.warn("The Security Manager denied our attempt to detect if the coprocessor '" + clazz + + "' requires legacy support; assuming it does. If you get later errors about legacy " + + "coprocessor use, consider updating your security policy to allow access to the package" + + " and declared members of your implementation."); + LOG.debug("Details of Security Manager rejection.", exception); + useLegacy = true; + } + return useLegacy; + } + + /** + * Used to limit legacy handling to once per Coprocessor class per classloader. + */ + private static final Set> legacyWarning = + new ConcurrentSkipListSet>( + new Comparator>() { + @Override + public int compare(Class c1, Class c2) { + if (c1.equals(c2)) { + return 0; + } + return c1.getName().compareTo(c2.getName()); + } + }); + + /** + * limits the amount of logging to once per coprocessor class. + * Used in concert with {@link #useLegacyMethod(Class, String, Class[])} when a runtime issue + * prevents properly supporting the legacy version of a coprocessor API. + * Since coprocessors can be in tight loops this serves to limit the amount of log spam we create. + */ + @InterfaceAudience.Private + protected void legacyWarning(final Class clazz, final String message) { + if(legacyWarning.add(clazz)) { + LOG.error("You have a legacy coprocessor loaded and there are events we can't map to the " + + " deprecated API. Your coprocessor will not see these events. Please update '" + clazz + + "'. Details of the problem: " + message); + } + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java index e526d63278c..ee43cba03f4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java @@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileScanner; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Pair; @@ -67,6 +68,9 @@ import com.google.common.collect.ImmutableList; */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) @InterfaceStability.Evolving +// TODO as method signatures need to break, update to +// ObserverContext +// so we can use additional environment state that isn't exposed to coprocessors. public interface RegionObserver extends Coprocessor { /** Mutation type for postMutationBeforeWAL hook */ @@ -1109,26 +1113,62 @@ public interface RegionObserver extends Coprocessor { /** * Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit} * replayed for this region. - * - * @param ctx - * @param info - * @param logKey - * @param logEdit - * @throws IOException */ + void preWALRestore(final ObserverContext ctx, + HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException; + + /** + * Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit} + * replayed for this region. + * + * This method is left in place to maintain binary compatibility with older + * {@link RegionObserver}s. If an implementation directly overrides + * {@link #preWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version + * won't be called at all, barring problems with the Security Manager. To work correctly + * in the presence of a strict Security Manager, or in the case of an implementation that + * relies on a parent class to implement preWALRestore, you should implement this method + * as a call to the non-deprecated version. + * + * Users of this method will see all edits that can be treated as HLogKey. If there are + * edits that can't be treated as HLogKey they won't be offered to coprocessors that rely + * on this method. If a coprocessor gets skipped because of this mechanism, a log message + * at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per + * classloader. + * + * @deprecated use {@link #preWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)} + */ + @Deprecated void preWALRestore(final ObserverContext ctx, HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException; /** * Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit} * replayed for this region. - * - * @param ctx - * @param info - * @param logKey - * @param logEdit - * @throws IOException */ + void postWALRestore(final ObserverContext ctx, + HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException; + + /** + * Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit} + * replayed for this region. + * + * This method is left in place to maintain binary compatibility with older + * {@link RegionObserver}s. If an implementation directly overrides + * {@link #postWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version + * won't be called at all, barring problems with the Security Manager. To work correctly + * in the presence of a strict Security Manager, or in the case of an implementation that + * relies on a parent class to implement preWALRestore, you should implement this method + * as a call to the non-deprecated version. + * + * Users of this method will see all edits that can be treated as HLogKey. If there are + * edits that can't be treated as HLogKey they won't be offered to coprocessors that rely + * on this method. If a coprocessor gets skipped because of this mechanism, a log message + * at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per + * classloader. + * + * @deprecated use {@link #postWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)} + */ + @Deprecated void postWALRestore(final ObserverContext ctx, HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java index d16eed80a8a..a4ce5f1a0bf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java @@ -23,11 +23,11 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.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.wal.WAL; @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) @InterfaceStability.Evolving public interface WALCoprocessorEnvironment extends CoprocessorEnvironment { - /** @return reference to the region server services */ - HLog getWAL(); + /** @return reference to the region server's WAL */ + WAL getWAL(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java index 49d84ed6327..bba83cc3838 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import java.io.IOException; @@ -40,7 +41,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.wal.WAL}. */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) @InterfaceStability.Evolving @@ -50,27 +51,65 @@ public interface WALObserver extends Coprocessor { * Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit} * is writen to WAL. * - * @param ctx - * @param info - * @param logKey - * @param logEdit * @return true if default behavior should be bypassed, false otherwise - * @throws IOException */ // TODO: return value is not used + boolean preWALWrite(ObserverContext ctx, + HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException; + + /** + * Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit} + * is writen to WAL. + * + * This method is left in place to maintain binary compatibility with older + * {@link WALObserver}s. If an implementation directly overrides + * {@link #preWALWrite(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version + * won't be called at all, barring problems with the Security Manager. To work correctly + * in the presence of a strict Security Manager, or in the case of an implementation that + * relies on a parent class to implement preWALWrite, you should implement this method + * as a call to the non-deprecated version. + * + * Users of this method will see all edits that can be treated as HLogKey. If there are + * edits that can't be treated as HLogKey they won't be offered to coprocessors that rely + * on this method. If a coprocessor gets skipped because of this mechanism, a log message + * at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per + * classloader. + * + * @return true if default behavior should be bypassed, false otherwise + * @deprecated use {@link #preWALWrite(ObserverContext, HRegionInfo, WALKey, WALEdit)} + */ + @Deprecated boolean preWALWrite(ObserverContext ctx, HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException; /** * Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit} * is writen to WAL. - * - * @param ctx - * @param info - * @param logKey - * @param logEdit - * @throws IOException */ + void postWALWrite(ObserverContext ctx, + HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException; + + /** + * Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit} + * is writen to WAL. + * + * This method is left in place to maintain binary compatibility with older + * {@link WALObserver}s. If an implementation directly overrides + * {@link #postWALWrite(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version + * won't be called at all, barring problems with the Security Manager. To work correctly + * in the presence of a strict Security Manager, or in the case of an implementation that + * relies on a parent class to implement preWALWrite, you should implement this method + * as a call to the non-deprecated version. + * + * Users of this method will see all edits that can be treated as HLogKey. If there are + * edits that can't be treated as HLogKey they won't be offered to coprocessors that rely + * on this method. If a coprocessor gets skipped because of this mechanism, a log message + * at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per + * classloader. + * + * @deprecated use {@link #postWALWrite(ObserverContext, HRegionInfo, WALKey, WALEdit)} + */ + @Deprecated void postWALWrite(ObserverContext ctx, HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java index f8cf7b30ccf..fb5836055d9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java @@ -40,7 +40,7 @@ import org.apache.hadoop.fs.FilterFileSystem; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.protocol.ClientProtocol; @@ -54,8 +54,8 @@ import org.apache.hadoop.util.ReflectionUtils; /** * An encapsulation for the FileSystem object that hbase uses to access * data. This class allows the flexibility of using - * separate filesystem objects for reading and writing hfiles and hlogs. - * In future, if we want to make hlogs be in a different filesystem, + * separate filesystem objects for reading and writing hfiles and wals. + * In future, if we want to make wals be in a different filesystem, * this is the place to make it happen. */ public class HFileSystem extends FilterFileSystem { @@ -322,7 +322,7 @@ public class HFileSystem extends FilterFileSystem { } /** - * We're putting at lowest priority the hlog files blocks that are on the same datanode + * We're putting at lowest priority the wal files blocks that are on the same datanode * as the original regionserver which created these files. This because we fear that the * datanode is actually dead, so if we use it it will timeout. */ @@ -330,17 +330,17 @@ public class HFileSystem extends FilterFileSystem { public void reorderBlocks(Configuration conf, LocatedBlocks lbs, String src) throws IOException { - ServerName sn = HLogUtil.getServerNameFromHLogDirectoryName(conf, src); + ServerName sn = DefaultWALProvider.getServerNameFromWALDirectoryName(conf, src); if (sn == null) { - // It's not an HLOG + // It's not an WAL return; } - // Ok, so it's an HLog + // Ok, so it's an WAL String hostName = sn.getHostname(); if (LOG.isTraceEnabled()) { LOG.trace(src + - " is an HLog file, so reordering blocks, last hostname will be:" + hostName); + " is an WAL file, so reordering blocks, last hostname will be:" + hostName); } // Just check for all blocks diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HLogLink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HLogLink.java deleted file mode 100644 index e62eb141960..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HLogLink.java +++ /dev/null @@ -1,69 +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.io; - -import java.io.IOException; - -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.util.FSUtils; - -/** - * HLogLink describes a link to a WAL. - * - * An hlog can be in /hbase/.logs// - * or it can be in /hbase/.oldlogs/ - * - * The link checks first in the original path, - * if it is not present it fallbacks to the archived path. - */ -@InterfaceAudience.Private -public class HLogLink extends FileLink { - /** - * @param conf {@link Configuration} from which to extract specific archive locations - * @param serverName Region Server owner of the log - * @param logName WAL file name - * @throws IOException on unexpected error. - */ - public HLogLink(final Configuration conf, - final String serverName, final String logName) throws IOException { - this(FSUtils.getRootDir(conf), serverName, logName); - } - - /** - * @param rootDir Path to the root directory where hbase files are stored - * @param serverName Region Server owner of the log - * @param logName WAL file name - */ - public HLogLink(final Path rootDir, final String serverName, final String logName) { - final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME); - final Path logDir = new Path(new Path(rootDir, HConstants.HREGION_LOGDIR_NAME), serverName); - setLocations(new Path(logDir, logName), new Path(oldLogDir, logName)); - } - - /** - * @param originPath Path to the wal in the log directory - * @param archivePath Path to the wal in the archived log directory - */ - public HLogLink(final Path originPath, final Path archivePath) { - setLocations(originPath, archivePath); - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java index 2c0efc8356d..e35071e9fbc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java @@ -186,12 +186,12 @@ public class HFileOutputFormat2 rollWriters(); } - // create a new HLog writer, if necessary + // create a new WAL writer, if necessary if (wl == null || wl.writer == null) { wl = getNewWriter(family, conf); } - // we now have the proper HLog writer. full steam ahead + // we now have the proper WAL writer. full steam ahead kv.updateLatestStamp(this.now); wl.writer.append(kv); wl.written += length; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HLogInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HLogInputFormat.java index 4f604f86f86..4ed0672ee26 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HLogInputFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HLogInputFormat.java @@ -17,26 +17,15 @@ */ package org.apache.hadoop.hbase.mapreduce; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.EOFException; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; -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.HLogFactory; +import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; -import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.JobContext; @@ -44,227 +33,51 @@ 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. + * @deprecated use {@link WALInputFormat} */ +@Deprecated @InterfaceAudience.Public public class HLogInputFormat extends InputFormat { private static final Log LOG = LogFactory.getLog(HLogInputFormat.class); - public static final String START_TIME_KEY = "hlog.start.time"; public static final String END_TIME_KEY = "hlog.end.time"; - /** - * {@link InputSplit} for {@link HLog} files. Each split represent - * exactly one log file. - */ - static class HLogSplit extends InputSplit implements Writable { - private String logFileName; - private long fileSize; - private long startTime; - private long endTime; - - /** for serialization */ - public HLogSplit() {} - - /** - * Represent an HLogSplit, i.e. a single HLog file. - * Start- and EndTime are managed by the split, so that HLog files can be - * filtered before WALEdits are passed to the mapper(s). - * @param logFileName - * @param fileSize - * @param startTime - * @param endTime - */ - public HLogSplit(String logFileName, long fileSize, long startTime, long endTime) { - this.logFileName = logFileName; - this.fileSize = fileSize; - this.startTime = startTime; - this.endTime = endTime; - } - - @Override - public long getLength() throws IOException, InterruptedException { - return fileSize; - } - - @Override - public String[] getLocations() throws IOException, InterruptedException { - // TODO: Find the data node with the most blocks for this HLog? - return new String[] {}; - } - - public String getLogFileName() { - return logFileName; - } - - public long getStartTime() { - return startTime; - } - - public long getEndTime() { - return endTime; - } - - @Override - public void readFields(DataInput in) throws IOException { - logFileName = in.readUTF(); - fileSize = in.readLong(); - startTime = in.readLong(); - endTime = in.readLong(); - } - - @Override - public void write(DataOutput out) throws IOException { - out.writeUTF(logFileName); - out.writeLong(fileSize); - out.writeLong(startTime); - out.writeLong(endTime); - } - - @Override - public String toString() { - return logFileName + " (" + startTime + ":" + endTime + ") length:" + fileSize; - } - } + // Delegate to WALInputFormat for implementation. + private final WALInputFormat delegate = new WALInputFormat(); /** - * {@link RecordReader} for an {@link HLog} file. + * {@link RecordReader} that pulls out the legacy HLogKey format directly. */ - static class HLogRecordReader extends RecordReader { - private HLog.Reader reader = null; - private HLog.Entry currentEntry = new HLog.Entry(); - private long startTime; - private long endTime; - - @Override - public void initialize(InputSplit split, TaskAttemptContext context) - throws IOException, InterruptedException { - HLogSplit hsplit = (HLogSplit)split; - Path logFile = new Path(hsplit.getLogFileName()); - Configuration conf = context.getConfiguration(); - LOG.info("Opening reader for "+split); - try { - this.reader = HLogFactory.createReader(logFile.getFileSystem(conf), - logFile, conf); - } catch (EOFException x) { - LOG.info("Ignoring corrupted HLog file: " + logFile - + " (This is normal when a RegionServer crashed.)"); - } - this.startTime = hsplit.getStartTime(); - this.endTime = hsplit.getEndTime(); - } - - @Override - public boolean nextKeyValue() throws IOException, InterruptedException { - if (reader == null) return false; - - HLog.Entry temp; - long i = -1; - do { - // skip older entries - try { - temp = reader.next(currentEntry); - i++; - } catch (EOFException x) { - LOG.info("Corrupted entry detected. Ignoring the rest of the file." - + " (This is normal when a RegionServer crashed.)"); - return false; - } - } - while(temp != null && temp.getKey().getWriteTime() < startTime); - - if (temp == null) { - if (i > 0) LOG.info("Skipped " + i + " entries."); - LOG.info("Reached end of file."); - return false; - } else if (i > 0) { - LOG.info("Skipped " + i + " entries, until ts: " + temp.getKey().getWriteTime() + "."); - } - boolean res = temp.getKey().getWriteTime() <= endTime; - if (!res) { - LOG.info("Reached ts: " + temp.getKey().getWriteTime() + " ignoring the rest of the file."); - } - return res; - } - + static class HLogKeyRecordReader extends WALInputFormat.WALRecordReader { @Override public HLogKey getCurrentKey() throws IOException, InterruptedException { - return currentEntry.getKey(); - } - - @Override - public WALEdit getCurrentValue() throws IOException, InterruptedException { - return currentEntry.getEdit(); - } - - @Override - public float getProgress() throws IOException, InterruptedException { - // N/A depends on total number of entries, which is unknown - return 0; - } - - @Override - public void close() throws IOException { - LOG.info("Closing reader"); - if (reader != null) this.reader.close(); + if (!(currentEntry.getKey() instanceof HLogKey)) { + final IllegalStateException exception = new IllegalStateException( + "HLogInputFormat only works when given entries that have HLogKey for keys. This" + + " one had '" + currentEntry.getKey().getClass() + "'"); + LOG.error("The deprecated HLogInputFormat has to work with the deprecated HLogKey class, " + + " but HBase internals read the wal entry using some other class." + + " This is a bug; please file an issue or email the developer mailing list. It is " + + "likely that you would not have this problem if you updated to use WALInputFormat. " + + "You will need the following exception details when seeking help from the HBase " + + "community.", + exception); + throw exception; + } + return (HLogKey)currentEntry.getKey(); } } @Override public List getSplits(JobContext context) throws IOException, InterruptedException { - Configuration conf = context.getConfiguration(); - Path inputDir = new Path(conf.get("mapreduce.input.fileinputformat.inputdir")); - - long startTime = conf.getLong(START_TIME_KEY, Long.MIN_VALUE); - long endTime = conf.getLong(END_TIME_KEY, Long.MAX_VALUE); - - FileSystem fs = inputDir.getFileSystem(conf); - List files = getFiles(fs, inputDir, startTime, endTime); - - List splits = new ArrayList(files.size()); - for (FileStatus file : files) { - splits.add(new HLogSplit(file.getPath().toString(), file.getLen(), startTime, endTime)); - } - return splits; - } - - private List getFiles(FileSystem fs, Path dir, long startTime, long endTime) - throws IOException { - List result = new ArrayList(); - LOG.debug("Scanning " + dir.toString() + " for HLog files"); - - FileStatus[] files = fs.listStatus(dir); - if (files == null) return Collections.emptyList(); - for (FileStatus file : files) { - if (file.isDirectory()) { - // recurse into sub directories - result.addAll(getFiles(fs, file.getPath(), startTime, endTime)); - } else { - String name = file.getPath().toString(); - int idx = name.lastIndexOf('.'); - if (idx > 0) { - try { - long fileStartTime = Long.parseLong(name.substring(idx+1)); - if (fileStartTime <= endTime) { - LOG.info("Found: " + name); - result.add(file); - } - } catch (NumberFormatException x) { - idx = 0; - } - } - if (idx == 0) { - LOG.warn("File " + name + " does not appear to be an HLog file. Skipping..."); - } - } - } - return result; + return delegate.getSplits(context, START_TIME_KEY, END_TIME_KEY); } @Override public RecordReader createRecordReader(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException { - return new HLogRecordReader(); + return new HLogKeyRecordReader(); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java index c1d837358e6..62a9626c926 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java @@ -51,7 +51,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext; *

    * *

    - * Write-ahead logging (HLog) for Puts can be disabled by setting + * Write-ahead logging (WAL) for Puts can be disabled by setting * {@link #WAL_PROPERTY} to {@link #WAL_OFF}. Default value is {@link #WAL_ON}. * Note that disabling write-ahead logging is only appropriate for jobs where * loss of data due to region server failure can be tolerated (for example, @@ -61,7 +61,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext; @InterfaceAudience.Public @InterfaceStability.Stable public class MultiTableOutputFormat extends OutputFormat { - /** Set this to {@link #WAL_OFF} to turn off write-ahead logging (HLog) */ + /** Set this to {@link #WAL_OFF} to turn off write-ahead logging (WAL) */ public static final String WAL_PROPERTY = "hbase.mapreduce.multitableoutputformat.wal"; /** Property value to use write-ahead logging */ public static final boolean WAL_ON = true; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java index 4d451a47c5c..79d52617c28 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java @@ -49,7 +49,7 @@ import com.google.common.annotations.VisibleForTesting; /** * TableSnapshotInputFormat allows a MapReduce job to run over a table snapshot. The job * bypasses HBase servers, and directly accesses the underlying files (hfile, recovered edits, - * hlogs, etc) directly to provide maximum performance. The snapshot is not required to be + * wals, etc) directly to provide maximum performance. The snapshot is not required to be * restored to the live cluster or cloned. This also allows to run the mapreduce job from an * online or offline hbase cluster. The snapshot files can be exported by using the * {@link ExportSnapshot} tool, to a pure-hdfs cluster, and this InputFormat can be used to diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java index cf9dc56adb5..26fab5a6517 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java @@ -23,6 +23,8 @@ import java.text.SimpleDateFormat; import java.util.Map; import java.util.TreeMap; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -39,7 +41,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapreduce.Job; @@ -63,11 +65,21 @@ import org.apache.hadoop.util.ToolRunner; @InterfaceAudience.Public @InterfaceStability.Stable public class WALPlayer extends Configured implements Tool { + final static Log LOG = LogFactory.getLog(WALPlayer.class); final static String NAME = "WALPlayer"; - final static String BULK_OUTPUT_CONF_KEY = "hlog.bulk.output"; - final static String HLOG_INPUT_KEY = "hlog.input.dir"; - final static String TABLES_KEY = "hlog.input.tables"; - final static String TABLE_MAP_KEY = "hlog.input.tablesmap"; + final static String BULK_OUTPUT_CONF_KEY = "wal.bulk.output"; + final static String TABLES_KEY = "wal.input.tables"; + final static String TABLE_MAP_KEY = "wal.input.tablesmap"; + + // This relies on Hadoop Configuration to handle warning about deprecated configs and + // to set the correct non-deprecated configs when an old one shows up. + static { + Configuration.addDeprecation("hlog.bulk.output", BULK_OUTPUT_CONF_KEY); + Configuration.addDeprecation("hlog.input.tables", TABLES_KEY); + Configuration.addDeprecation("hlog.input.tablesmap", TABLE_MAP_KEY); + Configuration.addDeprecation(HLogInputFormat.START_TIME_KEY, WALInputFormat.START_TIME_KEY); + Configuration.addDeprecation(HLogInputFormat.END_TIME_KEY, WALInputFormat.END_TIME_KEY); + } private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name"; @@ -75,12 +87,12 @@ public class WALPlayer extends Configured implements Tool { * A mapper that just writes out KeyValues. * This one can be used together with {@link KeyValueSortReducer} */ - static class HLogKeyValueMapper - extends Mapper { + static class WALKeyValueMapper + extends Mapper { private byte[] table; @Override - public void map(HLogKey key, WALEdit value, + public void map(WALKey key, WALEdit value, Context context) throws IOException { try { @@ -102,7 +114,7 @@ public class WALPlayer extends Configured implements Tool { // only a single table is supported when HFiles are generated with HFileOutputFormat String tables[] = context.getConfiguration().getStrings(TABLES_KEY); if (tables == null || tables.length != 1) { - // this can only happen when HLogMapper is used directly by a class other than WALPlayer + // this can only happen when WALMapper is used directly by a class other than WALPlayer throw new IOException("Exactly one table must be specified for bulk HFile case."); } table = Bytes.toBytes(tables[0]); @@ -113,13 +125,13 @@ public class WALPlayer extends Configured implements Tool { * A mapper that writes out {@link Mutation} to be directly applied to * a running HBase instance. */ - static class HLogMapper - extends Mapper { + static class WALMapper + extends Mapper { private Map tables = new TreeMap(); @Override - public void map(HLogKey key, WALEdit value, + public void map(WALKey key, WALEdit value, Context context) throws IOException { try { @@ -132,7 +144,7 @@ public class WALPlayer extends Configured implements Tool { Delete del = null; Cell lastCell = null; for (Cell cell : value.getCells()) { - // filtering HLog meta entries + // filtering WAL meta entries if (WALEdit.isMetaEditFamily(cell.getFamily())) continue; // A WALEdit may contain multiple operations (HBASE-3584) and/or @@ -172,7 +184,7 @@ public class WALPlayer extends Configured implements Tool { String[] tableMap = context.getConfiguration().getStrings(TABLE_MAP_KEY); String[] tablesToUse = context.getConfiguration().getStrings(TABLES_KEY); if (tablesToUse == null || tableMap == null || tablesToUse.length != tableMap.length) { - // this can only happen when HLogMapper is used directly by a class other than WALPlayer + // this can only happen when WALMapper is used directly by a class other than WALPlayer throw new IOException("No tables or incorrect table mapping specified."); } int i = 0; @@ -192,7 +204,7 @@ public class WALPlayer extends Configured implements Tool { void setupTime(Configuration conf, String option) throws IOException { String val = conf.get(option); - if (val == null) return; + if (null == val) return; long ms; try { // first try to parse in user friendly form @@ -239,7 +251,7 @@ public class WALPlayer extends Configured implements Tool { Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + inputDir)); job.setJarByClass(WALPlayer.class); FileInputFormat.setInputPaths(job, inputDir); - job.setInputFormatClass(HLogInputFormat.class); + job.setInputFormatClass(WALInputFormat.class); job.setMapOutputKeyClass(ImmutableBytesWritable.class); String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY); if (hfileOutPath != null) { @@ -248,7 +260,7 @@ public class WALPlayer extends Configured implements Tool { throw new IOException("Exactly one table must be specified for the bulk export option"); } HTable table = new HTable(conf, TableName.valueOf(tables[0])); - job.setMapperClass(HLogKeyValueMapper.class); + job.setMapperClass(WALKeyValueMapper.class); job.setReducerClass(KeyValueSortReducer.class); Path outputDir = new Path(hfileOutPath); FileOutputFormat.setOutputPath(job, outputDir); @@ -258,7 +270,7 @@ public class WALPlayer extends Configured implements Tool { com.google.common.base.Preconditions.class); } else { // output to live cluster - job.setMapperClass(HLogMapper.class); + job.setMapperClass(WALMapper.class); job.setOutputFormatClass(MultiTableOutputFormat.class); TableMapReduceUtil.addDependencyJars(job); TableMapReduceUtil.initCredentials(job); @@ -288,8 +300,8 @@ public class WALPlayer extends Configured implements Tool { System.err.println(" -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output"); System.err.println(" (Only one table can be specified, and no mapping is allowed!)"); System.err.println("Other options: (specify time range to WAL edit to consider)"); - System.err.println(" -D" + HLogInputFormat.START_TIME_KEY + "=[date|ms]"); - System.err.println(" -D" + HLogInputFormat.END_TIME_KEY + "=[date|ms]"); + System.err.println(" -D" + WALInputFormat.START_TIME_KEY + "=[date|ms]"); + System.err.println(" -D" + WALInputFormat.END_TIME_KEY + "=[date|ms]"); System.err.println(" -D " + JOB_NAME_CONF_KEY + "=jobName - use the specified mapreduce job name for the wal player"); System.err.println("For performance also consider the following options:\n" diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java index 7c7f0b67799..d23f1397729 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java @@ -76,8 +76,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Regio import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; import org.apache.hadoop.hbase.regionserver.RegionOpeningState; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.KeyLocker; @@ -463,17 +462,20 @@ public class AssignmentManager { } if (!failover) { // If we get here, we have a full cluster restart. It is a failover only - // if there are some HLogs are not split yet. For meta HLogs, they should have + // if there are some WALs are not split yet. For meta WALs, they should have // been split already, if any. We can walk through those queued dead servers, - // if they don't have any HLogs, this restart should be considered as a clean one + // if they don't have any WALs, this restart should be considered as a clean one Set queuedDeadServers = serverManager.getRequeuedDeadServers().keySet(); if (!queuedDeadServers.isEmpty()) { Configuration conf = server.getConfiguration(); Path rootdir = FSUtils.getRootDir(conf); FileSystem fs = rootdir.getFileSystem(conf); for (ServerName serverName: queuedDeadServers) { - Path logDir = new Path(rootdir, HLogUtil.getHLogDirectoryName(serverName.toString())); - Path splitDir = logDir.suffix(HLog.SPLITTING_EXT); + // In the case of a clean exit, the shutdown handler would have presplit any WALs and + // removed empty directories. + Path logDir = new Path(rootdir, + DefaultWALProvider.getWALDirectoryName(serverName.toString())); + Path splitDir = logDir.suffix(DefaultWALProvider.SPLITTING_EXT); if (fs.exists(logDir) || fs.exists(splitDir)) { LOG.debug("Found queued dead server " + serverName); failover = true; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java index 115cc3539da..9bf70e92e8b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java @@ -51,8 +51,8 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSTableDescriptors; @@ -94,14 +94,14 @@ public class MasterFileSystem { final static PathFilter META_FILTER = new PathFilter() { @Override public boolean accept(Path p) { - return HLogUtil.isMetaFile(p); + return DefaultWALProvider.isMetaFile(p); } }; final static PathFilter NON_META_FILTER = new PathFilter() { @Override public boolean accept(Path p) { - return !HLogUtil.isMetaFile(p); + return !DefaultWALProvider.isMetaFile(p); } }; @@ -216,7 +216,7 @@ public class MasterFileSystem { */ Set getFailedServersFromLogFolders() { boolean retrySplitting = !conf.getBoolean("hbase.hlog.split.skip.errors", - HLog.SPLIT_SKIP_ERRORS_DEFAULT); + WALSplitter.SPLIT_SKIP_ERRORS_DEFAULT); Set serverNames = new HashSet(); Path logsDirPath = new Path(this.rootdir, HConstants.HREGION_LOGDIR_NAME); @@ -239,13 +239,13 @@ public class MasterFileSystem { return serverNames; } 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()); - } - ServerName serverName = ServerName.parseServerName(sn); - if (!onlineServers.contains(serverName)) { + final ServerName serverName = DefaultWALProvider.getServerNameFromWALDirectoryName( + status.getPath()); + if (null == serverName) { + LOG.warn("Log folder " + status.getPath() + " doesn't look like its name includes a " + + "region server name; leaving in place. If you see later errors about missing " + + "write ahead logs they may be saved in this location."); + } else if (!onlineServers.contains(serverName)) { LOG.info("Log folder " + status.getPath() + " doesn't belong " + "to a known region server, splitting"); serverNames.add(serverName); @@ -283,7 +283,7 @@ public class MasterFileSystem { } /** - * Specialized method to handle the splitting for meta HLog + * Specialized method to handle the splitting for meta WAL * @param serverName * @throws IOException */ @@ -294,7 +294,7 @@ public class MasterFileSystem { } /** - * Specialized method to handle the splitting for meta HLog + * Specialized method to handle the splitting for meta WAL * @param serverNames * @throws IOException */ @@ -302,6 +302,9 @@ public class MasterFileSystem { splitLog(serverNames, META_FILTER); } + @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK", justification= + "We only release this lock when we set it. Updates to code that uses it should verify use " + + "of the guard boolean.") private List getLogDirs(final Set serverNames) throws IOException { List logDirs = new ArrayList(); boolean needReleaseLock = false; @@ -312,9 +315,10 @@ 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); - // Rename the directory so a rogue RS doesn't create more HLogs + Path logDir = new Path(this.rootdir, + DefaultWALProvider.getWALDirectoryName(serverName.toString())); + Path splitDir = logDir.suffix(DefaultWALProvider.SPLITTING_EXT); + // Rename the directory so a rogue RS doesn't create more WALs if (fs.exists(logDir)) { if (!this.fs.rename(logDir, splitDir)) { throw new IOException("Failed fs.rename for log split: " + logDir); @@ -367,9 +371,10 @@ public class MasterFileSystem { } /** - * This method is the base split method that splits HLog files matching a filter. Callers should - * pass the appropriate filter for meta and non-meta HLogs. - * @param serverNames + * This method is the base split method that splits WAL files matching a filter. Callers should + * pass the appropriate filter for meta and non-meta WALs. + * @param serverNames logs belonging to these servers will be split; this will rename the log + * directory out from under a soft-failed server * @param filter * @throws IOException */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystem.java index 34547ef5858..45dbeb8595b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystem.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystem.java @@ -31,7 +31,7 @@ public class MetricsMasterFileSystem { /** * Record a single instance of a split * @param time time that the split took - * @param size length of original HLogs that were split + * @param size length of original WALs that were split */ public synchronized void addSplit(long time, long size) { source.updateSplitTime(time); @@ -41,7 +41,7 @@ public class MetricsMasterFileSystem { /** * Record a single instance of a split * @param time time that the split took - * @param size length of original HLogs that were split + * @param size length of original WALs that were split */ public synchronized void addMetaWALSplit(long time, long size) { source.updateMetaWALSplitTime(time); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java index 95d41ed4332..b96aaee7b61 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java @@ -448,7 +448,7 @@ public class RegionStates { } /** - * A dead server's hlogs have been split so that all the regions + * A dead server's wals have been split so that all the regions * used to be open on it can be safely assigned now. Mark them assignable. */ public synchronized void logSplit(final ServerName serverName) { @@ -688,7 +688,7 @@ public class RegionStates { /** * Checking if a region was assigned to a server which is not online now. - * If so, we should hold re-assign this region till SSH has split its hlogs. + * If so, we should hold re-assign this region till SSH has split its wals. * Once logs are split, the last assignment of this region will be reset, * which means a null last assignment server is ok for re-assigning. * diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java index 369362b24af..39d0a0f00ac 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java @@ -591,7 +591,7 @@ public class ServerManager { this.processDeadServer(serverName, false); } - public synchronized void processDeadServer(final ServerName serverName, boolean shouldSplitHlog) { + public synchronized void processDeadServer(final ServerName serverName, boolean shouldSplitWal) { // When assignment manager is cleaning up the zookeeper nodes and rebuilding the // in-memory region states, region servers could be down. Meta table can and // should be re-assigned, log splitting can be done too. However, it is better to @@ -601,14 +601,14 @@ public class ServerManager { // the handler threads and meta table could not be re-assigned in case // the corresponding server is down. So we queue them up here instead. if (!services.getAssignmentManager().isFailoverCleanupDone()) { - requeuedDeadServers.put(serverName, shouldSplitHlog); + requeuedDeadServers.put(serverName, shouldSplitWal); return; } this.deadservers.add(serverName); this.services.getExecutorService().submit( new ServerShutdownHandler(this.master, this.services, this.deadservers, serverName, - shouldSplitHlog)); + shouldSplitWal)); } /** @@ -947,7 +947,7 @@ public class ServerManager { /** * During startup, if we figure it is not a failover, i.e. there is - * no more HLog files to split, we won't try to recover these dead servers. + * no more WAL files to split, we won't try to recover these dead servers. * So we just remove them from the queue. Use caution in calling this. */ void removeRequeuedDeadServers() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java index bf28a44493d..6dd5cf1233f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java @@ -59,7 +59,7 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; import org.apache.hadoop.hbase.regionserver.SplitLogWorker; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Pair; @@ -102,8 +102,7 @@ public class SplitLogManager { private Server server; private final Stoppable stopper; - private FileSystem fs; - private Configuration conf; + private final Configuration conf; public static final int DEFAULT_UNASSIGNED_TIMEOUT = (3 * 60 * 1000); // 3 min @@ -161,16 +160,34 @@ public class SplitLogManager { } private FileStatus[] getFileList(List logDirs, PathFilter filter) throws IOException { + return getFileList(conf, logDirs, filter); + } + + /** + * Get a list of paths that need to be split given a set of server-specific directories and + * optinally a filter. + * + * See {@link DefaultWALProvider#getServerNameFromWALDirectoryName} for more info on directory + * layout. + * + * Should be package-private, but is needed by + * {@link org.apache.hadoop.hbase.wal.WALSplitter#split(Path, Path, Path, FileSystem, + * Configuration, WALFactory)} for tests. + */ + @VisibleForTesting + public static FileStatus[] getFileList(final Configuration conf, final List logDirs, + final PathFilter filter) + throws IOException { List fileStatus = new ArrayList(); - for (Path hLogDir : logDirs) { - this.fs = hLogDir.getFileSystem(conf); - if (!fs.exists(hLogDir)) { - LOG.warn(hLogDir + " doesn't exist. Nothing to do!"); + for (Path logDir : logDirs) { + final FileSystem fs = logDir.getFileSystem(conf); + if (!fs.exists(logDir)) { + LOG.warn(logDir + " doesn't exist. Nothing to do!"); continue; } - FileStatus[] logfiles = FSUtils.listStatus(fs, hLogDir, filter); + FileStatus[] logfiles = FSUtils.listStatus(fs, logDir, filter); if (logfiles == null || logfiles.length == 0) { - LOG.info(hLogDir + " is empty dir, no logs to split"); + LOG.info(logDir + " is empty dir, no logs to split"); } else { Collections.addAll(fileStatus, logfiles); } @@ -180,7 +197,7 @@ public class SplitLogManager { } /** - * @param logDir one region sever hlog dir path in .logs + * @param logDir one region sever wal dir path in .logs * @throws IOException if there was an error while splitting any log file * @return cumulative size of the logfiles split * @throws IOException @@ -206,7 +223,7 @@ public class SplitLogManager { Set serverNames = new HashSet(); for (Path logDir : logDirs) { try { - ServerName serverName = HLogUtil.getServerNameFromHLogDirectoryName(logDir); + ServerName serverName = DefaultWALProvider.getServerNameFromWALDirectoryName(logDir); if (serverName != null) { serverNames.add(serverName); } @@ -273,6 +290,7 @@ public class SplitLogManager { } for (Path logDir : logDirs) { status.setStatus("Cleaning up log directory..."); + final FileSystem fs = logDir.getFileSystem(conf); try { if (fs.exists(logDir) && !fs.delete(logDir, false)) { LOG.warn("Unable to delete log src dir. Ignoring. " + logDir); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java index 6c8e4288ae6..f68bfa22db3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java @@ -26,10 +26,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Stoppable; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; /** - * This Chore, every time it runs, will attempt to delete the HLogs in the old logs folder. The HLog + * This Chore, every time it runs, will attempt to delete the WALs in the old logs folder. The WAL * is only deleted if none of the cleaner delegates says otherwise. * @see BaseLogCleanerDelegate */ @@ -51,6 +51,6 @@ public class LogCleaner extends CleanerChore { @Override protected boolean validate(Path file) { - return HLogUtil.validateHLogFilename(file.getName()); + return DefaultWALProvider.validateWALFilename(file.getName()); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/TimeToLiveLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/TimeToLiveLogCleaner.java index 3a39fb4492c..9d68601fb13 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/TimeToLiveLogCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/TimeToLiveLogCleaner.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; /** - * Log cleaner that uses the timestamp of the hlog to determine if it should + * Log cleaner that uses the timestamp of the wal to determine if it should * be deleted. By default they are allowed to live for 10 minutes. */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java index 648c835f7b7..73208bc15b1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java @@ -67,7 +67,7 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler { boolean distributedLogReplay = (this.services.getMasterFileSystem().getLogRecoveryMode() == RecoveryMode.LOG_REPLAY); try { - if (this.shouldSplitHlog) { + if (this.shouldSplitWal) { LOG.info("Splitting hbase:meta logs for " + serverName); if (distributedLogReplay) { Set regions = new HashSet(); @@ -95,7 +95,7 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler { } try { - if (this.shouldSplitHlog && distributedLogReplay) { + if (this.shouldSplitWal && distributedLogReplay) { if (!am.waitOnRegionToClearRegionsInTransition(HRegionInfo.FIRST_META_REGIONINFO, regionAssignmentWaitTimeout)) { // Wait here is to avoid log replay hits current dead server and incur a RPC timeout diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java index c44396896c7..5b7b27b3d2b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java @@ -56,19 +56,19 @@ public class ServerShutdownHandler extends EventHandler { protected final ServerName serverName; protected final MasterServices services; protected final DeadServer deadServers; - protected final boolean shouldSplitHlog; // whether to split HLog or not + protected final boolean shouldSplitWal; // whether to split WAL or not protected final int regionAssignmentWaitTimeout; public ServerShutdownHandler(final Server server, final MasterServices services, final DeadServer deadServers, final ServerName serverName, - final boolean shouldSplitHlog) { + final boolean shouldSplitWal) { this(server, services, deadServers, serverName, EventType.M_SERVER_SHUTDOWN, - shouldSplitHlog); + shouldSplitWal); } ServerShutdownHandler(final Server server, final MasterServices services, final DeadServer deadServers, final ServerName serverName, EventType type, - final boolean shouldSplitHlog) { + final boolean shouldSplitWal) { super(server, type); this.serverName = serverName; this.server = server; @@ -77,7 +77,7 @@ public class ServerShutdownHandler extends EventHandler { if (!this.deadServers.isDeadServer(this.serverName)) { LOG.warn(this.serverName + " is NOT in deadservers; it should be!"); } - this.shouldSplitHlog = shouldSplitHlog; + this.shouldSplitWal = shouldSplitWal; this.regionAssignmentWaitTimeout = server.getConfiguration().getInt( HConstants.LOG_REPLAY_WAIT_REGION_TIMEOUT, 15000); } @@ -133,7 +133,7 @@ public class ServerShutdownHandler extends EventHandler { AssignmentManager am = services.getAssignmentManager(); ServerManager serverManager = services.getServerManager(); if (isCarryingMeta() /* hbase:meta */ || !am.isFailoverCleanupDone()) { - serverManager.processDeadServer(serverName, this.shouldSplitHlog); + serverManager.processDeadServer(serverName, this.shouldSplitWal); return; } @@ -180,7 +180,7 @@ public class ServerShutdownHandler extends EventHandler { (this.services.getMasterFileSystem().getLogRecoveryMode() == RecoveryMode.LOG_REPLAY); try { - if (this.shouldSplitHlog) { + if (this.shouldSplitWal) { if (distributedLogReplay) { LOG.info("Mark regions in recovery for crashed server " + serverName + " before assignment; regions=" + hris); @@ -275,13 +275,13 @@ public class ServerShutdownHandler extends EventHandler { throw (InterruptedIOException)new InterruptedIOException().initCause(ie); } catch (IOException ioe) { LOG.info("Caught " + ioe + " during region assignment, will retry"); - // Only do HLog splitting if shouldSplitHlog and in DLR mode + // Only do wal splitting if shouldSplitWal and in DLR mode serverManager.processDeadServer(serverName, - this.shouldSplitHlog && distributedLogReplay); + this.shouldSplitWal && distributedLogReplay); return; } - if (this.shouldSplitHlog && distributedLogReplay) { + if (this.shouldSplitWal && distributedLogReplay) { // wait for region assignment completes for (HRegionInfo hri : toAssignRegions) { try { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotLogCleaner.java index d5e174d88df..a927db314cd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotLogCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotLogCleaner.java @@ -46,11 +46,11 @@ public class SnapshotLogCleaner extends BaseLogCleanerDelegate { * Conf key for the frequency to attempt to refresh the cache of hfiles currently used in * snapshots (ms) */ - static final String HLOG_CACHE_REFRESH_PERIOD_CONF_KEY = + static final String WAL_CACHE_REFRESH_PERIOD_CONF_KEY = "hbase.master.hlogcleaner.plugins.snapshot.period"; /** Refresh cache, by default, every 5 minutes */ - private static final long DEFAULT_HLOG_CACHE_REFRESH_PERIOD = 300000; + private static final long DEFAULT_WAL_CACHE_REFRESH_PERIOD = 300000; private SnapshotFileCache cache; @@ -77,14 +77,14 @@ public class SnapshotLogCleaner extends BaseLogCleanerDelegate { super.setConf(conf); try { long cacheRefreshPeriod = conf.getLong( - HLOG_CACHE_REFRESH_PERIOD_CONF_KEY, DEFAULT_HLOG_CACHE_REFRESH_PERIOD); + WAL_CACHE_REFRESH_PERIOD_CONF_KEY, DEFAULT_WAL_CACHE_REFRESH_PERIOD); final FileSystem fs = FSUtils.getCurrentFileSystem(conf); Path rootDir = FSUtils.getRootDir(conf); cache = new SnapshotFileCache(fs, rootDir, cacheRefreshPeriod, cacheRefreshPeriod, "snapshot-log-cleaner-cache-refresher", new SnapshotFileCache.SnapshotFileInspector() { public Collection filesUnderSnapshot(final Path snapshotDir) throws IOException { - return SnapshotReferenceUtil.getHLogNames(fs, snapshotDir); + return SnapshotReferenceUtil.getWALNames(fs, snapshotDir); } }); } catch (IOException e) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java index 0310733fbc0..19bfa8cc6c3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java @@ -50,9 +50,8 @@ import org.apache.hadoop.hbase.client.TableState; 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.HLogFactory; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.security.access.AccessControlLists; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; import org.apache.hadoop.hbase.util.Bytes; @@ -382,13 +381,11 @@ public class NamespaceUpgrade implements Tool { ServerName fakeServer = ServerName.valueOf("nsupgrade", 96, 123); - String metaLogName = HLogUtil.getHLogDirectoryName(fakeServer.toString()); - HLog metaHLog = HLogFactory.createMetaHLog(fs, rootDir, - metaLogName, conf, null, - fakeServer.toString()); + final WALFactory walFactory = new WALFactory(conf, null, fakeServer.toString()); + WAL metawal = walFactory.getMetaWAL(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes()); FSTableDescriptors fst = new FSTableDescriptors(conf); HRegion meta = HRegion.openHRegion(rootDir, HRegionInfo.FIRST_META_REGIONINFO, - fst.get(TableName.META_TABLE_NAME), metaHLog, conf); + fst.get(TableName.META_TABLE_NAME), metawal, conf); HRegion region = null; try { for(Path regionDir : FSUtils.getRegionDirs(fs, oldTablePath)) { @@ -405,7 +402,7 @@ public class NamespaceUpgrade implements Tool { new HRegion( HRegionFileSystem.openRegionFromFileSystem(conf, fs, oldTablePath, oldRegionInfo, false), - metaHLog, + metawal, conf, oldDesc, null); @@ -442,7 +439,7 @@ public class NamespaceUpgrade implements Tool { meta.flushcache(); meta.waitForFlushesAndCompactions(); meta.close(); - metaHLog.closeAndDelete(); + metawal.close(); if(region != null) { region.close(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java index 2e5fc418bc7..d6a120b1040 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java @@ -39,8 +39,8 @@ 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.HLogKey; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.Pair; @@ -50,14 +50,14 @@ import com.google.protobuf.ServiceException; @InterfaceAudience.Private public class ReplicationProtbufUtil { /** - * A helper to replicate a list of HLog entries using admin protocol. + * A helper to replicate a list of WAL entries using admin protocol. * * @param admin * @param entries * @throws java.io.IOException */ public static void replicateWALEntry(final AdminService.BlockingInterface admin, - final HLog.Entry[] entries) throws IOException { + final Entry[] entries) throws IOException { Pair p = buildReplicateWALEntryRequest(entries, null); PayloadCarryingRpcController controller = new PayloadCarryingRpcController(p.getSecond()); @@ -69,27 +69,27 @@ public class ReplicationProtbufUtil { } /** - * Create a new ReplicateWALEntryRequest from a list of HLog entries + * Create a new ReplicateWALEntryRequest from a list of WAL entries * - * @param entries the HLog entries to be replicated + * @param entries the WAL entries to be replicated * @return a pair of ReplicateWALEntryRequest and a CellScanner over all the WALEdit values * found. */ public static Pair - buildReplicateWALEntryRequest(final HLog.Entry[] entries) { + buildReplicateWALEntryRequest(final Entry[] entries) { return buildReplicateWALEntryRequest(entries, null); } /** - * Create a new ReplicateWALEntryRequest from a list of HLog entries + * Create a new ReplicateWALEntryRequest from a list of WAL entries * - * @param entries the HLog entries to be replicated + * @param entries the WAL entries to be replicated * @param encodedRegionName alternative region name to use if not null * @return a pair of ReplicateWALEntryRequest and a CellScanner over all the WALEdit values * found. */ public static Pair - buildReplicateWALEntryRequest(final HLog.Entry[] entries, byte[] encodedRegionName) { + buildReplicateWALEntryRequest(final Entry[] entries, byte[] encodedRegionName) { // Accumulate all the Cells seen in here. List> allCells = new ArrayList>(entries.length); int size = 0; @@ -98,11 +98,11 @@ public class ReplicationProtbufUtil { AdminProtos.ReplicateWALEntryRequest.Builder builder = AdminProtos.ReplicateWALEntryRequest.newBuilder(); HBaseProtos.UUID.Builder uuidBuilder = HBaseProtos.UUID.newBuilder(); - for (HLog.Entry entry: entries) { + for (Entry entry: entries) { entryBuilder.clear(); - // TODO: this duplicates a lot in HLogKey#getBuilder + // TODO: this duplicates a lot in WALKey#getBuilder WALProtos.WALKey.Builder keyBuilder = entryBuilder.getKeyBuilder(); - HLogKey key = entry.getKey(); + WALKey key = entry.getKey(); keyBuilder.setEncodedRegionName( ByteStringer.wrap(encodedRegionName == null ? key.getEncodedRegionName() diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 7952a87fec7..464ad7eaea2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -44,6 +44,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.net.InetAddress; @@ -124,9 +125,10 @@ import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager; import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress; import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler; import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.trace.SpanReceiverHost; @@ -330,15 +332,13 @@ public class HRegionServer extends HasThread implements */ Chore periodicFlusher; - // HLog and HLog roller. log is protected rather than private to avoid - // eclipse warning when accessed by inner classes - protected volatile HLog 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 WALFactory walFactory; - LogRoller hlogRoller; - LogRoller metaHLogRoller; + // WAL roller. log is protected rather than private to avoid + // eclipse warning when accessed by inner classes + final LogRoller walRoller; + // Lazily initialized if this RegionServer hosts a meta table. + final AtomicReference metawalRoller = new AtomicReference(); // flag set after we're done setting up server threads final AtomicBoolean online = new AtomicBoolean(false); @@ -546,6 +546,7 @@ public class HRegionServer extends HasThread implements rpcServices.start(); putUpWebUI(); + this.walRoller = new LogRoller(this, this); } protected void login(UserProvider user, String host) throws IOException { @@ -974,7 +975,7 @@ public class HRegionServer extends HasThread implements //fsOk flag may be changed when closing regions throws exception. if (this.fsOk) { - closeWAL(!abortRequested); + shutdownWAL(!abortRequested); } // Make sure the proxy is down. @@ -1076,7 +1077,8 @@ public class HRegionServer extends HasThread implements } } - ClusterStatusProtos.ServerLoad buildServerLoad(long reportStartTime, long reportEndTime) { + ClusterStatusProtos.ServerLoad buildServerLoad(long reportStartTime, long reportEndTime) + throws IOException { // We're getting the MetricsRegionServerWrapper here because the wrapper computes requests // per second, and other metrics As long as metrics are part of ServerLoad it's best to use // the wrapper to compute those numbers in one place. @@ -1095,7 +1097,7 @@ public class HRegionServer extends HasThread implements serverLoad.setTotalNumberOfRequests((int) regionServerWrapper.getTotalRequestCount()); serverLoad.setUsedHeapMB((int)(memory.getUsed() / 1024 / 1024)); serverLoad.setMaxHeapMB((int) (memory.getMax() / 1024 / 1024)); - Set coprocessors = this.hlog.getCoprocessorHost().getCoprocessors(); + Set coprocessors = getWAL(null).getCoprocessorHost().getCoprocessors(); for (String coprocessor : coprocessors) { serverLoad.addCoprocessors( Coprocessor.newBuilder().setName(coprocessor).build()); @@ -1104,6 +1106,10 @@ public class HRegionServer extends HasThread implements RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder(); for (HRegion region : regions) { serverLoad.addRegionLoads(createRegionLoad(region, regionLoadBldr, regionSpecifier)); + for (String coprocessor : + getWAL(region.getRegionInfo()).getCoprocessorHost().getCoprocessors()) { + serverLoad.addCoprocessors(Coprocessor.newBuilder().setName(coprocessor).build()); + } } serverLoad.setReportStartTime(reportStartTime); serverLoad.setReportEndTime(reportEndTime); @@ -1192,35 +1198,24 @@ public class HRegionServer extends HasThread implements return interrupted; } - private void closeWAL(final boolean delete) { - if (this.hlogForMeta != null) { - // All hlogs (meta and non-meta) are in the same directory. Don't call - // closeAndDelete here since that would delete all hlogs not just the - // meta ones. We will just 'close' the hlog for meta here, and leave - // the directory cleanup to the follow-on closeAndDelete call. + private void shutdownWAL(final boolean close) { + if (this.walFactory != null) { try { - this.hlogForMeta.close(); - } catch (Throwable e) { - e = e instanceof RemoteException ? ((RemoteException) e).unwrapRemoteException() : e; - LOG.error("Metalog close and delete failed", e); - } - } - if (this.hlog != null) { - try { - if (delete) { - hlog.closeAndDelete(); + if (close) { + walFactory.close(); } else { - hlog.close(); + walFactory.shutdown(); } } catch (Throwable e) { e = e instanceof RemoteException ? ((RemoteException) e).unwrapRemoteException() : e; - LOG.error("Close and delete failed", e); + LOG.error("Shutdown / close of WAL failed: " + e); + LOG.debug("Shutdown / close exception details:", e); } } } /* - * Run init. Sets up hlog and starts up all server threads. + * Run init. Sets up wal and starts up all server threads. * * @param c Extra configuration. */ @@ -1258,7 +1253,7 @@ public class HRegionServer extends HasThread implements ZNodeClearer.writeMyEphemeralNodeOnDisk(getMyEphemeralNodePath()); this.cacheConfig = new CacheConfig(conf); - this.hlog = setupWALAndReplication(); + this.walFactory = setupWALAndReplication(); // Init in here rather than in constructor after thread name has been set this.metricsRegionServer = new MetricsRegionServer(new MetricsRegionServerWrapperImpl(this)); @@ -1502,10 +1497,10 @@ public class HRegionServer extends HasThread implements * @return A WAL instance. * @throws IOException */ - private HLog setupWALAndReplication() throws IOException { + private WALFactory setupWALAndReplication() throws IOException { + // TODO Replication make assumptions here based on the default filesystem impl final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME); - final String logName - = HLogUtil.getHLogDirectoryName(this.serverName.toString()); + final String logName = DefaultWALProvider.getWALDirectoryName(this.serverName.toString()); Path logdir = new Path(rootDir, logName); if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir); @@ -1518,66 +1513,44 @@ public class HRegionServer extends HasThread implements // log directories. createNewReplicationInstance(conf, this, this.fs, logdir, oldLogDir); - return instantiateHLog(rootDir, logName); - } - - private HLog getMetaWAL() throws IOException { - if (this.hlogForMeta != null) return this.hlogForMeta; - final String logName = HLogUtil.getHLogDirectoryName(this.serverName.toString()); - Path logdir = new Path(rootDir, logName); - if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir); - this.hlogForMeta = HLogFactory.createMetaHLog(this.fs.getBackingFs(), rootDir, logName, - this.conf, getMetaWALActionListeners(), this.serverName.toString()); - return this.hlogForMeta; - } - - /** - * Called by {@link #setupWALAndReplication()} creating WAL instance. - * @param rootdir - * @param logName - * @return WAL instance. - * @throws IOException - */ - protected HLog instantiateHLog(Path rootdir, String logName) throws IOException { - return HLogFactory.createHLog(this.fs.getBackingFs(), rootdir, logName, this.conf, - getWALActionListeners(), this.serverName.toString()); - } - - /** - * 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. - */ - protected List getWALActionListeners() { - List listeners = new ArrayList(); - // Log roller. - this.hlogRoller = new LogRoller(this, this); - listeners.add(this.hlogRoller); + // listeners the wal factory will add to wals it creates. + final List listeners = new ArrayList(); + listeners.add(new MetricsWAL()); if (this.replicationSourceHandler != null && this.replicationSourceHandler.getWALActionsListener() != null) { // Replication handler is an implementation of WALActionsListener. listeners.add(this.replicationSourceHandler.getWALActionsListener()); } - return listeners; + + return new WALFactory(conf, listeners, serverName.toString()); } - protected List getMetaWALActionListeners() { - List listeners = new ArrayList(); + /** + * We initialize the roller for the wal that handles meta lazily + * since we don't know if this regionserver will handle it. All calls to + * this method return a reference to the that same roller. As newly referenced + * meta regions are brought online, they will be offered to the roller for maintenance. + * As a part of that registration process, the roller will add itself as a + * listener on the wal. + */ + protected LogRoller ensureMetaWALRoller() { // Using a tmp log roller to ensure metaLogRoller is alive once it is not // null - MetaLogRoller tmpLogRoller = new MetaLogRoller(this, this); - String n = Thread.currentThread().getName(); - Threads.setDaemonThreadRunning(tmpLogRoller.getThread(), - n + "-MetaLogRoller", uncaughtExceptionHandler); - this.metaHLogRoller = tmpLogRoller; - tmpLogRoller = null; - listeners.add(this.metaHLogRoller); - return listeners; - } - - protected LogRoller getLogRoller() { - return hlogRoller; + LogRoller roller = metawalRoller.get(); + if (null == roller) { + LogRoller tmpLogRoller = new LogRoller(this, this); + String n = Thread.currentThread().getName(); + Threads.setDaemonThreadRunning(tmpLogRoller.getThread(), + n + "-MetaLogRoller", uncaughtExceptionHandler); + if (metawalRoller.compareAndSet(null, tmpLogRoller)) { + roller = tmpLogRoller; + } else { + // Another thread won starting the roller + Threads.shutdown(tmpLogRoller.getThread()); + roller = metawalRoller.get(); + } + } + return roller; } public MetricsRegionServer getRegionServerMetrics() { @@ -1620,7 +1593,7 @@ public class HRegionServer extends HasThread implements this.service.startExecutorService(ExecutorType.RS_LOG_REPLAY_OPS, conf.getInt( "hbase.regionserver.wal.max.splitters", SplitLogWorkerCoordination.DEFAULT_MAX_SPLITTERS)); - Threads.setDaemonThreadRunning(this.hlogRoller.getThread(), getName() + ".logRoller", + Threads.setDaemonThreadRunning(this.walRoller.getThread(), getName() + ".logRoller", uncaughtExceptionHandler); this.cacheFlusher.start(uncaughtExceptionHandler); Threads.setDaemonThreadRunning(this.compactionChecker.getThread(), getName() + @@ -1667,7 +1640,7 @@ public class HRegionServer extends HasThread implements sinkConf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, conf.getInt("hbase.log.replay.rpc.timeout", 30000)); // default 30 seconds sinkConf.setInt("hbase.client.serverside.retries.multiplier", 1); - this.splitLogWorker = new SplitLogWorker(this, sinkConf, this, this); + this.splitLogWorker = new SplitLogWorker(this, sinkConf, this, this, walFactory); splitLogWorker.start(); } @@ -1730,38 +1703,37 @@ public class HRegionServer extends HasThread implements } // Verify that all threads are alive if (!(leases.isAlive() - && cacheFlusher.isAlive() && hlogRoller.isAlive() + && cacheFlusher.isAlive() && walRoller.isAlive() && this.compactionChecker.isAlive() && this.periodicFlusher.isAlive())) { stop("One or more threads are no longer alive -- stop"); return false; } - if (metaHLogRoller != null && !metaHLogRoller.isAlive()) { - stop("Meta HLog roller thread is no longer alive -- stop"); + final LogRoller metawalRoller = this.metawalRoller.get(); + if (metawalRoller != null && !metawalRoller.isAlive()) { + stop("Meta WAL roller thread is no longer alive -- stop"); return false; } return true; } - public HLog getWAL() { - try { - return getWAL(null); - } catch (IOException e) { - LOG.warn("getWAL threw exception " + e); - return null; - } - } + private static final byte[] UNSPECIFIED_REGION = new byte[]{}; @Override - public HLog 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) + public WAL getWAL(HRegionInfo regionInfo) throws IOException { + WAL wal; + LogRoller roller = walRoller; //_ROOT_ and hbase:meta regions have separate WAL. if (regionInfo != null && regionInfo.isMetaTable()) { - return getMetaWAL(); + roller = ensureMetaWALRoller(); + wal = walFactory.getMetaWAL(regionInfo.getEncodedNameAsBytes()); + } else if (regionInfo == null) { + wal = walFactory.getWAL(UNSPECIFIED_REGION); + } else { + wal = walFactory.getWAL(regionInfo.getEncodedNameAsBytes()); } - return this.hlog; + roller.addWAL(wal); + return wal; } @Override @@ -2006,11 +1978,12 @@ public class HRegionServer extends HasThread implements if (this.spanReceiverHost != null) { this.spanReceiverHost.closeReceivers(); } - if (this.hlogRoller != null) { - Threads.shutdown(this.hlogRoller.getThread()); + if (this.walRoller != null) { + Threads.shutdown(this.walRoller.getThread()); } - if (this.metaHLogRoller != null) { - Threads.shutdown(this.metaHLogRoller.getThread()); + final LogRoller metawalRoller = this.metawalRoller.get(); + if (metawalRoller != null) { + Threads.shutdown(metawalRoller.getThread()); } if (this.compactSplitThread != null) { this.compactSplitThread.join(); @@ -2518,7 +2491,7 @@ public class HRegionServer extends HasThread implements * @see org.apache.hadoop.hbase.regionserver.HRegionServerCommandLine */ public static void main(String[] args) throws Exception { - VersionInfo.logVersion(); + VersionInfo.logVersion(); Configuration conf = HBaseConfiguration.create(); @SuppressWarnings("unchecked") Class regionServerClass = (Class) conf @@ -2569,11 +2542,24 @@ public class HRegionServer extends HasThread implements // used by org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon (HBASE-4070). public String[] getRegionServerCoprocessors() { - TreeSet coprocessors = new TreeSet( - this.hlog.getCoprocessorHost().getCoprocessors()); + TreeSet coprocessors = new TreeSet(); + try { + coprocessors.addAll(getWAL(null).getCoprocessorHost().getCoprocessors()); + } catch (IOException exception) { + LOG.warn("Exception attempting to fetch wal coprocessor information for the common wal; " + + "skipping."); + LOG.debug("Exception details for failure to fetch wal coprocessor information.", exception); + } Collection regions = getOnlineRegionsLocalContext(); for (HRegion region: regions) { coprocessors.addAll(region.getCoprocessorHost().getCoprocessors()); + try { + coprocessors.addAll(getWAL(region.getRegionInfo()).getCoprocessorHost().getCoprocessors()); + } catch (IOException exception) { + LOG.warn("Exception attempting to fetch wal coprocessor information for region " + region + + "; skipping."); + LOG.debug("Exception details for failure to fetch wal coprocessor information.", exception); + } } return coprocessors.toArray(new String[coprocessors.size()]); } @@ -2696,16 +2682,22 @@ public class HRegionServer extends HasThread implements HRegion toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName()); if (destination != null) { - HLog 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. - closeSeqNum = r.getOpenSeqNum(); + try { + WAL wal = getWAL(r.getRegionInfo()); + long closeSeqNum = wal.getEarliestMemstoreSeqNum(r.getRegionInfo().getEncodedNameAsBytes()); if (closeSeqNum == HConstants.NO_SEQNUM) { - closeSeqNum = 0; + // No edits in WAL for this region; get the sequence number when the region was opened. + closeSeqNum = r.getOpenSeqNum(); + if (closeSeqNum == HConstants.NO_SEQNUM) { + closeSeqNum = 0; + } } + addToMovedRegions(r.getRegionInfo().getEncodedName(), destination, closeSeqNum); + } catch (IOException exception) { + LOG.error("Could not retrieve WAL information for region " + r.getRegionInfo() + + "; not adding to moved regions."); + LOG.debug("Exception details for failure to get wal", exception); } - addToMovedRegions(r.getRegionInfo().getEncodedName(), destination, closeSeqNum); } this.regionFavoredNodesMap.remove(r.getRegionInfo().getEncodedName()); return toReturn != null; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index 85b7676079f..381114220a7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -78,7 +78,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.OffPeakHours; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.security.EncryptionUtil; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; @@ -1216,7 +1216,7 @@ public class HStore implements Store { */ private void writeCompactionWalRecord(Collection filesCompacted, Collection newFiles) throws IOException { - if (region.getLog() == null) return; + if (region.getWAL() == null) return; List inputPaths = new ArrayList(filesCompacted.size()); for (StoreFile f : filesCompacted) { inputPaths.add(f.getPath()); @@ -1228,7 +1228,7 @@ public class HStore implements Store { HRegionInfo info = this.region.getRegionInfo(); CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(info, family.getName(), inputPaths, outputPaths, fs.getStoreDir(getFamily().getNameAsString())); - HLogUtil.writeCompactionMarker(region.getLog(), this.region.getTableDesc(), + WALUtil.writeCompactionMarker(region.getWAL(), this.region.getTableDesc(), this.region.getRegionInfo(), compactionDescriptor, this.region.getSequenceId()); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java index e8873ff6276..12c7c562dd8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java @@ -19,38 +19,39 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; +import java.util.Map.Entry; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; -import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.HasThread; import org.apache.hadoop.ipc.RemoteException; /** - * Runs periodically to determine if the HLog should be rolled. + * Runs periodically to determine if the WAL should be rolled. * * NOTE: This class extends Thread rather than Chore because the sleep time * can be interrupted when there is something to do, rather than the Chore * sleep time which is invariant. + * + * TODO: change to a pool of threads */ @InterfaceAudience.Private -class LogRoller extends HasThread implements WALActionsListener { +class LogRoller extends HasThread { static final Log LOG = LogFactory.getLog(LogRoller.class); private final ReentrantLock rollLock = new ReentrantLock(); private final AtomicBoolean rollLog = new AtomicBoolean(false); + private final ConcurrentHashMap walNeedsRoll = + new ConcurrentHashMap(); private final Server server; protected final RegionServerServices services; private volatile long lastrolltime = System.currentTimeMillis(); @@ -58,6 +59,32 @@ class LogRoller extends HasThread implements WALActionsListener { private final long rollperiod; private final int threadWakeFrequency; + public void addWAL(final WAL wal) { + if (null == walNeedsRoll.putIfAbsent(wal, Boolean.FALSE)) { + wal.registerWALActionsListener(new WALActionsListener.Base() { + @Override + public void logRollRequested() { + walNeedsRoll.put(wal, Boolean.TRUE); + // TODO logs will contend with each other here, replace with e.g. DelayedQueue + synchronized(rollLog) { + rollLog.set(true); + rollLog.notifyAll(); + } + } + }); + } + } + + public void requestRollAll() { + for (WAL wal : walNeedsRoll.keySet()) { + walNeedsRoll.put(wal, Boolean.TRUE); + } + synchronized(rollLog) { + rollLog.set(true); + rollLog.notifyAll(); + } + } + /** @param server */ public LogRoller(final Server server, final RegionServerServices services) { super(); @@ -88,19 +115,24 @@ class LogRoller extends HasThread implements WALActionsListener { } // Time for periodic roll if (LOG.isDebugEnabled()) { - LOG.debug("Hlog roll period " + this.rollperiod + "ms elapsed"); + LOG.debug("Wal roll period " + this.rollperiod + "ms elapsed"); } } else if (LOG.isDebugEnabled()) { - LOG.debug("HLog roll requested"); + LOG.debug("WAL roll requested"); } rollLock.lock(); // FindBugs UL_UNRELEASED_LOCK_EXCEPTION_PATH try { this.lastrolltime = now; - // Force the roll if the logroll.period is elapsed or if a roll was requested. - // The returned value is an array of actual region names. - byte [][] regionsToFlush = getWAL().rollWriter(periodic || rollLog.get()); - if (regionsToFlush != null) { - for (byte [] r: regionsToFlush) scheduleFlush(r); + for (Entry entry : walNeedsRoll.entrySet()) { + final WAL wal = entry.getKey(); + // Force the roll if the logroll.period is elapsed or if a roll was requested. + // The returned value is an array of actual region names. + final byte [][] regionsToFlush = wal.rollWriter(periodic || + entry.getValue().booleanValue()); + walNeedsRoll.put(wal, Boolean.FALSE); + if (regionsToFlush != null) { + for (byte [] r: regionsToFlush) scheduleFlush(r); + } } } catch (FailedLogCloseException e) { server.abort("Failed log close in log roller", e); @@ -145,51 +177,4 @@ class LogRoller extends HasThread implements WALActionsListener { } } - public void logRollRequested() { - synchronized (rollLog) { - rollLog.set(true); - rollLog.notifyAll(); - } - } - - protected HLog getWAL() throws IOException { - return this.services.getWAL(null); - } - - @Override - public void preLogRoll(Path oldPath, Path newPath) throws IOException { - // Not interested - } - - @Override - public void postLogRoll(Path oldPath, Path newPath) throws IOException { - // Not interested - } - - @Override - public void preLogArchive(Path oldPath, Path newPath) throws IOException { - // Not interested - } - - @Override - public void postLogArchive(Path oldPath, Path newPath) throws IOException { - // Not interested - } - - @Override - public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey, - WALEdit logEdit) { - // Not interested. - } - - @Override - public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, - WALEdit logEdit) { - //Not interested - } - - @Override - public void logCloseRequested() { - // not interested - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java index 34576f70484..4d4f1467086 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java @@ -460,11 +460,11 @@ class MemStoreFlusher implements FlushRequester { } } catch (DroppedSnapshotException ex) { // Cache flush can fail in a few places. If it fails in a critical - // section, we get a DroppedSnapshotException and a replay of hlog + // section, we get a DroppedSnapshotException and a replay of wal // is required. Currently the only way to do this is a restart of // the server. Abort because hdfs is probably bad (HBASE-644 is a case // where hdfs was bad but passed the hdfs check). - server.abort("Replay of HLog required. Forcing server shutdown", ex); + server.abort("Replay of WAL required. Forcing server shutdown", ex); return false; } catch (IOException ex) { ex = ex instanceof RemoteException ? ((RemoteException) ex).unwrapRemoteException() : ex; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetaLogRoller.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetaLogRoller.java deleted file mode 100644 index 467cfdf8ace..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetaLogRoller.java +++ /dev/null @@ -1,38 +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; - -import java.io.IOException; - -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.Server; -import org.apache.hadoop.hbase.regionserver.wal.HLog; - -@InterfaceAudience.Private -class MetaLogRoller extends LogRoller { - public MetaLogRoller(Server server, RegionServerServices services) { - super(server, services); - } - @Override - protected HLog 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 a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java index 327f55caf1a..52eafb94fd3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java @@ -33,6 +33,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.wal.DefaultWALProvider; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; @@ -53,8 +54,8 @@ class MetricsRegionServerWrapperImpl private BlockCache blockCache; private volatile long numStores = 0; - private volatile long numHLogFiles = 0; - private volatile long hlogFileSize = 0; + private volatile long numWALFiles = 0; + private volatile long walFileSize = 0; private volatile long numStoreFiles = 0; private volatile long memstoreSize = 0; private volatile long storeFileSize = 0; @@ -286,13 +287,13 @@ class MetricsRegionServerWrapperImpl } @Override - public long getNumHLogFiles() { - return numHLogFiles; + public long getNumWALFiles() { + return numWALFiles; } @Override - public long getHLogFileSize() { - return hlogFileSize; + public long getWALFileSize() { + return walFileSize; } @Override @@ -490,21 +491,11 @@ class MetricsRegionServerWrapperImpl } lastRan = currentTime; + numWALFiles = DefaultWALProvider.getNumLogFiles(regionServer.walFactory); + walFileSize = DefaultWALProvider.getLogFileSize(regionServer.walFactory); + //Copy over computed values so that no thread sees half computed values. numStores = tempNumStores; - long tempNumHLogFiles = regionServer.hlog.getNumLogFiles(); - // meta logs - if (regionServer.hlogForMeta != null) { - tempNumHLogFiles += regionServer.hlogForMeta.getNumLogFiles(); - } - numHLogFiles = tempNumHLogFiles; - - long tempHlogFileSize = regionServer.hlog.getLogFileSize(); - if (regionServer.hlogForMeta != null) { - tempHlogFileSize += regionServer.hlogForMeta.getLogFileSize(); - } - hlogFileSize = tempHlogFileSize; - numStoreFiles = tempNumStoreFiles; memstoreSize = tempMemstoreSize; storeFileSize = tempStoreFileSize; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 58f1a40aae0..41f1a99b0ea 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -149,9 +149,9 @@ import org.apache.hadoop.hbase.regionserver.HRegion.Operation; import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException; import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler; import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogKey; -import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.Bytes; @@ -693,13 +693,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * @throws IOException */ private OperationStatus [] doReplayBatchOp(final HRegion region, - final List mutations, long replaySeqId) throws IOException { + final List mutations, long replaySeqId) throws IOException { long before = EnvironmentEdgeManager.currentTime(); boolean batchContainsPuts = false, batchContainsDelete = false; try { - for (Iterator it = mutations.iterator(); it.hasNext();) { - HLogSplitter.MutationReplay m = it.next(); + for (Iterator it = mutations.iterator(); it.hasNext();) { + WALSplitter.MutationReplay m = it.next(); if (m.type == MutationType.PUT) { batchContainsPuts = true; @@ -724,7 +724,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, regionServer.cacheFlusher.reclaimMemStoreMemory(); } return region.batchReplay(mutations.toArray( - new HLogSplitter.MutationReplay[mutations.size()]), replaySeqId); + new WALSplitter.MutationReplay[mutations.size()]), replaySeqId); } finally { if (regionServer.metricsRegionServer != null) { long after = EnvironmentEdgeManager.currentTime(); @@ -1097,10 +1097,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler, return builder.build(); } catch (DroppedSnapshotException ex) { // Cache flush can fail in a few places. If it fails in a critical - // section, we get a DroppedSnapshotException and a replay of hlog + // section, we get a DroppedSnapshotException and a replay of wal // is required. Currently the only way to do this is a restart of // the server. - regionServer.abort("Replay of HLog required. Forcing server shutdown", ex); + regionServer.abort("Replay of WAL required. Forcing server shutdown", ex); throw new ServiceException(ex); } catch (IOException ie) { throw new ServiceException(ie); @@ -1431,7 +1431,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, ServerRegionReplicaUtil.isDefaultReplica(region.getRegionInfo()) ? region.getCoprocessorHost() : null; // do not invoke coprocessors if this is a secondary region replica - List> walEntries = new ArrayList>(); + List> walEntries = new ArrayList>(); // when tag is enabled, we need tag replay edits with log sequence number boolean needAddReplayTag = (HFile.getFormatVersion(regionServer.conf) >= 3); @@ -1451,9 +1451,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler, long nonce = entry.getKey().hasNonce() ? entry.getKey().getNonce() : HConstants.NO_NONCE; regionServer.nonceManager.reportOperationFromWal(nonceGroup, nonce, entry.getKey().getWriteTime()); } - Pair walEntry = (coprocessorHost == null) ? null : - new Pair(); - List edits = HLogSplitter.getMutationsFromWALEntry(entry, + Pair walEntry = (coprocessorHost == null) ? null : + new Pair(); + List edits = WALSplitter.getMutationsFromWALEntry(entry, cells, walEntry, needAddReplayTag, durability); if (coprocessorHost != null) { // Start coprocessor replay here. The coprocessor is for each WALEdit instead of a @@ -1482,7 +1482,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, region.syncWal(); if (coprocessorHost != null) { - for (Pair wal : walEntries) { + for (Pair wal : walEntries) { coprocessorHost.postWALRestore(region.getRegionInfo(), wal.getFirst(), wal.getSecond()); } @@ -1535,14 +1535,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler, checkOpen(); requestCount.increment(); regionServer.getRegionServerCoprocessorHost().preRollWALWriterRequest(); - HLog wal = regionServer.getWAL(); - byte[][] regionsToFlush = wal.rollWriter(true); + regionServer.walRoller.requestRollAll(); + regionServer.getRegionServerCoprocessorHost().postRollWALWriterRequest(); RollWALWriterResponse.Builder builder = RollWALWriterResponse.newBuilder(); - if (regionsToFlush != null) { - for (byte[] region: regionsToFlush) { - builder.addRegionToFlush(ByteStringer.wrap(region)); - } - } return builder.build(); } catch (IOException ie) { throw new ServiceException(ie); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java index d5f1ff84d31..e671e5052af 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java @@ -77,6 +77,7 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.regionserver.HRegion.Operation; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; @@ -107,6 +108,8 @@ public class RegionCoprocessorHost private static final int LATENCY_BUFFER_SIZE = 100; private final BlockingQueue coprocessorTimeNanos = new ArrayBlockingQueue( LATENCY_BUFFER_SIZE); + private final boolean useLegacyPre; + private final boolean useLegacyPost; /** * Constructor @@ -120,6 +123,14 @@ public class RegionCoprocessorHost this.region = region; this.rsServices = services; this.sharedData = sharedData; + // Pick which version of the WAL related events we'll call. + // This way we avoid calling the new version on older RegionObservers so + // we can maintain binary compatibility. + // See notes in javadoc for RegionObserver + useLegacyPre = useLegacyMethod(impl.getClass(), "preWALRestore", ObserverContext.class, + HRegionInfo.class, WALKey.class, WALEdit.class); + useLegacyPost = useLegacyMethod(impl.getClass(), "postWALRestore", ObserverContext.class, + HRegionInfo.class, WALKey.class, WALEdit.class); } /** @return the region */ @@ -1307,34 +1318,75 @@ public class RegionCoprocessorHost * @return true if default behavior should be bypassed, false otherwise * @throws IOException */ - public boolean preWALRestore(final HRegionInfo info, final HLogKey logKey, + public boolean preWALRestore(final HRegionInfo info, final WALKey logKey, final WALEdit logEdit) throws IOException { return execOperation(coprocessors.isEmpty() ? null : new RegionOperation() { @Override public void call(RegionObserver oserver, ObserverContext ctx) throws IOException { - oserver.preWALRestore(ctx, info, logKey, logEdit); + // Once we don't need to support the legacy call, replace RegionOperation with a version + // that's ObserverContext and avoid this cast. + final RegionEnvironment env = (RegionEnvironment)ctx.getEnvironment(); + if (env.useLegacyPre) { + if (logKey instanceof HLogKey) { + oserver.preWALRestore(ctx, info, (HLogKey)logKey, logEdit); + } else { + legacyWarning(oserver.getClass(), "There are wal keys present that are not HLogKey."); + } + } else { + oserver.preWALRestore(ctx, info, logKey, logEdit); + } } }); } + /** + * @return true if default behavior should be bypassed, false otherwise + * @deprecated use {@link #preWALRestore(HRegionInfo, WALKey, WALEdit)} + */ + @Deprecated + public boolean preWALRestore(final HRegionInfo info, final HLogKey logKey, + final WALEdit logEdit) throws IOException { + return preWALRestore(info, (WALKey)logKey, logEdit); + } + /** * @param info * @param logKey * @param logEdit * @throws IOException */ - public void postWALRestore(final HRegionInfo info, final HLogKey logKey, final WALEdit logEdit) + public void postWALRestore(final HRegionInfo info, final WALKey logKey, final WALEdit logEdit) throws IOException { execOperation(coprocessors.isEmpty() ? null : new RegionOperation() { @Override public void call(RegionObserver oserver, ObserverContext ctx) throws IOException { - oserver.postWALRestore(ctx, info, logKey, logEdit); + // Once we don't need to support the legacy call, replace RegionOperation with a version + // that's ObserverContext and avoid this cast. + final RegionEnvironment env = (RegionEnvironment)ctx.getEnvironment(); + if (env.useLegacyPost) { + if (logKey instanceof HLogKey) { + oserver.postWALRestore(ctx, info, (HLogKey)logKey, logEdit); + } else { + legacyWarning(oserver.getClass(), "There are wal keys present that are not HLogKey."); + } + } else { + oserver.postWALRestore(ctx, info, logKey, logEdit); + } } }); } + /** + * @deprecated use {@link #postWALRestore(HRegionInfo, WALKey, WALEdit)} + */ + @Deprecated + public void postWALRestore(final HRegionInfo info, final HLogKey logKey, final WALEdit logEdit) + throws IOException { + postWALRestore(info, (WALKey)logKey, logEdit); + } + /** * @param familyPaths pairs of { CF, file path } submitted for bulk load * @return true if the default operation should be bypassed diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java index 479aced5ab1..879b573e405 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java @@ -34,7 +34,7 @@ public class RegionServerAccounting { private final AtomicLong atomicGlobalMemstoreSize = new AtomicLong(0); - // Store the edits size during replaying HLog. Use this to roll back the + // Store the edits size during replaying WAL. Use this to roll back the // global memstore size once a region opening failed. private final ConcurrentMap replayEditsPerRegion = new ConcurrentSkipListMap(Bytes.BYTES_COMPARATOR); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java index f02b8baa8fc..08d038c8669 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java @@ -33,8 +33,8 @@ import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.ipc.RpcServerInterface; import org.apache.hadoop.hbase.master.TableLockManager; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; -import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager; +import org.apache.hadoop.hbase.wal.WAL; import org.apache.zookeeper.KeeperException; /** @@ -48,9 +48,9 @@ public interface RegionServerServices */ boolean isStopping(); - /** @return the HLog for a particular region. Pass null for getting the + /** @return the WAL for a particular region. Pass null for getting the * default (common) WAL */ - HLog getWAL(HRegionInfo regionInfo) throws IOException; + WAL getWAL(HRegionInfo regionInfo) throws IOException; /** * @return Implementation of {@link CompactionRequestor} or null. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java index 0052b002677..a182aa1a8d4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java @@ -36,7 +36,8 @@ import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager; import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination; import org.apache.hadoop.hbase.master.SplitLogManager; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; -import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hbase.util.CancelableProgressable; import org.apache.hadoop.hbase.util.ExceptionUtil; import org.apache.hadoop.hbase.util.FSUtils; @@ -70,6 +71,7 @@ public class SplitLogWorker implements Runnable { private SplitLogWorkerCoordination coordination; private Configuration conf; private RegionServerServices server; + public SplitLogWorker(Server hserver, Configuration conf, RegionServerServices server, TaskExecutor splitTaskExecutor) { this.server = server; @@ -82,7 +84,8 @@ public class SplitLogWorker implements Runnable { } public SplitLogWorker(final Server hserver, final Configuration conf, - final RegionServerServices server, final LastSequenceId sequenceIdChecker) { + final RegionServerServices server, final LastSequenceId sequenceIdChecker, + final WALFactory factory) { this(server, conf, server, new TaskExecutor() { @Override public Status exec(String filename, RecoveryMode mode, CancelableProgressable p) { @@ -99,8 +102,8 @@ public class SplitLogWorker implements Runnable { // interrupted or has encountered a transient error and when it has // encountered a bad non-retry-able persistent error. try { - if (!HLogSplitter.splitLogFile(rootdir, fs.getFileStatus(new Path(rootdir, filename)), - fs, conf, p, sequenceIdChecker, server.getCoordinatedStateManager(), mode)) { + if (!WALSplitter.splitLogFile(rootdir, fs.getFileStatus(new Path(rootdir, filename)), + fs, conf, p, sequenceIdChecker, server.getCoordinatedStateManager(), mode, factory)) { return Status.PREEMPTED; } } catch (InterruptedIOException iioe) { @@ -153,6 +156,7 @@ public class SplitLogWorker implements Runnable { LOG.info("SplitLogWorker " + server.getServerName() + " exiting"); } } + /** * If the worker is doing a task i.e. splitting a log file then stop the task. * It doesn't exit the worker thread. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/HLogSplitterHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/HLogSplitterHandler.java deleted file mode 100644 index b0f3f0b5396..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/HLogSplitterHandler.java +++ /dev/null @@ -1,106 +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.handler; - -import java.io.IOException; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.Server; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.SplitLogCounters; -import org.apache.hadoop.hbase.SplitLogTask; -import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination; -import org.apache.hadoop.hbase.executor.EventHandler; -import org.apache.hadoop.hbase.executor.EventType; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; -import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor; -import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor.Status; -import org.apache.hadoop.hbase.util.CancelableProgressable; - -/** - * Handles log splitting a wal - */ -@InterfaceAudience.Private -public class HLogSplitterHandler extends EventHandler { - private static final Log LOG = LogFactory.getLog(HLogSplitterHandler.class); - private final ServerName serverName; - private final CancelableProgressable reporter; - private final AtomicInteger inProgressTasks; - private final TaskExecutor splitTaskExecutor; - private final RecoveryMode mode; - private final SplitLogWorkerCoordination.SplitTaskDetails splitTaskDetails; - private final SplitLogWorkerCoordination coordination; - - - public HLogSplitterHandler(final Server server, SplitLogWorkerCoordination coordination, - SplitLogWorkerCoordination.SplitTaskDetails splitDetails, CancelableProgressable reporter, - AtomicInteger inProgressTasks, TaskExecutor splitTaskExecutor, RecoveryMode mode) { - super(server, EventType.RS_LOG_REPLAY); - this.splitTaskDetails = splitDetails; - this.coordination = coordination; - this.reporter = reporter; - this.inProgressTasks = inProgressTasks; - this.inProgressTasks.incrementAndGet(); - this.serverName = server.getServerName(); - this.splitTaskExecutor = splitTaskExecutor; - this.mode = mode; - } - - @Override - public void process() throws IOException { - long startTime = System.currentTimeMillis(); - try { - Status status = this.splitTaskExecutor.exec(splitTaskDetails.getWALFile(), mode, reporter); - switch (status) { - case DONE: - coordination.endTask(new SplitLogTask.Done(this.serverName,this.mode), - SplitLogCounters.tot_wkr_task_done, splitTaskDetails); - break; - case PREEMPTED: - SplitLogCounters.tot_wkr_preempt_task.incrementAndGet(); - LOG.warn("task execution prempted " + splitTaskDetails.getWALFile()); - break; - case ERR: - if (server != null && !server.isStopped()) { - coordination.endTask(new SplitLogTask.Err(this.serverName, this.mode), - SplitLogCounters.tot_wkr_task_err, splitTaskDetails); - break; - } - // if the RS is exiting then there is probably a tons of stuff - // that can go wrong. Resign instead of signaling error. - //$FALL-THROUGH$ - case RESIGNED: - if (server != null && server.isStopped()) { - LOG.info("task execution interrupted because worker is exiting " - + splitTaskDetails.toString()); - } - coordination.endTask(new SplitLogTask.Resigned(this.serverName, this.mode), - SplitLogCounters.tot_wkr_task_resigned, splitTaskDetails); - break; - } - } finally { - LOG.info("worker " + serverName + " done with task " + splitTaskDetails.toString() + " in " - + (System.currentTimeMillis() - startTime) + "ms"); - this.inProgressTasks.decrementAndGet(); - } - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java index d8da4127f02..12af61993c3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java @@ -26,17 +26,18 @@ import org.apache.hadoop.hbase.io.TagCompressionContext; import org.apache.hadoop.hbase.io.util.Dictionary; /** - * Context that holds the various dictionaries for compression in HLog. + * Context that holds the various dictionaries for compression in WAL. */ @InterfaceAudience.Private -class CompressionContext { +public class CompressionContext { static final String ENABLE_WAL_TAGS_COMPRESSION = "hbase.regionserver.wal.tags.enablecompression"; - final Dictionary regionDict; - final Dictionary tableDict; - final Dictionary familyDict; + // visible only for WALKey, until we move everything into o.a.h.h.wal + public final Dictionary regionDict; + public final Dictionary tableDict; + public final Dictionary familyDict; final Dictionary qualifierDict; final Dictionary rowDict; // Context used for compressing tags diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/Compressor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/Compressor.java index b75a7cf8ad0..4032cde5884 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/Compressor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/Compressor.java @@ -33,9 +33,13 @@ import org.apache.hadoop.io.WritableUtils; import com.google.common.base.Preconditions; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALProvider; + /** * A set of static functions for running our custom WAL compression/decompression. - * Also contains a command line tool to compress and uncompress HLogs. + * Also contains a command line tool to compress and uncompress WALs. */ @InterfaceAudience.Private public class Compressor { @@ -56,8 +60,8 @@ public class Compressor { private static void printHelp() { System.err.println("usage: Compressor "); - System.err.println("If HLog is compressed, will be decompressed."); - System.err.println("If HLog is uncompressed, will be compressed."); + System.err.println("If WAL is compressed, will be decompressed."); + System.err.println("If WAL is uncompressed, will be compressed."); return; } @@ -68,8 +72,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 = WALFactory.createReaderIgnoreCustomClass(inFS, input, conf); + WALProvider.Writer out = null; try { if (!(in instanceof ReaderBase)) { @@ -78,9 +82,9 @@ public class Compressor { } boolean compress = ((ReaderBase)in).hasCompression(); conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, !compress); - out = HLogFactory.createWALWriter(outFS, output, conf); + out = WALFactory.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 a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java index 2f820864e53..02cf41ec1e2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java @@ -43,6 +43,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; @@ -55,7 +56,7 @@ 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.fs.PathFilter; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -64,6 +65,15 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import static org.apache.hadoop.hbase.wal.DefaultWALProvider.WAL_FILE_NAME_DELIMITER; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.hbase.wal.WALPrettyPrinter; +import org.apache.hadoop.hbase.wal.WALProvider.Writer; +import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.hbase.util.DrainBarrier; @@ -89,8 +99,8 @@ import com.lmax.disruptor.dsl.Disruptor; import com.lmax.disruptor.dsl.ProducerType; /** - * Implementation of {@link HLog} to go against {@link FileSystem}; i.e. keep WALs in HDFS. - * Only one HLog/WAL is ever being written at a time. When a WAL hits a configured maximum size, + * Implementation of {@link WAL} to go against {@link FileSystem}; i.e. keep WALs in HDFS. + * Only one WAL is ever being written at a time. When a WAL hits a configured maximum size, * it is rolled. This is done internal to the implementation. * *

    As data is flushed from the MemStore to other on-disk structures (files sorted by @@ -102,11 +112,11 @@ import com.lmax.disruptor.dsl.ProducerType; * F when all of the edits in F have a log-sequence-id that's older * (smaller) than the most-recent flush. * - *

    To read an HLog, call {@link HLogFactory#createReader(org.apache.hadoop.fs.FileSystem, - * org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration)}. + *

    To read an WAL, call {@link WALFactory#createReader(org.apache.hadoop.fs.FileSystem, + * org.apache.hadoop.fs.Path)}. */ @InterfaceAudience.Private -class FSHLog implements HLog, Syncable { +public class FSHLog implements WAL { // IMPLEMENTATION NOTES: // // At the core is a ring buffer. Our ring buffer is the LMAX Disruptor. It tries to @@ -176,12 +186,6 @@ class FSHLog implements HLog, Syncable { */ private final Map syncFuturesByHandler; - private final FileSystem fs; - private final Path fullPathLogDir; - private final Path fullPathOldLogDir; - private final Configuration conf; - private final String logFilePrefix; - /** * The highest known outstanding unsync'd WALEdit sequence number where sequence number is the * ring buffer sequence. Maintained by the ring buffer consumer. @@ -195,8 +199,62 @@ class FSHLog implements HLog, Syncable { */ private final AtomicLong highestSyncedSequence = new AtomicLong(0); - private WALCoprocessorHost coprocessorHost; + /** + * file system instance + */ + private final FileSystem fs; + /** + * WAL directory, where all WAL files would be placed. + */ + private final Path fullPathLogDir; + /** + * dir path where old logs are kept. + */ + private final Path fullPathArchiveDir; + /** + * Matches just those wal files that belong to this wal instance. + */ + private final PathFilter ourFiles; + + /** + * Prefix of a WAL file, usually the region server name it is hosted on. + */ + private final String logFilePrefix; + + /** + * Suffix included on generated wal file names + */ + private final String logFileSuffix; + + /** + * Prefix used when checking for wal membership. + */ + private final String prefixPathStr; + + private final WALCoprocessorHost coprocessorHost; + + /** + * conf object + */ + private final Configuration conf; + /** Listeners that are called on WAL events. */ + private final List listeners = new CopyOnWriteArrayList(); + + @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 WALCoprocessorHost getCoprocessorHost() { + return coprocessorHost; + } /** * FSDataOutputStream associated with the current SequenceFile.writer */ @@ -244,37 +302,23 @@ class FSHLog implements HLog, Syncable { */ private final ReentrantLock rollWriterLock = new ReentrantLock(true); - // Listeners that are called on WAL events. - private final List listeners = - new CopyOnWriteArrayList(); - private volatile boolean closed = false; - - /** - * Set when this WAL is for meta only (we run a WAL for all regions except meta -- it has its - * own dedicated WAL). - */ - private final boolean forMeta; + private final AtomicBoolean shutdown = new AtomicBoolean(false); // The timestamp (in ms) when the log file was created. private final AtomicLong filenum = new AtomicLong(-1); - // Number of transactions in the current Hlog. + // Number of transactions in the current Wal. private final AtomicInteger numEntries = new AtomicInteger(0); // If > than this size, roll the log. private final long logrollsize; /** - * The total size of hlog + * The total size of wal */ private AtomicLong totalLogSize = new AtomicLong(0); - /** - * If WAL is enabled. - */ - private final boolean enabled; - /* * If more than this many logs, force flush of oldest region to oldest edit * goes to disk. If too many and we crash, then will take forever replaying. @@ -286,7 +330,6 @@ class FSHLog implements HLog, Syncable { private final int closeErrorsTolerated; private final AtomicInteger closeErrorCount = new AtomicInteger(); - private final MetricsWAL metrics; // Region sequence id accounting across flushes and for knowing when we can GC a WAL. These // sequence id numbers are by region and unrelated to the ring buffer sequence number accounting @@ -335,6 +378,7 @@ class FSHLog implements HLog, Syncable { /** * WAL Comparator; it compares the timestamp (log filenum), present in the log file name. + * Throws an IllegalArgumentException if used to compare paths from different wals. */ public final Comparator LOG_NAME_COMPARATOR = new Comparator() { @Override @@ -381,15 +425,14 @@ class FSHLog implements HLog, Syncable { * Constructor. * * @param fs filesystem handle - * @param root path for stored and archived hlogs - * @param logDir dir where hlogs are stored + * @param root path for stored and archived wals + * @param logDir dir where wals are stored * @param conf configuration to use * @throws IOException */ - public FSHLog(final FileSystem fs, final Path root, final String logDir, - final Configuration conf) - throws IOException { - this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, false); + public FSHLog(final FileSystem fs, final Path root, final String logDir, final Configuration conf) + throws IOException { + this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null); } /** @@ -397,46 +440,95 @@ class FSHLog implements HLog, Syncable { * * You should never have to load an existing log. If there is a log at * startup, it should have already been processed and deleted by the time the - * HLog object is started up. + * WAL object is started up. * * @param fs filesystem handle * @param rootDir path to where logs and oldlogs - * @param logDir dir where hlogs are stored - * @param oldLogDir dir where hlogs are archived + * @param logDir dir where wals are stored + * @param archiveDir dir where wals are archived * @param conf configuration to use * @param listeners Listeners on WAL events. Listeners passed here will * be registered before we do anything else; e.g. the * Constructor {@link #rollWriter()}. - * @param failIfLogDirExists If true IOException will be thrown if dir already exists. + * @param failIfWALExists If true IOException will be thrown if files related to this wal + * already exist. * @param prefix should always be hostname and port in distributed env and * it will be URL encoded before being used. - * If prefix is null, "hlog" will be used - * @param forMeta if this hlog is meant for meta updates + * If prefix is null, "wal" will be used + * @param suffix will be url encoded. null is treated as empty. non-empty must start with + * {@link DefaultWALProvider#WAL_FILE_NAME_DELIMITER} * @throws IOException */ public FSHLog(final FileSystem fs, final Path rootDir, final String logDir, - final String oldLogDir, final Configuration conf, + final String archiveDir, final Configuration conf, final List listeners, - final boolean failIfLogDirExists, final String prefix, boolean forMeta) - throws IOException { - super(); + final boolean failIfWALExists, final String prefix, final String suffix) + throws IOException { this.fs = fs; this.fullPathLogDir = new Path(rootDir, logDir); - this.fullPathOldLogDir = new Path(rootDir, oldLogDir); - this.forMeta = forMeta; + this.fullPathArchiveDir = new Path(rootDir, archiveDir); this.conf = conf; + if (!fs.exists(fullPathLogDir) && !fs.mkdirs(fullPathLogDir)) { + throw new IOException("Unable to mkdir " + fullPathLogDir); + } + + if (!fs.exists(this.fullPathArchiveDir)) { + if (!fs.mkdirs(this.fullPathArchiveDir)) { + throw new IOException("Unable to mkdir " + this.fullPathArchiveDir); + } + } + + // If prefix is null||empty then just name it wal + this.logFilePrefix = + prefix == null || prefix.isEmpty() ? "wal" : URLEncoder.encode(prefix, "UTF8"); + // we only correctly differentiate suffices when numeric ones start with '.' + if (suffix != null && !(suffix.isEmpty()) && !(suffix.startsWith(WAL_FILE_NAME_DELIMITER))) { + throw new IllegalArgumentException("wal suffix must start with '" + WAL_FILE_NAME_DELIMITER + + "' but instead was '" + suffix + "'"); + } + this.logFileSuffix = (suffix == null) ? "" : URLEncoder.encode(suffix, "UTF8"); + this.prefixPathStr = new Path(fullPathLogDir, + logFilePrefix + WAL_FILE_NAME_DELIMITER).toString(); + + this.ourFiles = new PathFilter() { + @Override + public boolean accept(final Path fileName) { + // The path should start with dir/ and end with our suffix + final String fileNameString = fileName.toString(); + if (!fileNameString.startsWith(prefixPathStr)) { + return false; + } + if (logFileSuffix.isEmpty()) { + // in the case of the null suffix, we need to ensure the filename ends with a timestamp. + return org.apache.commons.lang.StringUtils.isNumeric( + fileNameString.substring(prefixPathStr.length())); + } else if (!fileNameString.endsWith(logFileSuffix)) { + return false; + } + return true; + } + }; + + if (failIfWALExists) { + final FileStatus[] walFiles = FSUtils.listStatus(fs, fullPathLogDir, ourFiles); + if (null != walFiles && 0 != walFiles.length) { + throw new IOException("Target WAL already exists within directory " + fullPathLogDir); + } + } + // Register listeners. TODO: Should this exist anymore? We have CPs? if (listeners != null) { for (WALActionsListener i: listeners) { registerWALActionsListener(i); } } + this.coprocessorHost = new WALCoprocessorHost(this, conf); // Get size to roll log at. Roll at 95% of HDFS block size so we avoid crossing HDFS blocks // (it costs a little x'ing bocks) - long blocksize = this.conf.getLong("hbase.regionserver.hlog.blocksize", - FSUtils.getDefaultBlockSize(this.fs, this.fullPathLogDir)); + final long blocksize = this.conf.getLong("hbase.regionserver.hlog.blocksize", + FSUtils.getDefaultBlockSize(this.fs, this.fullPathLogDir)); this.logrollsize = (long)(blocksize * conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f)); @@ -445,31 +537,13 @@ class FSHLog implements HLog, Syncable { FSUtils.getDefaultReplication(fs, this.fullPathLogDir)); this.lowReplicationRollLimit = conf.getInt("hbase.regionserver.hlog.lowreplication.rolllimit", 5); - this.enabled = conf.getBoolean("hbase.regionserver.hlog.enabled", true); this.closeErrorsTolerated = conf.getInt("hbase.regionserver.logroll.errors.tolerated", 0); - // If prefix is null||empty then just name it hlog - this.logFilePrefix = - prefix == null || prefix.isEmpty() ? "hlog" : URLEncoder.encode(prefix, "UTF8"); int maxHandlersCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT, 200); LOG.info("WAL configuration: blocksize=" + StringUtils.byteDesc(blocksize) + ", rollsize=" + StringUtils.byteDesc(this.logrollsize) + - ", enabled=" + this.enabled + ", prefix=" + this.logFilePrefix + ", logDir=" + - this.fullPathLogDir + ", oldLogDir=" + this.fullPathOldLogDir); - - boolean dirExists = false; - if (failIfLogDirExists && (dirExists = this.fs.exists(fullPathLogDir))) { - throw new IOException("Target HLog directory already exists: " + fullPathLogDir); - } - if (!dirExists && !fs.mkdirs(fullPathLogDir)) { - throw new IOException("Unable to mkdir " + fullPathLogDir); - } - - if (!fs.exists(this.fullPathOldLogDir)) { - if (!fs.mkdirs(this.fullPathOldLogDir)) { - throw new IOException("Unable to mkdir " + this.fullPathOldLogDir); - } - } + ", prefix=" + this.logFilePrefix + ", suffix=" + logFileSuffix + ", logDir=" + + this.fullPathLogDir + ", archiveDir=" + this.fullPathArchiveDir); // rollWriter sets this.hdfs_out if it can. rollWriter(); @@ -482,9 +556,6 @@ class FSHLog implements HLog, Syncable { this.getNumCurrentReplicas = getGetNumCurrentReplicas(this.hdfs_out); this.getPipeLine = getGetPipeline(this.hdfs_out); - this.coprocessorHost = new WALCoprocessorHost(this, conf); - this.metrics = new MetricsWAL(); - // This is the 'writer' -- a single threaded executor. This single thread 'consumes' what is // put on the ring buffer. String hostingThreadName = Thread.currentThread().getName(); @@ -516,56 +587,22 @@ class FSHLog implements HLog, Syncable { } /** - * Find the 'getNumCurrentReplicas' on the passed os stream. - * @return Method or null. + * Get the backing files associated with this WAL. + * @return may be null if there are no files. */ - private static Method getGetNumCurrentReplicas(final FSDataOutputStream os) { - // TODO: Remove all this and use the now publically available - // HdfsDataOutputStream#getCurrentBlockReplication() - 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("No 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"); - } - 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.get(); + protected FileStatus[] getFiles() throws IOException { + return FSUtils.listStatus(fs, fullPathLogDir, ourFiles); } /** - * Method used internal to this class and for tests only. - * @return The wrapped stream our writer is using; its not the - * writer's 'out' FSDatoOutputStream but the stream that this 'out' wraps - * (In hdfs its an instance of DFSDataOutputStream). - * - * usage: see TestLogRolling.java + * 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. + * NOTE: This could be removed once Hadoop1 support is removed. + * @return null if underlying stream is not ready. */ + @VisibleForTesting OutputStream getOutputStream() { return this.hdfs_out.getWrappedStream(); } @@ -575,12 +612,16 @@ class FSHLog implements HLog, Syncable { return rollWriter(false); } + /** + * retrieve the next path to use for writing. + * Increments the internal filenum. + */ private Path getNewPath() throws IOException { this.filenum.set(System.currentTimeMillis()); - Path newPath = computeFilename(); + Path newPath = getCurrentFileName(); while (fs.exists(newPath)) { this.filenum.incrementAndGet(); - newPath = computeFilename(); + newPath = getCurrentFileName(); } return newPath; } @@ -589,7 +630,7 @@ class FSHLog implements HLog, Syncable { long currentFilenum = this.filenum.get(); Path oldPath = null; if (currentFilenum > 0) { - // ComputeFilename will take care of meta hlog filename + // ComputeFilename will take care of meta wal filename oldPath = computeFilename(currentFilenum); } // I presume if currentFilenum is <= 0, this is first file and null for oldPath if fine? return oldPath; @@ -645,11 +686,11 @@ class FSHLog implements HLog, Syncable { if (!force && (this.writer != null && this.numEntries.get() <= 0)) return null; byte [][] regionsToFlush = null; if (this.closed) { - LOG.debug("HLog closed. Skipping rolling of writer"); + LOG.debug("WAL closed. Skipping rolling of writer"); return regionsToFlush; } if (!closeBarrier.beginOp()) { - LOG.debug("HLog closing. Skipping rolling of writer"); + LOG.debug("WAL closing. Skipping rolling of writer"); return regionsToFlush; } TraceScope scope = Trace.startSpan("FSHLog.rollWriter"); @@ -657,7 +698,7 @@ class FSHLog implements HLog, Syncable { Path oldPath = getOldPath(); Path newPath = getNewPath(); // Any exception from here on is catastrophic, non-recoverable so we currently abort. - FSHLog.Writer nextWriter = this.createWriterInstance(fs, newPath, conf); + Writer nextWriter = this.createWriterInstance(newPath); FSDataOutputStream nextHdfsOut = null; if (nextWriter instanceof ProtobufLogWriter) { nextHdfsOut = ((ProtobufLogWriter)nextWriter).getStream(); @@ -689,18 +730,10 @@ class FSHLog implements HLog, Syncable { * This method allows subclasses to inject different writers without having to * extend other methods like rollWriter(). * - * @param fs - * @param path - * @param conf * @return Writer instance - * @throws IOException */ - protected Writer createWriterInstance(final FileSystem fs, final Path path, - final Configuration conf) throws IOException { - if (forMeta) { - //TODO: set a higher replication for the hlog files (HBASE-6773) - } - return HLogFactory.createWALWriter(fs, path, conf); + protected Writer createWriterInstance(final Path path) throws IOException { + return DefaultWALProvider.createWriter(conf, fs, path, false); } /** @@ -748,7 +781,7 @@ class FSHLog implements HLog, Syncable { * {@link #oldestUnflushedRegionSequenceIds} and {@link #lowestFlushingRegionSequenceIds}. 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 sequenceNums for a WAL, at the time when it was rolled. * @param oldestFlushingMap * @param oldestUnflushedMap * @return true if wal is eligible for archiving, false otherwise. @@ -817,7 +850,7 @@ class FSHLog implements HLog, Syncable { if (i > 0) sb.append(", "); sb.append(Bytes.toStringBinary(regions[i])); } - LOG.info("Too many hlogs: logs=" + logCount + ", maxlogs=" + + LOG.info("Too many wals: logs=" + logCount + ", maxlogs=" + this.maxLogs + "; forcing flush of " + regions.length + " regions(s): " + sb.toString()); } @@ -842,7 +875,7 @@ class FSHLog implements HLog, Syncable { * @return the passed in newPath * @throws IOException if there is a problem flushing or closing the underlying FS */ - Path replaceWriter(final Path oldPath, final Path newPath, FSHLog.Writer nextWriter, + Path replaceWriter(final Path oldPath, final Path newPath, Writer nextWriter, final FSDataOutputStream nextHdfsOut) throws IOException { // Ask the ring buffer writer to pause at a safe point. Once we do this, the writer @@ -912,7 +945,7 @@ class FSHLog implements HLog, Syncable { Thread.currentThread().interrupt(); } catch (IOException e) { long count = getUnflushedEntriesCount(); - LOG.error("Failed close of HLog writer " + oldPath + ", unflushedEntries=" + count, e); + LOG.error("Failed close of WAL writer " + oldPath + ", unflushedEntries=" + count, e); throw new FailedLogCloseException(oldPath + ", unflushedEntries=" + count, e); } finally { try { @@ -939,8 +972,16 @@ class FSHLog implements HLog, Syncable { return getUnflushedEntriesCount() > 0; } + /* + * only public so WALSplitter can use. + * @return archived location of a WAL file with the given path p + */ + public static Path getWALArchivePath(Path archiveDir, Path p) { + return new Path(archiveDir, p.getName()); + } + private void archiveLogFile(final Path p) throws IOException { - Path newPath = getHLogArchivePath(this.fullPathOldLogDir, p); + Path newPath = getWALArchivePath(this.fullPathArchiveDir, p); // Tell our listeners that a log is going to be archived. if (!this.listeners.isEmpty()) { for (WALActionsListener i : this.listeners) { @@ -965,24 +1006,25 @@ class FSHLog implements HLog, Syncable { * @return Path */ protected Path computeFilename(final long filenum) { - this.filenum.set(filenum); - return computeFilename(); + if (filenum < 0) { + throw new RuntimeException("wal file number can't be < 0"); + } + String child = logFilePrefix + WAL_FILE_NAME_DELIMITER + filenum + logFileSuffix; + return new Path(fullPathLogDir, child); } /** * This is a convenience method that computes a new filename with a given - * using the current HLog file-number + * using the current WAL file-number * @return Path */ - protected Path computeFilename() { - if (this.filenum.get() < 0) { - throw new RuntimeException("hlog file number can't be < 0"); - } - String child = logFilePrefix + "." + filenum; - if (forMeta) { - child += HLog.META_HLOG_FILE_EXTN; - } - return new Path(fullPathLogDir, child); + public Path getCurrentFileName() { + return computeFilename(this.filenum.get()); + } + + @Override + public String toString() { + return "FSHLog " + logFilePrefix + ":" + logFileSuffix + "(num " + filenum + ")"; } /** @@ -995,26 +1037,23 @@ 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(fullPathLogDir, logFilePrefix + ".").toString(); - if (!fileName.toString().startsWith(prefixPathStr)) { - throw new IllegalArgumentException("The log file " + fileName + " doesn't belong to" + - " this regionserver " + prefixPathStr); + if (!ourFiles.accept(fileName)) { + throw new IllegalArgumentException("The log file " + fileName + + " doesn't belong to this wal. (" + toString() + ")"); } - String chompedPath = fileName.toString().substring(prefixPathStr.length()); - if (forMeta) chompedPath = chompedPath.substring(0, chompedPath.indexOf(META_HLOG_FILE_EXTN)); + final String fileNameString = fileName.toString(); + String chompedPath = fileNameString.substring(prefixPathStr.length(), + (fileNameString.length() - logFileSuffix.length())); return Long.parseLong(chompedPath); } @Override - public void closeAndDelete() throws IOException { - close(); - if (!fs.exists(this.fullPathLogDir)) return; - FileStatus[] files = fs.listStatus(this.fullPathLogDir); - if (files != null) { - for(FileStatus file : files) { - - Path p = getHLogArchivePath(this.fullPathOldLogDir, file.getPath()); + public void close() throws IOException { + shutdown(); + final FileStatus[] files = getFiles(); + if (null != files && 0 != files.length) { + for (FileStatus file : files) { + Path p = getWALArchivePath(this.fullPathArchiveDir, file.getPath()); // Tell our listeners that a log is going to be archived. if (!this.listeners.isEmpty()) { for (WALActionsListener i : this.listeners) { @@ -1033,54 +1072,53 @@ class FSHLog implements HLog, Syncable { } } LOG.debug("Moved " + files.length + " WAL file(s) to " + - FSUtils.getPath(this.fullPathOldLogDir)); - } - if (!fs.delete(fullPathLogDir, true)) { - LOG.info("Unable to delete " + fullPathLogDir); + FSUtils.getPath(this.fullPathArchiveDir)); } + LOG.info("Closed WAL: " + toString() ); } @Override - public void close() throws IOException { - if (this.closed) return; - try { - // Prevent all further flushing and rolling. - closeBarrier.stopAndDrainOps(); - } catch (InterruptedException e) { - LOG.error("Exception while waiting for cache flushes and log rolls", e); - Thread.currentThread().interrupt(); - } - - // Shutdown the disruptor. Will stop after all entries have been processed. Make sure we have - // stopped incoming appends before calling this else it will not shutdown. We are - // conservative below waiting a long time and if not elapsed, then halting. - if (this.disruptor != null) { - long timeoutms = conf.getLong("hbase.wal.disruptor.shutdown.timeout.ms", 60000); + public void shutdown() throws IOException { + if (shutdown.compareAndSet(false, true)) { try { - this.disruptor.shutdown(timeoutms, TimeUnit.MILLISECONDS); - } catch (TimeoutException e) { - LOG.warn("Timed out bringing down disruptor after " + timeoutms + "ms; forcing halt " + - "(It is a problem if this is NOT an ABORT! -- DATALOSS!!!!)"); - this.disruptor.halt(); - this.disruptor.shutdown(); + // Prevent all further flushing and rolling. + closeBarrier.stopAndDrainOps(); + } catch (InterruptedException e) { + LOG.error("Exception while waiting for cache flushes and log rolls", e); + Thread.currentThread().interrupt(); } - } - // With disruptor down, this is safe to let go. - if (this.appendExecutor != null) this.appendExecutor.shutdown(); - // Tell our listeners that the log is closing - if (!this.listeners.isEmpty()) { - for (WALActionsListener i : this.listeners) { - i.logCloseRequested(); + // Shutdown the disruptor. Will stop after all entries have been processed. Make sure we + // have stopped incoming appends before calling this else it will not shutdown. We are + // conservative below waiting a long time and if not elapsed, then halting. + if (this.disruptor != null) { + long timeoutms = conf.getLong("hbase.wal.disruptor.shutdown.timeout.ms", 60000); + try { + this.disruptor.shutdown(timeoutms, TimeUnit.MILLISECONDS); + } catch (TimeoutException e) { + LOG.warn("Timed out bringing down disruptor after " + timeoutms + "ms; forcing halt " + + "(It is a problem if this is NOT an ABORT! -- DATALOSS!!!!)"); + this.disruptor.halt(); + this.disruptor.shutdown(); + } + } + // With disruptor down, this is safe to let go. + if (this.appendExecutor != null) this.appendExecutor.shutdown(); + + // Tell our listeners that the log is closing + if (!this.listeners.isEmpty()) { + for (WALActionsListener i : this.listeners) { + i.logCloseRequested(); + } + } + this.closed = true; + if (LOG.isDebugEnabled()) { + LOG.debug("Closing WAL writer in " + FSUtils.getPath(fullPathLogDir)); + } + if (this.writer != null) { + this.writer.close(); + this.writer = null; } - } - this.closed = true; - if (LOG.isDebugEnabled()) { - LOG.debug("Closing WAL writer in " + this.fullPathLogDir.toString()); - } - if (this.writer != null) { - this.writer.close(); - this.writer = null; } } @@ -1092,60 +1130,18 @@ class FSHLog implements HLog, Syncable { * @param clusterIds that have consumed the change * @return New log key. */ - protected HLogKey makeKey(byte[] encodedRegionName, TableName tableName, long seqnum, + protected WALKey makeKey(byte[] encodedRegionName, TableName tableName, long seqnum, long now, List clusterIds, long nonceGroup, long nonce) { + // we use HLogKey here instead of WALKey directly to support legacy coprocessors. return new HLogKey(encodedRegionName, tableName, seqnum, now, clusterIds, nonceGroup, nonce); } - @Override - @VisibleForTesting - public void append(HRegionInfo info, TableName tableName, WALEdit edits, - final long now, HTableDescriptor htd, AtomicLong sequenceId) - throws IOException { - HLogKey logKey = new HLogKey(info.getEncodedNameAsBytes(), tableName, now); - append(htd, info, logKey, edits, sequenceId, true, true, null); - } - - @Override - public long appendNoSync(final HRegionInfo info, TableName tableName, WALEdit edits, - List clusterIds, final long now, HTableDescriptor htd, AtomicLong sequenceId, - boolean inMemstore, long nonceGroup, long nonce) throws IOException { - HLogKey logKey = - new HLogKey(info.getEncodedNameAsBytes(), tableName, now, clusterIds, nonceGroup, nonce); - return append(htd, info, logKey, edits, sequenceId, false, inMemstore, null); - } - - @Override - public long appendNoSync(final HTableDescriptor htd, final HRegionInfo info, final HLogKey key, - final WALEdit edits, final AtomicLong sequenceId, final boolean inMemstore, - final List memstoreCells) - throws IOException { - return append(htd, info, key, edits, sequenceId, false, inMemstore, memstoreCells); - } - - /** - * Append a set of edits to the log. Log edits are keyed by (encoded) regionName, rowname, and - * log-sequence-id. - * @param key - * @param edits - * @param htd This comes in here just so it is available on a pre append for replications. Get - * rid of it. It is kinda crazy this comes in here when we have tablename and regioninfo. - * Replication gets its scope from the HTD. - * @param hri region info - * @param sync shall we sync after we call the append? - * @param inMemstore - * @param sequenceId The region sequence id reference. - * @param memstoreCells - * @return txid of this transaction or if nothing to do, the last txid - * @throws IOException - */ @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH_EXCEPTION", justification="Will never be null") - private long append(HTableDescriptor htd, final HRegionInfo hri, final HLogKey key, - WALEdit edits, AtomicLong sequenceId, boolean sync, boolean inMemstore, - List memstoreCells) - throws IOException { - if (!this.enabled) return this.highestUnsyncedSequence; + @Override + public long append(final HTableDescriptor htd, final HRegionInfo hri, final WALKey key, + final WALEdit edits, final AtomicLong sequenceId, final boolean inMemstore, + final List memstoreCells) throws IOException { if (this.closed) throw new IOException("Cannot append; log is closed"); // Make a trace scope for the append. It is closed on other side of the ring buffer by the // single consuming thread. Don't have to worry about it. @@ -1166,9 +1162,6 @@ class FSHLog implements HLog, Syncable { } finally { this.disruptor.getRingBuffer().publish(sequence); } - // doSync is set in tests. Usually we arrive in here via appendNoSync w/ the sync called after - // all edits on a handler have been added. - if (sync) sync(sequence); return sequence; } @@ -1312,7 +1305,7 @@ class FSHLog implements HLog, Syncable { Trace.addTimelineAnnotation("writer synced"); currentSequence = updateHighestSyncedSequence(currentSequence); } catch (IOException e) { - LOG.error("Error syncing, request close of hlog ", e); + LOG.error("Error syncing, request close of wal ", e); t = e; } catch (Exception e) { LOG.warn("UNEXPECTED", e); @@ -1373,7 +1366,7 @@ class FSHLog implements HLog, Syncable { LOG.warn("HDFS pipeline error detected. " + "Found " + numCurrentReplicas + " replicas but expecting no less than " + this.minTolerableReplication + " replicas. " - + " Requesting close of hlog."); + + " Requesting close of wal."); logRollNeeded = true; // If rollWriter is requested, increase consecutiveLogRolls. Once it // is larger than lowReplicationRollLimit, disable the @@ -1457,10 +1450,7 @@ class FSHLog implements HLog, Syncable { return syncFuture.reset(sequence, span); } - @Override - public void postSync(final long timeInNanos, final int handlerSyncs) { - // TODO: Add metric for handler syncs done at a time. - if (this.metrics != null) metrics.finishSync(timeInNanos/1000000); + private void postSync(final long timeInNanos, final int handlerSyncs) { if (timeInNanos > this.slowSyncNs) { String msg = new StringBuilder().append("Slow sync cost: ") @@ -1469,19 +1459,57 @@ class FSHLog implements HLog, Syncable { Trace.addTimelineAnnotation(msg); LOG.info(msg); } + if (!listeners.isEmpty()) { + for (WALActionsListener listener : listeners) { + listener.postSync(timeInNanos, handlerSyncs); + } + } } - @Override - public long postAppend(final Entry e, final long elapsedTime) { + private long postAppend(final Entry e, final long elapsedTime) { long len = 0; - if (this.metrics == null) return len; - for (Cell cell : e.getEdit().getCells()) len += CellUtil.estimatedSerializedSizeOf(cell); - metrics.finishAppend(elapsedTime, len); + if (!listeners.isEmpty()) { + for (Cell cell : e.getEdit().getCells()) { + len += CellUtil.estimatedSerializedSizeOf(cell); + } + for (WALActionsListener listener : listeners) { + listener.postAppend(len, elapsedTime); + } + } return len; } /** - * This method gets the datanode replication count for the current HLog. + * Find the 'getNumCurrentReplicas' on the passed os stream. + * This is used for getting current replicas of a file being written. + * @return Method or null. + */ + private Method getGetNumCurrentReplicas(final FSDataOutputStream os) { + // TODO: Remove all this and use the now publically available + // HdfsDataOutputStream#getCurrentBlockReplication() + 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("No 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"); + } + return m; + } + + /** + * This method gets the datanode replication count for the current WAL. * * If the pipeline isn't started yet or is empty, you will get the default * replication factor. Therefore, if this function returns 0, it means you @@ -1492,10 +1520,12 @@ class FSHLog implements HLog, Syncable { * * @throws Exception */ + @VisibleForTesting int getLogReplication() throws IllegalArgumentException, IllegalAccessException, InvocationTargetException { - if (this.getNumCurrentReplicas != null && this.hdfs_out != null) { - Object repl = this.getNumCurrentReplicas.invoke(getOutputStream(), NO_ARGS); + final OutputStream stream = getOutputStream(); + if (this.getNumCurrentReplicas != null && stream != null) { + Object repl = this.getNumCurrentReplicas.invoke(stream, NO_ARGS); if (repl instanceof Integer) { return ((Integer)repl).intValue(); } @@ -1503,32 +1533,6 @@ class FSHLog implements HLog, Syncable { return 0; } - boolean canGetCurReplicas() { - return this.getNumCurrentReplicas != null; - } - - @Override - public void hsync() throws IOException { - TraceScope scope = Trace.startSpan("FSHLog.hsync"); - try { - scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach())); - } finally { - assert scope == NullScope.INSTANCE || !scope.isDetached(); - scope.close(); - } - } - - @Override - public void hflush() throws IOException { - TraceScope scope = Trace.startSpan("FSHLog.hflush"); - try { - scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach())); - } finally { - assert scope == NullScope.INSTANCE || !scope.isDetached(); - scope.close(); - } - } - @Override public void sync() throws IOException { TraceScope scope = Trace.startSpan("FSHLog.sync"); @@ -1555,7 +1559,8 @@ class FSHLog implements HLog, Syncable { } } - void requestLogRoll() { + // public only until class moves to o.a.h.h.wal + public void requestLogRoll() { if (!this.listeners.isEmpty()) { for (WALActionsListener i: this.listeners) { i.logRollRequested(); @@ -1563,25 +1568,21 @@ class FSHLog implements HLog, Syncable { } } - /** @return How many items have been added to the log */ - int getNumEntries() { - return numEntries.get(); - } - + // public only until class moves to o.a.h.h.wal /** @return the number of rolled log files */ public int getNumRolledLogFiles() { return byWalRegionSequenceIds.size(); } + // public only until class moves to o.a.h.h.wal /** @return the number of log files in use */ - @Override public int getNumLogFiles() { // +1 for current use log return getNumRolledLogFiles() + 1; } + // public only until class moves to o.a.h.h.wal /** @return the size of log files in use */ - @Override public long getLogFileSize() { return this.totalLogSize.get(); } @@ -1645,28 +1646,11 @@ class FSHLog implements HLog, Syncable { } } - @Override - public boolean isLowReplicationRollEnabled() { + @VisibleForTesting + boolean isLowReplicationRollEnabled() { return lowReplicationRollEnabled; } - /** - * Get the directory we are making logs in. - * - * @return dir - */ - protected Path getDir() { - return fullPathLogDir; - } - - 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)); @@ -1682,14 +1666,10 @@ class FSHLog implements HLog, Syncable { } final Path baseDir = FSUtils.getRootDir(conf); - final Path oldLogDir = new Path(baseDir, HConstants.HREGION_OLDLOGDIR_NAME); - HLogSplitter.split(baseDir, p, oldLogDir, fs, conf); + final Path archiveDir = new Path(baseDir, HConstants.HREGION_OLDLOGDIR_NAME); + WALSplitter.split(baseDir, p, archiveDir, fs, conf, WALFactory.getInstance(conf)); } - @Override - public WALCoprocessorHost getCoprocessorHost() { - return coprocessorHost; - } @Override public long getEarliestMemstoreSeqNum(byte[] encodedRegionName) { @@ -1941,7 +1921,7 @@ class FSHLog implements HLog, Syncable { long start = EnvironmentEdgeManager.currentTime(); byte [] encodedRegionName = entry.getKey().getEncodedRegionName(); - long regionSequenceId = HLog.NO_SEQUENCE_ID; + long regionSequenceId = WALKey.NO_SEQUENCE_ID; try { // We are about to append this edit; update the region-scoped sequence number. Do it // here inside this single appending/writing thread. Events are ordered on the ringbuffer @@ -1984,7 +1964,7 @@ class FSHLog implements HLog, Syncable { // Update metrics. postAppend(entry, EnvironmentEdgeManager.currentTime() - start); } catch (Exception e) { - LOG.fatal("Could not append. Requesting close of hlog", e); + LOG.fatal("Could not append. Requesting close of wal", e); requestLogRoll(); throw e; } @@ -2015,7 +1995,7 @@ class FSHLog implements HLog, Syncable { } private static void usage() { - System.err.println("Usage: HLog "); + System.err.println("Usage: FSHLog "); System.err.println("Arguments:"); System.err.println(" --dump Dump textual representation of passed one or more files"); System.err.println(" For example: " + @@ -2023,7 +2003,6 @@ class FSHLog implements HLog, Syncable { System.err.println(" --split Split the passed directory of WAL logs"); System.err.println(" For example: " + "FSHLog --split hdfs://example.com:9000/hbase/.logs/DIR"); - System.err.println(" --perf Write the same key times to a WAL: e.g. FSHLog --perf 10"); } /** @@ -2038,30 +2017,14 @@ class FSHLog implements HLog, Syncable { usage(); System.exit(-1); } - // either dump using the HLogPrettyPrinter or split, depending on args + // either dump using the WALPrettyPrinter or split, depending on args if (args[0].compareTo("--dump") == 0) { - HLogPrettyPrinter.run(Arrays.copyOfRange(args, 1, args.length)); + WALPrettyPrinter.run(Arrays.copyOfRange(args, 1, args.length)); } else if (args[0].compareTo("--perf") == 0) { - final int count = Integer.parseInt(args[1]); - // Put up a WAL and just keep adding same edit to it. Simple perf test. - Configuration conf = HBaseConfiguration.create(); - Path rootDir = FSUtils.getRootDir(conf); - FileSystem fs = rootDir.getFileSystem(conf); - FSHLog wal = - new FSHLog(fs, rootDir, "perflog", "oldPerflog", conf, null, false, "perf", false); - long start = System.nanoTime(); - WALEdit walEdit = new WALEdit(); - walEdit.add(new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("family"), - Bytes.toBytes("qualifier"), -1, new byte [1000])); - FSTableDescriptors fst = new FSTableDescriptors(conf); - for (AtomicLong i = new AtomicLong(0); i.get() < count; i.incrementAndGet()) { - wal.append(HRegionInfo.FIRST_META_REGIONINFO, - TableName.META_TABLE_NAME, walEdit, start, - fst.get(TableName.META_TABLE_NAME), i); - wal.sync(); - } - wal.close(); - LOG.info("Write " + count + " 1k edits in " + (System.nanoTime() - start) + "nanos"); + LOG.fatal("Please use the WALPerformanceEvaluation tool instead. i.e.:"); + LOG.fatal("\thbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation --iterations " + + args[1]); + System.exit(-1); } else if (args[0].compareTo("--split") == 0) { Configuration conf = HBaseConfiguration.create(); for (int i = 1; i < args.length; i++) { @@ -2109,9 +2072,9 @@ class FSHLog implements HLog, Syncable { } /** - * This method gets the pipeline for the current HLog. - * @return + * This method gets the pipeline for the current WAL. */ + @VisibleForTesting DatanodeInfo[] getPipeLine() { if (this.getPipeLine != null && this.hdfs_out != null) { Object repl; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java index 0325f78f687..d9942b3273b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java @@ -27,16 +27,19 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALKey; + /** * A WAL Entry for {@link FSHLog} implementation. Immutable. - * A subclass of {@link HLog.Entry} that carries extra info across the ring buffer such as + * A subclass of {@link Entry} that carries extra info across the ring buffer such as * region sequence id (we want to use this later, just before we write the WAL to ensure region * edits maintain order). The extra info added here is not 'serialized' as part of the WALEdit * hence marked 'transient' to underline this fact. It also adds mechanism so we can wait on * the assign of the region sequence id. See {@link #stampRegionSequenceId()}. */ @InterfaceAudience.Private -class FSWALEntry extends HLog.Entry { +class FSWALEntry extends Entry { // The below data members are denoted 'transient' just to highlight these are not persisted; // they are only in memory and held here while passing over the ring buffer. private final transient long sequence; @@ -46,7 +49,7 @@ class FSWALEntry extends HLog.Entry { private final transient HRegionInfo hri; private final transient List memstoreCells; - FSWALEntry(final long sequence, final HLogKey key, final WALEdit edit, + FSWALEntry(final long sequence, final WALKey key, final WALEdit edit, final AtomicLong referenceToRegionSequenceId, final boolean inMemstore, final HTableDescriptor htd, final HRegionInfo hri, List memstoreCells) { super(key, edit); @@ -98,7 +101,7 @@ class FSWALEntry extends HLog.Entry { CellUtil.setSequenceId(cell, regionSequenceId); } } - HLogKey key = getKey(); + WALKey key = getKey(); key.setLogSeqNum(regionSequenceId); return regionSequenceId; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java deleted file mode 100644 index eb3692e6722..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java +++ /dev/null @@ -1,445 +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.hbase.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.Cell; -import org.apache.hadoop.hbase.HBaseInterfaceAudience; -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; - -/** - * HLog records all the edits to HStore. It is the hbase write-ahead-log (WAL). - */ -@InterfaceAudience.Private -// TODO: Rename interface to WAL -public interface HLog { - Log LOG = LogFactory.getLog(HLog.class); - public static final long NO_SEQUENCE_ID = -1; - - /** 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.*/ - // TODO: Implementation detail. Does not belong in here. - 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: Implementation detail. Why in here? - Pattern EDITFILES_NAME_PATTERN = Pattern.compile("-?[0-9]+"); - String RECOVERED_LOG_TMPFILE_SUFFIX = ".temp"; - String SEQUENCE_ID_FILE_SUFFIX = "_seqid"; - - /** - * WAL Reader Interface - */ - 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? It won't be present on last WAL most of the time. - // What then? - WALTrailer getWALTrailer(); - } - - /** - * WAL Writer Intrface. - */ - 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/WAL's WALTrailer. This trailer is appended at the end of WAL on closing. - * @param walTrailer trailer to append to WAL. - */ - // TODO: Why a trailer on the log? - void setWALTrailer(WALTrailer walTrailer); - } - - /** - * Utility class that lets us keep track of the edit and it's associated key. Only used when - * splitting logs. - */ - // TODO: Remove this Writable. - // TODO: Why is this in here? Implementation detail? - @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION) - 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) { - 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. - * Used by tests only and in rare cases where we need a log just temporarily while bootstrapping - * a region or running migrations. - * - * @throws IOException - */ - void closeAndDelete() throws IOException; - - /** - * Same as {@link #appendNoSync(HRegionInfo, TableName, WALEdit, List, long, HTableDescriptor, - * AtomicLong, boolean, long, long)} - * except it causes a sync on the log - * @param info - * @param tableName - * @param edits - * @param now - * @param htd - * @param sequenceId - * @throws IOException - * @deprecated For tests only and even then, should use - * {@link #appendNoSync(HTableDescriptor, HRegionInfo, HLogKey, WALEdit, AtomicLong, boolean, - * List)} and {@link #sync()} instead. - */ - @Deprecated - @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. Used by metrics system at least. - * @param entry - * @param elapsedTime - * @return Size of this append. - */ - long postAppend(final Entry entry, final long elapsedTime); - - /** - * For notification post writer sync. Used by metrics system at least. - * @param timeInMillis How long the filesystem sync took in milliseconds. - * @param handlerSyncs How many sync handler calls were released by this call to filesystem - * sync. - */ - void postSync(final long timeInMillis, final int handlerSyncs); - - /** - * Append a set of edits to the WAL. WAL edits are keyed by (encoded) regionName, rowname, and - * log-sequence-id. The WAL is not flushed/sync'd after this transaction completes BUT on return - * this edit must have its region edit/sequence id assigned else it messes up our unification - * of mvcc and sequenceid. - * @param info - * @param tableName - * @param edits - * @param clusterIds - * @param now - * @param htd - * @param sequenceId A reference to the atomic long the info region is using as - * source of its incrementing edits sequence id. Inside in this call we will increment it and - * attach the sequence to the edit we apply the WAL. - * @param isInMemstore Always true except for case where we are writing a compaction completion - * record into the WAL; in this case the entry is just so we can finish an unfinished compaction - * -- it is not an edit for memstore. - * @param nonceGroup - * @param nonce - * @return Returns a 'transaction id'. Do not use. This is an internal implementation detail and - * cannot be respected in all implementations; i.e. the append/sync machine may or may not be - * able to sync an explicit edit only (the current default implementation syncs up to the time - * of the sync call syncing whatever is behind the sync). - * @throws IOException - * @deprecated Use {@link #appendNoSync(HTableDescriptor, HRegionInfo, HLogKey, WALEdit, AtomicLong, boolean, List)} - * instead because you can get back the region edit/sequenceid; it is set into the passed in - * key. - */ - @Deprecated - 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; - - /** - * Append a set of edits to the WAL. The WAL is not flushed/sync'd after this transaction - * completes BUT on return this edit must have its region edit/sequence id assigned - * else it messes up our unification of mvcc and sequenceid. On return key will - * have the region edit/sequence id filled in. - * @param info - * @param key Modified by this call; we add to it this edits region edit/sequence id. - * @param edits Edits to append. MAY CONTAIN NO EDITS for case where we want to get an edit - * sequence id that is after all currently appended edits. - * @param htd - * @param sequenceId A reference to the atomic long the info region is using as - * source of its incrementing edits sequence id. Inside in this call we will increment it and - * attach the sequence to the edit we apply the WAL. - * @param inMemstore Always true except for case where we are writing a compaction completion - * record into the WAL; in this case the entry is just so we can finish an unfinished compaction - * -- it is not an edit for memstore. - * @param memstoreCells list of Cells added into memstore - * @return Returns a 'transaction id' and key will have the region edit/sequence id - * in it. - * @throws IOException - */ - long appendNoSync(HTableDescriptor htd, HRegionInfo info, HLogKey key, WALEdit edits, - AtomicLong sequenceId, boolean inMemstore, List memstoreCells) throws IOException; - - // TODO: Do we need all these versions of sync? - void hsync() throws IOException; - - void hflush() throws IOException; - - /** - * Sync what we have in the WAL. - * @throws IOException - */ - void sync() throws IOException; - - /** - * Sync the WAL if the txId was not already sync'd. - * @param txid Transaction id to sync to. - * @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 a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java deleted file mode 100644 index a54091da926..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java +++ /dev/null @@ -1,207 +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.IOException; -import java.util.Arrays; -import java.io.InterruptedIOException; -import java.util.List; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.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.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, - 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, - final String oldLogName, final Configuration conf) throws IOException { - return new FSHLog(fs, root, logName, oldLogName, conf, null, true, null, false); -} - - public static HLog 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, HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners, - true, prefix, false); - } - - 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); - } - - /* - * WAL Reader - */ - private static Class logReaderClass; - - static void resetLogReaderClass() { - logReaderClass = null; - } - - public static HLog.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 - * 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, - Configuration conf, CancelableProgressable reporter) throws IOException { - return createReader(fs, path, conf, reporter, true); - } - - public static HLog.Reader createReader(final FileSystem fs, final Path path, - Configuration conf, CancelableProgressable reporter, boolean allowCustom) - throws IOException { - if (allowCustom && (logReaderClass == null)) { - logReaderClass = conf.getClass("hbase.regionserver.hlog.reader.impl", - ProtobufLogReader.class, Reader.class); - } - Class lrClass = allowCustom ? logReaderClass : ProtobufLogReader.class; - - try { - // A hlog file could be under recovery, so it may take several - // tries to get it open. Instead of claiming it is corrupted, retry - // to open it up to 5 minutes by default. - long startWaiting = EnvironmentEdgeManager.currentTime(); - long openTimeout = conf.getInt("hbase.hlog.open.timeout", 300000) + startWaiting; - int nbAttempt = 0; - while (true) { - try { - if (lrClass != ProtobufLogReader.class) { - // User is overriding the WAL reader, let them. - HLog.Reader reader = lrClass.newInstance(); - reader.init(fs, path, conf, null); - return reader; - } else { - FSDataInputStream stream = fs.open(path); - // Note that zero-length file will fail to read PB magic, and attempt to create - // a non-PB reader and fail the same way existing code expects it to. If we get - // rid of the old reader entirely, we need to handle 0-size files differently from - // merely non-PB files. - 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 = - isPbWal ? new ProtobufLogReader() : new SequenceFileLogReader(); - reader.init(fs, path, conf, stream); - return reader; - } - } catch (IOException e) { - String msg = e.getMessage(); - if (msg != null && (msg.contains("Cannot obtain block length") - || msg.contains("Could not obtain the last block") - || msg.matches("Blocklist for [^ ]* has changed.*"))) { - if (++nbAttempt == 1) { - LOG.warn("Lease should have recovered. This is not expected. Will retry", e); - } - if (reporter != null && !reporter.progress()) { - throw new InterruptedIOException("Operation is cancelled"); - } - if (nbAttempt > 2 && openTimeout < EnvironmentEdgeManager.currentTime()) { - LOG.error("Can't open after " + nbAttempt + " attempts and " - + (EnvironmentEdgeManager.currentTime() - startWaiting) - + "ms " + " for " + path); - } else { - try { - Thread.sleep(nbAttempt < 3 ? 500 : 1000); - continue; // retry - } catch (InterruptedException ie) { - InterruptedIOException iioe = new InterruptedIOException(); - iioe.initCause(ie); - throw iioe; - } - } - } - throw e; - } - } - } catch (IOException ie) { - throw ie; - } catch (Exception e) { - throw new IOException("Cannot get log reader", e); - } - } - - /* - * WAL writer - */ - private static Class logWriterClass; - - static void resetLogWriterClass() { - logWriterClass = null; - } - - /** - * Create a writer for the WAL. - * @return A WAL writer. Close when done with it. - * @throws IOException - */ - public static HLog.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, - final Path path, Configuration conf) throws IOException { - return createWriter(fs, path, conf, true); - } - - private static HLog.Writer createWriter(final FileSystem fs, - final Path path, Configuration conf, boolean overwritable) - throws IOException { - try { - if (logWriterClass == null) { - logWriterClass = conf.getClass("hbase.regionserver.hlog.writer.impl", - ProtobufLogWriter.class, Writer.class); - } - HLog.Writer writer = (HLog.Writer)logWriterClass.newInstance(); - writer.init(fs, path, conf, overwritable); - return writer; - } catch (Exception e) { - throw new IOException("cannot get log writer", e); - } - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java index bda197285c4..9fd2a376d10 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java @@ -22,39 +22,22 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.EOFException; import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; import java.util.Iterator; import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.TreeMap; import java.util.UUID; -import java.util.concurrent.CountDownLatch; - -import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; -import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FamilyScope; -import org.apache.hadoop.hbase.protobuf.generated.WALProtos.ScopeType; -import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey; -import org.apache.hadoop.hbase.regionserver.SequenceId; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableUtils; import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.ByteString; /** * A Key for an entry in the change log. @@ -65,97 +48,29 @@ import com.google.protobuf.ByteString; * *

    Some Transactional edits (START, COMMIT, ABORT) will not have an * associated row. + * @deprecated use WALKey */ -// TODO: Key and WALEdit are never used separately, or in one-to-many relation, for practical -// purposes. They need to be merged into HLogEntry. @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION) -public class HLogKey implements WritableComparable, SequenceId { +@Deprecated +public class HLogKey extends WALKey implements Writable { public static final Log LOG = LogFactory.getLog(HLogKey.class); - // should be < 0 (@see #readFields(DataInput)) - // version 2 supports HLog compression - enum Version { - UNVERSIONED(0), - // Initial number we put on HLogKey when we introduced versioning. - INITIAL(-1), - // Version -2 introduced a dictionary compression facility. Only this - // dictionary-based compression is available in version -2. - COMPRESSED(-2); - - final int code; - static final Version[] byCode; - static { - byCode = Version.values(); - for (int i = 0; i < byCode.length; i++) { - if (byCode[i].code != -1 * i) { - throw new AssertionError("Values in this enum should be descending by one"); - } - } - } - - Version(int code) { - this.code = code; - } - - boolean atLeast(Version other) { - return code <= other.code; - } - - static Version fromCode(int code) { - return byCode[code * -1]; - } - } - - /* - * This is used for reading the log entries created by the previous releases - * (0.94.11) which write the clusters information to the scopes of WALEdit. - */ - private static final String PREFIX_CLUSTER_KEY = "."; - - - private static final Version VERSION = Version.COMPRESSED; - - // The encoded region name. - private byte [] encodedRegionName; - private TableName tablename; - private long logSeqNum; - private long origLogSeqNum = 0; - private CountDownLatch seqNumAssignedLatch = new CountDownLatch(1); - // Time at which this edit was written. - private long writeTime; - - // The first element in the list is the cluster id on which the change has originated - private List clusterIds; - - private NavigableMap scopes; - - private long nonceGroup = HConstants.NO_NONCE; - private long nonce = HConstants.NO_NONCE; - static final List EMPTY_UUIDS = Collections.unmodifiableList(new ArrayList()); - - private CompressionContext compressionContext; - public HLogKey() { - init(null, null, 0L, HConstants.LATEST_TIMESTAMP, - new ArrayList(), HConstants.NO_NONCE, HConstants.NO_NONCE); + super(); } @VisibleForTesting public HLogKey(final byte[] encodedRegionName, final TableName tablename, long logSeqNum, final long now, UUID clusterId) { - List clusterIds = new ArrayList(); - clusterIds.add(clusterId); - init(encodedRegionName, tablename, logSeqNum, now, clusterIds, - HConstants.NO_NONCE, HConstants.NO_NONCE); + super(encodedRegionName, tablename, logSeqNum, now, clusterId); } public HLogKey(final byte[] encodedRegionName, final TableName tablename) { - this(encodedRegionName, tablename, System.currentTimeMillis()); + super(encodedRegionName, tablename); } public HLogKey(final byte[] encodedRegionName, final TableName tablename, final long now) { - init(encodedRegionName, tablename, HLog.NO_SEQUENCE_ID, now, - EMPTY_UUIDS, HConstants.NO_NONCE, HConstants.NO_NONCE); + super(encodedRegionName, tablename, now); } /** @@ -173,7 +88,7 @@ public class HLogKey implements WritableComparable, SequenceId { */ public HLogKey(final byte [] encodedRegionName, final TableName tablename, long logSeqNum, final long now, List clusterIds, long nonceGroup, long nonce) { - init(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce); + super(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce); } /** @@ -191,8 +106,7 @@ public class HLogKey implements WritableComparable, SequenceId { */ public HLogKey(final byte [] encodedRegionName, final TableName tablename, final long now, List clusterIds, long nonceGroup, long nonce) { - init(encodedRegionName, tablename, HLog.NO_SEQUENCE_ID, now, clusterIds, - nonceGroup, nonce); + super(encodedRegionName, tablename, now, clusterIds, nonceGroup, nonce); } /** @@ -209,240 +123,7 @@ public class HLogKey implements WritableComparable, SequenceId { */ public HLogKey(final byte [] encodedRegionName, final TableName tablename, long logSeqNum, long nonceGroup, long nonce) { - init(encodedRegionName, tablename, logSeqNum, EnvironmentEdgeManager.currentTime(), - EMPTY_UUIDS, nonceGroup, nonce); - } - - protected void init(final byte [] encodedRegionName, final TableName tablename, - long logSeqNum, final long now, List clusterIds, long nonceGroup, long nonce) { - this.logSeqNum = logSeqNum; - this.writeTime = now; - this.clusterIds = clusterIds; - this.encodedRegionName = encodedRegionName; - this.tablename = tablename; - this.nonceGroup = nonceGroup; - this.nonce = nonce; - } - - /** - * @param compressionContext Compression context to use - */ - public void setCompressionContext(CompressionContext compressionContext) { - this.compressionContext = compressionContext; - } - - /** @return encoded region name */ - public byte [] getEncodedRegionName() { - return encodedRegionName; - } - - /** @return table name */ - public TableName getTablename() { - return tablename; - } - - /** @return log sequence number */ - public long getLogSeqNum() { - return this.logSeqNum; - } - - /** - * Allow that the log sequence id to be set post-construction and release all waiters on assigned - * sequence number. - * @param sequence - */ - void setLogSeqNum(final long sequence) { - this.logSeqNum = sequence; - this.seqNumAssignedLatch.countDown(); - } - - /** - * Used to set original seq Id for HLogKey during wal replay - * @param seqId - */ - public void setOrigLogSeqNum(final long seqId) { - this.origLogSeqNum = seqId; - } - - /** - * Return a positive long if current HLogKey is created from a replay edit - * @return original sequence number of the WALEdit - */ - public long getOrigLogSeqNum() { - return this.origLogSeqNum; - } - - /** - * Wait for sequence number is assigned & return the assigned value - * @return long the new assigned sequence number - * @throws InterruptedException - */ - @Override - public long getSequenceId() throws IOException { - try { - this.seqNumAssignedLatch.await(); - } catch (InterruptedException ie) { - LOG.warn("Thread interrupted waiting for next log sequence number"); - InterruptedIOException iie = new InterruptedIOException(); - iie.initCause(ie); - throw iie; - } - return this.logSeqNum; - } - - /** - * @return the write time - */ - public long getWriteTime() { - return this.writeTime; - } - - public NavigableMap getScopes() { - return scopes; - } - - /** @return The nonce group */ - public long getNonceGroup() { - return nonceGroup; - } - - /** @return The nonce */ - public long getNonce() { - return nonce; - } - - public void setScopes(NavigableMap scopes) { - this.scopes = scopes; - } - - public void readOlderScopes(NavigableMap scopes) { - if (scopes != null) { - Iterator> iterator = scopes.entrySet() - .iterator(); - while (iterator.hasNext()) { - Map.Entry scope = iterator.next(); - String key = Bytes.toString(scope.getKey()); - if (key.startsWith(PREFIX_CLUSTER_KEY)) { - addClusterId(UUID.fromString(key.substring(PREFIX_CLUSTER_KEY - .length()))); - iterator.remove(); - } - } - if (scopes.size() > 0) { - this.scopes = scopes; - } - } - } - - /** - * Marks that the cluster with the given clusterId has consumed the change - */ - public void addClusterId(UUID clusterId) { - if (!clusterIds.contains(clusterId)) { - clusterIds.add(clusterId); - } - } - - /** - * @return the set of cluster Ids that have consumed the change - */ - public List getClusterIds() { - return clusterIds; - } - - /** - * @return the cluster id on which the change has originated. It there is no such cluster, it - * returns DEFAULT_CLUSTER_ID (cases where replication is not enabled) - */ - public UUID getOriginatingClusterId(){ - return clusterIds.isEmpty() ? HConstants.DEFAULT_CLUSTER_ID : clusterIds.get(0); - } - - @Override - public String toString() { - return tablename + "/" + Bytes.toString(encodedRegionName) + "/" + - logSeqNum; - } - - /** - * Produces a string map for this key. Useful for programmatic use and - * manipulation of the data stored in an HLogKey, for example, printing - * as JSON. - * - * @return a Map containing data from this key - */ - public Map toStringMap() { - Map stringMap = new HashMap(); - stringMap.put("table", tablename); - stringMap.put("region", Bytes.toStringBinary(encodedRegionName)); - stringMap.put("sequence", logSeqNum); - return stringMap; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - return compareTo((HLogKey)obj) == 0; - } - - @Override - public int hashCode() { - int result = Bytes.hashCode(this.encodedRegionName); - result ^= this.logSeqNum; - result ^= this.writeTime; - return result; - } - - @Override - public int compareTo(HLogKey o) { - int result = Bytes.compareTo(this.encodedRegionName, o.encodedRegionName); - if (result == 0) { - if (this.logSeqNum < o.logSeqNum) { - result = -1; - } else if (this.logSeqNum > o.logSeqNum) { - result = 1; - } - if (result == 0) { - if (this.writeTime < o.writeTime) { - result = -1; - } else if (this.writeTime > o.writeTime) { - return 1; - } - } - } - // why isn't cluster id accounted for? - return result; - } - - /** - * Drop this instance's tablename byte array and instead - * hold a reference to the provided tablename. This is not - * meant to be a general purpose setter - it's only used - * to collapse references to conserve memory. - */ - void internTableName(TableName tablename) { - // We should not use this as a setter - only to swap - // in a new reference to the same table name. - assert tablename.equals(this.tablename); - this.tablename = tablename; - } - - /** - * Drop this instance's region name byte array and instead - * hold a reference to the provided region name. This is not - * meant to be a general purpose setter - it's only used - * to collapse references to conserve memory. - */ - void internEncodedRegionName(byte []encodedRegionName) { - // We should not use this as a setter - only to swap - // in a new reference to the same table name. - assert Bytes.equals(this.encodedRegionName, encodedRegionName); - this.encodedRegionName = encodedRegionName; + super(encodedRegionName, tablename, logSeqNum, nonceGroup, nonce); } @Override @@ -486,13 +167,13 @@ public class HLogKey implements WritableComparable, SequenceId { // encodes the length of encodedRegionName. // If < 0 we just read the version and the next vint is the length. // @see Bytes#readByteArray(DataInput) - this.scopes = null; // writable HLogKey does not contain scopes + setScopes(null); // writable HLogKey does not contain scopes int len = WritableUtils.readVInt(in); byte[] tablenameBytes = null; if (len < 0) { // what we just read was the version version = Version.fromCode(len); - // We only compress V2 of HLogkey. + // We only compress V2 of WALkey. // If compression is on, the length is handled by the dictionary if (compressionContext == null || !version.atLeast(Version.COMPRESSED)) { len = WritableUtils.readVInt(in); @@ -541,87 +222,4 @@ public class HLogKey implements WritableComparable, SequenceId { // Do not need to read the clusters information as we are using protobufs from 0.95 } - public WALKey.Builder getBuilder(WALCellCodec.ByteStringCompressor compressor) - throws IOException { - WALKey.Builder builder = WALKey.newBuilder(); - if (compressionContext == null) { - builder.setEncodedRegionName(ByteStringer.wrap(this.encodedRegionName)); - builder.setTableName(ByteStringer.wrap(this.tablename.getName())); - } else { - builder.setEncodedRegionName(compressor.compress(this.encodedRegionName, - compressionContext.regionDict)); - builder.setTableName(compressor.compress(this.tablename.getName(), - compressionContext.tableDict)); - } - builder.setLogSequenceNumber(this.logSeqNum); - builder.setWriteTime(writeTime); - if(this.origLogSeqNum > 0) { - builder.setOrigSequenceNumber(this.origLogSeqNum); - } - if (this.nonce != HConstants.NO_NONCE) { - builder.setNonce(nonce); - } - if (this.nonceGroup != HConstants.NO_NONCE) { - builder.setNonceGroup(nonceGroup); - } - HBaseProtos.UUID.Builder uuidBuilder = HBaseProtos.UUID.newBuilder(); - for (UUID clusterId : clusterIds) { - uuidBuilder.setLeastSigBits(clusterId.getLeastSignificantBits()); - uuidBuilder.setMostSigBits(clusterId.getMostSignificantBits()); - builder.addClusterIds(uuidBuilder.build()); - } - if (scopes != null) { - for (Map.Entry e : scopes.entrySet()) { - ByteString family = (compressionContext == null) ? ByteStringer.wrap(e.getKey()) - : compressor.compress(e.getKey(), compressionContext.familyDict); - builder.addScopes(FamilyScope.newBuilder() - .setFamily(family).setScopeType(ScopeType.valueOf(e.getValue()))); - } - } - return builder; - } - - public void readFieldsFromPb( - WALKey walKey, WALCellCodec.ByteStringUncompressor uncompressor) throws IOException { - if (this.compressionContext != null) { - this.encodedRegionName = uncompressor.uncompress( - walKey.getEncodedRegionName(), compressionContext.regionDict); - byte[] tablenameBytes = uncompressor.uncompress( - walKey.getTableName(), compressionContext.tableDict); - this.tablename = TableName.valueOf(tablenameBytes); - } else { - this.encodedRegionName = walKey.getEncodedRegionName().toByteArray(); - this.tablename = TableName.valueOf(walKey.getTableName().toByteArray()); - } - clusterIds.clear(); - if (walKey.hasClusterId()) { - //When we are reading the older log (0.95.1 release) - //This is definitely the originating cluster - clusterIds.add(new UUID(walKey.getClusterId().getMostSigBits(), walKey.getClusterId() - .getLeastSigBits())); - } - for (HBaseProtos.UUID clusterId : walKey.getClusterIdsList()) { - clusterIds.add(new UUID(clusterId.getMostSigBits(), clusterId.getLeastSigBits())); - } - if (walKey.hasNonceGroup()) { - this.nonceGroup = walKey.getNonceGroup(); - } - if (walKey.hasNonce()) { - this.nonce = walKey.getNonce(); - } - this.scopes = null; - if (walKey.getScopesCount() > 0) { - this.scopes = new TreeMap(Bytes.BYTES_COMPARATOR); - for (FamilyScope scope : walKey.getScopesList()) { - byte[] family = (compressionContext == null) ? scope.getFamily().toByteArray() : - uncompressor.uncompress(scope.getFamily(), compressionContext.familyDict); - this.scopes.put(family, scope.getScopeType().getNumber()); - } - } - this.logSeqNum = walKey.getLogSequenceNumber(); - this.writeTime = walKey.getWriteTime(); - if(walKey.hasOrigSequenceNumber()) { - this.origLogSeqNum = walKey.getOrigSequenceNumber(); - } - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogPrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogPrettyPrinter.java index 48915cfd3ec..914ac4706d1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogPrettyPrinter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogPrettyPrinter.java @@ -23,7 +23,6 @@ import java.io.PrintStream; import java.util.ArrayList; import java.util.Date; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; @@ -38,13 +37,11 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.Tag; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.wal.WALPrettyPrinter; import org.codehaus.jackson.map.ObjectMapper; /** @@ -59,36 +56,18 @@ import org.codehaus.jackson.map.ObjectMapper; * * It can also toggle output of values. * + * @deprecated use the "hbase wal" command */ @InterfaceAudience.Public @InterfaceStability.Evolving -public class HLogPrettyPrinter { - private boolean outputValues; - private boolean outputJSON; - // The following enable filtering by sequence, region, and row, respectively - private long sequence; - private String region; - private String row; - // enable in order to output a single list of transactions from several files - private boolean persistentOutput; - private boolean firstTxn; - // useful for programatic capture of JSON output - private PrintStream out; - // for JSON encoding - private static final ObjectMapper MAPPER = new ObjectMapper(); +@Deprecated +public class HLogPrettyPrinter extends WALPrettyPrinter { /** * Basic constructor that simply initializes values to reasonable defaults. */ public HLogPrettyPrinter() { - outputValues = false; - outputJSON = false; - sequence = -1; - region = null; - row = null; - persistentOutput = false; - firstTxn = true; - out = System.out; + this(false, false, -1l, null, null, false, System.out); } /** @@ -119,289 +98,11 @@ public class HLogPrettyPrinter { public HLogPrettyPrinter(boolean outputValues, boolean outputJSON, long sequence, String region, String row, boolean persistentOutput, PrintStream out) { - this.outputValues = outputValues; - this.outputJSON = outputJSON; - this.sequence = sequence; - this.region = region; - this.row = row; - this.persistentOutput = persistentOutput; - if (persistentOutput) { - beginPersistentOutput(); - } - this.out = out; - this.firstTxn = true; - } - - /** - * turns value output on - */ - public void enableValues() { - outputValues = true; - } - - /** - * turns value output off - */ - public void disableValues() { - outputValues = false; - } - - /** - * turns JSON output on - */ - public void enableJSON() { - outputJSON = true; - } - - /** - * turns JSON output off, and turns on "pretty strings" for human consumption - */ - public void disableJSON() { - outputJSON = false; - } - - /** - * sets the region by which output will be filtered - * - * @param sequence - * when nonnegative, serves as a filter; only log entries with this - * sequence id will be printed - */ - public void setSequenceFilter(long sequence) { - this.sequence = sequence; - } - - /** - * sets the region by which output will be filtered - * - * @param region - * when not null, serves as a filter; only log entries from this - * region will be printed - */ - public void setRegionFilter(String region) { - this.region = region; - } - - /** - * sets the region by which output will be filtered - * - * @param row - * when not null, serves as a filter; only log entries from this row - * will be printed - */ - public void setRowFilter(String row) { - this.row = row; - } - - /** - * enables output as a single, persistent list. at present, only relevant in - * the case of JSON output. - */ - public void beginPersistentOutput() { - if (persistentOutput) - return; - persistentOutput = true; - firstTxn = true; - if (outputJSON) - out.print("["); - } - - /** - * ends output of a single, persistent list. at present, only relevant in the - * case of JSON output. - */ - public void endPersistentOutput() { - if (!persistentOutput) - return; - persistentOutput = false; - if (outputJSON) - out.print("]"); - } - - /** - * reads a log file and outputs its contents, one transaction at a time, as - * specified by the currently configured options - * - * @param conf - * the HBase configuration relevant to this log file - * @param p - * the path of the log file to be read - * @throws IOException - * may be unable to access the configured filesystem or requested - * file. - */ - public void processFile(final Configuration conf, final Path p) - throws IOException { - FileSystem fs = FileSystem.get(conf); - if (!fs.exists(p)) { - throw new FileNotFoundException(p.toString()); - } - if (!fs.isFile(p)) { - throw new IOException(p + " is not a file"); - } - if (outputJSON && !persistentOutput) { - out.print("["); - firstTxn = true; - } - Reader log = HLogFactory.createReader(fs, p, conf); - try { - FSHLog.Entry entry; - while ((entry = log.next()) != null) { - HLogKey key = entry.getKey(); - WALEdit edit = entry.getEdit(); - // begin building a transaction structure - Map txn = key.toStringMap(); - long writeTime = key.getWriteTime(); - // check output filters - if (sequence >= 0 && ((Long) txn.get("sequence")) != sequence) - continue; - if (region != null && !((String) txn.get("region")).equals(region)) - continue; - // initialize list into which we will store atomic actions - List actions = new ArrayList(); - for (Cell cell : edit.getCells()) { - // add atomic operation to txn - Map op = new HashMap(toStringMap(cell)); - if (outputValues) op.put("value", Bytes.toStringBinary(cell.getValue())); - // check row output filter - if (row == null || ((String) op.get("row")).equals(row)) - actions.add(op); - } - if (actions.size() == 0) - continue; - txn.put("actions", actions); - if (outputJSON) { - // JSON output is a straightforward "toString" on the txn object - if (firstTxn) - firstTxn = false; - else - out.print(","); - // encode and print JSON - out.print(MAPPER.writeValueAsString(txn)); - } else { - // Pretty output, complete with indentation by atomic action - out.println("Sequence " + txn.get("sequence") + " " - + "from region " + txn.get("region") + " " + "in table " - + txn.get("table") + " at write timestamp: " + new Date(writeTime)); - for (int i = 0; i < actions.size(); i++) { - Map op = actions.get(i); - out.println(" Action:"); - out.println(" row: " + op.get("row")); - out.println(" column: " + op.get("family") + ":" - + op.get("qualifier")); - out.println(" timestamp: " - + (new Date((Long) op.get("timestamp")))); - if(op.get("tag") != null) { - out.println(" tag: " + op.get("tag")); - } - if (outputValues) - out.println(" value: " + op.get("value")); - } - } - } - } finally { - log.close(); - } - if (outputJSON && !persistentOutput) { - out.print("]"); - } - } - - private static Map toStringMap(Cell cell) { - Map stringMap = new HashMap(); - stringMap.put("row", - Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())); - stringMap.put("family", Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(), - cell.getFamilyLength())); - stringMap.put("qualifier", - Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(), - cell.getQualifierLength())); - stringMap.put("timestamp", cell.getTimestamp()); - stringMap.put("vlen", cell.getValueLength()); - if (cell.getTagsLength() > 0) { - List tagsString = new ArrayList(); - Iterator tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), - cell.getTagsLength()); - while (tagsIterator.hasNext()) { - Tag tag = tagsIterator.next(); - tagsString.add((tag.getType()) + ":" - + Bytes.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength())); - } - stringMap.put("tag", tagsString); - } - return stringMap; + super(outputValues, outputJSON, sequence, region, row, persistentOutput, out); } public static void main(String[] args) throws IOException { - run(args); + WALPrettyPrinter.main(args); } - /** - * Pass one or more log file names and formatting options and it will dump out - * a text version of the contents on stdout. - * - * @param args - * Command line arguments - * @throws IOException - * Thrown upon file system errors etc. - * @throws ParseException - * Thrown if command-line parsing fails. - */ - public static void run(String[] args) throws IOException { - // create options - Options options = new Options(); - options.addOption("h", "help", false, "Output help message"); - options.addOption("j", "json", false, "Output JSON"); - options.addOption("p", "printvals", false, "Print values"); - options.addOption("r", "region", true, - "Region to filter by. Pass region name; e.g. 'hbase:meta,,1'"); - options.addOption("s", "sequence", true, - "Sequence to filter by. Pass sequence number."); - options.addOption("w", "row", true, "Row to filter by. Pass row name."); - - HLogPrettyPrinter printer = new HLogPrettyPrinter(); - CommandLineParser parser = new PosixParser(); - List files = null; - try { - CommandLine cmd = parser.parse(options, args); - files = cmd.getArgList(); - if (files.size() == 0 || cmd.hasOption("h")) { - HelpFormatter formatter = new HelpFormatter(); - formatter.printHelp("HLog ", options, true); - System.exit(-1); - } - // configure the pretty printer using command line options - if (cmd.hasOption("p")) - printer.enableValues(); - if (cmd.hasOption("j")) - printer.enableJSON(); - if (cmd.hasOption("r")) - printer.setRegionFilter(cmd.getOptionValue("r")); - if (cmd.hasOption("s")) - printer.setSequenceFilter(Long.parseLong(cmd.getOptionValue("s"))); - if (cmd.hasOption("w")) - printer.setRowFilter(cmd.getOptionValue("w")); - } catch (ParseException e) { - e.printStackTrace(); - HelpFormatter formatter = new HelpFormatter(); - formatter.printHelp("HFile filename(s) ", options, true); - System.exit(-1); - } - // get configuration, file system, and process the given files - Configuration conf = HBaseConfiguration.create(); - FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf)); - - // begin output - printer.beginPersistentOutput(); - for (Object f : files) { - Path file = new Path((String) f); - FileSystem fs = file.getFileSystem(conf); - if (!fs.exists(file)) { - System.err.println("ERROR, file doesnt exist: " + file); - return; - } - printer.processFile(conf, file); - } - printer.endPersistentOutput(); - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java deleted file mode 100644 index 29bffd1fa87..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java +++ /dev/null @@ -1,367 +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.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.NavigableSet; -import java.util.TreeSet; -import java.util.concurrent.atomic.AtomicLong; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -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.fs.PathFilter; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor; -import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor; -import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor; -import org.apache.hadoop.hbase.util.FSUtils; - -import com.google.protobuf.TextFormat; - -@InterfaceAudience.Private -public class HLogUtil { - static final Log LOG = LogFactory.getLog(HLogUtil.class); - - /** - * Pattern used to validate a HLog file name - */ - private static final Pattern pattern = - Pattern.compile(".*\\.\\d*("+HLog.META_HLOG_FILE_EXTN+")*"); - - /** - * @param filename - * name of the file to validate - * @return true if the filename matches an HLog, false - * otherwise - */ - public static boolean validateHLogFilename(String filename) { - return pattern.matcher(filename).matches(); - } - - /** - * Construct the HLog directory name - * - * @param serverName - * Server name formatted as described in {@link ServerName} - * @return the relative HLog directory name, e.g. - * .logs/1.example.org,60030,12345 if - * serverName passed is - * 1.example.org,60030,12345 - */ - public static String getHLogDirectoryName(final String serverName) { - StringBuilder dirName = new StringBuilder(HConstants.HREGION_LOGDIR_NAME); - dirName.append("/"); - dirName.append(serverName); - return dirName.toString(); - } - - /** - * @param regiondir - * This regions directory in the filesystem. - * @return The directory that holds recovered edits files for the region - * regiondir - */ - public static Path getRegionDirRecoveredEditsDir(final Path regiondir) { - return new Path(regiondir, HConstants.RECOVERED_EDITS_DIR); - } - - /** - * Move aside a bad edits file. - * - * @param fs - * @param edits - * Edits file to move aside. - * @return The name of the moved aside file. - * @throws IOException - */ - public static Path moveAsideBadEditsFile(final FileSystem fs, final Path edits) - throws IOException { - Path moveAsideName = new Path(edits.getParent(), edits.getName() + "." - + System.currentTimeMillis()); - if (!fs.rename(edits, moveAsideName)) { - LOG.warn("Rename failed from " + edits + " to " + moveAsideName); - } - return moveAsideName; - } - - /** - * @param path - * - the path to analyze. Expected format, if it's in hlog directory: - * / [base directory for hbase] / hbase / .logs / ServerName / - * logfile - * @return null if it's not a log file. Returns the ServerName of the region - * server that created this log file otherwise. - */ - public static ServerName getServerNameFromHLogDirectoryName( - Configuration conf, String path) throws IOException { - if (path == null - || path.length() <= HConstants.HREGION_LOGDIR_NAME.length()) { - return null; - } - - if (conf == null) { - throw new IllegalArgumentException("parameter conf must be set"); - } - - final String rootDir = conf.get(HConstants.HBASE_DIR); - if (rootDir == null || rootDir.isEmpty()) { - throw new IllegalArgumentException(HConstants.HBASE_DIR - + " key not found in conf."); - } - - final StringBuilder startPathSB = new StringBuilder(rootDir); - if (!rootDir.endsWith("/")) - startPathSB.append('/'); - startPathSB.append(HConstants.HREGION_LOGDIR_NAME); - if (!HConstants.HREGION_LOGDIR_NAME.endsWith("/")) - startPathSB.append('/'); - final String startPath = startPathSB.toString(); - - String fullPath; - try { - fullPath = FileSystem.get(conf).makeQualified(new Path(path)).toString(); - } catch (IllegalArgumentException e) { - LOG.info("Call to makeQualified failed on " + path + " " + e.getMessage()); - return null; - } - - if (!fullPath.startsWith(startPath)) { - return null; - } - - final String serverNameAndFile = fullPath.substring(startPath.length()); - - if (serverNameAndFile.indexOf('/') < "a,0,0".length()) { - // Either it's a file (not a directory) or it's not a ServerName format - return null; - } - - Path p = new Path(path); - return getServerNameFromHLogDirectoryName(p); - } - - /** - * This function returns region server name from a log file name which is in either format: - * hdfs:///hbase/.logs/-splitting/... or hdfs:///hbase/.logs//... - * @param logFile - * @return null if the passed in logFile isn't a valid HLog file path - */ - public static ServerName getServerNameFromHLogDirectoryName(Path logFile) { - Path logDir = logFile.getParent(); - String logDirName = logDir.getName(); - if (logDirName.equals(HConstants.HREGION_LOGDIR_NAME)) { - logDir = logFile; - logDirName = logDir.getName(); - } - ServerName serverName = null; - if (logDirName.endsWith(HLog.SPLITTING_EXT)) { - logDirName = logDirName.substring(0, logDirName.length() - HLog.SPLITTING_EXT.length()); - } - try { - serverName = ServerName.parseServerName(logDirName); - } catch (IllegalArgumentException ex) { - serverName = null; - LOG.warn("Cannot parse a server name from path=" + logFile + "; " + ex.getMessage()); - } - if (serverName != null && serverName.getStartcode() < 0) { - LOG.warn("Invalid log file path=" + logFile); - return null; - } - return serverName; - } - - /** - * Returns sorted set of edit files made by wal-log splitter, excluding files - * with '.temp' suffix. - * - * @param fs - * @param regiondir - * @return Files in passed regiondir as a sorted set. - * @throws IOException - */ - public static NavigableSet getSplitEditFilesSorted(final FileSystem fs, - final Path regiondir) throws IOException { - NavigableSet filesSorted = new TreeSet(); - Path editsdir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir); - if (!fs.exists(editsdir)) - return filesSorted; - FileStatus[] files = FSUtils.listStatus(fs, editsdir, new PathFilter() { - @Override - public boolean accept(Path p) { - boolean result = false; - try { - // Return files and only files that match the editfile names pattern. - // 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()); - 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)) { - result = false; - } - } catch (IOException e) { - LOG.warn("Failed isFile check on " + p); - } - return result; - } - }); - if (files == null) - return filesSorted; - for (FileStatus status : files) { - filesSorted.add(status.getPath()); - } - return filesSorted; - } - - public static boolean isMetaFile(Path p) { - return isMetaFile(p.getName()); - } - - public static boolean isMetaFile(String p) { - if (p != null && p.endsWith(HLog.META_HLOG_FILE_EXTN)) { - return true; - } - return false; - } - - /** - * Write the marker that a compaction has succeeded and is about to be committed. - * This provides info to the HMaster to allow it to recover the compaction if - * this regionserver dies in the middle (This part is not yet implemented). It also prevents - * 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, - final CompactionDescriptor c, AtomicLong sequenceId) throws IOException { - TableName tn = TableName.valueOf(c.getTableName().toByteArray()); - HLogKey key = new HLogKey(info.getEncodedNameAsBytes(), tn); - log.appendNoSync(htd, info, key, WALEdit.createCompaction(info, c), sequenceId, false, null); - log.sync(); - if (LOG.isTraceEnabled()) { - LOG.trace("Appended compaction marker " + TextFormat.shortDebugString(c)); - } - } - - /** - * Write a flush marker indicating a start / abort or a complete of a region flush - */ - public static long writeFlushMarker(HLog log, HTableDescriptor htd, HRegionInfo info, - final FlushDescriptor f, AtomicLong sequenceId, boolean sync) throws IOException { - TableName tn = TableName.valueOf(f.getTableName().toByteArray()); - HLogKey key = new HLogKey(info.getEncodedNameAsBytes(), tn); - long trx = log.appendNoSync(htd, info, key, WALEdit.createFlushWALEdit(info, f), sequenceId, false, null); - if (sync) log.sync(trx); - if (LOG.isTraceEnabled()) { - LOG.trace("Appended flush marker " + TextFormat.shortDebugString(f)); - } - return trx; - } - - /** - * Write a region open marker indicating that the region is opened - */ - public static long writeRegionEventMarker(HLog log, HTableDescriptor htd, HRegionInfo info, - final RegionEventDescriptor r, AtomicLong sequenceId) throws IOException { - TableName tn = TableName.valueOf(r.getTableName().toByteArray()); - HLogKey key = new HLogKey(info.getEncodedNameAsBytes(), tn); - long trx = log.appendNoSync(htd, info, key, WALEdit.createRegionEventWALEdit(info, r), - sequenceId, false, null); - log.sync(trx); - if (LOG.isTraceEnabled()) { - LOG.trace("Appended region event marker " + TextFormat.shortDebugString(r)); - } - return trx; - } - - /** - * Create a file with name as region open sequence id - * - * @param fs - * @param regiondir - * @param newSeqId - * @param saftyBumper - * @return long new sequence Id value - * @throws IOException - */ - public static long writeRegionOpenSequenceIdFile(final FileSystem fs, final Path regiondir, - long newSeqId, long saftyBumper) throws IOException { - - Path editsdir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir); - long maxSeqId = 0; - FileStatus[] files = null; - if (fs.exists(editsdir)) { - files = FSUtils.listStatus(fs, editsdir, new PathFilter() { - @Override - public boolean accept(Path p) { - if (p.getName().endsWith(HLog.SEQUENCE_ID_FILE_SUFFIX)) { - return true; - } - return false; - } - }); - if (files != null) { - for (FileStatus status : files) { - String fileName = status.getPath().getName(); - try { - Long tmpSeqId = Long.parseLong(fileName.substring(0, fileName.length() - - HLog.SEQUENCE_ID_FILE_SUFFIX.length())); - maxSeqId = Math.max(tmpSeqId, maxSeqId); - } catch (NumberFormatException ex) { - LOG.warn("Invalid SeqId File Name=" + fileName); - } - } - } - } - if (maxSeqId > newSeqId) { - newSeqId = maxSeqId; - } - newSeqId += saftyBumper; // bump up SeqId - - // write a new seqId file - Path newSeqIdFile = new Path(editsdir, newSeqId + HLog.SEQUENCE_ID_FILE_SUFFIX); - if (!fs.createNewFile(newSeqIdFile)) { - throw new IOException("Failed to create SeqId file:" + newSeqIdFile); - } - // remove old ones - if(files != null) { - for (FileStatus status : files) { - fs.delete(status.getPath(), false); - } - } - return newSeqId; - } - -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWAL.java index cf4b7a6fe6f..ad549f03df7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWAL.java @@ -31,7 +31,7 @@ import org.apache.hadoop.util.StringUtils; * single function call and turn it into multiple manipulations of the hadoop metrics system. */ @InterfaceAudience.Private -public class MetricsWAL { +public class MetricsWAL extends WALActionsListener.Base { static final Log LOG = LogFactory.getLog(MetricsWAL.class); private final MetricsWALSource source; @@ -40,19 +40,20 @@ public class MetricsWAL { source = CompatibilitySingletonFactory.getInstance(MetricsWALSource.class); } - public void finishSync(long time) { - source.incrementSyncTime(time); + @Override + public void postSync(final long timeInNanos, final int handlerSyncs) { + source.incrementSyncTime(timeInNanos/1000000l); } - public void finishAppend(long time, long size) { - + @Override + public void postAppend(final long size, final long time) { source.incrementAppendCount(); source.incrementAppendTime(time); source.incrementAppendSize(size); if (time > 1000) { source.incrementSlowAppendCount(); - LOG.warn(String.format("%s took %d ms appending an edit to hlog; len~=%s", + LOG.warn(String.format("%s took %d ms appending an edit to wal; len~=%s", Thread.currentThread().getName(), time, StringUtils.humanReadableInt(size))); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java index 39f1d9f9739..285f69b6a40 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java @@ -31,6 +31,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.hbase.codec.Codec; import org.apache.hadoop.hbase.HBaseInterfaceAudience; @@ -40,6 +42,7 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader.Builder; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL.Entry; import com.google.protobuf.CodedInputStream; import com.google.protobuf.InvalidProtocolBufferException; @@ -58,17 +61,32 @@ import com.google.protobuf.InvalidProtocolBufferException; @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX, HBaseInterfaceAudience.CONFIG}) public class ProtobufLogReader extends ReaderBase { private static final Log LOG = LogFactory.getLog(ProtobufLogReader.class); - static final byte[] PB_WAL_MAGIC = Bytes.toBytes("PWAL"); - static final byte[] PB_WAL_COMPLETE_MAGIC = Bytes.toBytes("LAWP"); + // public for WALFactory until we move everything to o.a.h.h.wal + @InterfaceAudience.Private + public static final byte[] PB_WAL_MAGIC = Bytes.toBytes("PWAL"); + // public for TestWALSplit + @InterfaceAudience.Private + public static final byte[] PB_WAL_COMPLETE_MAGIC = Bytes.toBytes("LAWP"); + /** + * Configuration name of WAL Trailer's warning size. If a waltrailer's size is greater than the + * configured size, providers should log a warning. e.g. this is used with Protobuf reader/writer. + */ + static final String WAL_TRAILER_WARN_SIZE = "hbase.regionserver.waltrailer.warn.size"; + static final int DEFAULT_WAL_TRAILER_WARN_SIZE = 1024 * 1024; // 1MB + protected FSDataInputStream inputStream; protected Codec.Decoder cellDecoder; protected WALCellCodec.ByteStringUncompressor byteStringUncompressor; protected boolean hasCompression = false; protected boolean hasTagCompression = false; // walEditsStopOffset is the position of the last byte to read. After reading the last WALEdit entry - // in the hlog, the inputstream's position is equal to walEditsStopOffset. + // in the wal, the inputstream's position is equal to walEditsStopOffset. private long walEditsStopOffset; private boolean trailerPresent; + protected WALTrailer trailer; + // maximum size of the wal Trailer in bytes. If a user writes/reads a trailer with size larger + // than this size, it is written/read respectively, with a WARN message in the log. + protected int trailerWarnSize; private static List writerClsNames = new ArrayList(); static { writerClsNames.add(ProtobufLogWriter.class.getSimpleName()); @@ -120,6 +138,13 @@ public class ProtobufLogReader extends ReaderBase { initAfterCompression(clsName); // We need a new decoder (at least). } + @Override + public void init(FileSystem fs, Path path, Configuration conf, FSDataInputStream stream) + throws IOException { + this.trailerWarnSize = conf.getInt(WAL_TRAILER_WARN_SIZE, DEFAULT_WAL_TRAILER_WARN_SIZE); + super.init(fs, path, conf, stream); + } + @Override protected String initReader(FSDataInputStream stream) throws IOException { return initInternal(stream, true); @@ -268,7 +293,7 @@ public class ProtobufLogReader extends ReaderBase { } @Override - protected boolean readNext(HLog.Entry entry) throws IOException { + protected boolean readNext(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(); @@ -332,7 +357,7 @@ public class ProtobufLogReader extends ReaderBase { initCause(realEofEx != null ? realEofEx : ex); } if (trailerPresent && this.inputStream.getPos() > this.walEditsStopOffset) { - LOG.error("Read WALTrailer while reading WALEdits. hlog: " + this.path + LOG.error("Read WALTrailer while reading WALEdits. wal: " + this.path + ", inputStream.getPos(): " + this.inputStream.getPos() + ", walEditsStopOffset: " + this.walEditsStopOffset); throw new EOFException("Read WALTrailer while reading WALEdits"); @@ -369,11 +394,6 @@ public class ProtobufLogReader extends ReaderBase { return null; } - @Override - public WALTrailer getWALTrailer() { - return trailer; - } - @Override protected void seekOnFs(long pos) throws IOException { this.inputStream.seek(pos); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java index fe2eac92fa0..ca80e4c32b8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java @@ -34,6 +34,10 @@ import org.apache.hadoop.hbase.codec.Codec; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.wal.WAL.Entry; + +import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.WAL_TRAILER_WARN_SIZE; +import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.DEFAULT_WAL_TRAILER_WARN_SIZE; /** * Writer for protobuf-based WAL. @@ -77,8 +81,7 @@ 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_TRAILER_WARN_SIZE, DEFAULT_WAL_TRAILER_WARN_SIZE); int bufferSize = FSUtils.getDefaultBufferSize(fs); short replication = (short)conf.getInt( "hbase.regionserver.hlog.replication", FSUtils.getDefaultReplication(fs, path)); @@ -110,7 +113,7 @@ public class ProtobufLogWriter extends WriterBase { } @Override - public void append(HLog.Entry entry) throws IOException { + public void append(Entry entry) throws IOException { entry.setCompressionContext(compressionContext); entry.getKey().getBuilder(compressor). setFollowingKvCount(entry.getEdit().size()).build().writeDelimitedTo(output); @@ -134,7 +137,7 @@ public class ProtobufLogWriter extends WriterBase { } } - protected WALTrailer buildWALTrailer(WALTrailer.Builder builder) { + WALTrailer buildWALTrailer(WALTrailer.Builder builder) { return builder.build(); } @@ -188,8 +191,7 @@ public class ProtobufLogWriter extends WriterBase { return this.output; } - @Override - public void setWALTrailer(WALTrailer walTrailer) { + void setWALTrailer(WALTrailer walTrailer) { this.trailer = walTrailer; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java index 7fe5a819dbd..5f1e90471c1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java @@ -31,21 +31,19 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.io.util.LRUDictionary; -import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.wal.WAL.Entry; + @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX}) -public abstract class ReaderBase implements HLog.Reader { +public abstract class ReaderBase implements DefaultWALProvider.Reader { private static final Log LOG = LogFactory.getLog(ReaderBase.class); protected Configuration conf; protected FileSystem fs; protected Path path; protected long edit = 0; protected long fileLength; - protected WALTrailer trailer; - // maximum size of the wal Trailer in bytes. If a user writes/reads a trailer with size larger - // than this size, it is written/read respectively, with a WARN message in the log. - protected int trailerWarnSize; /** * Compression context to use reading. Can be null if no compression. */ @@ -65,8 +63,6 @@ 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); String cellCodecClsName = initReader(stream); boolean compression = hasCompression(); @@ -87,15 +83,17 @@ public abstract class ReaderBase implements HLog.Reader { } @Override - public HLog.Entry next() throws IOException { + public Entry next() throws IOException { return next(null); } @Override - public HLog.Entry next(HLog.Entry reuse) throws IOException { - HLog.Entry e = reuse; + public Entry next(Entry reuse) throws IOException { + Entry e = reuse; if (e == null) { - e = new HLog.Entry(new HLogKey(), new WALEdit()); + // we use HLogKey here instead of WALKey directly to support legacy coprocessors, + // seqencefile based readers, and HLogInputFormat. + e = new Entry(new HLogKey(), new WALEdit()); } if (compressionContext != null) { e.setCompressionContext(compressionContext); @@ -165,15 +163,10 @@ 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(Entry e) throws IOException; /** * Performs a filesystem-level seek to a certain position in an underlying file. */ protected abstract void seekOnFs(long pos) throws IOException; - - @Override - public WALTrailer getWALTrailer() { - return null; - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureProtobufLogWriter.java index 985c0bb1aa0..03d1608717c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureProtobufLogWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureProtobufLogWriter.java @@ -52,10 +52,10 @@ public class SecureProtobufLogWriter extends ProtobufLogWriter { builder.setWriterClsName(SecureProtobufLogWriter.class.getSimpleName()); if (conf.getBoolean(HConstants.ENABLE_WAL_ENCRYPTION, false)) { // Get an instance of our cipher - Cipher cipher = Encryption.getCipher(conf, - conf.get(HConstants.CRYPTO_WAL_ALGORITHM_CONF_KEY, DEFAULT_CIPHER)); + final String cipherName = conf.get(HConstants.CRYPTO_WAL_ALGORITHM_CONF_KEY, DEFAULT_CIPHER); + Cipher cipher = Encryption.getCipher(conf, cipherName); if (cipher == null) { - throw new RuntimeException("Cipher '" + cipher + "' is not available"); + throw new RuntimeException("Cipher '" + cipherName + "' is not available"); } // Generate an encryption key for this WAL diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java index 128274a5f63..11312b11c92 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java @@ -33,7 +33,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry; +import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile.Metadata; import org.apache.hadoop.io.Text; @@ -222,10 +222,27 @@ public class SequenceFileLogReader extends ReaderBase { } + /** + * fill in the passed entry with teh next key/value. + * Note that because this format deals with our legacy storage, the provided + * Entery MUST use an {@link HLogKey} for the key. + * @return boolean indicating if the contents of Entry have been filled in. + */ @Override protected boolean readNext(Entry e) throws IOException { try { - boolean hasNext = this.reader.next(e.getKey(), e.getEdit()); + if (!(e.getKey() instanceof HLogKey)) { + final IllegalArgumentException exception = new IllegalArgumentException( + "SequenceFileLogReader only works when given entries that have HLogKey for keys. This" + + " one had '" + e.getKey().getClass() + "'"); + LOG.error("We need to use the legacy SequenceFileLogReader to handle a " + + " pre-0.96 style WAL, but HBase internals failed to use the deprecated HLogKey class." + + " This is a bug; please file an issue or email the developer mailing list. You will " + + "need the following exception details when seeking help from the HBase community.", + exception); + throw exception; + } + boolean hasNext = this.reader.next((HLogKey)e.getKey(), e.getEdit()); if (!hasNext) return false; // Scopes are probably in WAL edit, move to key NavigableMap scopes = e.getEdit().getAndRemoveScopes(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java index eddb92dc000..2194ce977a4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java @@ -25,8 +25,10 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.wal.WALKey; + /** - * Get notification of {@link FSHLog}/WAL log events. The invocations are inline + * Get notification of WAL events. The invocations are inline * so make sure your implementation is fast else you'll slow hbase. */ @InterfaceAudience.Private @@ -35,30 +37,30 @@ public interface WALActionsListener { /** * The WAL is going to be rolled. The oldPath can be null if this is * the first log file from the regionserver. - * @param oldPath the path to the old hlog - * @param newPath the path to the new hlog + * @param oldPath the path to the old wal + * @param newPath the path to the new wal */ void preLogRoll(Path oldPath, Path newPath) throws IOException; /** * The WAL has been rolled. The oldPath can be null if this is * the first log file from the regionserver. - * @param oldPath the path to the old hlog - * @param newPath the path to the new hlog + * @param oldPath the path to the old wal + * @param newPath the path to the new wal */ void postLogRoll(Path oldPath, Path newPath) throws IOException; /** * The WAL is going to be archived. - * @param oldPath the path to the old hlog - * @param newPath the path to the new hlog + * @param oldPath the path to the old wal + * @param newPath the path to the new wal */ void preLogArchive(Path oldPath, Path newPath) throws IOException; /** * The WAL has been archived. - * @param oldPath the path to the old hlog - * @param newPath the path to the new hlog + * @param oldPath the path to the old wal + * @param newPath the path to the new wal */ void postLogArchive(Path oldPath, Path newPath) throws IOException; @@ -79,7 +81,7 @@ public interface WALActionsListener { * @param logEdit */ void visitLogEntryBeforeWrite( - HRegionInfo info, HLogKey logKey, WALEdit logEdit + HRegionInfo info, WALKey logKey, WALEdit logEdit ); /** @@ -87,11 +89,59 @@ public interface WALActionsListener { * @param htd * @param logKey * @param logEdit - * TODO: Retire this in favor of {@link #visitLogEntryBeforeWrite(HRegionInfo, HLogKey, WALEdit)} - * It only exists to get scope when replicating. Scope should be in the HLogKey and not need + * TODO: Retire this in favor of {@link #visitLogEntryBeforeWrite(HRegionInfo, WALKey, WALEdit)} + * It only exists to get scope when replicating. Scope should be in the WALKey and not need * us passing in a htd. */ void visitLogEntryBeforeWrite( - HTableDescriptor htd, HLogKey logKey, WALEdit logEdit + HTableDescriptor htd, WALKey logKey, WALEdit logEdit ); + + /** + * For notification post append to the writer. Used by metrics system at least. + * TODO: Combine this with above. + * @param entryLen approx length of cells in this append. + * @param elapsedTimeMillis elapsed time in milliseconds. + */ + void postAppend(final long entryLen, final long elapsedTimeMillis); + + /** + * For notification post writer sync. Used by metrics system at least. + * @param timeInNanos How long the filesystem sync took in nanoseconds. + * @param handlerSyncs How many sync handler calls were released by this call to filesystem + * sync. + */ + void postSync(final long timeInNanos, final int handlerSyncs); + + static class Base implements WALActionsListener { + @Override + public void preLogRoll(Path oldPath, Path newPath) throws IOException {} + + @Override + public void postLogRoll(Path oldPath, Path newPath) throws IOException {} + + @Override + public void preLogArchive(Path oldPath, Path newPath) throws IOException {} + + @Override + public void postLogArchive(Path oldPath, Path newPath) throws IOException {} + + @Override + public void logRollRequested() {} + + @Override + public void logCloseRequested() {} + + @Override + public void visitLogEntryBeforeWrite(HRegionInfo info, WALKey logKey, WALEdit logEdit) {} + + @Override + public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit) {} + + @Override + public void postAppend(final long entryLen, final long elapsedTimeMillis) {} + + @Override + public void postSync(final long timeInNanos, final int handlerSyncs) {} + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java index 89f4b869fc1..56137e8d975 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java @@ -131,7 +131,7 @@ public class WALCellCodec implements Codec { byte[] uncompress(ByteString data, Dictionary dict) throws IOException; } - // TODO: it sucks that compression context is in HLog.Entry. It'd be nice if it was here. + // TODO: it sucks that compression context is in WAL.Entry. It'd be nice if it was here. // Dictionary could be gotten by enum; initially, based on enum, context would create // an array of dictionaries. static class BaosAndCompressor extends ByteArrayOutputStream implements ByteStringCompressor { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java index 521e5f388ea..52dcee0e031 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java @@ -28,9 +28,12 @@ import org.apache.hadoop.hbase.coprocessor.*; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALKey; + /** * Implements the coprocessor environment and runtime support for coprocessors - * loaded within a {@link FSHLog}. + * loaded within a {@link WAL}. */ @InterfaceAudience.Private public class WALCoprocessorHost @@ -42,10 +45,13 @@ public class WALCoprocessorHost static class WALEnvironment extends CoprocessorHost.Environment implements WALCoprocessorEnvironment { - private FSHLog wal; + private final WAL wal; + + final boolean useLegacyPre; + final boolean useLegacyPost; @Override - public FSHLog getWAL() { + public WAL getWAL() { return wal; } @@ -56,23 +62,32 @@ public class WALCoprocessorHost * @param priority chaining priority * @param seq load sequence * @param conf configuration - * @param hlog HLog + * @param wal WAL */ public WALEnvironment(Class implClass, final Coprocessor impl, final int priority, final int seq, final Configuration conf, - final FSHLog hlog) { + final WAL wal) { super(impl, priority, seq, conf); - this.wal = hlog; + this.wal = wal; + // Pick which version of the API we'll call. + // This way we avoid calling the new version on older WALObservers so + // we can maintain binary compatibility. + // See notes in javadoc for WALObserver + useLegacyPre = useLegacyMethod(impl.getClass(), "preWALWrite", ObserverContext.class, + HRegionInfo.class, WALKey.class, WALEdit.class); + useLegacyPost = useLegacyMethod(impl.getClass(), "postWALWrite", ObserverContext.class, + HRegionInfo.class, WALKey.class, WALEdit.class); } } - FSHLog wal; + private final WAL wal; + /** * Constructor * @param log the write ahead log * @param conf the configuration */ - public WALCoprocessorHost(final FSHLog log, final Configuration conf) { + public WALCoprocessorHost(final WAL 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 @@ -100,21 +115,29 @@ public class WALCoprocessorHost * @return true if default behavior should be bypassed, false otherwise * @throws IOException */ - public boolean preWALWrite(final HRegionInfo info, final HLogKey logKey, final WALEdit logEdit) + public boolean preWALWrite(final HRegionInfo info, final WALKey logKey, final WALEdit logEdit) throws IOException { boolean bypass = false; if (this.coprocessors == null || this.coprocessors.isEmpty()) return bypass; ObserverContext ctx = null; for (WALEnvironment env: coprocessors) { - if (env.getInstance() instanceof - org.apache.hadoop.hbase.coprocessor.WALObserver) { + if (env.getInstance() instanceof WALObserver) { + final WALObserver observer = (WALObserver)env.getInstance(); ctx = ObserverContext.createAndPrepare(env, ctx); Thread currentThread = Thread.currentThread(); ClassLoader cl = currentThread.getContextClassLoader(); try { currentThread.setContextClassLoader(env.getClassLoader()); - ((org.apache.hadoop.hbase.coprocessor.WALObserver)env.getInstance()). - preWALWrite(ctx, info, logKey, logEdit); + if (env.useLegacyPre) { + if (logKey instanceof HLogKey) { + observer.preWALWrite(ctx, info, (HLogKey)logKey, logEdit); + } else { + legacyWarning(observer.getClass(), + "There are wal keys present that are not HLogKey."); + } + } else { + observer.preWALWrite(ctx, info, logKey, logEdit); + } } catch (Throwable e) { handleCoprocessorThrowable(env, e); } finally { @@ -135,20 +158,28 @@ public class WALCoprocessorHost * @param logEdit * @throws IOException */ - public void postWALWrite(final HRegionInfo info, final HLogKey logKey, final WALEdit logEdit) + public void postWALWrite(final HRegionInfo info, final WALKey logKey, final WALEdit logEdit) throws IOException { if (this.coprocessors == null || this.coprocessors.isEmpty()) return; ObserverContext ctx = null; for (WALEnvironment env: coprocessors) { - if (env.getInstance() instanceof - org.apache.hadoop.hbase.coprocessor.WALObserver) { + if (env.getInstance() instanceof WALObserver) { + final WALObserver observer = (WALObserver)env.getInstance(); ctx = ObserverContext.createAndPrepare(env, ctx); Thread currentThread = Thread.currentThread(); ClassLoader cl = currentThread.getContextClassLoader(); try { currentThread.setContextClassLoader(env.getClassLoader()); - ((org.apache.hadoop.hbase.coprocessor.WALObserver)env.getInstance()). - postWALWrite(ctx, info, logKey, logEdit); + if (env.useLegacyPost) { + if (logKey instanceof HLogKey) { + observer.postWALWrite(ctx, info, (HLogKey)logKey, logEdit); + } else { + legacyWarning(observer.getClass(), + "There are wal keys present that are not HLogKey."); + } + } else { + observer.postWALWrite(ctx, info, logKey, logEdit); + } } catch (Throwable e) { handleCoprocessorThrowable(env, e); } finally { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java index 172e47898fa..05cead2ad7d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java @@ -52,7 +52,7 @@ import org.apache.hadoop.io.Writable; * for serializing/deserializing a set of KeyValue items. * * Previously, if a transaction contains 3 edits to c1, c2, c3 for a row R, - * the HLog would have three log entries as follows: + * the WAL would have three log entries as follows: * * : * : @@ -73,7 +73,7 @@ import org.apache.hadoop.io.Writable; * <-1, 3, , , > * * The -1 marker is just a special way of being backward compatible with - * an old HLog which would have contained a single . + * an old WAL which would have contained a single . * * The deserializer for WALEdit backward compatibly detects if the record * is an old style KeyValue or the new style WALEdit. @@ -168,7 +168,7 @@ public class WALEdit implements Writable, HeapSize { int versionOrLength = in.readInt(); // TODO: Change version when we protobuf. Also, change way we serialize KV! Pb it too. if (versionOrLength == VERSION_2) { - // this is new style HLog entry containing multiple KeyValues. + // this is new style WAL entry containing multiple KeyValues. int numEdits = in.readInt(); for (int idx = 0; idx < numEdits; idx++) { if (compressionContext != null) { @@ -189,7 +189,7 @@ public class WALEdit implements Writable, HeapSize { } } } else { - // this is an old style HLog entry. The int that we just + // this is an old style WAL entry. The int that we just // read is actually the length of a single KeyValue this.add(KeyValue.create(versionOrLength, in)); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java index 9c0b8a92e94..ff5f2f53ecd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.wal.WAL.Entry; import com.google.protobuf.ServiceException; @@ -96,17 +97,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; + Entry entry = null; + List regionEntries = null; // Build the action list. for (int i = 0; i < batchSize; i++) { loc = entries.get(i).getFirst(); @@ -114,7 +115,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); @@ -123,9 +124,9 @@ public class WALEditsReplaySink { long startTime = EnvironmentEdgeManager.currentTime(); // 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; @@ -159,7 +160,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 = @@ -182,11 +183,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; @@ -203,11 +204,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()]; + Entry[] entriesArray = new Entry[entries.size()]; entriesArray = entries.toArray(entriesArray); AdminService.BlockingInterface remoteSvr = conn.getAdmin(getLocation().getServerName()); @@ -228,11 +229,11 @@ 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 (Entry entry : this.entries) { WALEdit edit = entry.getEdit(); List cells = edit.getCells(); for (Cell cell : cells) { - // filtering HLog meta entries + // filtering WAL meta entries setLocation(conn.locateRegion(tableName, cell.getRow())); skip = true; break; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WriterBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WriterBase.java index cd3aeaf6751..8188e02c6dc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WriterBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WriterBase.java @@ -28,12 +28,14 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.io.util.LRUDictionary; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; + /** * Context used by our wal dictionary compressor. Null if we're not to do our * custom dictionary compression. */ @InterfaceAudience.Private -public abstract class WriterBase implements HLog.Writer { +public abstract class WriterBase implements DefaultWALProvider.Writer { protected CompressionContext compressionContext; protected Configuration conf; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEntryFilter.java index 281ba635341..6a3981ae7cc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEntryFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEntryFilter.java @@ -24,7 +24,7 @@ import java.util.List; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry; +import org.apache.hadoop.hbase.wal.WAL.Entry; /** * A {@link WALEntryFilter} which contains multiple filters and applies them diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationEndpoint.java index 03b66d2eeb2..c3ec976ac27 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationEndpoint.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.hbase.replication.regionserver.MetricsSource; import com.google.common.util.concurrent.Service; @@ -128,13 +128,13 @@ public interface ReplicationEndpoint extends Service { */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION) class ReplicateContext { - List entries; + List entries; int size; @InterfaceAudience.Private public ReplicateContext() { } - public ReplicateContext setEntries(List entries) { + public ReplicateContext setEntries(List entries) { this.entries = entries; return this; } @@ -142,7 +142,7 @@ public interface ReplicationEndpoint extends Service { this.size = size; return this; } - public List getEntries() { + public List getEntries() { return entries; } public int getSize() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java index 5df7b25e1ab..166dc37210b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java @@ -24,7 +24,7 @@ import java.util.NavigableMap; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry; +import org.apache.hadoop.hbase.wal.WAL.Entry; /** * Keeps KVs that are scoped other than local diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/SystemTableWALEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/SystemTableWALEntryFilter.java index b683ad6f17f..46b8b81e98e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/SystemTableWALEntryFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/SystemTableWALEntryFilter.java @@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.replication; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry; +import org.apache.hadoop.hbase.wal.WAL.Entry; /** * Skips WAL edits for all System tables including META diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/TableCfWALEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/TableCfWALEntryFilter.java index 0ea267dc7e3..b8925125423 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/TableCfWALEntryFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/TableCfWALEntryFilter.java @@ -26,7 +26,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry; +import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.hbase.util.Bytes; public class TableCfWALEntryFilter implements WALEntryFilter { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/WALEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/WALEntryFilter.java index 60797c9cea0..b66dddedb3a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/WALEntryFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/WALEntryFilter.java @@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.replication; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.wal.WAL.Entry; /** * A Filter for WAL entries before being sent over to replication. Multiple @@ -30,12 +30,12 @@ import org.apache.hadoop.hbase.regionserver.wal.HLog; public interface WALEntryFilter { /** - * Applies the filter, possibly returning a different HLog.Entry instance. + * Applies the filter, possibly returning a different Entry instance. * If null is returned, the entry will be skipped. - * @param entry WAL Entry to filter - * @return a (possibly modified) HLog.Entry to use. Returning null or an entry with + * @param entry Entry to filter + * @return a (possibly modified) Entry to use. Returning null or an entry with * no cells will cause the entry to be skipped for replication. */ - public HLog.Entry filter(HLog.Entry entry); + public Entry filter(Entry entry); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java index 8f099d79307..525b7ad9fe6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java @@ -61,17 +61,17 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate implements Abo return files; } - final Set hlogs = loadHLogsFromQueues(); + final Set wals = loadWALsFromQueues(); return Iterables.filter(files, new Predicate() { @Override public boolean apply(FileStatus file) { - String hlog = file.getPath().getName(); - boolean logInReplicationQueue = hlogs.contains(hlog); + String wal = file.getPath().getName(); + boolean logInReplicationQueue = wals.contains(wal); if (LOG.isDebugEnabled()) { if (logInReplicationQueue) { - LOG.debug("Found log in ZK, keeping: " + hlog); + LOG.debug("Found log in ZK, keeping: " + wal); } else { - LOG.debug("Didn't find this log in ZK, deleting: " + hlog); + LOG.debug("Didn't find this log in ZK, deleting: " + wal); } } return !logInReplicationQueue; @@ -79,15 +79,15 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate implements Abo } /** - * Load all hlogs in all replication queues from ZK + * Load all wals in all replication queues from ZK */ - private Set loadHLogsFromQueues() { + private Set loadWALsFromQueues() { List rss = replicationQueues.getListOfReplicators(); if (rss == null) { LOG.debug("Didn't find any region server that replicates, won't prevent any deletions."); return ImmutableSet.of(); } - Set hlogs = Sets.newHashSet(); + Set wals = Sets.newHashSet(); for (String rs: rss) { List listOfPeers = replicationQueues.getAllQueues(rs); // if rs just died, this will be null @@ -95,13 +95,13 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate implements Abo continue; } for (String id : listOfPeers) { - List peersHlogs = replicationQueues.getLogsInQueue(rs, id); - if (peersHlogs != null) { - hlogs.addAll(peersHlogs); + List peersWals = replicationQueues.getLogsInQueue(rs, id); + if (peersWals != null) { + wals.addAll(peersWals); } } } - return hlogs; + return wals; } @Override @@ -109,7 +109,7 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate implements Abo // If replication is disabled, keep all members null if (!config.getBoolean(HConstants.REPLICATION_ENABLE_KEY, HConstants.REPLICATION_ENABLE_DEFAULT)) { - LOG.warn("Not configured - allowing all hlogs to be deleted"); + LOG.warn("Not configured - allowing all wals to be deleted"); return; } // Make my own Configuration. Then I'll have my own connection to zk that diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java index ff74fd38875..b70d513a6b7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java @@ -34,7 +34,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.wal.WAL.Entry; import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; @@ -136,7 +136,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi */ @Override public boolean replicate(ReplicateContext replicateContext) { - List entries = replicateContext.getEntries(); + List entries = replicateContext.getEntries(); int sleepMultiplier = 1; while (this.isRunning()) { if (!peersSelected) { @@ -159,7 +159,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi " entries of total size " + replicateContext.getSize()); } ReplicationProtbufUtil.replicateWALEntry(rrs, - entries.toArray(new HLog.Entry[entries.size()])); + entries.toArray(new Entry[entries.size()])); // update metrics this.metrics.setAgeOfLastShippedOp(entries.get(entries.size()-1).getKey().getWriteTime()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java index 9df85f9ca23..efe79aeb2b3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java @@ -60,14 +60,14 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.PipelineController; -import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.RegionEntryBuffer; -import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.SinkWriter; import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry; -import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.EntryBuffers; -import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.OutputSink; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALSplitter.EntryBuffers; +import org.apache.hadoop.hbase.wal.WALSplitter.OutputSink; +import org.apache.hadoop.hbase.wal.WALSplitter.PipelineController; +import org.apache.hadoop.hbase.wal.WALSplitter.RegionEntryBuffer; +import org.apache.hadoop.hbase.wal.WALSplitter.SinkWriter; import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; import org.apache.hadoop.hbase.replication.WALEntryFilter; @@ -92,7 +92,7 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint { private Configuration conf; private ClusterConnection connection; - // Reuse HLogSplitter constructs as a WAL pipe + // Reuse WALSplitter constructs as a WAL pipe private PipelineController controller; private RegionReplicaOutputSink outputSink; private EntryBuffers entryBuffers; @@ -211,7 +211,7 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint { * WAL file filling up a buffer of heap size "replication.source.size.capacity"(64MB) or at most * "replication.source.nb.capacity" entries or until it sees the end of file (in live tailing). * Then RS passes all the buffered edits in this replicate() call context. RRRE puts the edits - * to the HLogSplitter.EntryBuffers which is a blocking buffer space of up to + * to the WALSplitter.EntryBuffers which is a blocking buffer space of up to * "hbase.region.replica.replication.buffersize" (128MB) in size. This buffer splits the edits * based on regions. * @@ -456,7 +456,7 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint { // replicaId of the region replica that we want to replicate to private final int replicaId; - private final List entries; + private final List entries; private final byte[] initialEncodedRegionName; private final AtomicLong skippedEntries; private final RpcControllerFactory rpcControllerFactory; @@ -464,7 +464,7 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint { public RegionReplicaReplayCallable(ClusterConnection connection, RpcControllerFactory rpcControllerFactory, TableName tableName, - HRegionLocation location, HRegionInfo regionInfo, byte[] row,List entries, + HRegionLocation location, HRegionInfo regionInfo, byte[] row,List entries, AtomicLong skippedEntries) { super(connection, location, tableName, row); this.replicaId = regionInfo.getReplicaId(); @@ -502,14 +502,14 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint { return replayToServer(this.entries, timeout); } - private ReplicateWALEntryResponse replayToServer(List entries, int timeout) + private ReplicateWALEntryResponse replayToServer(List entries, int timeout) throws IOException { if (entries.isEmpty() || skip) { skippedEntries.incrementAndGet(); return ReplicateWALEntryResponse.newBuilder().build(); } - HLog.Entry[] entriesArray = new HLog.Entry[entries.size()]; + Entry[] entriesArray = new Entry[entries.size()]; entriesArray = entries.toArray(entriesArray); // set the region name for the target region replica diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java index 2c413f49cba..47296440e1c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java @@ -47,7 +47,7 @@ import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry; import org.apache.hadoop.hbase.regionserver.ReplicationSinkService; import org.apache.hadoop.hbase.regionserver.ReplicationSourceService; -import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.replication.ReplicationException; @@ -66,7 +66,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; * Gateway to Replication. Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}. */ @InterfaceAudience.Private -public class Replication implements WALActionsListener, +public class Replication extends WALActionsListener.Base implements ReplicationSourceService, ReplicationSinkService { private static final Log LOG = LogFactory.getLog(Replication.class); @@ -155,7 +155,7 @@ public class Replication implements WALActionsListener, } /* - * Returns an object to listen to new hlog changes + * Returns an object to listen to new wal changes **/ public WALActionsListener getWALActionsListener() { return this; @@ -222,13 +222,7 @@ public class Replication implements WALActionsListener, } @Override - public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey, - WALEdit logEdit) { - // Not interested - } - - @Override - public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, + public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit) { scopeWALEdits(htd, logKey, logEdit); } @@ -240,7 +234,7 @@ public class Replication implements WALActionsListener, * @param logKey Key that may get scoped according to its edits * @param logEdit Edits used to lookup the scopes */ - public static void scopeWALEdits(HTableDescriptor htd, HLogKey logKey, + public static void scopeWALEdits(HTableDescriptor htd, WALKey logKey, WALEdit logEdit) { NavigableMap scopes = new TreeMap(Bytes.BYTES_COMPARATOR); @@ -273,16 +267,6 @@ public class Replication implements WALActionsListener, getReplicationManager().postLogRoll(newPath); } - @Override - public void preLogArchive(Path oldPath, Path newPath) throws IOException { - // Not interested - } - - @Override - public void postLogArchive(Path oldPath, Path newPath) throws IOException { - // Not interested - } - /** * This method modifies the master's configuration in order to inject * replication-related features @@ -299,16 +283,6 @@ public class Replication implements WALActionsListener, } } - @Override - public void logRollRequested() { - // Not interested - } - - @Override - public void logCloseRequested() { - // not interested - } - /* * Statistics thread. Periodically prints the cache statistics to the log. */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationHLogReaderManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationHLogReaderManager.java deleted file mode 100644 index ccae169a4f3..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationHLogReaderManager.java +++ /dev/null @@ -1,144 +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.replication.regionserver; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.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.HLogFactory; - -import java.io.IOException; - -/** - * Wrapper class around HLog to help manage the implementation details - * such as compression. - */ -@InterfaceAudience.Private -public class ReplicationHLogReaderManager { - - private static final Log LOG = LogFactory.getLog(ReplicationHLogReaderManager.class); - private final FileSystem fs; - private final Configuration conf; - private long position = 0; - private HLog.Reader reader; - private Path lastPath; - - /** - * Creates the helper but doesn't open any file - * Use setInitialPosition after using the constructor if some content needs to be skipped - * @param fs - * @param conf - */ - public ReplicationHLogReaderManager(FileSystem fs, Configuration conf) { - this.fs = fs; - this.conf = conf; - } - - /** - * Opens the file at the current position - * @param path - * @return an HLog reader. - * @throws IOException - */ - public HLog.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)) { - this.closeReader(); - this.reader = HLogFactory.createReader(this.fs, path, this.conf); - this.lastPath = path; - } else { - try { - this.reader.reset(); - } catch (NullPointerException npe) { - throw new IOException("NPE resetting reader, likely HDFS-4380", npe); - } - } - return this.reader; - } - - /** - * Get the next entry, returned and also added in the array - * @return a new entry or null - * @throws IOException - */ - public HLog.Entry readNextAndSetPosition() throws IOException { - HLog.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 - // from the last known good position - this.position = this.reader.getPosition(); - // We need to set the CC to null else it will be compressed when sent to the sink - if (entry != null) { - entry.setCompressionContext(null); - } - return entry; - } - - /** - * Advance the reader to the current position - * @throws IOException - */ - public void seek() throws IOException { - if (this.position != 0) { - this.reader.seek(this.position); - } - } - - /** - * Get the position that we stopped reading at - * @return current position, cannot be negative - */ - public long getPosition() { - return this.position; - } - - public void setPosition(long pos) { - this.position = pos; - } - - /** - * Close the current reader - * @throws IOException - */ - public void closeReader() throws IOException { - if (this.reader != null) { - this.reader.close(); - this.reader = null; - } - } - - /** - * Tell the helper to reset internal state - */ - void finishCurrentFile() { - this.position = 0; - try { - this.closeReader(); - } catch (IOException e) { - LOG.warn("Unable to close reader", e); - } - } - -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java index 7ed7bec8083..9a6013188da 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java @@ -58,7 +58,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; *

    * This replication process is currently waiting for the edits to be applied * before the method can return. This means that the replication of edits - * is synchronized (after reading from HLogs in ReplicationSource) and that a + * is synchronized (after reading from WALs in ReplicationSource) and that a * single region server cannot receive edits from two sources at the same time *

    * This class uses the native HBase client in order to replicate entries. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java index 02b97f2309f..ee43956f482 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java @@ -41,8 +41,9 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Stoppable; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.replication.ChainWALEntryFilter; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; @@ -52,6 +53,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; import org.apache.hadoop.hbase.replication.ReplicationQueues; import org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter; import org.apache.hadoop.hbase.replication.WALEntryFilter; +import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Threads; import com.google.common.collect.Lists; @@ -94,8 +96,8 @@ public class ReplicationSource extends Thread private long replicationQueueSizeCapacity; // Max number of entries in entriesArray private int replicationQueueNbCapacity; - // Our reader for the current log - private HLog.Reader reader; + // Our reader for the current log. open/close handled by repLogReader + private WAL.Reader reader; // Last position in the log that we sent to ZooKeeper private long lastLoggedPosition = -1; // Path of the current log @@ -122,7 +124,7 @@ public class ReplicationSource extends Thread // Metrics for this source private MetricsSource metrics; // Handle on the log reader helper - private ReplicationHLogReaderManager repLogReader; + private ReplicationWALReaderManager repLogReader; //WARN threshold for the number of queued logs, defaults to 2 private int logQueueWarnThreshold; // ReplicationEndpoint which will handle the actual replication @@ -176,7 +178,7 @@ public class ReplicationSource extends Thread this.manager = manager; this.fs = fs; this.metrics = metrics; - this.repLogReader = new ReplicationHLogReaderManager(this.fs, this.conf); + this.repLogReader = new ReplicationWALReaderManager(this.fs, this.conf); this.clusterId = clusterId; this.peerClusterZnode = peerClusterZnode; @@ -342,7 +344,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)) { @@ -420,7 +422,7 @@ public class ReplicationSource extends Thread * @throws IOException */ protected boolean readAllEntriesToReplicateOrNextFile(boolean currentWALisBeingWrittenTo, - List entries) throws IOException{ + List entries) throws IOException { long seenEntries = 0; if (LOG.isTraceEnabled()) { LOG.trace("Seeking in " + this.currentPath + " at position " @@ -428,7 +430,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) { this.metrics.incrLogEditsRead(); @@ -440,7 +442,7 @@ public class ReplicationSource extends Thread // Remove all KVs that should not be replicated entry = walEntryFilter.filter(entry); WALEdit edit = null; - HLogKey logKey = null; + WALKey logKey = null; if (entry != null) { edit = entry.getEdit(); logKey = entry.getKey(); @@ -521,12 +523,13 @@ public class ReplicationSource extends Thread // to look at) List deadRegionServers = this.replicationQueueInfo.getDeadRegionServers(); LOG.info("NB dead servers : " + deadRegionServers.size()); + final Path rootDir = FSUtils.getRootDir(this.conf); for (String curDeadServerName : deadRegionServers) { - Path deadRsDirectory = - new Path(manager.getLogDir().getParent(), curDeadServerName); + final Path deadRsDirectory = new Path(rootDir, + DefaultWALProvider.getWALDirectoryName(curDeadServerName)); Path[] locs = new Path[] { new Path(deadRsDirectory, currentPath.getName()), - new Path(deadRsDirectory.suffix(HLog.SPLITTING_EXT), + new Path(deadRsDirectory.suffix(DefaultWALProvider.SPLITTING_EXT), currentPath.getName()), }; for (Path possibleLogLocation : locs) { @@ -536,6 +539,7 @@ public class ReplicationSource extends Thread LOG.info("Log " + this.currentPath + " still exists at " + possibleLogLocation); // Breaking here will make us sleep since reader is null + // TODO why don't we need to set currentPath and call openReader here? return true; } } @@ -543,6 +547,8 @@ public class ReplicationSource extends Thread // In the case of disaster/recovery, HMaster may be shutdown/crashed before flush data // from .logs to .oldlogs. Loop into .logs folders and check whether a match exists if (stopper instanceof ReplicationSyncUp.DummyServer) { + // N.B. the ReplicationSyncUp tool sets the manager.getLogDir to the root of the wal + // area rather than to the wal area for a particular region server. FileStatus[] rss = fs.listStatus(manager.getLogDir()); for (FileStatus rs : rss) { Path p = rs.getPath(); @@ -551,7 +557,7 @@ public class ReplicationSource extends Thread p = new Path(p, log.getPath().getName()); if (p.getName().equals(currentPath.getName())) { currentPath = p; - LOG.info("Log " + this.currentPath + " exists under " + manager.getLogDir()); + LOG.info("Log " + currentPath.getName() + " found at " + currentPath); // Open the log at the new location this.openReader(sleepMultiplier); return true; @@ -591,7 +597,7 @@ public class ReplicationSource extends Thread if (ioe.getCause() instanceof NullPointerException) { // Workaround for race condition in HDFS-4380 // which throws a NPE if we open a file before any data node has the most recent block - // Just sleep and retry. Will require re-reading compressed HLogs for compressionContext. + // Just sleep and retry. Will require re-reading compressed WALs for compressionContext. LOG.warn("Got NPE opening reader, will retry."); } else if (sleepMultiplier == this.maxRetriesMultiplier) { // TODO Need a better way to determine if a file is really gone but @@ -656,7 +662,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"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java index ad1b088aeff..4908ebc7c9a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java @@ -63,7 +63,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; * sources. There are two classes of sources: *

  14. Normal sources are persistent and one per peer cluster
  15. *
  16. Old sources are recovered from a failed region server and our - * only goal is to finish replicating the HLog queue it had up in ZK
  17. + * only goal is to finish replicating the WAL queue it had up in ZK * * When a region server dies, this class uses a watcher to get notified and it * tries to grab a lock in order to transfer all the queues in a local @@ -88,16 +88,16 @@ public class ReplicationSourceManager implements ReplicationListener { // All about stopping private final Server server; // All logs we are currently tracking - private final Map> hlogsById; + private final Map> walsById; // Logs for recovered sources we are currently tracking - private final Map> hlogsByIdRecoveredQueues; + private final Map> walsByIdRecoveredQueues; private final Configuration conf; private final FileSystem fs; // The path to the latest log we saw, for new coming sources private Path latestPath; - // Path to the hlogs directories + // Path to the wals directories private final Path logDir; - // Path to the hlog archive + // Path to the wal archive private final Path oldLogDir; // The number of ms that we wait before moving znodes, HBASE-3596 private final long sleepBeforeFailover; @@ -115,7 +115,7 @@ public class ReplicationSourceManager implements ReplicationListener { * @param conf the configuration to use * @param server the server for this region server * @param fs the file system to use - * @param logDir the directory that contains all hlog directories of live RSs + * @param logDir the directory that contains all wal directories of live RSs * @param oldLogDir the directory where old logs are archived * @param clusterId */ @@ -130,8 +130,8 @@ public class ReplicationSourceManager implements ReplicationListener { this.replicationPeers = replicationPeers; this.replicationTracker = replicationTracker; this.server = server; - this.hlogsById = new HashMap>(); - this.hlogsByIdRecoveredQueues = new ConcurrentHashMap>(); + this.walsById = new HashMap>(); + this.walsByIdRecoveredQueues = new ConcurrentHashMap>(); this.oldsources = new CopyOnWriteArrayList(); this.conf = conf; this.fs = fs; @@ -159,7 +159,7 @@ public class ReplicationSourceManager implements ReplicationListener { /** * Provide the id of the peer and a log key and this method will figure which - * hlog it belongs to and will log, for this region server, the current + * wal it belongs to and will log, for this region server, the current * position. It will also clean old logs from the queue. * @param log Path to the log currently being replicated from * replication status in zookeeper. It will also delete older entries. @@ -187,32 +187,32 @@ public class ReplicationSourceManager implements ReplicationListener { */ public void cleanOldLogs(String key, String id, boolean queueRecovered) { if (queueRecovered) { - SortedSet hlogs = hlogsByIdRecoveredQueues.get(id); - if (hlogs != null && !hlogs.first().equals(key)) { - cleanOldLogs(hlogs, key, id); + SortedSet wals = walsByIdRecoveredQueues.get(id); + if (wals != null && !wals.first().equals(key)) { + cleanOldLogs(wals, key, id); } } else { - synchronized (this.hlogsById) { - SortedSet hlogs = hlogsById.get(id); - if (!hlogs.first().equals(key)) { - cleanOldLogs(hlogs, key, id); + synchronized (this.walsById) { + SortedSet wals = walsById.get(id); + if (!wals.first().equals(key)) { + cleanOldLogs(wals, key, id); } } } } - private void cleanOldLogs(SortedSet hlogs, String key, String id) { - SortedSet hlogSet = hlogs.headSet(key); - LOG.debug("Removing " + hlogSet.size() + " logs in the list: " + hlogSet); - for (String hlog : hlogSet) { - this.replicationQueues.removeLog(id, hlog); + private void cleanOldLogs(SortedSet wals, String key, String id) { + SortedSet walSet = wals.headSet(key); + LOG.debug("Removing " + walSet.size() + " logs in the list: " + walSet); + for (String wal : walSet) { + this.replicationQueues.removeLog(id, wal); } - hlogSet.clear(); + walSet.clear(); } /** * Adds a normal source per registered peer cluster and tries to process all - * old region server hlog queues + * old region server wal queues */ protected void init() throws IOException, ReplicationException { for (String id : this.replicationPeers.getPeerIds()) { @@ -248,13 +248,13 @@ public class ReplicationSourceManager implements ReplicationListener { ReplicationSourceInterface src = getReplicationSource(this.conf, this.fs, this, this.replicationQueues, this.replicationPeers, server, id, this.clusterId, peerConfig, peer); - synchronized (this.hlogsById) { + synchronized (this.walsById) { this.sources.add(src); - this.hlogsById.put(id, new TreeSet()); - // Add the latest hlog to that source's queue + this.walsById.put(id, new TreeSet()); + // Add the latest wal to that source's queue if (this.latestPath != null) { String name = this.latestPath.getName(); - this.hlogsById.get(id).add(name); + this.walsById.get(id).add(name); try { this.replicationQueues.addLog(src.getPeerClusterZnode(), name); } catch (ReplicationException e) { @@ -272,8 +272,8 @@ public class ReplicationSourceManager implements ReplicationListener { } /** - * Delete a complete queue of hlogs associated with a peer cluster - * @param peerId Id of the peer cluster queue of hlogs to delete + * Delete a complete queue of wals associated with a peer cluster + * @param peerId Id of the peer cluster queue of wals to delete */ public void deleteSource(String peerId, boolean closeConnection) { this.replicationQueues.removeQueue(peerId); @@ -296,19 +296,19 @@ public class ReplicationSourceManager implements ReplicationListener { } /** - * Get a copy of the hlogs of the first source on this rs - * @return a sorted set of hlog names + * Get a copy of the wals of the first source on this rs + * @return a sorted set of wal names */ - protected Map> getHLogs() { - return Collections.unmodifiableMap(hlogsById); + protected Map> getWALs() { + return Collections.unmodifiableMap(walsById); } /** - * Get a copy of the hlogs of the recovered sources on this rs - * @return a sorted set of hlog names + * Get a copy of the wals of the recovered sources on this rs + * @return a sorted set of wal names */ - protected Map> getHlogsByIdRecoveredQueues() { - return Collections.unmodifiableMap(hlogsByIdRecoveredQueues); + protected Map> getWalsByIdRecoveredQueues() { + return Collections.unmodifiableMap(walsByIdRecoveredQueues); } /** @@ -328,7 +328,7 @@ public class ReplicationSourceManager implements ReplicationListener { } void preLogRoll(Path newLog) throws IOException { - synchronized (this.hlogsById) { + synchronized (this.walsById) { String name = newLog.getName(); for (ReplicationSourceInterface source : this.sources) { try { @@ -338,13 +338,13 @@ public class ReplicationSourceManager implements ReplicationListener { + source.getPeerClusterZnode() + ", filename=" + name, e); } } - for (SortedSet hlogs : this.hlogsById.values()) { + for (SortedSet wals : this.walsById.values()) { if (this.sources.isEmpty()) { - // If there's no slaves, don't need to keep the old hlogs since + // If there's no slaves, don't need to keep the old wals since // we only consider the last one when a new slave comes in - hlogs.clear(); + wals.clear(); } - hlogs.add(name); + wals.add(name); } } @@ -452,7 +452,7 @@ public class ReplicationSourceManager implements ReplicationListener { LOG.info("Done with the recovered queue " + src.getPeerClusterZnode()); this.oldsources.remove(src); deleteSource(src.getPeerClusterZnode(), false); - this.hlogsByIdRecoveredQueues.remove(src.getPeerClusterZnode()); + this.walsByIdRecoveredQueues.remove(src.getPeerClusterZnode()); } /** @@ -569,7 +569,7 @@ public class ReplicationSourceManager implements ReplicationListener { // Copying over the failed queue is completed. if (newQueues.isEmpty()) { // We either didn't get the lock or the failed region server didn't have any outstanding - // HLogs to replicate, so we are done. + // WALs to replicate, so we are done. return; } @@ -600,12 +600,12 @@ public class ReplicationSourceManager implements ReplicationListener { break; } oldsources.add(src); - SortedSet hlogsSet = entry.getValue(); - for (String hlog : hlogsSet) { - src.enqueueLog(new Path(oldLogDir, hlog)); + SortedSet walsSet = entry.getValue(); + for (String wal : walsSet) { + src.enqueueLog(new Path(oldLogDir, wal)); } src.startup(); - hlogsByIdRecoveredQueues.put(peerId, hlogsSet); + walsByIdRecoveredQueues.put(peerId, walsSet); } catch (IOException e) { // TODO manage it LOG.error("Failed creating a source", e); @@ -615,16 +615,16 @@ public class ReplicationSourceManager implements ReplicationListener { } /** - * Get the directory where hlogs are archived - * @return the directory where hlogs are archived + * Get the directory where wals are archived + * @return the directory where wals are archived */ public Path getOldLogDir() { return this.oldLogDir; } /** - * Get the directory where hlogs are stored by their RSs - * @return the directory where hlogs are stored by their RSs + * Get the directory where wals are stored by their RSs + * @return the directory where wals are stored by their RSs */ public Path getLogDir() { return this.logDir; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java index e665887fa5b..9d196abfac9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java @@ -30,7 +30,7 @@ import java.util.TreeSet; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/HbaseObjectWritableFor96Migration.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/HbaseObjectWritableFor96Migration.java index c1542b101f5..a4f6054d926 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/HbaseObjectWritableFor96Migration.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/HbaseObjectWritableFor96Migration.java @@ -86,8 +86,10 @@ 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.HLogKey; +import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ProtoUtil; import org.apache.hadoop.io.DataOutputOutputStream; @@ -229,8 +231,8 @@ class HbaseObjectWritableFor96Migration implements Writable, WritableWithSize, C addToMap(Delete [].class, code++); - addToMap(HLog.Entry.class, code++); - addToMap(HLog.Entry[].class, code++); + addToMap(Entry.class, code++); + addToMap(Entry[].class, code++); addToMap(HLogKey.class, code++); addToMap(List.class, code++); @@ -540,6 +542,26 @@ class HbaseObjectWritableFor96Migration implements Writable, WritableWithSize, C byte [] scanBytes = ProtobufUtil.toScan(scan).toByteArray(); out.writeInt(scanBytes.length); out.write(scanBytes); + } else if (Entry.class.isAssignableFrom(declClass)) { + // Entry is no longer Writable, maintain compatible serialization. + // Writables write their exact runtime class + Class c = instanceObj.getClass(); + Integer code = CLASS_TO_CODE.get(c); + if (code == null) { + out.writeByte(NOT_ENCODED); + Text.writeString(out, c.getName()); + } else { + writeClassCode(out, c); + } + final Entry entry = (Entry)instanceObj; + // We only support legacy HLogKey + WALKey key = entry.getKey(); + if (!(key instanceof HLogKey)) { + throw new IOException("Can't write Entry '" + instanceObj + "' due to key class '" + + key.getClass() + "'"); + } + ((HLogKey)key).write(out); + entry.getEdit().write(out); } else { throw new IOException("Can't write: "+instanceObj+" as "+declClass); } @@ -674,6 +696,9 @@ class HbaseObjectWritableFor96Migration implements Writable, WritableWithSize, C int b = (byte)WritableUtils.readVInt(in); if (b == NOT_ENCODED) { String className = Text.readString(in); + if ("org.apache.hadoop.hbase.regionserver.wal.HLog$Entry".equals(className)) { + className = Entry.class.getName(); + } try { instanceClass = getClassByName(conf, className); } catch (ClassNotFoundException e) { @@ -696,6 +721,13 @@ class HbaseObjectWritableFor96Migration implements Writable, WritableWithSize, C declaredClass = ((NullInstance)instance).declaredClass; instance = null; } + } else if (Entry.class.isAssignableFrom(instanceClass)) { + // Entry stopped being Writable; maintain serialization support. + final HLogKey key = new HLogKey(); + final WALEdit edit = new WALEdit(); + key.readFields(in); + edit.readFields(in); + instance = new Entry(key, edit); } else { int length = in.readInt(); byte[] objectBytes = new byte[length]; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java index 51dac3c320b..95a5d5282dd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java @@ -52,7 +52,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.io.FileLink; import org.apache.hadoop.hbase.io.HFileLink; -import org.apache.hadoop.hbase.io.HLogLink; +import org.apache.hadoop.hbase.io.WALLink; import org.apache.hadoop.hbase.mapreduce.JobUtil; import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; @@ -84,7 +84,7 @@ import org.apache.hadoop.util.ToolRunner; * Export the specified snapshot to a given FileSystem. * * The .snapshot/name folder is copied to the destination cluster - * and then all the hfiles/hlogs are copied using a Map-Reduce Job in the .archive/ location. + * and then all the hfiles/wals are copied using a Map-Reduce Job in the .archive/ location. * When everything is done, the second cluster can restore the snapshot. */ @InterfaceAudience.Public @@ -407,7 +407,7 @@ public class ExportSnapshot extends Configured implements Tool { case WAL: String serverName = fileInfo.getWalServer(); String logName = fileInfo.getWalName(); - link = new HLogLink(inputRoot, serverName, logName); + link = new WALLink(inputRoot, serverName, logName); break; default: throw new IOException("Invalid File Type: " + fileInfo.getType().toString()); @@ -430,7 +430,7 @@ public class ExportSnapshot extends Configured implements Tool { link = new HFileLink(inputRoot, inputArchive, inputPath); break; case WAL: - link = new HLogLink(inputRoot, fileInfo.getWalServer(), fileInfo.getWalName()); + link = new WALLink(inputRoot, fileInfo.getWalServer(), fileInfo.getWalName()); break; default: throw new IOException("Invalid File Type: " + fileInfo.getType().toString()); @@ -482,7 +482,7 @@ public class ExportSnapshot extends Configured implements Tool { // ========================================================================== /** - * Extract the list of files (HFiles/HLogs) to copy using Map-Reduce. + * Extract the list of files (HFiles/WALs) to copy using Map-Reduce. * @return list of files referenced by the snapshot (pair of path and size) */ private static List> getSnapshotFiles(final Configuration conf, @@ -530,7 +530,7 @@ public class ExportSnapshot extends Configured implements Tool { .setWalName(logfile) .build(); - long size = new HLogLink(conf, server, logfile).getFileStatus(fs).getLen(); + long size = new WALLink(conf, server, logfile).getFileStatus(fs).getLen(); files.add(new Pair(fileInfo, size)); } }); @@ -786,7 +786,7 @@ public class ExportSnapshot extends Configured implements Tool { } /** - * Execute the export snapshot by copying the snapshot metadata, hfiles and hlogs. + * Execute the export snapshot by copying the snapshot metadata, hfiles and wals. * @return 0 on success, and != 0 upon failure. */ @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java index f33f5b07483..a5e0d851a6d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java @@ -47,7 +47,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.io.HFileLink; -import org.apache.hadoop.hbase.io.HLogLink; +import org.apache.hadoop.hbase.io.WALLink; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; import org.apache.hadoop.hbase.util.FSUtils; @@ -57,7 +57,7 @@ import org.apache.hadoop.hbase.util.FSUtils; *
      *
    1. Table Descriptor *
    2. Snapshot creation time, type, format version, ... - *
    3. List of hfiles and hlogs + *
    4. List of hfiles and wals *
    5. Stats about hfiles and logs sizes, percentage of shared with the source table, ... *
    */ @@ -244,7 +244,7 @@ public final class SnapshotInfo extends Configured implements Tool { * @return the log information */ FileInfo addLogFile(final String server, final String logfile) throws IOException { - HLogLink logLink = new HLogLink(conf, server, logfile); + WALLink logLink = new WALLink(conf, server, logfile); long size = -1; try { size = logLink.getFileStatus(fs).getLen(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java index 2c3913a8882..9297ea00b4f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java @@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.util.FSVisitor; @@ -73,7 +73,7 @@ public final class SnapshotReferenceUtil { * @return path to the log home directory for the archive files. */ public static Path getLogsDir(Path snapshotDir, String serverName) { - return new Path(snapshotDir, HLogUtil.getHLogDirectoryName(serverName)); + return new Path(snapshotDir, DefaultWALProvider.getWALDirectoryName(serverName)); } /** @@ -364,9 +364,9 @@ public final class SnapshotReferenceUtil { * @param fs {@link FileSystem} * @param snapshotDir {@link Path} to the Snapshot directory * @throws IOException if an error occurred while scanning the directory - * @return the names of hlogs in the specified snaphot + * @return the names of wals in the specified snaphot */ - public static Set getHLogNames(final FileSystem fs, final Path snapshotDir) + public static Set getWALNames(final FileSystem fs, final Path snapshotDir) throws IOException { final Set names = new HashSet(); visitLogFiles(fs, snapshotDir, new FSVisitor.LogFileVisitor() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java index f904565d6ff..8bdac1521f8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java @@ -262,7 +262,7 @@ public class FSHDFSUtils extends FSUtils { } catch (IOException e) { if (e instanceof LeaseExpiredException && e.getMessage().contains("File does not exist")) { // This exception comes out instead of FNFE, fix it - throw new FileNotFoundException("The given HLog wasn't found at " + p); + throw new FileNotFoundException("The given WAL wasn't found at " + p); } else if (e instanceof FileNotFoundException) { throw (FileNotFoundException)e; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSVisitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSVisitor.java index 5e14db6c863..f0cc0c102f2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSVisitor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSVisitor.java @@ -29,7 +29,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.wal.WALSplitter; /** * Utility methods for interacting with the hbase.root file system. @@ -179,7 +179,7 @@ public final class FSVisitor { */ public static void visitRegionRecoveredEdits(final FileSystem fs, final Path regionDir, final FSVisitor.RecoveredEditsVisitor visitor) throws IOException { - NavigableSet files = HLogUtil.getSplitEditFilesSorted(fs, regionDir); + NavigableSet files = WALSplitter.getSplitEditFilesSorted(fs, regionDir); if (files == null || files.size() == 0) return; for (Path source: files) { @@ -213,16 +213,16 @@ public final class FSVisitor { for (FileStatus serverLogs: logServerDirs) { String serverName = serverLogs.getPath().getName(); - FileStatus[] hlogs = FSUtils.listStatus(fs, serverLogs.getPath()); - if (hlogs == null) { + FileStatus[] wals = FSUtils.listStatus(fs, serverLogs.getPath()); + if (wals == null) { if (LOG.isTraceEnabled()) { - LOG.trace("No hfiles found for server: " + serverName + ", skipping."); + LOG.trace("No wals found for server: " + serverName + ", skipping."); } continue; } - for (FileStatus hlogRef: hlogs) { - visitor.logFile(serverName, hlogRef.getPath().getName()); + for (FileStatus walRef: wals) { + visitor.logFile(serverName, walRef.getPath().getName()); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index 9373130c983..f08800b6d77 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -104,7 +104,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.Block import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator; import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE; @@ -708,7 +708,7 @@ public class HBaseFsck extends Configured { /** * Orphaned regions are regions without a .regioninfo file in them. We "adopt" * these orphans by creating a new region, and moving the column families, - * recovered edits, HLogs, into the new region dir. We determine the region + * recovered edits, WALs, into the new region dir. We determine the region * startkey and endkeys by looking at all of the hfiles inside the column * families to identify the min and max keys. The resulting region will * likely violate table integrity but will be dealt with by merging @@ -3675,7 +3675,7 @@ public class HBaseFsck extends Configured { // This is special case if a region is left after split he.hdfsOnlyEdits = true; FileStatus[] subDirs = fs.listStatus(regionDir.getPath()); - Path ePath = HLogUtil.getRegionDirRecoveredEditsDir(regionDir.getPath()); + Path ePath = WALSplitter.getRegionDirRecoveredEditsDir(regionDir.getPath()); for (FileStatus subDir : subDirs) { String sdName = subDir.getPath().getName(); if (!sdName.startsWith(".") && !sdName.equals(ePath.getName())) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java index 5294b8509a3..c577abfab70 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java @@ -47,8 +47,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; +import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.ipc.RemoteException; /** @@ -143,7 +142,7 @@ class HMerge { protected final FileSystem fs; protected final Path rootDir; protected final HTableDescriptor htd; - protected final HLog hlog; + protected final WALFactory walFactory; private final long maxFilesize; @@ -160,7 +159,9 @@ class HMerge { .getHTableDescriptor(); String logname = "merge_" + System.currentTimeMillis() + HConstants.HREGION_LOGDIR_NAME; - this.hlog = HLogFactory.createHLog(fs, tabledir, logname, conf); + final Configuration walConf = new Configuration(conf); + FSUtils.setRootDir(walConf, tabledir); + this.walFactory = new WALFactory(walConf, null, logname); } void process() throws IOException { @@ -174,8 +175,7 @@ class HMerge { } } finally { try { - hlog.closeAndDelete(); - + walFactory.close(); } catch(IOException e) { LOG.error(e); } @@ -194,10 +194,12 @@ class HMerge { long nextSize = 0; for (int i = 0; i < info.length - 1; i++) { if (currentRegion == null) { - currentRegion = HRegion.openHRegion(conf, fs, this.rootDir, info[i], this.htd, hlog); + currentRegion = HRegion.openHRegion(conf, fs, this.rootDir, info[i], this.htd, + walFactory.getWAL(info[i].getEncodedNameAsBytes())); currentSize = currentRegion.getLargestHStoreSize(); } - nextRegion = HRegion.openHRegion(conf, fs, this.rootDir, info[i + 1], this.htd, hlog); + nextRegion = HRegion.openHRegion(conf, fs, this.rootDir, info[i + 1], this.htd, + walFactory.getWAL(info[i+1].getEncodedNameAsBytes())); nextSize = nextRegion.getLargestHStoreSize(); if ((currentSize + nextSize) <= (maxFilesize / 2)) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java index 33aa569ae91..7a6ee566963 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java @@ -42,7 +42,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.wal.WAL; import org.apache.hadoop.io.WritableComparator; import org.apache.hadoop.util.GenericOptionsParser; import org.apache.hadoop.util.Tool; @@ -181,10 +181,9 @@ public class Merge extends Configured implements Tool { Bytes.toStringBinary(meta.getRegionName())); } HRegion merged = null; - HLog log = utils.getLog(); - HRegion r1 = HRegion.openHRegion(info1, htd, log, getConf()); + HRegion r1 = HRegion.openHRegion(info1, htd, utils.getLog(info1), getConf()); try { - HRegion r2 = HRegion.openHRegion(info2, htd, log, getConf()); + HRegion r2 = HRegion.openHRegion(info2, htd, utils.getLog(info2), getConf()); try { merged = HRegion.merge(r1, r2); } finally { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java index b4454edfc85..60e85379b05 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java @@ -35,8 +35,8 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; /** * Contains utility methods for manipulating HBase meta tables. @@ -51,7 +51,7 @@ public class MetaUtils { private final Configuration conf; private final FSTableDescriptors descriptors; private FileSystem fs; - private HLog log; + private WALFactory walFactory; private HRegion metaRegion; private Map metaRegions = Collections.synchronizedSortedMap( new TreeMap(Bytes.BYTES_COMPARATOR)); @@ -84,17 +84,19 @@ public class MetaUtils { } /** - * @return the HLog + * @return the WAL associated with the given region * @throws IOException e */ - public synchronized HLog getLog() throws IOException { - if (this.log == null) { + public synchronized WAL getLog(HRegionInfo info) throws IOException { + if (this.walFactory == null) { String logName = HConstants.HREGION_LOGDIR_NAME + "_" + System.currentTimeMillis(); - this.log = HLogFactory.createHLog(this.fs, this.fs.getHomeDirectory(), - logName, this.conf); + final Configuration walConf = new Configuration(this.conf); + FSUtils.setRootDir(walConf, fs.getHomeDirectory()); + this.walFactory = new WALFactory(walConf, null, logName); } - return this.log; + final byte[] region = info.getEncodedNameAsBytes(); + return info.isMetaRegion() ? walFactory.getMetaWAL(region) : walFactory.getWAL(region); } /** @@ -109,11 +111,11 @@ public class MetaUtils { } /** - * Closes catalog regions if open. Also closes and deletes the HLog. You + * Closes catalog regions if open. Also closes and deletes the WAL. You * must call this method if you want to persist changes made during a * MetaUtils edit session. */ - public void shutdown() { + public synchronized void shutdown() { if (this.metaRegion != null) { try { this.metaRegion.close(); @@ -134,14 +136,11 @@ public class MetaUtils { metaRegions.clear(); } try { - if (this.log != null) { - this.log.rollWriter(); - this.log.closeAndDelete(); + if (this.walFactory != null) { + this.walFactory.close(); } } catch (IOException e) { - LOG.error("closing HLog", e); - } finally { - this.log = null; + LOG.error("closing WAL", e); } } @@ -150,7 +149,7 @@ public class MetaUtils { return this.metaRegion; } this.metaRegion = HRegion.openHRegion(HRegionInfo.FIRST_META_REGIONINFO, - descriptors.get(TableName.META_TABLE_NAME), getLog(), + descriptors.get(TableName.META_TABLE_NAME), getLog(HRegionInfo.FIRST_META_REGIONINFO), this.conf); this.metaRegion.compactStores(); return this.metaRegion; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java index 0a75a1652d7..bffda57928c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java @@ -102,7 +102,7 @@ import com.google.common.collect.Sets; * profiling is much easier with manual splits. It is hard to trace the logs to * understand region level problems if it keeps splitting and getting renamed. *
  18. Data offlining bugs + unknown number of split regions == oh crap! If an - * HLog or StoreFile was mistakenly unprocessed by HBase due to a weird bug and + * WAL or StoreFile was mistakenly unprocessed by HBase due to a weird bug and * you notice it a day or so later, you can be assured that the regions * specified in these files are the same as the current regions and you have * less headaches trying to restore/replay your data. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java index 1bbfdb3b14e..9571e4d7653 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java @@ -157,7 +157,7 @@ public class ZKSplitLog { public static long parseLastFlushedSequenceIdFrom(final byte[] bytes) { long lastRecordedFlushedSequenceId = -1l; try { - lastRecordedFlushedSequenceId = ZKUtil.parseHLogPositionFrom(bytes); + lastRecordedFlushedSequenceId = ZKUtil.parseWALPositionFrom(bytes); } catch (DeserializationException e) { lastRecordedFlushedSequenceId = -1l; LOG.warn("Can't parse last flushed sequence Id", e); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java index d0553fc5b97..3705f3bfe8d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java @@ -150,7 +150,7 @@ public abstract class HBaseTestCase extends TestCase { /** * You must call close on the returned region and then close on the log file - * it created. Do {@link HRegion#close()} followed by {@link HRegion#getLog()} + * it created. Do {@link HRegion#close()} followed by {@link HRegion#getWAL()} * and on it call close. * @param desc * @param startKey diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 0c12b551ff8..aa319b73748 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -90,7 +90,7 @@ 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.RegionServerStoppedException; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.tool.Canary; import org.apache.hadoop.hbase.util.Bytes; @@ -573,7 +573,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { return this.dfsCluster; } - public MiniDFSCluster startMiniDFSClusterForTestHLog(int namenodePort) throws IOException { + public MiniDFSCluster startMiniDFSClusterForTestWAL(int namenodePort) throws IOException { createDirsAndSetProperties(); dfsCluster = new MiniDFSCluster(namenodePort, conf, 5, false, true, true, null, null, null, null); @@ -1651,18 +1651,18 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { } /** - * Create an HRegion that writes to the local tmp dirs with specified hlog + * Create an HRegion that writes to the local tmp dirs with specified wal * @param info regioninfo * @param desc table descriptor - * @param hlog hlog for this region. + * @param wal wal for this region. * @return created hregion * @throws IOException */ - public HRegion createLocalHRegion(HRegionInfo info, HTableDescriptor desc, HLog hlog) throws IOException { - return HRegion.createHRegion(info, getDataTestDir(), getConfiguration(), desc, hlog); + public HRegion createLocalHRegion(HRegionInfo info, HTableDescriptor desc, WAL wal) + throws IOException { + return HRegion.createHRegion(info, getDataTestDir(), getConfiguration(), desc, wal); } - /** * @param tableName * @param startKey @@ -1677,7 +1677,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 { + WAL wal, byte[]... families) throws IOException { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); htd.setReadOnly(isReadOnly); for (byte[] family : families) { @@ -1688,7 +1688,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { } htd.setDurability(durability); HRegionInfo info = new HRegionInfo(htd.getTableName(), startKey, stopKey, false); - return createLocalHRegion(info, htd, hlog); + return createLocalHRegion(info, htd, wal); } // // ========================================================================== @@ -1919,7 +1919,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { Put put = new Put(k); put.setDurability(Durability.SKIP_WAL); put.add(f, null, k); - if (r.getLog() == null) put.setDurability(Durability.SKIP_WAL); + if (r.getWAL() == null) put.setDurability(Durability.SKIP_WAL); int preRowCount = rowCount; int pause = 10; @@ -2979,7 +2979,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * Set maxRecoveryErrorCount in DFSClient. In 0.20 pre-append its hard-coded to 5 and * makes tests linger. Here is the exception you'll see: *
    -   * 2010-06-15 11:52:28,511 WARN  [DataStreamer for file /hbase/.logs/hlog.1276627923013 block blk_928005470262850423_1021] hdfs.DFSClient$DFSOutputStream(2657): Error Recovery for block blk_928005470262850423_1021 failed  because recovery from primary datanode 127.0.0.1:53683 failed 4 times.  Pipeline was 127.0.0.1:53687, 127.0.0.1:53683. Will retry...
    +   * 2010-06-15 11:52:28,511 WARN  [DataStreamer for file /hbase/.logs/wal.1276627923013 block blk_928005470262850423_1021] hdfs.DFSClient$DFSOutputStream(2657): Error Recovery for block blk_928005470262850423_1021 failed  because recovery from primary datanode 127.0.0.1:53683 failed 4 times.  Pipeline was 127.0.0.1:53687, 127.0.0.1:53683. Will retry...
        * 
    * @param stream A DFSClient.DFSOutputStream. * @param max diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java index 0078ebf9b88..d1ac1fdace0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java @@ -42,7 +42,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.wal.WAL; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; @@ -218,7 +218,7 @@ class MockRegionServerServices implements RegionServerServices { } @Override - public HLog getWAL(HRegionInfo regionInfo) throws IOException { + public WAL getWAL(HRegionInfo regionInfo) throws IOException { return null; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java index e19b8fef277..76bc94b998f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java @@ -44,8 +44,8 @@ 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.HLogUtil; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; @@ -90,7 +90,6 @@ public class TestIOFencing { //((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); } public abstract static class CompactionBlockerRegion extends HRegion { @@ -99,7 +98,7 @@ public class TestIOFencing { volatile CountDownLatch compactionsWaiting = new CountDownLatch(0); @SuppressWarnings("deprecation") - public CompactionBlockerRegion(Path tableDir, HLog log, + public CompactionBlockerRegion(Path tableDir, WAL log, FileSystem fs, Configuration confParam, HRegionInfo info, HTableDescriptor htd, RegionServerServices rsServices) { super(tableDir, log, fs, confParam, info, htd, rsServices); @@ -146,7 +145,7 @@ public class TestIOFencing { */ public static class BlockCompactionsInPrepRegion extends CompactionBlockerRegion { - public BlockCompactionsInPrepRegion(Path tableDir, HLog log, + public BlockCompactionsInPrepRegion(Path tableDir, WAL log, FileSystem fs, Configuration confParam, HRegionInfo info, HTableDescriptor htd, RegionServerServices rsServices) { super(tableDir, log, fs, confParam, info, htd, rsServices); @@ -169,7 +168,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, WAL log, FileSystem fs, Configuration confParam, HRegionInfo info, HTableDescriptor htd, RegionServerServices rsServices) { super(tableDir, log, fs, confParam, info, htd, rsServices); @@ -285,7 +284,7 @@ public class TestIOFencing { CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(oldHri, FAMILY, Lists.newArrayList(new Path("/a")), Lists.newArrayList(new Path("/b")), new Path("store_dir")); - HLogUtil.writeCompactionMarker(compactingRegion.getLog(), table.getTableDescriptor(), + WALUtil.writeCompactionMarker(compactingRegion.getWAL(), table.getTableDescriptor(), oldHri, compactionDescriptor, new AtomicLong(Long.MAX_VALUE-100)); // Wait till flush has happened, otherwise there won't be multiple store files diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java index 8e6518b59d8..cbc97e17228 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java @@ -437,7 +437,7 @@ public class TestHFileArchiving { // remove all the non-storefile named files for the region for (int i = 0; i < storeFiles.size(); i++) { String file = storeFiles.get(i); - if (file.contains(HRegionFileSystem.REGION_INFO_FILE) || file.contains("hlog")) { + if (file.contains(HRegionFileSystem.REGION_INFO_FILE) || file.contains("wal")) { storeFiles.remove(i--); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index 396f8ea43a1..9daf6856f0b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -53,11 +53,11 @@ import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtilsForTests; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; @@ -535,7 +535,7 @@ public class TestAdmin2 { } @Test (timeout=300000) - public void testHLogRollWriting() throws Exception { + public void testWALRollWriting() throws Exception { setUpforLogRolling(); String className = this.getClass().getName(); StringBuilder v = new StringBuilder(className); @@ -545,7 +545,7 @@ public class TestAdmin2 { byte[] value = Bytes.toBytes(v.toString()); HRegionServer regionServer = startAndWriteData(TableName.valueOf("TestLogRolling"), value); LOG.info("after writing there are " - + HLogUtilsForTests.getNumRolledLogFiles(regionServer.getWAL()) + " log files"); + + DefaultWALProvider.getNumRolledLogFiles(regionServer.getWAL(null)) + " log files"); // flush all regions @@ -554,8 +554,8 @@ public class TestAdmin2 { for (HRegion r : regions) { r.flushcache(); } - admin.rollHLogWriter(regionServer.getServerName().getServerName()); - int count = HLogUtilsForTests.getNumRolledLogFiles(regionServer.getWAL()); + admin.rollWALWriter(regionServer.getServerName()); + int count = DefaultWALProvider.getNumRolledLogFiles(regionServer.getWAL(null)); LOG.info("after flushing all regions and rolling logs there are " + count + " log files"); assertTrue(("actual count: " + count), count <= 2); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALObserver.java index 3f983ed161d..d7852f17f8e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALObserver.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; @@ -60,6 +61,12 @@ implements WALObserver { private boolean preWALRestoreCalled = false; private boolean postWALRestoreCalled = false; + // Deprecated versions + private boolean preWALWriteDeprecatedCalled = false; + private boolean postWALWriteDeprecatedCalled = false; + private boolean preWALRestoreDeprecatedCalled = false; + private boolean postWALRestoreDeprecatedCalled = false; + /** * Set values: with a table name, a column name which will be ignored, and * a column name which will be added to WAL. @@ -74,18 +81,32 @@ implements WALObserver { this.addedQualifier = addq; this.changedFamily = chf; this.changedQualifier = chq; + preWALWriteCalled = false; + postWALWriteCalled = false; + preWALRestoreCalled = false; + postWALRestoreCalled = false; + preWALWriteDeprecatedCalled = false; + postWALWriteDeprecatedCalled = false; + preWALRestoreDeprecatedCalled = false; + postWALRestoreDeprecatedCalled = false; } - @Override - public void postWALWrite(ObserverContext env, - HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException { + public void postWALWrite(ObserverContext env, + HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException { postWALWriteCalled = true; } @Override - public boolean preWALWrite(ObserverContext env, + public void postWALWrite(ObserverContext env, HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException { + postWALWriteDeprecatedCalled = true; + postWALWrite(env, info, (WALKey)logKey, logEdit); + } + + @Override + public boolean preWALWrite(ObserverContext env, + HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException { boolean bypass = false; // check table name matches or not. if (!Bytes.equals(info.getTableName(), this.tableName)) { @@ -122,24 +143,45 @@ implements WALObserver { return bypass; } + @Override + public boolean preWALWrite(ObserverContext env, + HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException { + preWALWriteDeprecatedCalled = true; + return preWALWrite(env, info, (WALKey)logKey, logEdit); + } + /** * Triggered before {@link org.apache.hadoop.hbase.regionserver.HRegion} when WAL is * Restoreed. */ + @Override + public void preWALRestore(ObserverContext env, + HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException { + preWALRestoreCalled = true; + } + @Override public void preWALRestore(ObserverContext env, HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException { - preWALRestoreCalled = true; + preWALRestoreDeprecatedCalled = true; + preWALRestore(env, info, (WALKey)logKey, logEdit); } /** * Triggered after {@link org.apache.hadoop.hbase.regionserver.HRegion} when WAL is * Restoreed. */ + @Override + public void postWALRestore(ObserverContext env, + HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException { + postWALRestoreCalled = true; + } + @Override public void postWALRestore(ObserverContext env, HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException { - postWALRestoreCalled = true; + postWALRestoreDeprecatedCalled = true; + postWALRestore(env, info, (WALKey)logKey, logEdit); } public boolean isPreWALWriteCalled() { @@ -161,4 +203,27 @@ implements WALObserver { ".isPostWALRestoreCalled is called."); return postWALRestoreCalled; } + + public boolean isPreWALWriteDeprecatedCalled() { + return preWALWriteDeprecatedCalled; + } + + public boolean isPostWALWriteDeprecatedCalled() { + return postWALWriteDeprecatedCalled; + } + + public boolean isPreWALRestoreDeprecatedCalled() { + return preWALRestoreDeprecatedCalled; + } + + public boolean isPostWALRestoreDeprecatedCalled() { + return postWALRestoreDeprecatedCalled; + } + + /** + * This class should trigger our legacy support since it does not directly implement the + * newer API methods. + */ + static class Legacy extends SampleRegionWALObserver { + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java index bf5351820c5..7100ae7c4fd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java @@ -66,6 +66,7 @@ import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFile.Reader; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; @@ -115,8 +116,6 @@ public class SimpleRegionObserver extends BaseRegionObserver { final AtomicInteger ctPreCheckAndDelete = new AtomicInteger(0); final AtomicInteger ctPreCheckAndDeleteAfterRowLock = new AtomicInteger(0); final AtomicInteger ctPostCheckAndDelete = new AtomicInteger(0); - final AtomicInteger ctPreWALRestored = new AtomicInteger(0); - final AtomicInteger ctPostWALRestored = new AtomicInteger(0); final AtomicInteger ctPreScannerNext = new AtomicInteger(0); final AtomicInteger ctPostScannerNext = new AtomicInteger(0); final AtomicInteger ctPreScannerClose = new AtomicInteger(0); @@ -130,6 +129,8 @@ public class SimpleRegionObserver extends BaseRegionObserver { final AtomicInteger ctPostBatchMutate = new AtomicInteger(0); final AtomicInteger ctPreWALRestore = new AtomicInteger(0); final AtomicInteger ctPostWALRestore = new AtomicInteger(0); + final AtomicInteger ctPreWALRestoreDeprecated = new AtomicInteger(0); + final AtomicInteger ctPostWALRestoreDeprecated = new AtomicInteger(0); final AtomicInteger ctPreSplitBeforePONR = new AtomicInteger(0); final AtomicInteger ctPreSplitAfterPONR = new AtomicInteger(0); final AtomicInteger ctPreStoreFileReaderOpen = new AtomicInteger(0); @@ -661,8 +662,8 @@ public class SimpleRegionObserver extends BaseRegionObserver { } @Override - public void preWALRestore(ObserverContext env, HRegionInfo info, - HLogKey logKey, WALEdit logEdit) throws IOException { + public void preWALRestore(ObserverContext env, + HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException { String tableName = logKey.getTablename().getNameAsString(); if (tableName.equals(TABLE_SKIPPED)) { // skip recovery of TABLE_SKIPPED for testing purpose @@ -672,10 +673,24 @@ public class SimpleRegionObserver extends BaseRegionObserver { ctPreWALRestore.incrementAndGet(); } + @Override + public void preWALRestore(ObserverContext env, HRegionInfo info, + HLogKey logKey, WALEdit logEdit) throws IOException { + preWALRestore(env, info, (WALKey)logKey, logEdit); + ctPreWALRestoreDeprecated.incrementAndGet(); + } + + @Override + public void postWALRestore(ObserverContext env, + HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException { + ctPostWALRestore.incrementAndGet(); + } + @Override public void postWALRestore(ObserverContext env, HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException { - ctPostWALRestore.incrementAndGet(); + postWALRestore(env, info, (WALKey)logKey, logEdit); + ctPostWALRestoreDeprecated.incrementAndGet(); } @Override @@ -794,13 +809,14 @@ public class SimpleRegionObserver extends BaseRegionObserver { return ctPrePrepareDeleteTS.get() > 0; } - public boolean hadPreWALRestored() { - return ctPreWALRestored.get() > 0; + public boolean hadPreWALRestore() { + return ctPreWALRestore.get() > 0; } - public boolean hadPostWALRestored() { - return ctPostWALRestored.get() > 0; + public boolean hadPostWALRestore() { + return ctPostWALRestore.get() > 0; } + public boolean wasScannerNextCalled() { return ctPreScannerNext.get() > 0 && ctPostScannerNext.get() > 0; } @@ -939,7 +955,22 @@ public class SimpleRegionObserver extends BaseRegionObserver { return ctPostWALRestore.get(); } + public int getCtPreWALRestoreDeprecated() { + return ctPreWALRestoreDeprecated.get(); + } + + public int getCtPostWALRestoreDeprecated() { + return ctPostWALRestoreDeprecated.get(); + } + public boolean wasStoreFileReaderOpenCalled() { return ctPreStoreFileReaderOpen.get() > 0 && ctPostStoreFileReaderOpen.get() > 0; } + + /** + * This implementation should trigger our legacy support because it does not directly + * implement the newer API calls. + */ + public static class Legacy extends SimpleRegionObserver { + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java index 4890eddc188..c747f0e32e0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java @@ -99,8 +99,9 @@ public class TestRegionObserverInterface { // set configure to indicate which cp should be loaded Configuration conf = util.getConfiguration(); conf.setBoolean("hbase.master.distributed.log.replay", true); - conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, - "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver"); + conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, + "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver", + "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver$Legacy"); util.startMiniCluster(); cluster = util.getMiniHBaseCluster(); @@ -619,9 +620,10 @@ public class TestRegionObserverInterface { ); verifyMethodResult(SimpleRegionObserver.class, - new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut", "getCtPostPut"}, + new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut", "getCtPostPut", + "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"}, tableName, - new Integer[] {0, 0, 1, 1}); + new Integer[] {0, 0, 1, 1, 0, 0}); cluster.killRegionServer(rs1.getRegionServer().getServerName()); Threads.sleep(1000); // Let the kill soak in. @@ -629,9 +631,60 @@ public class TestRegionObserverInterface { LOG.info("All regions assigned"); verifyMethodResult(SimpleRegionObserver.class, - new String[]{"getCtPrePut", "getCtPostPut"}, + new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut", "getCtPostPut", + "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"}, tableName, - new Integer[]{0, 0}); + new Integer[]{1, 1, 0, 0, 0, 0}); + } finally { + util.deleteTable(tableName); + table.close(); + } + } + + @Test + public void testLegacyRecovery() throws Exception { + LOG.info(TestRegionObserverInterface.class.getName() +".testLegacyRecovery"); + TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testLegacyRecovery"); + HTable table = util.createTable(tableName, new byte[][] {A, B, C}); + try { + JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer(); + ServerName sn2 = rs1.getRegionServer().getServerName(); + String regEN = table.getRegionLocations().firstEntry().getKey().getEncodedName(); + + util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes()); + while (!sn2.equals(table.getRegionLocations().firstEntry().getValue() )){ + Thread.sleep(100); + } + + Put put = new Put(ROW); + put.add(A, A, A); + put.add(B, B, B); + put.add(C, C, C); + table.put(put); + + verifyMethodResult(SimpleRegionObserver.Legacy.class, + new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut", + "hadPreBatchMutate", "hadPostBatchMutate", "hadDelete"}, + tableName, + new Boolean[] {false, false, true, true, true, true, false} + ); + + verifyMethodResult(SimpleRegionObserver.Legacy.class, + new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut", "getCtPostPut", + "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"}, + tableName, + new Integer[] {0, 0, 1, 1, 0, 0}); + + cluster.killRegionServer(rs1.getRegionServer().getServerName()); + Threads.sleep(1000); // Let the kill soak in. + util.waitUntilAllRegionsAssigned(tableName); + LOG.info("All regions assigned"); + + verifyMethodResult(SimpleRegionObserver.Legacy.class, + new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut", "getCtPostPut", + "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"}, + tableName, + new Integer[]{1, 1, 0, 0, 1, 1}); } finally { util.deleteTable(tableName); table.close(); @@ -665,7 +718,9 @@ public class TestRegionObserverInterface { util.waitUntilAllRegionsAssigned(tableName); verifyMethodResult(SimpleRegionObserver.class, new String[] { "getCtPreWALRestore", - "getCtPostWALRestore" }, tableName, new Integer[] { 0, 0 }); + "getCtPostWALRestore", "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"}, + tableName, + new Integer[] {0, 0, 0, 0}); util.deleteTable(tableName); table.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java index 11bcda48506..76e3209738d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java +++ b/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.wal.WAL; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -217,7 +217,7 @@ public class TestRegionObserverScannerOpenHook { private static volatile CountDownLatch compactionStateChangeLatch = null; @SuppressWarnings("deprecation") - public CompactionCompletionNotifyingRegion(Path tableDir, HLog log, + public CompactionCompletionNotifyingRegion(Path tableDir, WAL log, FileSystem fs, Configuration confParam, HRegionInfo info, HTableDescriptor htd, RegionServerServices rsServices) { super(tableDir, log, fs, confParam, info, htd, rsServices); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java index 2bb56b5136e..cdcdeed391a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java @@ -48,11 +48,14 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; -import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; +import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -64,7 +67,9 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TestName; import org.junit.experimental.categories.Category; /** @@ -86,6 +91,9 @@ public class TestWALObserver { Bytes.toBytes("v2"), Bytes.toBytes("v3"), }; private static byte[] TEST_ROW = Bytes.toBytes("testRow"); + @Rule + public TestName currentTest = new TestName(); + private Configuration conf; private FileSystem fs; private Path dir; @@ -93,12 +101,13 @@ public class TestWALObserver { private String logName; private Path oldLogDir; private Path logDir; + private WALFactory wals; @BeforeClass public static void setupBeforeClass() throws Exception { Configuration conf = TEST_UTIL.getConfiguration(); - conf.set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY, - SampleRegionWALObserver.class.getName()); + conf.setStrings(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY, + SampleRegionWALObserver.class.getName(), SampleRegionWALObserver.Legacy.class.getName()); conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, SampleRegionWALObserver.class.getName()); conf.setBoolean("dfs.support.append", true); @@ -125,16 +134,25 @@ public class TestWALObserver { this.dir = new Path(this.hbaseRootDir, TestWALObserver.class.getName()); this.oldLogDir = new Path(this.hbaseRootDir, HConstants.HREGION_OLDLOGDIR_NAME); - this.logDir = new Path(this.hbaseRootDir, HConstants.HREGION_LOGDIR_NAME); + this.logDir = new Path(this.hbaseRootDir, + DefaultWALProvider.getWALDirectoryName(currentTest.getMethodName())); this.logName = HConstants.HREGION_LOGDIR_NAME; if (TEST_UTIL.getDFSCluster().getFileSystem().exists(this.hbaseRootDir)) { TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true); } + this.wals = new WALFactory(conf, null, currentTest.getMethodName()); } @After public void tearDown() throws Exception { + try { + wals.shutdown(); + } catch (IOException exception) { + // one of our tests splits out from under our wals. + LOG.warn("Ignoring failure to close wal factory. " + exception.getMessage()); + LOG.debug("details of failure to close wal factory.", exception); + } TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true); } @@ -145,7 +163,23 @@ public class TestWALObserver { */ @Test public void testWALObserverWriteToWAL() throws Exception { + final WAL log = wals.getWAL(UNSPECIFIED_REGION); + verifyWritesSeen(log, getCoprocessor(log, SampleRegionWALObserver.class), false); + } + /** + * Test WAL write behavior with WALObserver. The coprocessor monitors a + * WALEdit written to WAL, and ignore, modify, and add KeyValue's for the + * WALEdit. + */ + @Test + public void testLegacyWALObserverWriteToWAL() throws Exception { + final WAL log = wals.getWAL(UNSPECIFIED_REGION); + verifyWritesSeen(log, getCoprocessor(log, SampleRegionWALObserver.Legacy.class), true); + } + + private void verifyWritesSeen(final WAL log, final SampleRegionWALObserver cp, + final boolean seesLegacy) throws Exception { HRegionInfo hri = createBasic3FamilyHRegionInfo(Bytes.toString(TEST_TABLE)); final HTableDescriptor htd = createBasic3FamilyHTD(Bytes .toString(TEST_TABLE)); @@ -155,10 +189,6 @@ public class TestWALObserver { fs.mkdirs(new Path(basedir, hri.getEncodedName())); final AtomicLong sequenceId = new AtomicLong(0); - HLog log = HLogFactory.createHLog(this.fs, hbaseRootDir, - TestWALObserver.class.getName(), this.conf); - SampleRegionWALObserver cp = getCoprocessor(log); - // TEST_FAMILY[0] shall be removed from WALEdit. // TEST_FAMILY[1] value shall be changed. // TEST_FAMILY[2] shall be added to WALEdit, although it's not in the put. @@ -167,6 +197,8 @@ public class TestWALObserver { assertFalse(cp.isPreWALWriteCalled()); assertFalse(cp.isPostWALWriteCalled()); + assertFalse(cp.isPreWALWriteDeprecatedCalled()); + assertFalse(cp.isPostWALWriteDeprecatedCalled()); // TEST_FAMILY[2] is not in the put, however it shall be added by the tested // coprocessor. @@ -202,7 +234,10 @@ public class TestWALObserver { // it's where WAL write cp should occur. long now = EnvironmentEdgeManager.currentTime(); - log.append(hri, hri.getTable(), edit, now, htd, sequenceId); + // we use HLogKey here instead of WALKey directly to support legacy coprocessors. + long txid = log.append(htd, hri, new HLogKey(hri.getEncodedNameAsBytes(), hri.getTable(), now), + edit, sequenceId, true, null); + log.sync(txid); // the edit shall have been change now by the coprocessor. foundFamily0 = false; @@ -227,6 +262,83 @@ public class TestWALObserver { assertTrue(cp.isPreWALWriteCalled()); assertTrue(cp.isPostWALWriteCalled()); + assertEquals(seesLegacy, cp.isPreWALWriteDeprecatedCalled()); + assertEquals(seesLegacy, cp.isPostWALWriteDeprecatedCalled()); + } + + @Test + public void testNonLegacyWALKeysDoNotExplode() throws Exception { + TableName tableName = TableName.valueOf(TEST_TABLE); + final HTableDescriptor htd = createBasic3FamilyHTD(Bytes + .toString(TEST_TABLE)); + final HRegionInfo hri = new HRegionInfo(tableName, null, null); + final AtomicLong sequenceId = new AtomicLong(0); + + fs.mkdirs(new Path(FSUtils.getTableDir(hbaseRootDir, tableName), hri.getEncodedName())); + + final Configuration newConf = HBaseConfiguration.create(this.conf); + + final WAL wal = wals.getWAL(UNSPECIFIED_REGION); + final SampleRegionWALObserver newApi = getCoprocessor(wal, SampleRegionWALObserver.class); + newApi.setTestValues(TEST_TABLE, TEST_ROW, null, null, null, null, null, null); + final SampleRegionWALObserver oldApi = getCoprocessor(wal, + SampleRegionWALObserver.Legacy.class); + oldApi.setTestValues(TEST_TABLE, TEST_ROW, null, null, null, null, null, null); + + LOG.debug("ensuring wal entries haven't happened before we start"); + assertFalse(newApi.isPreWALWriteCalled()); + assertFalse(newApi.isPostWALWriteCalled()); + assertFalse(newApi.isPreWALWriteDeprecatedCalled()); + assertFalse(newApi.isPostWALWriteDeprecatedCalled()); + assertFalse(oldApi.isPreWALWriteCalled()); + assertFalse(oldApi.isPostWALWriteCalled()); + assertFalse(oldApi.isPreWALWriteDeprecatedCalled()); + assertFalse(oldApi.isPostWALWriteDeprecatedCalled()); + + LOG.debug("writing to WAL with non-legacy keys."); + final int countPerFamily = 5; + for (HColumnDescriptor hcd : htd.getFamilies()) { + addWALEdits(tableName, hri, TEST_ROW, hcd.getName(), countPerFamily, + EnvironmentEdgeManager.getDelegate(), wal, htd, sequenceId); + } + + LOG.debug("Verify that only the non-legacy CP saw edits."); + assertTrue(newApi.isPreWALWriteCalled()); + assertTrue(newApi.isPostWALWriteCalled()); + assertFalse(newApi.isPreWALWriteDeprecatedCalled()); + assertFalse(newApi.isPostWALWriteDeprecatedCalled()); + // wish we could test that the log message happened :/ + assertFalse(oldApi.isPreWALWriteCalled()); + assertFalse(oldApi.isPostWALWriteCalled()); + assertFalse(oldApi.isPreWALWriteDeprecatedCalled()); + assertFalse(oldApi.isPostWALWriteDeprecatedCalled()); + + LOG.debug("reseting cp state."); + newApi.setTestValues(TEST_TABLE, TEST_ROW, null, null, null, null, null, null); + oldApi.setTestValues(TEST_TABLE, TEST_ROW, null, null, null, null, null, null); + + LOG.debug("write a log edit that supports legacy cps."); + final long now = EnvironmentEdgeManager.currentTime(); + final WALKey legacyKey = new HLogKey(hri.getEncodedNameAsBytes(), hri.getTable(), now); + final WALEdit edit = new WALEdit(); + final byte[] nonce = Bytes.toBytes("1772"); + edit.add(new KeyValue(TEST_ROW, TEST_FAMILY[0], nonce, now, nonce)); + final long txid = wal.append(htd, hri, legacyKey, edit, sequenceId, true, null); + wal.sync(txid); + + LOG.debug("Make sure legacy cps can see supported edits after having been skipped."); + assertTrue("non-legacy WALObserver didn't see pre-write.", newApi.isPreWALWriteCalled()); + assertTrue("non-legacy WALObserver didn't see post-write.", newApi.isPostWALWriteCalled()); + assertFalse("non-legacy WALObserver shouldn't have seen legacy pre-write.", + newApi.isPreWALWriteDeprecatedCalled()); + assertFalse("non-legacy WALObserver shouldn't have seen legacy post-write.", + newApi.isPostWALWriteDeprecatedCalled()); + assertTrue("legacy WALObserver didn't see pre-write.", oldApi.isPreWALWriteCalled()); + assertTrue("legacy WALObserver didn't see post-write.", oldApi.isPostWALWriteCalled()); + assertTrue("legacy WALObserver didn't see legacy pre-write.", + oldApi.isPreWALWriteDeprecatedCalled()); + assertTrue("legacy WALObserver didn't see legacy post-write.", + oldApi.isPostWALWriteDeprecatedCalled()); } /** @@ -238,10 +350,9 @@ public class TestWALObserver { final HTableDescriptor htd = createBasic3FamilyHTD(Bytes.toString(TEST_TABLE)); final AtomicLong sequenceId = new AtomicLong(0); - HLog log = HLogFactory.createHLog(this.fs, hbaseRootDir, - TestWALObserver.class.getName(), this.conf); + WAL log = wals.getWAL(UNSPECIFIED_REGION); try { - SampleRegionWALObserver cp = getCoprocessor(log); + SampleRegionWALObserver cp = getCoprocessor(log, SampleRegionWALObserver.class); cp.setTestValues(TEST_TABLE, null, null, null, null, null, null, null); @@ -249,13 +360,14 @@ public class TestWALObserver { assertFalse(cp.isPostWALWriteCalled()); final long now = EnvironmentEdgeManager.currentTime(); - log.append(hri, hri.getTable(), new WALEdit(), now, htd, sequenceId); - log.sync(); + long txid = log.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), hri.getTable(), now), + new WALEdit(), sequenceId, true, null); + log.sync(txid); assertFalse("Empty WALEdit should skip coprocessor evaluation.", cp.isPreWALWriteCalled()); assertFalse("Empty WALEdit should skip coprocessor evaluation.", cp.isPostWALWriteCalled()); } finally { - log.closeAndDelete(); + log.close(); } } @@ -282,8 +394,8 @@ 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); + // WAL wal = new WAL(this.fs, this.dir, this.oldLogDir, this.conf); + WAL wal = wals.getWAL(UNSPECIFIED_REGION); // Put p = creatPutWith2Families(TEST_ROW); WALEdit edit = new WALEdit(); long now = EnvironmentEdgeManager.currentTime(); @@ -291,12 +403,11 @@ public class TestWALObserver { final int countPerFamily = 1000; // for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) { for (HColumnDescriptor hcd : htd.getFamilies()) { - // addWALEdits(tableName, hri, TEST_ROW, hcd.getName(), countPerFamily, - // EnvironmentEdgeManager.getDelegate(), wal); addWALEdits(tableName, hri, TEST_ROW, hcd.getName(), countPerFamily, EnvironmentEdgeManager.getDelegate(), wal, htd, sequenceId); } - wal.append(hri, tableName, edit, now, htd, sequenceId); + wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now), edit, sequenceId, + true, null); // sync to fs. wal.sync(); @@ -308,7 +419,8 @@ 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); + final WALFactory wals2 = new WALFactory(conf, null, currentTest.getMethodName()+"2"); + WAL wal2 = wals2.getWAL(UNSPECIFIED_REGION);; HRegion region = HRegion.openHRegion(newConf, FileSystem.get(newConf), hbaseRootDir, hri, htd, wal2, TEST_UTIL.getHBaseCluster().getRegionServer(0), null); long seqid2 = region.getOpenSeqNum(); @@ -320,8 +432,10 @@ public class TestWALObserver { assertNotNull(cp2); assertTrue(cp2.isPreWALRestoreCalled()); assertTrue(cp2.isPostWALRestoreCalled()); + assertFalse(cp2.isPreWALRestoreDeprecatedCalled()); + assertFalse(cp2.isPostWALRestoreDeprecatedCalled()); region.close(); - wal2.closeAndDelete(); + wals2.close(); return null; } }); @@ -330,19 +444,18 @@ public class TestWALObserver { /** * Test to see CP loaded successfully or not. There is a duplication at * TestHLog, but the purpose of that one is to see whether the loaded CP will - * impact existing HLog tests or not. + * impact existing WAL tests or not. */ @Test public void testWALObserverLoaded() throws Exception { - HLog log = HLogFactory.createHLog(fs, hbaseRootDir, - TestWALObserver.class.getName(), conf); - assertNotNull(getCoprocessor(log)); + WAL log = wals.getWAL(UNSPECIFIED_REGION); + assertNotNull(getCoprocessor(log, SampleRegionWALObserver.class)); } - private SampleRegionWALObserver getCoprocessor(HLog wal) throws Exception { + private SampleRegionWALObserver getCoprocessor(WAL wal, + Class clazz) throws Exception { WALCoprocessorHost host = wal.getCoprocessorHost(); - Coprocessor c = host.findCoprocessor(SampleRegionWALObserver.class - .getName()); + Coprocessor c = host.findCoprocessor(clazz.getName()); return (SampleRegionWALObserver) c; } @@ -400,8 +513,8 @@ public class TestWALObserver { } private Path runWALSplit(final Configuration c) throws IOException { - List splits = HLogSplitter.split( - hbaseRootDir, logDir, oldLogDir, FileSystem.get(c), c); + List splits = WALSplitter.split( + hbaseRootDir, logDir, oldLogDir, FileSystem.get(c), c, wals); // Split should generate only 1 file since there's only 1 region assertEquals(1, splits.size()); // Make sure the file exists @@ -410,21 +523,25 @@ public class TestWALObserver { return splits.get(0); } - private HLog createWAL(final Configuration c) throws IOException { - return HLogFactory.createHLog(FileSystem.get(c), hbaseRootDir, logName, c); - } + private static final byte[] UNSPECIFIED_REGION = new byte[]{}; - 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 WAL wal, + final HTableDescriptor htd, final AtomicLong sequenceId) throws IOException { String familyStr = Bytes.toString(family); + long txid = -1; for (int j = 0; j < count; j++) { byte[] qualifierBytes = Bytes.toBytes(Integer.toString(j)); byte[] columnBytes = Bytes.toBytes(familyStr + ":" + Integer.toString(j)); WALEdit edit = new WALEdit(); edit.add(new KeyValue(rowName, family, qualifierBytes, ee.currentTime(), columnBytes)); - wal.append(hri, tableName, edit, ee.currentTime(), htd, sequenceId); + // uses WALKey instead of HLogKey on purpose. will only work for tests where we don't care + // about legacy coprocessors + txid = wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, + ee.currentTime()), edit, sequenceId, true, null); + } + if (-1 != txid) { + wal.sync(txid); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java index c6489073064..33965873394 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java @@ -48,7 +48,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.wal.WAL; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -217,9 +217,9 @@ public class TestFilter { @After public void tearDown() throws Exception { - HLog hlog = region.getLog(); + WAL wal = region.getWAL(); region.close(); - hlog.closeAndDelete(); + wal.close(); } @Test @@ -1488,9 +1488,9 @@ public class TestFilter { assertEquals(2, resultCount); scanner.close(); - HLog hlog = testRegion.getLog(); + WAL wal = testRegion.getWAL(); testRegion.close(); - hlog.closeAndDelete(); + wal.close(); } @Test @@ -2096,8 +2096,8 @@ public class TestFilter { results.clear(); } assertFalse(scanner.next(results)); - HLog hlog = testRegion.getLog(); + WAL wal = testRegion.getWAL(); testRegion.close(); - hlog.closeAndDelete(); + wal.close(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java index a44125df25b..70ef51fc325 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java @@ -34,7 +34,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.wal.WAL; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -150,9 +150,9 @@ public class TestInvocationRecordFilter { @After public void tearDown() throws Exception { - HLog hlog = region.getLog(); + WAL wal = region.getWAL(); region.close(); - hlog.closeAndDelete(); + wal.close(); } /** @@ -180,4 +180,4 @@ public class TestInvocationRecordFilter { return true; } } -} \ No newline at end of file +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java index 157b77ea9eb..db751b22fd4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java @@ -18,12 +18,15 @@ package org.apache.hadoop.hbase.fs; - +import java.io.FileNotFoundException; +import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.net.BindException; import java.net.ServerSocket; +import java.util.List; +import java.util.concurrent.CountDownLatch; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -35,14 +38,17 @@ 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.ipc.RemoteException; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.FSUtils; @@ -278,7 +284,32 @@ public class TestBlockReorder { int nbTest = 0; while (nbTest < 10) { - htu.getHBaseAdmin().rollHLogWriter(targetRs.getServerName().toString()); + final List regions = targetRs.getOnlineRegions(h.getName()); + final CountDownLatch latch = new CountDownLatch(regions.size()); + // listen for successful log rolls + final WALActionsListener listener = new WALActionsListener.Base() { + @Override + public void postLogRoll(final Path oldPath, final Path newPath) throws IOException { + latch.countDown(); + } + }; + for (HRegion region : regions) { + region.getWAL().registerWALActionsListener(listener); + } + + htu.getHBaseAdmin().rollWALWriter(targetRs.getServerName()); + + // wait + try { + latch.await(); + } catch (InterruptedException exception) { + LOG.warn("Interrupted while waiting for the wal of '" + targetRs + "' to roll. If later " + + "tests fail, it's probably because we should still be waiting."); + Thread.currentThread().interrupt(); + } + for (HRegion region : regions) { + region.getWAL().unregisterWALActionsListener(listener); + } // We need a sleep as the namenode is informed asynchronously Thread.sleep(100); @@ -294,37 +325,52 @@ public class TestBlockReorder { // As we wrote a put, we should have at least one log file. Assert.assertTrue(hfs.length >= 1); for (HdfsFileStatus hf : hfs) { - LOG.info("Log file found: " + hf.getLocalName() + " in " + rootDir); - String logFile = rootDir + "/" + hf.getLocalName(); - FileStatus fsLog = rfs.getFileStatus(new Path(logFile)); + // Because this is a live cluster, log files might get archived while we're processing + try { + LOG.info("Log file found: " + hf.getLocalName() + " in " + rootDir); + String logFile = rootDir + "/" + hf.getLocalName(); + FileStatus fsLog = rfs.getFileStatus(new Path(logFile)); - LOG.info("Checking log file: " + logFile); - // Now checking that the hook is up and running - // We can't call directly getBlockLocations, it's not available in HFileSystem - // We're trying multiple times to be sure, as the order is random + LOG.info("Checking log file: " + logFile); + // Now checking that the hook is up and running + // We can't call directly getBlockLocations, it's not available in HFileSystem + // We're trying multiple times to be sure, as the order is random - BlockLocation[] bls = rfs.getFileBlockLocations(fsLog, 0, 1); - if (bls.length > 0) { - BlockLocation bl = bls[0]; + BlockLocation[] bls = rfs.getFileBlockLocations(fsLog, 0, 1); + if (bls.length > 0) { + BlockLocation bl = bls[0]; - LOG.info(bl.getHosts().length + " replicas for block 0 in " + logFile + " "); - for (int i = 0; i < bl.getHosts().length - 1; i++) { - LOG.info(bl.getHosts()[i] + " " + logFile); - Assert.assertNotSame(bl.getHosts()[i], host4); - } - String last = bl.getHosts()[bl.getHosts().length - 1]; - LOG.info(last + " " + logFile); - if (host4.equals(last)) { - nbTest++; - LOG.info(logFile + " is on the new datanode and is ok"); - if (bl.getHosts().length == 3) { - // We can test this case from the file system as well - // Checking the underlying file system. Multiple times as the order is random - testFromDFS(dfs, logFile, repCount, host4); - - // now from the master - testFromDFS(mdfs, logFile, repCount, host4); + LOG.info(bl.getHosts().length + " replicas for block 0 in " + logFile + " "); + for (int i = 0; i < bl.getHosts().length - 1; i++) { + LOG.info(bl.getHosts()[i] + " " + logFile); + Assert.assertNotSame(bl.getHosts()[i], host4); } + String last = bl.getHosts()[bl.getHosts().length - 1]; + LOG.info(last + " " + logFile); + if (host4.equals(last)) { + nbTest++; + LOG.info(logFile + " is on the new datanode and is ok"); + if (bl.getHosts().length == 3) { + // We can test this case from the file system as well + // Checking the underlying file system. Multiple times as the order is random + testFromDFS(dfs, logFile, repCount, host4); + + // now from the master + testFromDFS(mdfs, logFile, repCount, host4); + } + } + } + } catch (FileNotFoundException exception) { + LOG.debug("Failed to find log file '" + hf.getLocalName() + "'; it probably was " + + "archived out from under us so we'll ignore and retry. If this test hangs " + + "indefinitely you should treat this failure as a symptom.", exception); + } catch (RemoteException exception) { + if (exception.unwrapRemoteException() instanceof FileNotFoundException) { + LOG.debug("Failed to find log file '" + hf.getLocalName() + "'; it probably was " + + "archived out from under us so we'll ignore and retry. If this test hangs " + + "indefinitely you should treat this failure as a symptom.", exception); + } else { + throw exception; } } } @@ -414,7 +460,7 @@ public class TestBlockReorder { // Check that it will be possible to extract a ServerName from our construction Assert.assertNotNull("log= " + pseudoLogFile, - HLogUtil.getServerNameFromHLogDirectoryName(dfs.getConf(), pseudoLogFile)); + DefaultWALProvider.getServerNameFromWALDirectoryName(dfs.getConf(), pseudoLogFile)); // And check we're doing the right reorder. lrb.reorderBlocks(conf, l, pseudoLogFile); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java index de842926670..d6423e80f5a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java @@ -67,7 +67,7 @@ public class TestHeapSize { static final Log LOG = LogFactory.getLog(TestHeapSize.class); // List of classes implementing HeapSize // BatchOperation, BatchUpdate, BlockIndex, Entry, Entry, HStoreKey - // KeyValue, LruBlockCache, LruHashMap, Put, HLogKey + // KeyValue, LruBlockCache, LruHashMap, Put, WALKey @BeforeClass public static void beforeClass() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java index bfe5aa7eda4..d82f36bae20 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java @@ -17,233 +17,27 @@ */ package org.apache.hadoop.hbase.mapreduce; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import java.util.List; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -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.Cell; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.mapreduce.HLogInputFormat.HLogRecordReader; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; -import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.mapreduce.WALInputFormat.WALRecordReader; +import org.apache.hadoop.hbase.mapreduce.HLogInputFormat.HLogKeyRecordReader; +import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.MapReduceTestUtil; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; import org.junit.experimental.categories.Category; /** - * JUnit tests for the HLogRecordReader + * JUnit tests for the record reader in HLogInputFormat */ @Category({MapReduceTests.class, MediumTests.class}) -public class TestHLogRecordReader { - private final Log LOG = LogFactory.getLog(getClass()); - private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static Configuration conf; - private static FileSystem fs; - private static Path hbaseDir; - private static final TableName tableName = - TableName.valueOf(getName()); - private static final byte [] rowName = tableName.getName(); - private static final HRegionInfo info = new HRegionInfo(tableName, - Bytes.toBytes(""), Bytes.toBytes(""), false); - private static final byte [] family = Bytes.toBytes("column"); - private static final byte [] value = Bytes.toBytes("value"); - private static HTableDescriptor htd; - private static Path logDir; - private static String logName; +public class TestHLogRecordReader extends TestWALRecordReader { - private static String getName() { - return "TestHLogRecordReader"; + @Override + protected WALKey getWalKey(final long sequenceid) { + return new HLogKey(info.getEncodedNameAsBytes(), tableName, sequenceid); } - @Before - public void setUp() throws Exception { - FileStatus[] entries = fs.listStatus(hbaseDir); - for (FileStatus dir : entries) { - fs.delete(dir.getPath(), true); - } - + @Override + protected WALRecordReader getReader() { + return new HLogKeyRecordReader(); } - @BeforeClass - public static void setUpBeforeClass() throws Exception { - // Make block sizes small. - conf = TEST_UTIL.getConfiguration(); - conf.setInt("dfs.blocksize", 1024 * 1024); - conf.setInt("dfs.replication", 1); - TEST_UTIL.startMiniDFSCluster(1); - - conf = TEST_UTIL.getConfiguration(); - fs = TEST_UTIL.getDFSCluster().getFileSystem(); - - hbaseDir = TEST_UTIL.createRootDir(); - - logName = HConstants.HREGION_LOGDIR_NAME; - logDir = new Path(hbaseDir, logName); - - htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(family)); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - TEST_UTIL.shutdownMiniCluster(); - } - - /** - * Test partial reads from the log based on passed time range - * @throws Exception - */ - @Test - public void testPartialRead() throws Exception { - HLog log = HLogFactory.createHLog(fs, hbaseDir, logName, conf); - // This test depends on timestamp being millisecond based and the filename of the WAL also - // being millisecond based. - long ts = System.currentTimeMillis(); - WALEdit edit = new WALEdit(); - final AtomicLong sequenceId = new AtomicLong(0); - edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"), ts, value)); - log.append(info, tableName, edit, ts, htd, sequenceId); - edit = new WALEdit(); - edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"), ts+1, value)); - log.append(info, tableName, edit, ts+1, htd, sequenceId); - LOG.info("Before 1st WAL roll " + log.getFilenum()); - log.rollWriter(); - LOG.info("Past 1st WAL roll " + log.getFilenum()); - - Thread.sleep(1); - long ts1 = System.currentTimeMillis(); - - edit = new WALEdit(); - edit.add(new KeyValue(rowName, family, Bytes.toBytes("3"), ts1+1, value)); - log.append(info, tableName, edit, ts1+1, htd, sequenceId); - edit = new WALEdit(); - edit.add(new KeyValue(rowName, family, Bytes.toBytes("4"), ts1+2, value)); - log.append(info, tableName, edit, ts1+2, htd, sequenceId); - log.close(); - LOG.info("Closed WAL " + log.getFilenum()); - - - HLogInputFormat input = new HLogInputFormat(); - Configuration jobConf = new Configuration(conf); - jobConf.set("mapreduce.input.fileinputformat.inputdir", logDir.toString()); - jobConf.setLong(HLogInputFormat.END_TIME_KEY, ts); - - // only 1st file is considered, and only its 1st entry is used - List splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf)); - - assertEquals(1, splits.size()); - testSplit(splits.get(0), Bytes.toBytes("1")); - - jobConf.setLong(HLogInputFormat.START_TIME_KEY, ts+1); - jobConf.setLong(HLogInputFormat.END_TIME_KEY, ts1+1); - splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf)); - // both files need to be considered - assertEquals(2, splits.size()); - // only the 2nd entry from the 1st file is used - testSplit(splits.get(0), Bytes.toBytes("2")); - // only the 1nd entry from the 2nd file is used - testSplit(splits.get(1), Bytes.toBytes("3")); - } - - /** - * Test basic functionality - * @throws Exception - */ - @Test - public void testHLogRecordReader() throws Exception { - HLog log = HLogFactory.createHLog(fs, hbaseDir, logName, conf); - byte [] value = Bytes.toBytes("value"); - final AtomicLong sequenceId = new AtomicLong(0); - WALEdit edit = new WALEdit(); - edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"), - System.currentTimeMillis(), value)); - log.append(info, tableName, edit, - System.currentTimeMillis(), htd, sequenceId); - - Thread.sleep(1); // make sure 2nd log gets a later timestamp - long secondTs = System.currentTimeMillis(); - log.rollWriter(); - - edit = new WALEdit(); - edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"), - System.currentTimeMillis(), value)); - log.append(info, tableName, edit, - System.currentTimeMillis(), htd, sequenceId); - log.close(); - long thirdTs = System.currentTimeMillis(); - - // should have 2 log files now - HLogInputFormat input = new HLogInputFormat(); - Configuration jobConf = new Configuration(conf); - jobConf.set("mapreduce.input.fileinputformat.inputdir", logDir.toString()); - - // make sure both logs are found - List splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf)); - assertEquals(2, splits.size()); - - // should return exactly one KV - testSplit(splits.get(0), Bytes.toBytes("1")); - // same for the 2nd split - testSplit(splits.get(1), Bytes.toBytes("2")); - - // now test basic time ranges: - - // set an endtime, the 2nd log file can be ignored completely. - jobConf.setLong(HLogInputFormat.END_TIME_KEY, secondTs-1); - splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf)); - assertEquals(1, splits.size()); - testSplit(splits.get(0), Bytes.toBytes("1")); - - // now set a start time - jobConf.setLong(HLogInputFormat.END_TIME_KEY, Long.MAX_VALUE); - jobConf.setLong(HLogInputFormat.START_TIME_KEY, thirdTs); - splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf)); - // both logs need to be considered - assertEquals(2, splits.size()); - // but both readers skip all edits - testSplit(splits.get(0)); - testSplit(splits.get(1)); - } - - /** - * Create a new reader from the split, and match the edits against the passed columns. - */ - private void testSplit(InputSplit split, byte[]... columns) throws Exception { - HLogRecordReader reader = new HLogRecordReader(); - reader.initialize(split, MapReduceTestUtil.createDummyMapTaskAttemptContext(conf)); - - for (byte[] column : columns) { - assertTrue(reader.nextKeyValue()); - Cell cell = reader.getCurrentValue().getCells().get(0); - if (!Bytes.equals(column, cell.getQualifier())) { - assertTrue("expected [" + Bytes.toString(column) + "], actual [" - + Bytes.toString(cell.getQualifier()) + "]", false); - } - } - assertFalse(reader.nextKeyValue()); - reader.close(); - } - } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java index 6f39ea54852..a2df10583eb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java @@ -58,10 +58,10 @@ import org.apache.hadoop.hbase.filter.FilterBase; import org.apache.hadoop.hbase.filter.PrefixFilter; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.Import.KeyValueImporter; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests; import org.apache.hadoop.hbase.util.Bytes; @@ -644,10 +644,10 @@ public class TestImportExport { String importTableName = "importTestDurability1"; Table importTable = UTIL.createTable(TableName.valueOf(importTableName), FAMILYA, 3); - // Register the hlog listener for the import table + // Register the wal listener for the import table TableWALActionListener walListener = new TableWALActionListener(importTableName); - HLog hLog = UTIL.getMiniHBaseCluster().getRegionServer(0).getWAL(); - hLog.registerWALActionsListener(walListener); + WAL wal = UTIL.getMiniHBaseCluster().getRegionServer(0).getWAL(null); + wal.registerWALActionsListener(walListener); // Run the import with SKIP_WAL args = @@ -662,9 +662,9 @@ public class TestImportExport { // Run the import with the default durability option importTableName = "importTestDurability2"; importTable = UTIL.createTable(TableName.valueOf(importTableName), FAMILYA, 3); - hLog.unregisterWALActionsListener(walListener); + wal.unregisterWALActionsListener(walListener); walListener = new TableWALActionListener(importTableName); - hLog.registerWALActionsListener(walListener); + wal.registerWALActionsListener(walListener); args = new String[] { importTableName, FQ_OUTPUT_DIR }; assertTrue(runImport(args)); //Assert that the wal is visisted @@ -674,10 +674,10 @@ public class TestImportExport { } /** - * This listens to the {@link #visitLogEntryBeforeWrite(HTableDescriptor, HLogKey, WALEdit)} to + * This listens to the {@link #visitLogEntryBeforeWrite(HTableDescriptor, WALKey, WALEdit)} to * identify that an entry is written to the Write Ahead Log for the given table. */ - private static class TableWALActionListener implements WALActionsListener { + private static class TableWALActionListener extends WALActionsListener.Base { private String tableName; private boolean isVisited = false; @@ -687,42 +687,7 @@ public class TestImportExport { } @Override - public void preLogRoll(Path oldPath, Path newPath) throws IOException { - // Not interested in this method. - } - - @Override - public void postLogRoll(Path oldPath, Path newPath) throws IOException { - // Not interested in this method. - } - - @Override - public void preLogArchive(Path oldPath, Path newPath) throws IOException { - // Not interested in this method. - } - - @Override - public void postLogArchive(Path oldPath, Path newPath) throws IOException { - // Not interested in this method. - } - - @Override - public void logRollRequested() { - // Not interested in this method. - } - - @Override - public void logCloseRequested() { - // Not interested in this method. - } - - @Override - public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit) { - // Not interested in this method. - } - - @Override - public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, WALEdit logEdit) { + public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit) { if (tableName.equalsIgnoreCase(htd.getNameAsString())) { isVisited = true; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceUtil.java index 8c0b26c9acd..303a144d16f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceUtil.java @@ -46,8 +46,8 @@ public class TestTableMapReduceUtil { Job job = new Job(configuration, "tableName"); // test TableMapReduceUtil.initTableMapperJob("Table", new Scan(), Import.Importer.class, Text.class, - Text.class, job, false, HLogInputFormat.class); - assertEquals(HLogInputFormat.class, job.getInputFormatClass()); + Text.class, job, false, WALInputFormat.class); + assertEquals(WALInputFormat.class, job.getInputFormatClass()); assertEquals(Import.Importer.class, job.getMapperClass()); assertEquals(LongWritable.class, job.getOutputKeyClass()); assertEquals(Text.class, job.getOutputValueClass()); @@ -60,8 +60,8 @@ public class TestTableMapReduceUtil { Configuration configuration = new Configuration(); Job job = new Job(configuration, "tableName"); TableMapReduceUtil.initTableMapperJob(Bytes.toBytes("Table"), new Scan(), - Import.Importer.class, Text.class, Text.class, job, false, HLogInputFormat.class); - assertEquals(HLogInputFormat.class, job.getInputFormatClass()); + Import.Importer.class, Text.class, Text.class, job, false, WALInputFormat.class); + assertEquals(WALInputFormat.class, job.getInputFormatClass()); assertEquals(Import.Importer.class, job.getMapperClass()); assertEquals(LongWritable.class, job.getOutputKeyClass()); assertEquals(Text.class, job.getOutputValueClass()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java index bde3bc6fc29..68cf8ba9c04 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java @@ -44,9 +44,9 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; 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.HLogKey; +import org.apache.hadoop.hbase.mapreduce.WALPlayer.WALKeyValueMapper; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests; @@ -107,7 +107,7 @@ public class TestWALPlayer { t1.delete(d); // replay the WAL, map table 1 to table 2 - HLog log = cluster.getRegionServer(0).getWAL(); + WAL log = cluster.getRegionServer(0).getWAL(null); log.rollWriter(); String walInputDir = new Path(cluster.getMaster().getMasterFileSystem() .getRootDir(), HConstants.HREGION_LOGDIR_NAME).toString(); @@ -130,17 +130,26 @@ public class TestWALPlayer { } /** - * Test HLogKeyValueMapper setup and map + * Test WALKeyValueMapper setup and map */ @Test - public void testHLogKeyValueMapper() throws Exception { + public void testWALKeyValueMapper() throws Exception { + testWALKeyValueMapper(WALPlayer.TABLES_KEY); + } + + @Test + public void testWALKeyValueMapperWithDeprecatedConfig() throws Exception { + testWALKeyValueMapper("hlog.input.tables"); + } + + private void testWALKeyValueMapper(final String tableConfigKey) throws Exception { Configuration configuration = new Configuration(); - configuration.set(WALPlayer.TABLES_KEY, "table"); - HLogKeyValueMapper mapper = new HLogKeyValueMapper(); - HLogKey key = mock(HLogKey.class); + configuration.set(tableConfigKey, "table"); + WALKeyValueMapper mapper = new WALKeyValueMapper(); + WALKey key = mock(WALKey.class); when(key.getTablename()).thenReturn(TableName.valueOf("table")); @SuppressWarnings("unchecked") - Mapper.Context context = + Mapper.Context context = mock(Context.class); when(context.getConfiguration()).thenReturn(configuration); @@ -192,7 +201,7 @@ public class TestWALPlayer { assertTrue(data.toString().contains("ERROR: Wrong number of arguments:")); assertTrue(data.toString().contains("Usage: WALPlayer [options] " + " []")); - assertTrue(data.toString().contains("-Dhlog.bulk.output=/path/for/output")); + assertTrue(data.toString().contains("-Dwal.bulk.output=/path/for/output")); } } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java index 98b59bb711c..5d8cd19cc5e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java @@ -98,7 +98,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.wal.WAL; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; @@ -539,7 +539,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { } @Override - public HLog getWAL(HRegionInfo regionInfo) throws IOException { + public WAL getWAL(HRegionInfo regionInfo) throws IOException { // TODO Auto-generated method stub return null; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java index bc065210551..a16d7e5888d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java @@ -89,10 +89,13 @@ 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.HLogFactory; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.Bytes; @@ -234,7 +237,7 @@ public class TestDistributedLogSplitting { } if (foundRs) break; } - final Path logDir = new Path(rootdir, HLogUtil.getHLogDirectoryName(hrs + final Path logDir = new Path(rootdir, DefaultWALProvider.getWALDirectoryName(hrs .getServerName().toString())); LOG.info("#regions = " + regions.size()); @@ -246,7 +249,8 @@ public class TestDistributedLogSplitting { it.remove(); } } - makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100); + + makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100); slm.splitLogDistributed(logDir); @@ -255,12 +259,13 @@ public class TestDistributedLogSplitting { Path tdir = FSUtils.getTableDir(rootdir, table); Path editsdir = - HLogUtil.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, hri.getEncodedName())); + WALSplitter.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, hri.getEncodedName())); LOG.debug("checking edits dir " + editsdir); FileStatus[] files = fs.listStatus(editsdir); - assertTrue(files.length > 1); + assertTrue("edits dir should have more than a single file in it. instead has " + files.length, + files.length > 1); for (int i = 0; i < files.length; i++) { - int c = countHLog(files[i].getPath(), fs, conf); + int c = countWAL(files[i].getPath(), fs, conf); count += c; } LOG.info(count + " edits in " + files.length + " recovered edits files."); @@ -289,7 +294,7 @@ public class TestDistributedLogSplitting { HRegionServer hrs = findRSToKill(false, "table"); List regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()); - makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100); + makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100); // wait for abort completes this.abortRSAndVerifyRecovery(hrs, ht, zkw, NUM_REGIONS_TO_CREATE, NUM_LOG_LINES); @@ -387,7 +392,7 @@ public class TestDistributedLogSplitting { HRegionServer hrs = findRSToKill(true, "table"); List regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()); - makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100); + makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100); this.abortRSAndVerifyRecovery(hrs, ht, zkw, NUM_REGIONS_TO_CREATE, NUM_LOG_LINES); ht.close(); @@ -455,7 +460,7 @@ public class TestDistributedLogSplitting { HRegionServer hrs = findRSToKill(false, "table"); List regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()); - makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100); + makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100); // abort master abortMaster(cluster); @@ -512,7 +517,7 @@ public class TestDistributedLogSplitting { HRegionServer hrs = findRSToKill(false, "table"); List regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()); - makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100); + makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100); // abort master abortMaster(cluster); @@ -575,7 +580,7 @@ public class TestDistributedLogSplitting { HRegionServer hrs1 = findRSToKill(false, "table"); regions = ProtobufUtil.getOnlineRegions(hrs1.getRSRpcServices()); - makeHLog(hrs1.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100); + makeWAL(hrs1, regions, "table", "family", NUM_LOG_LINES, 100); // abort RS1 LOG.info("Aborting region server: " + hrs1.getServerName()); @@ -789,8 +794,8 @@ public class TestDistributedLogSplitting { it.remove(); } } - makeHLog(hrs.getWAL(), regions, "disableTable", "family", NUM_LOG_LINES, 100, false); - makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100); + makeWAL(hrs, regions, "disableTable", "family", NUM_LOG_LINES, 100, false); + makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100); LOG.info("Disabling table\n"); TEST_UTIL.getHBaseAdmin().disableTable(TableName.valueOf("disableTable")); @@ -834,13 +839,13 @@ public class TestDistributedLogSplitting { Path tdir = FSUtils.getTableDir(rootdir, TableName.valueOf("disableTable")); for (HRegionInfo hri : regions) { Path editsdir = - HLogUtil.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, hri.getEncodedName())); + WALSplitter.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, hri.getEncodedName())); LOG.debug("checking edits dir " + editsdir); if(!fs.exists(editsdir)) continue; FileStatus[] files = fs.listStatus(editsdir); if(files != null) { for(FileStatus file : files) { - int c = countHLog(file.getPath(), fs, conf); + int c = countWAL(file.getPath(), fs, conf); count += c; LOG.info(c + " edits in " + file.getPath()); } @@ -855,7 +860,7 @@ public class TestDistributedLogSplitting { // clean up for (HRegionInfo hri : regions) { Path editsdir = - HLogUtil.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, hri.getEncodedName())); + WALSplitter.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, hri.getEncodedName())); fs.delete(editsdir, true); } disablingHT.close(); @@ -960,12 +965,12 @@ public class TestDistributedLogSplitting { HRegionServer hrs = findRSToKill(false, "table"); Path rootdir = FSUtils.getRootDir(conf); final Path logDir = new Path(rootdir, - HLogUtil.getHLogDirectoryName(hrs.getServerName().toString())); + DefaultWALProvider.getWALDirectoryName(hrs.getServerName().toString())); installTable(new ZooKeeperWatcher(conf, "table-creation", null), "table", "family", 40); - makeHLog(hrs.getWAL(), ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()), + makeWAL(hrs, ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()), "table", "family", NUM_LOG_LINES, 100); new Thread() { @@ -1239,15 +1244,17 @@ public class TestDistributedLogSplitting { long timeStamp = System.currentTimeMillis(); HTableDescriptor htd = new HTableDescriptor(); htd.addFamily(new HColumnDescriptor(family)); + final WAL wal = hrs.getWAL(curRegionInfo); for (int i = 0; i < NUM_LOG_LINES; i += 1) { WALEdit e = new WALEdit(); value++; e.add(new KeyValue(row, family, qualifier, timeStamp, Bytes.toBytes(value))); - hrs.getWAL().append(curRegionInfo, tableName, e, - System.currentTimeMillis(), htd, sequenceId); + wal.append(htd, curRegionInfo, + new HLogKey(curRegionInfo.getEncodedNameAsBytes(), tableName, System.currentTimeMillis()), + e, sequenceId, true, null); } - hrs.getWAL().sync(); - hrs.getWAL().close(); + wal.sync(); + wal.shutdown(); // wait for abort completes this.abortRSAndWaitForRecovery(hrs, zkw, NUM_REGIONS_TO_CREATE); @@ -1331,15 +1338,16 @@ public class TestDistributedLogSplitting { long timeStamp = System.currentTimeMillis(); HTableDescriptor htd = new HTableDescriptor(tableName); htd.addFamily(new HColumnDescriptor(family)); + final WAL wal = hrs.getWAL(curRegionInfo); for (int i = 0; i < NUM_LOG_LINES; i += 1) { WALEdit e = new WALEdit(); value++; e.add(new KeyValue(row, family, qualifier, timeStamp, Bytes.toBytes(value))); - hrs.getWAL().append(curRegionInfo, tableName, e, - System.currentTimeMillis(), htd, sequenceId); + wal.append(htd, curRegionInfo, new HLogKey(curRegionInfo.getEncodedNameAsBytes(), + tableName, System.currentTimeMillis()), e, sequenceId, true, null); } - hrs.getWAL().sync(); - hrs.getWAL().close(); + wal.sync(); + wal.shutdown(); // wait for abort completes this.abortRSAndWaitForRecovery(hrs, zkw, NUM_REGIONS_TO_CREATE); @@ -1379,27 +1387,24 @@ public class TestDistributedLogSplitting { FileSystem fs = master.getMasterFileSystem().getFileSystem(); Path tableDir = FSUtils.getTableDir(FSUtils.getRootDir(conf), TableName.valueOf("table")); List regionDirs = FSUtils.getRegionDirs(fs, tableDir); - HLogUtil.writeRegionOpenSequenceIdFile(fs, regionDirs.get(0) , 1L, 1000L); + WALSplitter.writeRegionOpenSequenceIdFile(fs, regionDirs.get(0) , 1L, 1000L); // current SeqId file has seqid=1001 - HLogUtil.writeRegionOpenSequenceIdFile(fs, regionDirs.get(0) , 1L, 1000L); + WALSplitter.writeRegionOpenSequenceIdFile(fs, regionDirs.get(0) , 1L, 1000L); // current SeqId file has seqid=2001 - assertEquals(3001, HLogUtil.writeRegionOpenSequenceIdFile(fs, regionDirs.get(0) , 3L, 1000L)); + assertEquals(3001, WALSplitter.writeRegionOpenSequenceIdFile(fs, regionDirs.get(0), 3L, 1000L)); - Path editsdir = HLogUtil.getRegionDirRecoveredEditsDir(regionDirs.get(0)); + Path editsdir = WALSplitter.getRegionDirRecoveredEditsDir(regionDirs.get(0)); FileStatus[] files = FSUtils.listStatus(fs, editsdir, new PathFilter() { @Override public boolean accept(Path p) { - if (p.getName().endsWith(HLog.SEQUENCE_ID_FILE_SUFFIX)) { - return true; - } - return false; + return WALSplitter.isSequenceIdFile(p); } }); // only one seqid file should exist assertEquals(1, files.length); // verify all seqId files aren't treated as recovered.edits files - NavigableSet recoveredEdits = HLogUtil.getSplitEditFilesSorted(fs, regionDirs.get(0)); + NavigableSet recoveredEdits = WALSplitter.getSplitEditFilesSorted(fs, regionDirs.get(0)); assertEquals(0, recoveredEdits.size()); ht.close(); @@ -1486,13 +1491,13 @@ public class TestDistributedLogSplitting { } } - public void makeHLog(HLog log, List regions, String tname, String fname, + public void makeWAL(HRegionServer hrs, List regions, String tname, String fname, int num_edits, int edit_size) throws IOException { - makeHLog(log, regions, tname, fname, num_edits, edit_size, true); + makeWAL(hrs, regions, tname, fname, num_edits, edit_size, true); } - public void makeHLog(HLog log, List regions, String tname, String fname, - int num_edits, int edit_size, boolean closeLog) throws IOException { + public void makeWAL(HRegionServer hrs, List regions, String tname, String fname, + int num_edits, int edit_size, boolean cleanShutdown) throws IOException { TableName fullTName = TableName.valueOf(tname); // remove root and meta region regions.remove(HRegionInfo.FIRST_META_REGIONINFO); @@ -1524,10 +1529,13 @@ public class TestDistributedLogSplitting { } int n = hris.size(); int[] counts = new int[n]; + // sync every ~30k to line up with desired wal rolls + final int syncEvery = 30 * 1024 / edit_size; if (n > 0) { for (int i = 0; i < num_edits; i += 1) { WALEdit e = new WALEdit(); HRegionInfo curRegionInfo = hris.get(i % n); + final WAL log = hrs.getWAL(curRegionInfo); byte[] startRow = curRegionInfo.getStartKey(); if (startRow == null || startRow.length == 0) { startRow = new byte[] { 0, 0, 0, 0, 1 }; @@ -1538,13 +1546,25 @@ public class TestDistributedLogSplitting { // key byte[] qualifier = Bytes.toBytes("c" + Integer.toString(i)); e.add(new KeyValue(row, family, qualifier, System.currentTimeMillis(), value)); - log.append(curRegionInfo, fullTName, e, System.currentTimeMillis(), htd, sequenceId); + log.append(htd, curRegionInfo, new HLogKey(curRegionInfo.getEncodedNameAsBytes(), fullTName, + System.currentTimeMillis()), e, sequenceId, true, null); + if (0 == i % syncEvery) { + log.sync(); + } counts[i % n] += 1; } } - log.sync(); - if(closeLog) { - log.close(); + // done as two passes because the regions might share logs. shutdown is idempotent, but sync + // will cause errors if done after. + for (HRegionInfo info : hris) { + final WAL log = hrs.getWAL(info); + log.sync(); + } + if (cleanShutdown) { + for (HRegionInfo info : hris) { + final WAL log = hrs.getWAL(info); + log.shutdown(); + } } for (int i = 0; i < n; i++) { LOG.info("region " + hris.get(i).getRegionNameAsString() + " has " + counts[i] + " edits"); @@ -1552,14 +1572,23 @@ public class TestDistributedLogSplitting { return; } - private int countHLog(Path log, FileSystem fs, Configuration conf) + private int countWAL(Path log, FileSystem fs, Configuration conf) throws IOException { int count = 0; - HLog.Reader in = HLogFactory.createReader(fs, log, conf); - HLog.Entry e; - while ((e = in.next()) != null) { - if (!WALEdit.isMetaEditFamily(e.getEdit().getCells().get(0))) { - count++; + WAL.Reader in = WALFactory.createReader(fs, log, conf); + try { + WAL.Entry e; + while ((e = in.next()) != null) { + if (!WALEdit.isMetaEditFamily(e.getEdit().getCells().get(0))) { + count++; + } + } + } finally { + try { + in.close(); + } catch (IOException exception) { + LOG.warn("Problem closing wal: " + exception.getMessage()); + LOG.debug("exception details.", exception); } } return count; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java index 6c17686c42d..a2dbcb62134 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java @@ -64,7 +64,7 @@ public class TestMasterFailover { final HTableDescriptor htd) throws IOException { HRegion r = HRegion.createHRegion(hri, rootdir, c, htd); - // The above call to create a region will create an hlog file. Each + // The above call to create a region will create an wal file. Each // log file create will also create a running thread to do syncing. We need // to close out this log else we will have a running thread trying to sync // the file system continuously which is ugly when dfs is taken away at the diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java index 198ce6a6a50..efaef9de6c4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils.SnapshotMock; @@ -99,7 +98,7 @@ public class TestSnapshotFileCache { "test-snapshot-file-cache-refresh", new SnapshotFileCache.SnapshotFileInspector() { public Collection filesUnderSnapshot(final Path snapshotDir) throws IOException { - return SnapshotReferenceUtil.getHLogNames(fs, snapshotDir); + return SnapshotReferenceUtil.getWALNames(fs, snapshotDir); } }); @@ -111,7 +110,7 @@ public class TestSnapshotFileCache { fs.createNewFile(file1); // and another file in the logs directory - Path logs = getSnapshotHLogsDir(snapshot, "server"); + Path logs = SnapshotReferenceUtil.getLogsDir(snapshot, "server"); Path log = new Path(logs, "me.hbase.com%2C58939%2C1350424310315.1350424315552"); fs.createNewFile(log); @@ -123,16 +122,6 @@ public class TestSnapshotFileCache { assertTrue("Cache didn't find:" + log, cache.contains(log.getName())); } - /** - * Get the log directory for a specific snapshot - * @param snapshotDir directory where the specific snapshot will be store - * @param serverName name of the parent regionserver for the log files - * @return path to the log home directory for the archive files. - */ - public static Path getSnapshotHLogsDir(Path snapshotDir, String serverName) { - return new Path(snapshotDir, HLogUtil.getHLogDirectoryName(serverName)); - } - @Test public void testReloadModifiedDirectory() throws IOException { // don't refresh the cache unless we tell it to @@ -170,7 +159,7 @@ public class TestSnapshotFileCache { class SnapshotFiles implements SnapshotFileCache.SnapshotFileInspector { public Collection filesUnderSnapshot(final Path snapshotDir) throws IOException { Collection files = new HashSet(); - files.addAll(SnapshotReferenceUtil.getHLogNames(fs, snapshotDir)); + files.addAll(SnapshotReferenceUtil.getWALNames(fs, snapshotDir)); files.addAll(SnapshotReferenceUtil.getHFileNames(UTIL.getConfiguration(), fs, snapshotDir)); return files; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java index ca95c4a551f..58c61c6559d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java @@ -202,12 +202,12 @@ public class MetricsRegionServerWrapperStub implements MetricsRegionServerWrappe } @Override - public long getNumHLogFiles() { + public long getNumWALFiles() { return 10; } @Override - public long getHLogFileSize() { + public long getWALFileSize() { return 1024000; } @@ -250,4 +250,4 @@ public class MetricsRegionServerWrapperStub implements MetricsRegionServerWrappe public long getHedgedReadWins() { return 10; } -} \ No newline at end of file +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java index 86cca009406..a4b4959696c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java @@ -60,7 +60,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.wal.WAL; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -607,7 +607,7 @@ public class TestAtomicOperation { public static class MockHRegion extends HRegion { - public MockHRegion(Path tableDir, HLog log, FileSystem fs, Configuration conf, + public MockHRegion(Path tableDir, WAL log, FileSystem fs, Configuration conf, final HRegionInfo regionInfo, final HTableDescriptor htd, RegionServerServices rsServices) { super(tableDir, log, fs, conf, regionInfo, htd, rsServices); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java index 1f63a48ab47..dc142d6b0c6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java @@ -51,9 +51,10 @@ 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.HLogFactory; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; import org.junit.After; import org.junit.Before; import org.junit.Rule; @@ -126,7 +127,7 @@ public class TestCacheOnWriteInSchema { private final String testDescription; private HRegion region; private HStore store; - private HLog hlog; + private WALFactory walFactory; private FileSystem fs; public TestCacheOnWriteInSchema(CacheOnWriteType cowType) { @@ -165,15 +166,16 @@ public class TestCacheOnWriteInSchema { htd.addFamily(hcd); // Create a store based on the schema - Path basedir = new Path(DIR); - String logName = "logs"; - Path logdir = new Path(DIR, logName); + final String id = TestCacheOnWriteInSchema.class.getName(); + final Path logdir = new Path(FSUtils.getRootDir(conf), + DefaultWALProvider.getWALDirectoryName(id)); fs.delete(logdir, true); HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); - hlog = HLogFactory.createHLog(fs, basedir, logName, conf); + walFactory = new WALFactory(conf, null, id); - region = TEST_UTIL.createLocalHRegion(info, htd, hlog); + region = TEST_UTIL.createLocalHRegion(info, htd, + walFactory.getWAL(info.getEncodedNameAsBytes())); store = new HStore(region, hcd, conf); } @@ -187,7 +189,7 @@ public class TestCacheOnWriteInSchema { ex = e; } try { - hlog.closeAndDelete(); + walFactory.close(); } catch (IOException e) { LOG.warn("Caught Exception", e); ex = e; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java index 4900ef43f85..7cfa4756fa4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java @@ -59,7 +59,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileScanner; 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.wal.WAL; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Threads; @@ -118,9 +118,9 @@ public class TestCompaction { @After public void tearDown() throws Exception { - HLog hlog = r.getLog(); + WAL wal = r.getWAL(); this.r.close(); - hlog.closeAndDelete(); + wal.close(); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java index a78e1d5ed6b..43bc9f1f0c8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java @@ -39,8 +39,8 @@ import org.apache.hadoop.hbase.testclassification.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.HLogFactory; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.junit.After; @@ -65,7 +65,7 @@ public class TestDefaultCompactSelection extends TestCase { protected static final long minSize = 10; protected static final long maxSize = 2100; - private HLog hlog; + private WALFactory wals; private HRegion region; @Override @@ -82,9 +82,9 @@ public class TestDefaultCompactSelection extends TestCase { this.conf.unset("hbase.hstore.compaction.min.size"); //Setting up a Store + final String id = TestDefaultCompactSelection.class.getName(); Path basedir = new Path(DIR); - String logName = "logs"; - Path logdir = new Path(DIR, logName); + final Path logdir = new Path(basedir, DefaultWALProvider.getWALDirectoryName(id)); HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes("family")); FileSystem fs = FileSystem.get(conf); @@ -94,11 +94,14 @@ public class TestDefaultCompactSelection extends TestCase { htd.addFamily(hcd); HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); - hlog = HLogFactory.createHLog(fs, basedir, logName, conf); + final Configuration walConf = new Configuration(conf); + FSUtils.setRootDir(walConf, basedir); + wals = new WALFactory(walConf, null, id); region = HRegion.createHRegion(info, basedir, conf, htd); HRegion.closeHRegion(region); Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName()); - region = new HRegion(tableDir, hlog, fs, conf, info, htd, null); + region = new HRegion(tableDir, wals.getWAL(info.getEncodedNameAsBytes()), fs, conf, info, htd, + null); store = new HStore(region, hcd, conf); @@ -116,7 +119,7 @@ public class TestDefaultCompactSelection extends TestCase { ex = e; } try { - hlog.closeAndDelete(); + wals.close(); } catch (IOException e) { LOG.warn("Caught Exception", e); ex = e; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java index b270a7db660..92351f4af00 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java @@ -142,7 +142,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { } catch (Exception e) { e.printStackTrace(); } - mr.getLog().closeAndDelete(); + mr.getWAL().close(); } } } @@ -293,7 +293,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { } catch (Exception e) { e.printStackTrace(); } - region.getLog().closeAndDelete(); + region.getWAL().close(); } } } @@ -349,7 +349,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { } catch (Exception e) { e.printStackTrace(); } - region.getLog().closeAndDelete(); + region.getWAL().close(); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index b4e94bff9f8..f7936c3ff28 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -124,13 +124,17 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescripto import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl; import org.apache.hadoop.hbase.regionserver.HRegion.RowLock; import org.apache.hadoop.hbase.regionserver.TestStore.FaultyFileSystem; -import org.apache.hadoop.hbase.regionserver.wal.FaultyHLog; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; -import org.apache.hadoop.hbase.regionserver.wal.HLogKey; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource; +import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.wal.FaultyFSLog; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.hbase.wal.WALProvider; +import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -254,9 +258,9 @@ public class TestHRegion { */ @Test (timeout=60000) public void testMemstoreSnapshotSize() throws IOException { - class MyFaultyHLog extends FaultyHLog { + class MyFaultyFSLog extends FaultyFSLog { StoreFlushContext storeFlushCtx; - public MyFaultyHLog(FileSystem fs, Path rootDir, String logName, Configuration conf) + public MyFaultyFSLog(FileSystem fs, Path rootDir, String logName, Configuration conf) throws IOException { super(fs, rootDir, logName, conf); } @@ -274,7 +278,7 @@ public class TestHRegion { FileSystem fs = FileSystem.get(CONF); Path rootDir = new Path(dir + "testMemstoreSnapshotSize"); - MyFaultyHLog faultyLog = new MyFaultyHLog(fs, rootDir, "testMemstoreSnapshotSize", CONF); + MyFaultyFSLog faultyLog = new MyFaultyFSLog(fs, rootDir, "testMemstoreSnapshotSize", CONF); HRegion region = initHRegion(tableName, null, null, name.getMethodName(), CONF, false, Durability.SYNC_WAL, faultyLog, COLUMN_FAMILY_BYTES); @@ -285,7 +289,7 @@ public class TestHRegion { Put put = new Put(value); put.add(COLUMN_FAMILY_BYTES, Bytes.toBytes("abc"), value); - faultyLog.setFailureType(FaultyHLog.FailureType.SYNC); + faultyLog.setFailureType(FaultyFSLog.FailureType.SYNC); boolean threwIOE = false; try { @@ -512,12 +516,13 @@ public class TestHRegion { TableName tableName = TableName.valueOf(method); byte[] family = Bytes.toBytes("family"); this.region = initHRegion(tableName, method, CONF, family); + final WALFactory wals = new WALFactory(CONF, null, method); try { Path regiondir = region.getRegionFileSystem().getRegionDir(); FileSystem fs = region.getRegionFileSystem().getFileSystem(); byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes(); - Path recoveredEditsDir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir); + Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir); long maxSeqId = 1050; long minSeqId = 1000; @@ -525,13 +530,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); + WALProvider.Writer writer = wals.createRecoveredEditsWriter(fs, recoveredEdits); 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(); @@ -554,6 +559,7 @@ public class TestHRegion { } finally { HRegion.closeHRegion(this.region); this.region = null; + wals.close(); } } @@ -563,12 +569,13 @@ public class TestHRegion { TableName tableName = TableName.valueOf(method); byte[] family = Bytes.toBytes("family"); this.region = initHRegion(tableName, method, CONF, family); + final WALFactory wals = new WALFactory(CONF, null, method); try { Path regiondir = region.getRegionFileSystem().getRegionDir(); FileSystem fs = region.getRegionFileSystem().getFileSystem(); byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes(); - Path recoveredEditsDir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir); + Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir); long maxSeqId = 1050; long minSeqId = 1000; @@ -576,13 +583,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); + WALProvider.Writer writer = wals.createRecoveredEditsWriter(fs, recoveredEdits); 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(); @@ -610,6 +617,7 @@ public class TestHRegion { } finally { HRegion.closeHRegion(this.region); this.region = null; + wals.close(); } } @@ -621,7 +629,7 @@ public class TestHRegion { Path regiondir = region.getRegionFileSystem().getRegionDir(); FileSystem fs = region.getRegionFileSystem().getFileSystem(); - Path recoveredEditsDir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir); + Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir); for (int i = 1000; i < 1050; i += 10) { Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i)); FSDataOutputStream dos = fs.create(recoveredEdits); @@ -651,6 +659,7 @@ public class TestHRegion { TableName tableName = TableName.valueOf(method); byte[] family = Bytes.toBytes("family"); this.region = initHRegion(tableName, method, CONF, family); + final WALFactory wals = new WALFactory(CONF, null, method); try { Path regiondir = region.getRegionFileSystem().getRegionDir(); FileSystem fs = region.getRegionFileSystem().getFileSystem(); @@ -659,7 +668,7 @@ public class TestHRegion { assertEquals(0, region.getStoreFileList( region.getStores().keySet().toArray(new byte[0][])).size()); - Path recoveredEditsDir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir); + Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir); long maxSeqId = 1050; long minSeqId = 1000; @@ -667,7 +676,7 @@ 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); + WALProvider.Writer writer = wals.createRecoveredEditsWriter(fs, recoveredEdits); long time = System.nanoTime(); WALEdit edit = null; @@ -685,7 +694,7 @@ public class TestHRegion { 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(); } @@ -706,7 +715,9 @@ public class TestHRegion { } finally { HRegion.closeHRegion(this.region); this.region = null; - } } + wals.close(); + } + } @Test public void testRecoveredEditsReplayCompaction() throws Exception { @@ -714,6 +725,7 @@ public class TestHRegion { TableName tableName = TableName.valueOf(method); byte[] family = Bytes.toBytes("family"); this.region = initHRegion(tableName, method, CONF, family); + final WALFactory wals = new WALFactory(CONF, null, method); try { Path regiondir = region.getRegionFileSystem().getRegionDir(); FileSystem fs = region.getRegionFileSystem().getFileSystem(); @@ -758,18 +770,18 @@ public class TestHRegion { .getRegionInfo(), family, storeFiles, Lists.newArrayList(newFile), region .getRegionFileSystem().getStoreDir(Bytes.toString(family))); - HLogUtil.writeCompactionMarker(region.getLog(), this.region.getTableDesc(), + WALUtil.writeCompactionMarker(region.getWAL(), this.region.getTableDesc(), this.region.getRegionInfo(), compactionDescriptor, new AtomicLong(1)); - Path recoveredEditsDir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir); + Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir); Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", 1000)); fs.create(recoveredEdits); - HLog.Writer writer = HLogFactory.createRecoveredEditsWriter(fs, recoveredEdits, CONF); + WALProvider.Writer writer = wals.createRecoveredEditsWriter(fs, recoveredEdits); 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(region.getRegionInfo(), compactionDescriptor))); writer.close(); @@ -798,6 +810,7 @@ public class TestHRegion { } finally { HRegion.closeHRegion(this.region); this.region = null; + wals.close(); } } @@ -808,11 +821,13 @@ public class TestHRegion { TableName tableName = TableName.valueOf(method); byte[] family = Bytes.toBytes("family"); Path logDir = TEST_UTIL.getDataTestDirOnTestFS(method + ".log"); - HLog hlog = HLogFactory.createHLog(FILESYSTEM, logDir, "logs", - TEST_UTIL.getConfiguration()); + final Configuration walConf = new Configuration(TEST_UTIL.getConfiguration()); + FSUtils.setRootDir(walConf, logDir); + final WALFactory wals = new WALFactory(walConf, null, method); + final WAL wal = wals.getWAL(tableName.getName()); this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW, - HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, hlog, family); + HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, wal, family); try { Path regiondir = region.getRegionFileSystem().getRegionDir(); FileSystem fs = region.getRegionFileSystem().getFileSystem(); @@ -836,59 +851,69 @@ public class TestHRegion { } // now verify that the flush markers are written - hlog.close(); - HLog.Reader reader = HLogFactory.createReader(fs, - fs.listStatus(new Path(logDir, "logs"))[0].getPath(), + wal.shutdown(); + WAL.Reader reader = wals.createReader(fs, DefaultWALProvider.getCurrentFileName(wal), TEST_UTIL.getConfiguration()); + try { + List flushDescriptors = new ArrayList(); + long lastFlushSeqId = -1; + while (true) { + WAL.Entry entry = reader.next(); + if (entry == null) { + break; + } + Cell cell = entry.getEdit().getCells().get(0); + if (WALEdit.isMetaEditFamily(cell)) { + FlushDescriptor flushDesc = WALEdit.getFlushDescriptor(cell); + assertNotNull(flushDesc); + assertArrayEquals(tableName.getName(), flushDesc.getTableName().toByteArray()); + if (flushDesc.getAction() == FlushAction.START_FLUSH) { + assertTrue(flushDesc.getFlushSequenceNumber() > lastFlushSeqId); + } else if (flushDesc.getAction() == FlushAction.COMMIT_FLUSH) { + assertTrue(flushDesc.getFlushSequenceNumber() == lastFlushSeqId); + } + lastFlushSeqId = flushDesc.getFlushSequenceNumber(); + assertArrayEquals(regionName, flushDesc.getEncodedRegionName().toByteArray()); + assertEquals(1, flushDesc.getStoreFlushesCount()); //only one store + StoreFlushDescriptor storeFlushDesc = flushDesc.getStoreFlushes(0); + assertArrayEquals(family, storeFlushDesc.getFamilyName().toByteArray()); + assertEquals("family", storeFlushDesc.getStoreHomeDir()); + if (flushDesc.getAction() == FlushAction.START_FLUSH) { + assertEquals(0, storeFlushDesc.getFlushOutputCount()); + } else { + assertEquals(1, storeFlushDesc.getFlushOutputCount()); //only one file from flush + assertTrue(storeFiles.contains(storeFlushDesc.getFlushOutput(0))); + } - List flushDescriptors = new ArrayList(); - long lastFlushSeqId = -1; - while (true) { - HLog.Entry entry = reader.next(); - if (entry == null) { - break; + flushDescriptors.add(entry); + } } - Cell cell = entry.getEdit().getCells().get(0); - if (WALEdit.isMetaEditFamily(cell)) { - FlushDescriptor flushDesc = WALEdit.getFlushDescriptor(cell); - assertNotNull(flushDesc); - assertArrayEquals(tableName.getName(), flushDesc.getTableName().toByteArray()); - if (flushDesc.getAction() == FlushAction.START_FLUSH) { - assertTrue(flushDesc.getFlushSequenceNumber() > lastFlushSeqId); - } else if (flushDesc.getAction() == FlushAction.COMMIT_FLUSH) { - assertTrue(flushDesc.getFlushSequenceNumber() == lastFlushSeqId); - } - lastFlushSeqId = flushDesc.getFlushSequenceNumber(); - assertArrayEquals(regionName, flushDesc.getEncodedRegionName().toByteArray()); - assertEquals(1, flushDesc.getStoreFlushesCount()); //only one store - StoreFlushDescriptor storeFlushDesc = flushDesc.getStoreFlushes(0); - assertArrayEquals(family, storeFlushDesc.getFamilyName().toByteArray()); - assertEquals("family", storeFlushDesc.getStoreHomeDir()); - if (flushDesc.getAction() == FlushAction.START_FLUSH) { - assertEquals(0, storeFlushDesc.getFlushOutputCount()); - } else { - assertEquals(1, storeFlushDesc.getFlushOutputCount()); //only one file from flush - assertTrue(storeFiles.contains(storeFlushDesc.getFlushOutput(0))); - } - flushDescriptors.add(entry); + assertEquals(3 * 2, flushDescriptors.size()); // START_FLUSH and COMMIT_FLUSH per flush + + // now write those markers to the recovered edits again. + + Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir); + + Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", 1000)); + fs.create(recoveredEdits); + WALProvider.Writer writer = wals.createRecoveredEditsWriter(fs, recoveredEdits); + + for (WAL.Entry entry : flushDescriptors) { + writer.append(entry); + } + writer.close(); + } finally { + if (null != reader) { + try { + reader.close(); + } catch (IOException exception) { + LOG.warn("Problem closing wal: " + exception.getMessage()); + LOG.debug("exception details", exception); + } } } - assertEquals(3 * 2, flushDescriptors.size()); // START_FLUSH and COMMIT_FLUSH per flush - - // now write those markers to the recovered edits again. - - Path recoveredEditsDir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir); - - Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", 1000)); - fs.create(recoveredEdits); - HLog.Writer writer = HLogFactory.createRecoveredEditsWriter(fs, recoveredEdits, CONF); - - for (HLog.Entry entry : flushDescriptors) { - writer.append(entry); - } - writer.close(); // close the region now, and reopen again region.close(); @@ -904,6 +929,7 @@ public class TestHRegion { } finally { HRegion.closeHRegion(this.region); this.region = null; + wals.close(); } } @@ -953,11 +979,13 @@ public class TestHRegion { // spy an actual WAL implementation to throw exception (was not able to mock) Path logDir = TEST_UTIL.getDataTestDirOnTestFS(method + "log"); - HLog hlog = spy(HLogFactory.createHLog(FILESYSTEM, logDir, "logs", - TEST_UTIL.getConfiguration())); + final Configuration walConf = new Configuration(TEST_UTIL.getConfiguration()); + FSUtils.setRootDir(walConf, logDir); + final WALFactory wals = new WALFactory(walConf, null, method); + WAL wal = spy(wals.getWAL(tableName.getName())); this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW, - HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, hlog, family); + HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, wal, family); try { int i = 0; Put put = new Put(Bytes.toBytes(i)); @@ -969,7 +997,7 @@ public class TestHRegion { IsFlushWALMarker isFlushWALMarker = new IsFlushWALMarker(FlushAction.START_FLUSH); // throw exceptions if the WalEdit is a start flush action - when(hlog.appendNoSync((HTableDescriptor)any(), (HRegionInfo)any(), (HLogKey)any(), + when(wal.append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any(), (WALEdit)argThat(isFlushWALMarker), (AtomicLong)any(), Mockito.anyBoolean(), (List)any())) .thenThrow(new IOException("Fail to append flush marker")); @@ -1000,7 +1028,7 @@ public class TestHRegion { region.close(); this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW, - HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, hlog, family); + HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, wal, family); region.put(put); // 3. Test case where ABORT_FLUSH will throw exception. @@ -4486,10 +4514,12 @@ 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(FILESYSTEM, logDir, UUID.randomUUID().toString(), conf); - final HLog log = spy(hlog); + final Configuration walConf = new Configuration(conf); + FSUtils.setRootDir(walConf, logDir); + final WALFactory wals = new WALFactory(walConf, null, UUID.randomUUID().toString()); + final WAL wal = spy(wals.getWAL(tableName.getName())); this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW, - HConstants.EMPTY_END_ROW, method, conf, false, tableDurability, log, + HConstants.EMPTY_END_ROW, method, conf, false, tableDurability, wal, new byte[][] { family }); Put put = new Put(Bytes.toBytes("r1")); @@ -4498,8 +4528,8 @@ public class TestHRegion { region.put(put); //verify append called or not - verify(log, expectAppend ? times(1) : never()) - .appendNoSync((HTableDescriptor)any(), (HRegionInfo)any(), (HLogKey)any(), + verify(wal, expectAppend ? times(1) : never()) + .append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any(), (WALEdit)any(), (AtomicLong)any(), Mockito.anyBoolean(), (List)any()); // verify sync called or not @@ -4509,9 +4539,9 @@ public class TestHRegion { public boolean evaluate() throws Exception { try { if (expectSync) { - verify(log, times(1)).sync(anyLong()); // Hregion calls this one + verify(wal, times(1)).sync(anyLong()); // Hregion calls this one } else if (expectSyncFromLogSyncer) { - verify(log, times(1)).sync(); // log syncer calls this one + verify(wal, times(1)).sync(); // wal syncer calls this one } } catch (Throwable ignore) { } @@ -4519,8 +4549,8 @@ public class TestHRegion { } }); } else { - //verify(log, never()).sync(anyLong()); - verify(log, never()).sync(); + //verify(wal, never()).sync(anyLong()); + verify(wal, never()).sync(); } HRegion.closeHRegion(this.region); @@ -4532,7 +4562,7 @@ public class TestHRegion { // create a primary region, load some data and flush // create a secondary region, and do a get against that Path rootDir = new Path(dir + "testRegionReplicaSecondary"); - TEST_UTIL.getConfiguration().set(HConstants.HBASE_DIR, rootDir.toString()); + FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir); byte[][] families = new byte[][] { Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3") @@ -4582,7 +4612,7 @@ public class TestHRegion { // create a primary region, load some data and flush // create a secondary region, and do a put against that Path rootDir = new Path(dir + "testRegionReplicaSecondary"); - TEST_UTIL.getConfiguration().set(HConstants.HBASE_DIR, rootDir.toString()); + FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir); byte[][] families = new byte[][] { Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3") @@ -4635,7 +4665,7 @@ public class TestHRegion { @Test public void testCompactionFromPrimary() throws IOException { Path rootDir = new Path(dir + "testRegionReplicaSecondary"); - TEST_UTIL.getConfiguration().set(HConstants.HBASE_DIR, rootDir.toString()); + FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir); byte[][] families = new byte[][] { Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3") @@ -4889,8 +4919,9 @@ 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 { - return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, callingMethod, conf, isReadOnly, durability, hlog, families); + WAL wal, byte[]... families) throws IOException { + return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, callingMethod, conf, + isReadOnly, durability, wal, families); } /** @@ -5503,7 +5534,7 @@ public class TestHRegion { HRegionInfo hri = new HRegionInfo(htd.getTableName(), HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY); - // open the region w/o rss and log and flush some files + // open the region w/o rss and wal and flush some files HRegion region = HRegion.createHRegion(hri, TEST_UTIL.getDataTestDir(), TEST_UTIL .getConfiguration(), htd); @@ -5516,15 +5547,15 @@ public class TestHRegion { ArgumentCaptor editCaptor = ArgumentCaptor.forClass(WALEdit.class); - // capture appendNoSync() calls - HLog log = mock(HLog.class); - when(rss.getWAL((HRegionInfo) any())).thenReturn(log); + // capture append() calls + WAL wal = mock(WAL.class); + when(rss.getWAL((HRegionInfo) any())).thenReturn(wal); try { region = HRegion.openHRegion(hri, htd, rss.getWAL(hri), TEST_UTIL.getConfiguration(), rss, null); - verify(log, times(1)).appendNoSync((HTableDescriptor)any(), (HRegionInfo)any(), (HLogKey)any() + verify(wal, times(1)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any() , editCaptor.capture(), (AtomicLong)any(), anyBoolean(), (List)any()); WALEdit edit = editCaptor.getValue(); @@ -5576,9 +5607,9 @@ public class TestHRegion { ArgumentCaptor editCaptor = ArgumentCaptor.forClass(WALEdit.class); - // capture appendNoSync() calls - HLog log = mock(HLog.class); - when(rss.getWAL((HRegionInfo) any())).thenReturn(log); + // capture append() calls + WAL wal = mock(WAL.class); + when(rss.getWAL((HRegionInfo) any())).thenReturn(wal); // open a region first so that it can be closed later region = HRegion.openHRegion(hri, htd, rss.getWAL(hri), @@ -5588,7 +5619,7 @@ public class TestHRegion { region.close(false); // 2 times, one for region open, the other close region - verify(log, times(2)).appendNoSync((HTableDescriptor)any(), (HRegionInfo)any(), (HLogKey)any(), + verify(wal, times(2)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any(), editCaptor.capture(), (AtomicLong)any(), anyBoolean(), (List)any()); WALEdit edit = editCaptor.getAllValues().get(1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java index 0469f0e5e14..03acfdc73e8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java @@ -57,7 +57,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress; 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.wal.WAL; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; @@ -110,9 +110,9 @@ public class TestMajorCompaction { @After public void tearDown() throws Exception { - HLog hlog = r.getLog(); + WAL wal = r.getWAL(); this.r.close(); - hlog.closeAndDelete(); + wal.close(); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinorCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinorCompaction.java index 49d04454453..7ac6eefd71e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinorCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinorCompaction.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; @@ -90,9 +90,9 @@ public class TestMinorCompaction { @After public void tearDown() throws Exception { - HLog hlog = r.getLog(); + WAL wal = r.getWAL(); this.r.close(); - hlog.closeAndDelete(); + wal.close(); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java index 9e65751f671..8b5b4a3ee9f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; @@ -43,11 +44,11 @@ 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.HLogFactory; +import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; import org.apache.zookeeper.KeeperException; import org.junit.After; import org.junit.Before; @@ -69,7 +70,7 @@ public class TestRegionMergeTransaction { private HRegion region_a; private HRegion region_b; private HRegion region_c; - private HLog wal; + private WALFactory wals; private FileSystem fs; // Start rows of region_a,region_b,region_c private static final byte[] STARTROW_A = new byte[] { 'a', 'a', 'a' }; @@ -82,11 +83,12 @@ public class TestRegionMergeTransaction { public void setup() throws IOException { this.fs = FileSystem.get(TEST_UTIL.getConfiguration()); this.fs.delete(this.testdir, true); - this.wal = HLogFactory.createHLog(fs, this.testdir, "logs", - TEST_UTIL.getConfiguration()); - this.region_a = createRegion(this.testdir, this.wal, STARTROW_A, STARTROW_B); - this.region_b = createRegion(this.testdir, this.wal, STARTROW_B, STARTROW_C); - this.region_c = createRegion(this.testdir, this.wal, STARTROW_C, ENDROW); + final Configuration walConf = new Configuration(TEST_UTIL.getConfiguration()); + FSUtils.setRootDir(walConf, this.testdir); + this.wals = new WALFactory(walConf, null, TestRegionMergeTransaction.class.getName()); + this.region_a = createRegion(this.testdir, this.wals, STARTROW_A, STARTROW_B); + this.region_b = createRegion(this.testdir, this.wals, STARTROW_B, STARTROW_C); + this.region_c = createRegion(this.testdir, this.wals, STARTROW_C, ENDROW); assert region_a != null && region_b != null && region_c != null; TEST_UTIL.getConfiguration().setBoolean("hbase.testing.nocluster", true); } @@ -101,8 +103,9 @@ public class TestRegionMergeTransaction { + region.getRegionFileSystem().getRegionDir()); } } - if (this.wal != null) - this.wal.closeAndDelete(); + if (this.wals != null) { + this.wals.close(); + } this.fs.delete(this.testdir, true); } @@ -401,7 +404,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 WALFactory wals, final byte[] startrow, final byte[] endrow) throws IOException { // Make a region with start and end keys. @@ -412,7 +415,7 @@ public class TestRegionMergeTransaction { HRegion a = HRegion.createHRegion(hri, testdir, TEST_UTIL.getConfiguration(), htd); HRegion.closeHRegion(a); - return HRegion.openHRegion(testdir, hri, htd, wal, + return HRegion.openHRegion(testdir, hri, htd, wals.getWAL(hri.getEncodedNameAsBytes()), TEST_UTIL.getConfiguration()); } @@ -457,7 +460,7 @@ public class TestRegionMergeTransaction { } Put put = new Put(k); put.add(f, null, k); - if (r.getLog() == null) + if (r.getWAL() == null) put.setDurability(Durability.SKIP_WAL); r.put(put); rowCount++; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java index 4479ce6dbda..ca97c3e8c17 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; @@ -45,11 +46,11 @@ 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.HLogFactory; +import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.PairOfSameType; import org.apache.zookeeper.KeeperException; import org.junit.After; @@ -70,7 +71,7 @@ public class TestSplitTransaction { private final Path testdir = TEST_UTIL.getDataTestDir(this.getClass().getName()); private HRegion parent; - private HLog wal; + private WALFactory wals; private FileSystem fs; private static final byte [] STARTROW = new byte [] {'a', 'a', 'a'}; // '{' is next ascii after 'z'. @@ -85,10 +86,11 @@ public class TestSplitTransaction { this.fs = FileSystem.get(TEST_UTIL.getConfiguration()); TEST_UTIL.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, CustomObserver.class.getName()); this.fs.delete(this.testdir, true); - this.wal = HLogFactory.createHLog(fs, this.testdir, "logs", - TEST_UTIL.getConfiguration()); + final Configuration walConf = new Configuration(TEST_UTIL.getConfiguration()); + FSUtils.setRootDir(walConf, this.testdir); + this.wals = new WALFactory(walConf, null, this.getClass().getName()); - this.parent = createRegion(this.testdir, this.wal); + this.parent = createRegion(this.testdir, this.wals); RegionCoprocessorHost host = new RegionCoprocessorHost(this.parent, null, TEST_UTIL.getConfiguration()); this.parent.setCoprocessorHost(host); TEST_UTIL.getConfiguration().setBoolean("hbase.testing.nocluster", true); @@ -100,7 +102,9 @@ public class TestSplitTransaction { if (this.fs.exists(regionDir) && !this.fs.delete(regionDir, true)) { throw new IOException("Failed delete of " + regionDir); } - if (this.wal != null) this.wal.closeAndDelete(); + if (this.wals != null) { + this.wals.close(); + } this.fs.delete(this.testdir, true); } @@ -327,7 +331,7 @@ public class TestSplitTransaction { return rowcount; } - HRegion createRegion(final Path testdir, final HLog wal) + HRegion createRegion(final Path testdir, final WALFactory wals) 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'. @@ -337,7 +341,7 @@ public class TestSplitTransaction { HRegionInfo hri = new HRegionInfo(htd.getTableName(), STARTROW, ENDROW); HRegion r = HRegion.createHRegion(hri, testdir, TEST_UTIL.getConfiguration(), htd); HRegion.closeHRegion(r); - return HRegion.openHRegion(testdir, hri, htd, wal, + return HRegion.openHRegion(testdir, hri, htd, wals.getWAL(hri.getEncodedNameAsBytes()), TEST_UTIL.getConfiguration()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java index be99f37ab1b..b5bc927e6bb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java @@ -70,8 +70,8 @@ import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration; import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -168,8 +168,7 @@ public class TestStore { //Setting up a Store Path basedir = new Path(DIR+methodName); Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName()); - String logName = "logs"; - Path logdir = new Path(basedir, logName); + final Path logdir = new Path(basedir, DefaultWALProvider.getWALDirectoryName(methodName)); FileSystem fs = FileSystem.get(conf); @@ -181,8 +180,11 @@ public class TestStore { htd.addFamily(hcd); } HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); - HLog hlog = HLogFactory.createHLog(fs, basedir, logName, conf); - HRegion region = new HRegion(tableDir, hlog, fs, conf, info, htd, null); + final Configuration walConf = new Configuration(conf); + FSUtils.setRootDir(walConf, basedir); + final WALFactory wals = new WALFactory(walConf, null, methodName); + HRegion region = new HRegion(tableDir, wals.getWAL(info.getEncodedNameAsBytes()), fs, conf, + info, htd, null); store = new HStore(region, hcd, conf); return store; @@ -783,7 +785,7 @@ public class TestStore { LOG.info("After failed flush, we should still have no files!"); files = store.getRegionFileSystem().getStoreFiles(store.getColumnFamilyName()); Assert.assertEquals(0, files != null ? files.size() : 0); - store.getHRegion().getLog().closeAndDelete(); + store.getHRegion().getWAL().close(); return null; } }); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java index e66a069ef0b..0319051238e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java @@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; +import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.StoppableImplementation; @@ -62,10 +62,10 @@ public class TestStoreFileRefresherChore { private Path testDir; @Before - public void setUp() { + public void setUp() throws IOException { TEST_UTIL = new HBaseTestingUtility(); testDir = TEST_UTIL.getDataTestDir("TestStoreFileRefresherChore"); - TEST_UTIL.getConfiguration().set(HConstants.HBASE_DIR, testDir.toString()); + FSUtils.setRootDir(TEST_UTIL.getConfiguration(), testDir); } private HTableDescriptor getTableDesc(TableName tableName, byte[]... families) { @@ -101,8 +101,10 @@ public class TestStoreFileRefresherChore { HRegionInfo info = new HRegionInfo(htd.getTableName(), startKey, stopKey, false, 0, replicaId); HRegionFileSystem fs = new FailingHRegionFileSystem(conf, tableDir.getFileSystem(conf), tableDir, info); - HRegion region = new HRegion(fs, HLogFactory.createHLog(fs.getFileSystem(), - tableDir, "log_" + replicaId, conf), conf, htd, null); + final Configuration walConf = new Configuration(conf); + FSUtils.setRootDir(walConf, tableDir); + final WALFactory wals = new WALFactory(walConf, null, "log_" + replicaId); + HRegion region = new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes()), conf, htd, null); region.initialize(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultyHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultyHLog.java deleted file mode 100644 index 10ba82faaec..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultyHLog.java +++ /dev/null @@ -1,70 +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.IOException; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.hadoop.conf.Configuration; -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; - -/* - * This is a utility class, used by tests, which fails operation specified by FailureType enum - */ -public class FaultyHLog extends FSHLog { - public enum FailureType { - NONE, APPENDNOSYNC, SYNC - } - FailureType ft = FailureType.NONE; - - public FaultyHLog(FileSystem fs, Path rootDir, String logName, Configuration conf) - throws IOException { - super(fs, rootDir, logName, conf); - } - - public void setFailureType(FailureType fType) { - this.ft = fType; - } - - @Override - public void sync(long txid) throws IOException { - if (this.ft == FailureType.SYNC) { - throw new IOException("sync"); - } - super.sync(txid); - } - @Override - public 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 { - if (this.ft == FailureType.APPENDNOSYNC) { - throw new IOException("appendNoSync"); - } - return super.appendNoSync(info, tableName, edits, clusterIds, now, htd, sequenceId, - isInMemstore, nonceGroup, nonce); - } -} - diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java index 2164a435818..a0e44909b65 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java @@ -23,12 +23,12 @@ 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.wal.WAL.Entry; public class FaultySequenceFileLogReader extends SequenceFileLogReader { - enum FailureType { + // public until class relocates to o.a.h.h.wal + public enum FailureType { BEGINNING, MIDDLE, END, NONE } @@ -40,17 +40,17 @@ 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); } - b = this.reader.next(e.getKey(), e.getEdit()); + b = readNext(e); nextQueue.offer(e); numberOfFileEntries++; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java deleted file mode 100644 index f56ef98b65b..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java +++ /dev/null @@ -1,566 +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.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.UUID; -import java.util.concurrent.TimeUnit; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.conf.Configured; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; -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.trace.SpanReceiverHost; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.util.Tool; -import org.apache.hadoop.util.ToolRunner; -import org.htrace.Sampler; -import org.htrace.Trace; -import org.htrace.TraceScope; -import org.htrace.impl.ProbabilitySampler; - -import com.yammer.metrics.core.Histogram; -import com.yammer.metrics.core.Meter; -import com.yammer.metrics.core.MetricsRegistry; -import com.yammer.metrics.reporting.ConsoleReporter; - -/** - * This class runs performance benchmarks for {@link HLog}. - * See usage for this tool by running: - * $ hbase org.apache.hadoop.hbase.regionserver.wal.HLogPerformanceEvaluation -h - */ -@InterfaceAudience.Private -public final class HLogPerformanceEvaluation extends Configured implements Tool { - static final Log LOG = LogFactory.getLog(HLogPerformanceEvaluation.class.getName()); - private final MetricsRegistry metrics = new MetricsRegistry(); - private final Meter syncMeter = - metrics.newMeter(HLogPerformanceEvaluation.class, "syncMeter", "syncs", TimeUnit.MILLISECONDS); - private final Histogram syncHistogram = - metrics.newHistogram(HLogPerformanceEvaluation.class, "syncHistogram", "nanos-between-syncs", - true); - private final Histogram syncCountHistogram = - metrics.newHistogram(HLogPerformanceEvaluation.class, "syncCountHistogram", "countPerSync", - true); - private final Meter appendMeter = - metrics.newMeter(HLogPerformanceEvaluation.class, "appendMeter", "bytes", - TimeUnit.MILLISECONDS); - private final Histogram latencyHistogram = - metrics.newHistogram(HLogPerformanceEvaluation.class, "latencyHistogram", "nanos", true); - - private HBaseTestingUtility TEST_UTIL; - - static final String TABLE_NAME = "HLogPerformanceEvaluation"; - static final String QUALIFIER_PREFIX = "q"; - static final String FAMILY_PREFIX = "cf"; - - private int numQualifiers = 1; - private int valueSize = 512; - private int keySize = 16; - - @Override - public void setConf(Configuration conf) { - super.setConf(conf); - TEST_UTIL = new HBaseTestingUtility(conf); - } - - /** - * Perform HLog.append() of Put object, for the number of iterations requested. - * Keys and Vaues are generated randomly, the number of column families, - * qualifiers and key/value size is tunable by the user. - */ - class HLogPutBenchmark implements Runnable { - private final long numIterations; - private final int numFamilies; - private final boolean noSync; - private final HRegion region; - private final int syncInterval; - private final HTableDescriptor htd; - private final Sampler loopSampler; - - HLogPutBenchmark(final HRegion region, final HTableDescriptor htd, - final long numIterations, final boolean noSync, final int syncInterval, - final double traceFreq) { - this.numIterations = numIterations; - this.noSync = noSync; - this.syncInterval = syncInterval; - this.numFamilies = htd.getColumnFamilies().length; - this.region = region; - this.htd = htd; - String spanReceivers = getConf().get("hbase.trace.spanreceiver.classes"); - if (spanReceivers == null || spanReceivers.isEmpty()) { - loopSampler = Sampler.NEVER; - } else { - if (traceFreq <= 0.0) { - LOG.warn("Tracing enabled but traceFreq=0."); - loopSampler = Sampler.NEVER; - } else if (traceFreq >= 1.0) { - loopSampler = Sampler.ALWAYS; - if (numIterations > 1000) { - LOG.warn("Full tracing of all iterations will produce a lot of data. Be sure your" - + " SpanReciever can keep up."); - } - } else { - loopSampler = new ProbabilitySampler(traceFreq); - } - } - } - - @Override - public void run() { - byte[] key = new byte[keySize]; - byte[] value = new byte[valueSize]; - Random rand = new Random(Thread.currentThread().getId()); - HLog hlog = region.getLog(); - ArrayList clusters = new ArrayList(); - long nonce = HConstants.NO_NONCE; - - TraceScope threadScope = - Trace.startSpan("HLogPerfEval." + Thread.currentThread().getName()); - try { - long startTime = System.currentTimeMillis(); - int lastSync = 0; - for (int i = 0; i < numIterations; ++i) { - assert Trace.currentSpan() == threadScope.getSpan() : "Span leak detected."; - TraceScope loopScope = Trace.startSpan("runLoopIter" + i, loopSampler); - try { - long now = System.nanoTime(); - Put put = setupPut(rand, key, value, numFamilies); - WALEdit walEdit = new WALEdit(); - addFamilyMapToWALEdit(put.getFamilyCellMap(), walEdit); - HRegionInfo hri = region.getRegionInfo(); - hlog.appendNoSync(hri, hri.getTable(), walEdit, clusters, now, htd, - region.getSequenceId(), true, nonce, nonce); - if (!this.noSync) { - if (++lastSync >= this.syncInterval) { - hlog.sync(); - lastSync = 0; - } - } - latencyHistogram.update(System.nanoTime() - now); - } finally { - loopScope.close(); - } - } - long totalTime = (System.currentTimeMillis() - startTime); - logBenchmarkResult(Thread.currentThread().getName(), numIterations, totalTime); - } catch (Exception e) { - LOG.error(getClass().getSimpleName() + " Thread failed", e); - } finally { - threadScope.close(); - } - } - } - - @Override - public int run(String[] args) throws Exception { - Path rootRegionDir = null; - int numThreads = 1; - long numIterations = 1000000; - int numFamilies = 1; - int syncInterval = 0; - boolean noSync = false; - boolean verify = false; - boolean verbose = false; - boolean cleanup = true; - boolean noclosefs = false; - long roll = Long.MAX_VALUE; - boolean compress = false; - String cipher = null; - String spanReceivers = getConf().get("hbase.trace.spanreceiver.classes"); - boolean trace = spanReceivers != null && !spanReceivers.isEmpty(); - double traceFreq = 1.0; - // Process command line args - for (int i = 0; i < args.length; i++) { - String cmd = args[i]; - try { - if (cmd.equals("-threads")) { - numThreads = Integer.parseInt(args[++i]); - } else if (cmd.equals("-iterations")) { - numIterations = Long.parseLong(args[++i]); - } else if (cmd.equals("-path")) { - rootRegionDir = new Path(args[++i]); - } else if (cmd.equals("-families")) { - numFamilies = Integer.parseInt(args[++i]); - } else if (cmd.equals("-qualifiers")) { - numQualifiers = Integer.parseInt(args[++i]); - } else if (cmd.equals("-keySize")) { - keySize = Integer.parseInt(args[++i]); - } else if (cmd.equals("-valueSize")) { - valueSize = Integer.parseInt(args[++i]); - } else if (cmd.equals("-syncInterval")) { - syncInterval = Integer.parseInt(args[++i]); - } else if (cmd.equals("-nosync")) { - noSync = true; - } else if (cmd.equals("-verify")) { - verify = true; - } else if (cmd.equals("-verbose")) { - verbose = true; - } else if (cmd.equals("-nocleanup")) { - cleanup = false; - } else if (cmd.equals("-noclosefs")) { - noclosefs = true; - } else if (cmd.equals("-roll")) { - roll = Long.parseLong(args[++i]); - } else if (cmd.equals("-compress")) { - compress = true; - } else if (cmd.equals("-encryption")) { - cipher = args[++i]; - } else if (cmd.equals("-traceFreq")) { - traceFreq = Double.parseDouble(args[++i]); - } else if (cmd.equals("-h")) { - printUsageAndExit(); - } else if (cmd.equals("--help")) { - printUsageAndExit(); - } else { - System.err.println("UNEXPECTED: " + cmd); - printUsageAndExit(); - } - } catch (Exception e) { - printUsageAndExit(); - } - } - - if (compress) { - Configuration conf = getConf(); - conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true); - } - - if (cipher != null) { - // Set up HLog for encryption - Configuration conf = getConf(); - 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); - conf.setClass("hbase.regionserver.hlog.writer.impl", SecureProtobufLogWriter.class, - HLog.Writer.class); - conf.setBoolean(HConstants.ENABLE_WAL_ENCRYPTION, true); - conf.set(HConstants.CRYPTO_WAL_ALGORITHM_CONF_KEY, cipher); - } - - // Internal config. goes off number of threads; if more threads than handlers, stuff breaks. - // In regionserver, number of handlers == number of threads. - getConf().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, numThreads); - - // Run HLog Performance Evaluation - // First set the fs from configs. In case we are on hadoop1 - FSUtils.setFsDefault(getConf(), FSUtils.getRootDir(getConf())); - FileSystem fs = FileSystem.get(getConf()); - LOG.info("FileSystem: " + fs); - - SpanReceiverHost receiverHost = trace ? SpanReceiverHost.getInstance(getConf()) : null; - TraceScope scope = Trace.startSpan("HLogPerfEval", trace ? Sampler.ALWAYS : Sampler.NEVER); - - try { - if (rootRegionDir == null) { - rootRegionDir = TEST_UTIL.getDataTestDirOnTestFS("HLogPerformanceEvaluation"); - } - rootRegionDir = rootRegionDir.makeQualified(fs); - cleanRegionRootDir(fs, rootRegionDir); - // Initialize Table Descriptor - HTableDescriptor htd = createHTableDescriptor(numFamilies); - final long whenToRoll = roll; - final HLog hlog = new FSHLog(fs, rootRegionDir, "wals", getConf()) { - - @Override - public void postSync(final long timeInNanos, final int handlerSyncs) { - super.postSync(timeInNanos, handlerSyncs); - syncMeter.mark(); - syncHistogram.update(timeInNanos); - syncCountHistogram.update(handlerSyncs); - } - - @Override - public long postAppend(final HLog.Entry entry, final long elapsedTime) { - long size = super.postAppend(entry, elapsedTime); - appendMeter.mark(size); - return size; - } - }; - hlog.registerWALActionsListener(new WALActionsListener() { - private int appends = 0; - - @Override - public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, - WALEdit logEdit) { - this.appends++; - if (this.appends % whenToRoll == 0) { - LOG.info("Rolling after " + appends + " edits"); - // We used to do explicit call to rollWriter but changed it to a request - // to avoid dead lock (there are less threads going on in this class than - // in the regionserver -- regionserver does not have the issue). - ((FSHLog)hlog).requestLogRoll(); - } - } - - @Override - public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit) { - } - - @Override - public void preLogRoll(Path oldPath, Path newPath) throws IOException { - } - - @Override - public void preLogArchive(Path oldPath, Path newPath) throws IOException { - } - - @Override - public void postLogRoll(Path oldPath, Path newPath) throws IOException { - } - - @Override - public void postLogArchive(Path oldPath, Path newPath) throws IOException { - } - - @Override - public void logRollRequested() { - } - - @Override - public void logCloseRequested() { - } - }); - hlog.rollWriter(); - HRegion region = null; - - try { - region = openRegion(fs, rootRegionDir, htd, hlog); - ConsoleReporter.enable(this.metrics, 30, TimeUnit.SECONDS); - long putTime = - runBenchmark(Trace.wrap( - new HLogPutBenchmark(region, htd, numIterations, noSync, syncInterval, traceFreq)), - numThreads); - logBenchmarkResult("Summary: threads=" + numThreads + ", iterations=" + numIterations + - ", syncInterval=" + syncInterval, numIterations * numThreads, putTime); - - if (region != null) { - closeRegion(region); - region = null; - } - if (verify) { - Path dir = ((FSHLog) hlog).getDir(); - long editCount = 0; - FileStatus [] fsss = fs.listStatus(dir); - if (fsss.length == 0) throw new IllegalStateException("No WAL found"); - for (FileStatus fss: fsss) { - Path p = fss.getPath(); - if (!fs.exists(p)) throw new IllegalStateException(p.toString()); - editCount += verify(p, verbose); - } - long expected = numIterations * numThreads; - if (editCount != expected) { - throw new IllegalStateException("Counted=" + editCount + ", expected=" + expected); - } - } - } finally { - if (region != null) closeRegion(region); - // Remove the root dir for this test region - if (cleanup) cleanRegionRootDir(fs, rootRegionDir); - } - } finally { - // We may be called inside a test that wants to keep on using the fs. - if (!noclosefs) fs.close(); - scope.close(); - if (receiverHost != null) receiverHost.closeReceivers(); - } - - return(0); - } - - private static HTableDescriptor createHTableDescriptor(final int numFamilies) { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE_NAME)); - for (int i = 0; i < numFamilies; ++i) { - HColumnDescriptor colDef = new HColumnDescriptor(FAMILY_PREFIX + i); - htd.addFamily(colDef); - } - return htd; - } - - /** - * Verify the content of the WAL file. - * Verify that the file has expected number of edits. - * @param wal - * @return Count of edits. - * @throws IOException - */ - private long verify(final Path wal, final boolean verbose) throws IOException { - HLog.Reader reader = HLogFactory.createReader(wal.getFileSystem(getConf()), wal, getConf()); - long count = 0; - Map sequenceIds = new HashMap(); - try { - while (true) { - Entry e = reader.next(); - if (e == null) { - LOG.debug("Read count=" + count + " from " + wal); - break; - } - count++; - long seqid = e.getKey().getLogSeqNum(); - if (sequenceIds.containsKey(Bytes.toString(e.getKey().getEncodedRegionName()))) { - // sequenceIds should be increasing for every regions - if (sequenceIds.get(Bytes.toString(e.getKey().getEncodedRegionName())) >= seqid) { - throw new IllegalStateException("wal = " + wal.getName() + ", " + "previous seqid = " - + sequenceIds.get(Bytes.toString(e.getKey().getEncodedRegionName())) - + ", current seqid = " + seqid); - } - } - // update the sequence Id. - sequenceIds.put(Bytes.toString(e.getKey().getEncodedRegionName()), seqid); - if (verbose) LOG.info("seqid=" + seqid); - } - } finally { - reader.close(); - } - return count; - } - - private static void logBenchmarkResult(String testName, long numTests, long totalTime) { - float tsec = totalTime / 1000.0f; - LOG.info(String.format("%s took %.3fs %.3fops/s", testName, tsec, numTests / tsec)); - - } - - private void printUsageAndExit() { - System.err.printf("Usage: bin/hbase %s [options]\n", getClass().getName()); - System.err.println(" where [options] are:"); - System.err.println(" -h|-help Show this help and exit."); - System.err.println(" -threads Number of threads writing on the WAL."); - System.err.println(" -iterations Number of iterations per thread."); - System.err.println(" -path Path where region's root directory is created."); - System.err.println(" -families Number of column families to write."); - System.err.println(" -qualifiers Number of qualifiers to write."); - System.err.println(" -keySize Row key size in byte."); - System.err.println(" -valueSize Row/Col value size in byte."); - System.err.println(" -nocleanup Do NOT remove test data when done."); - System.err.println(" -noclosefs Do NOT close the filesystem when done."); - System.err.println(" -nosync Append without syncing"); - System.err.println(" -syncInterval Append N edits and then sync. " + - "Default=0, i.e. sync every edit."); - System.err.println(" -verify Verify edits written in sequence"); - System.err.println(" -verbose Output extra info; " + - "e.g. all edit seq ids when verifying"); - System.err.println(" -roll Roll the way every N appends"); - System.err.println(" -encryption Encrypt the WAL with algorithm A, e.g. AES"); - System.err.println(" -traceFreq Rate of trace sampling. Default: 1.0, " + - "only respected when tracing is enabled, ie -Dhbase.trace.spanreceiver.classes=..."); - System.err.println(""); - System.err.println("Examples:"); - System.err.println(""); - System.err.println(" To run 100 threads on hdfs with log rolling every 10k edits and " + - "verification afterward do:"); - System.err.println(" $ ./bin/hbase org.apache.hadoop.hbase.regionserver.wal." + - "HLogPerformanceEvaluation \\"); - System.err.println(" -conf ./core-site.xml -path hdfs://example.org:7000/tmp " + - "-threads 100 -roll 10000 -verify"); - System.exit(1); - } - - private HRegion openRegion(final FileSystem fs, final Path dir, final HTableDescriptor htd, - final HLog hlog) - throws IOException { - // Initialize HRegion - HRegionInfo regionInfo = new HRegionInfo(htd.getTableName()); - return HRegion.createHRegion(regionInfo, dir, getConf(), htd, hlog); - } - - private void closeRegion(final HRegion region) throws IOException { - if (region != null) { - region.close(); - HLog wal = region.getLog(); - if (wal != null) wal.close(); - } - } - - private void cleanRegionRootDir(final FileSystem fs, final Path dir) throws IOException { - if (fs.exists(dir)) { - fs.delete(dir, true); - } - } - - private Put setupPut(Random rand, byte[] key, byte[] value, final int numFamilies) { - rand.nextBytes(key); - Put put = new Put(key); - for (int cf = 0; cf < numFamilies; ++cf) { - for (int q = 0; q < numQualifiers; ++q) { - rand.nextBytes(value); - put.add(Bytes.toBytes(FAMILY_PREFIX + cf), Bytes.toBytes(QUALIFIER_PREFIX + q), value); - } - } - return put; - } - - private void addFamilyMapToWALEdit(Map> familyMap, - WALEdit walEdit) { - for (List edits : familyMap.values()) { - for (Cell cell : edits) { - walEdit.add(cell); - } - } - } - - private long runBenchmark(Runnable runnable, final int numThreads) throws InterruptedException { - Thread[] threads = new Thread[numThreads]; - long startTime = System.currentTimeMillis(); - for (int i = 0; i < numThreads; ++i) { - threads[i] = new Thread(runnable, "t" + i); - threads[i].start(); - } - for (Thread t : threads) t.join(); - long endTime = System.currentTimeMillis(); - return(endTime - startTime); - } - - /** - * The guts of the {@link #main} method. - * Call this method to avoid the {@link #main(String[])} System.exit. - * @param args - * @return errCode - * @throws Exception - */ - static int innerMain(final Configuration c, final String [] args) throws Exception { - return ToolRunner.run(c, new HLogPerformanceEvaluation(), args); - } - - public static void main(String[] args) throws Exception { - System.exit(innerMain(HBaseConfiguration.create(), args)); - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtilsForTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtilsForTests.java deleted file mode 100644 index f2fcf80bf58..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtilsForTests.java +++ /dev/null @@ -1,43 +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; - - -/** - * An Utility testcase that returns the number of log files that - * were rolled to be accessed from outside packages. - * - * This class makes available methods that are package protected. - * 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 getNumEntries(HLog log) { - return ((FSHLog) log).getNumEntries(); - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/InstrumentedSequenceFileLogWriter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/InstrumentedSequenceFileLogWriter.java deleted file mode 100644 index d240e66afbc..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/InstrumentedSequenceFileLogWriter.java +++ /dev/null @@ -1,40 +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.IOException; - -import org.apache.hadoop.hbase.util.Bytes; - -public class InstrumentedSequenceFileLogWriter extends ProtobufLogWriter { - - public InstrumentedSequenceFileLogWriter() { - super(); - } - - public static boolean activateFailure = false; - @Override - public void append(HLog.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..."); - throw(new IOException("This exception is instrumented and should only be thrown for testing")); - } - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java index 221f76eaabc..7c13c004817 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java @@ -32,8 +32,9 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALProvider; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.SequenceFile.Metadata; @@ -42,8 +43,13 @@ 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 WALProvider.Writer} that delegates to * SequenceFile.Writer. Legacy implementation only used for compat tests. + * + * Note that because this class writes to the legacy hadoop-specific SequenceFile + * format, users of it must write {@link HLogKey} keys and not arbitrary + * {@link WALKey}s because the latter are not Writables (nor made to work with + * Hadoop serialization). */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) public class SequenceFileLogWriter extends WriterBase { @@ -163,7 +169,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()); @@ -213,11 +219,4 @@ public class SequenceFileLogWriter extends WriterBase { public FSDataOutputStream getWriterFSDataOutputStream() { return this.writer_out; } - - /** - * This method is empty as trailer is added only in Protobuf based hlog readers/writers. - */ - @Override - public void setWALTrailer(WALTrailer walTrailer) { - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java index 9cc8f2fc917..10e7e3d66f0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java @@ -38,6 +38,10 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -45,7 +49,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; /** - * Tests for HLog write durability + * Tests for WAL write durability */ @Category({RegionServerTests.class, MediumTests.class}) public class TestDurability { @@ -68,6 +72,7 @@ public class TestDurability { CLUSTER = TEST_UTIL.getDFSCluster(); FS = CLUSTER.getFileSystem(); DIR = TEST_UTIL.getDataTestDirOnTestFS("TestDurability"); + FSUtils.setRootDir(CONF, DIR); } @AfterClass @@ -77,14 +82,14 @@ public class TestDurability { @Test public void testDurability() throws Exception { - HLog wal = HLogFactory.createHLog(FS, DIR, "hlogdir", - "hlogdir_archive", CONF); + final WALFactory wals = new WALFactory(CONF, null, "TestDurability"); byte[] tableName = Bytes.toBytes("TestDurability"); + final WAL wal = wals.getWAL(tableName); HRegion region = createHRegion(tableName, "region", wal, Durability.USE_DEFAULT); HRegion deferredRegion = createHRegion(tableName, "deferredRegion", wal, Durability.ASYNC_WAL); region.put(newPut(null)); - verifyHLogCount(wal, 1); + verifyWALCount(wals, wal, 1); // a put through the deferred table does not write to the wal immediately, // but maybe has been successfully sync-ed by the underlying AsyncWriter + @@ -92,44 +97,44 @@ public class TestDurability { deferredRegion.put(newPut(null)); // but will after we sync the wal wal.sync(); - verifyHLogCount(wal, 2); + verifyWALCount(wals, wal, 2); // a put through a deferred table will be sync with the put sync'ed put deferredRegion.put(newPut(null)); wal.sync(); - verifyHLogCount(wal, 3); + verifyWALCount(wals, wal, 3); region.put(newPut(null)); - verifyHLogCount(wal, 4); + verifyWALCount(wals, wal, 4); // a put through a deferred table will be sync with the put sync'ed put deferredRegion.put(newPut(Durability.USE_DEFAULT)); wal.sync(); - verifyHLogCount(wal, 5); + verifyWALCount(wals, wal, 5); region.put(newPut(Durability.USE_DEFAULT)); - verifyHLogCount(wal, 6); + verifyWALCount(wals, wal, 6); // SKIP_WAL never writes to the wal region.put(newPut(Durability.SKIP_WAL)); deferredRegion.put(newPut(Durability.SKIP_WAL)); - verifyHLogCount(wal, 6); + verifyWALCount(wals, wal, 6); wal.sync(); - verifyHLogCount(wal, 6); + verifyWALCount(wals, wal, 6); // Async overrides sync table default region.put(newPut(Durability.ASYNC_WAL)); deferredRegion.put(newPut(Durability.ASYNC_WAL)); wal.sync(); - verifyHLogCount(wal, 8); + verifyWALCount(wals, wal, 8); // sync overrides async table default region.put(newPut(Durability.SYNC_WAL)); deferredRegion.put(newPut(Durability.SYNC_WAL)); - verifyHLogCount(wal, 10); + verifyWALCount(wals, wal, 10); // fsync behaves like sync region.put(newPut(Durability.FSYNC_WAL)); deferredRegion.put(newPut(Durability.FSYNC_WAL)); - verifyHLogCount(wal, 12); + verifyWALCount(wals, wal, 12); } @Test @@ -140,9 +145,9 @@ public class TestDurability { byte[] col3 = Bytes.toBytes("col3"); // Setting up region - HLog wal = HLogFactory.createHLog(FS, DIR, "myhlogdir", - "myhlogdir_archive", CONF); + final WALFactory wals = new WALFactory(CONF, null, "TestIncrement"); byte[] tableName = Bytes.toBytes("TestIncrement"); + final WAL wal = wals.getWAL(tableName); HRegion region = createHRegion(tableName, "increment", wal, Durability.USE_DEFAULT); // col1: amount = 1, 1 write back to WAL @@ -151,7 +156,7 @@ public class TestDurability { Result res = region.increment(inc1); assertEquals(1, res.size()); assertEquals(1, Bytes.toLong(res.getValue(FAMILY, col1))); - verifyHLogCount(wal, 1); + verifyWALCount(wals, wal, 1); // col1: amount = 0, 0 write back to WAL inc1 = new Increment(row1); @@ -159,7 +164,7 @@ public class TestDurability { res = region.increment(inc1); assertEquals(1, res.size()); assertEquals(1, Bytes.toLong(res.getValue(FAMILY, col1))); - verifyHLogCount(wal, 1); + verifyWALCount(wals, wal, 1); // col1: amount = 0, col2: amount = 0, col3: amount = 0 // 0 write back to WAL @@ -172,7 +177,7 @@ public class TestDurability { assertEquals(1, Bytes.toLong(res.getValue(FAMILY, col1))); assertEquals(0, Bytes.toLong(res.getValue(FAMILY, col2))); assertEquals(0, Bytes.toLong(res.getValue(FAMILY, col3))); - verifyHLogCount(wal, 1); + verifyWALCount(wals, wal, 1); // col1: amount = 5, col2: amount = 4, col3: amount = 3 // 1 write back to WAL @@ -185,7 +190,7 @@ public class TestDurability { assertEquals(6, Bytes.toLong(res.getValue(FAMILY, col1))); assertEquals(4, Bytes.toLong(res.getValue(FAMILY, col2))); assertEquals(3, Bytes.toLong(res.getValue(FAMILY, col3))); - verifyHLogCount(wal, 2); + verifyWALCount(wals, wal, 2); } private Put newPut(Durability durability) { @@ -197,11 +202,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 verifyWALCount(WALFactory wals, WAL log, int expected) throws Exception { + Path walPath = DefaultWALProvider.getCurrentFileName(log); + WAL.Reader reader = wals.createReader(FS, walPath); 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); @@ -209,7 +214,7 @@ public class TestDurability { // lifted from TestAtomicOperation private HRegion createHRegion (byte [] tableName, String callingMethod, - HLog log, Durability durability) + WAL log, Durability durability) throws IOException { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); htd.setDurability(durability); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java deleted file mode 100644 index fe1f740a7aa..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java +++ /dev/null @@ -1,1343 +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 static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.io.IOException; -import java.lang.reflect.Method; -import java.net.BindException; -import java.util.ArrayList; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.commons.lang.mutable.MutableBoolean; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.commons.logging.impl.Log4JLogger; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -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.hbase.Cell; -import org.apache.hadoop.hbase.CellScanner; -import org.apache.hadoop.hbase.Coprocessor; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; -import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdge; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.hdfs.DFSClient; -import org.apache.hadoop.hdfs.DistributedFileSystem; -import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.hdfs.protocol.FSConstants; -import org.apache.hadoop.hdfs.server.datanode.DataNode; -import org.apache.hadoop.hdfs.server.namenode.LeaseManager; -import org.apache.log4j.Level; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -/** JUnit test case for HLog */ -@Category({RegionServerTests.class, LargeTests.class}) -@SuppressWarnings("deprecation") -public class TestHLog { - private static final Log LOG = LogFactory.getLog(TestHLog.class); - { - // Uncomment the following lines if more verbosity is needed for - // debugging (see HBASE-12285 for details). - //((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); - } - - private static Configuration conf; - private static FileSystem fs; - private static Path dir; - private static MiniDFSCluster cluster; - private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static Path hbaseDir; - private static Path oldLogDir; - - @Before - public void setUp() throws Exception { - FileStatus[] entries = fs.listStatus(new Path("/")); - for (FileStatus dir : entries) { - fs.delete(dir.getPath(), true); - } - } - - @After - public void tearDown() throws Exception { - } - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - // Make block sizes small. - TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024); - // needed for testAppendClose() - TEST_UTIL.getConfiguration().setBoolean("dfs.support.broken.append", true); - TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true); - // quicker heartbeat interval for faster DN death notification - TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000); - TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1); - TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000); - - // faster failover with cluster.shutdown();fs.close() idiom - TEST_UTIL.getConfiguration() - .setInt("hbase.ipc.client.connect.max.retries", 1); - TEST_UTIL.getConfiguration().setInt( - "dfs.client.block.recovery.retries", 1); - TEST_UTIL.getConfiguration().setInt( - "hbase.ipc.client.connection.maxidletime", 500); - TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY, - SampleRegionWALObserver.class.getName()); - TEST_UTIL.startMiniDFSCluster(3); - - conf = TEST_UTIL.getConfiguration(); - cluster = TEST_UTIL.getDFSCluster(); - fs = cluster.getFileSystem(); - - hbaseDir = TEST_UTIL.createRootDir(); - oldLogDir = new Path(hbaseDir, HConstants.HREGION_OLDLOGDIR_NAME); - dir = new Path(hbaseDir, getName()); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - TEST_UTIL.shutdownMiniCluster(); - } - - private static String getName() { - // TODO Auto-generated method stub - return "TestHLog"; - } - - /** - * Test flush for sure has a sequence id that is beyond the last edit appended. We do this - * by slowing appends in the background ring buffer thread while in foreground we call - * flush. The addition of the sync over HRegion in flush should fix an issue where flush was - * returning before all of its appends had made it out to the WAL (HBASE-11109). - * @throws IOException - * @see HBASE-11109 - */ - @Test - public void testFlushSequenceIdIsGreaterThanAllEditsInHFile() throws IOException { - String testName = "testFlushSequenceIdIsGreaterThanAllEditsInHFile"; - final TableName tableName = TableName.valueOf(testName); - final HRegionInfo hri = new HRegionInfo(tableName); - final byte[] rowName = tableName.getName(); - final HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor("f")); - HRegion r = HRegion.createHRegion(hri, TEST_UTIL.getDefaultRootDirPath(), - TEST_UTIL.getConfiguration(), htd); - HRegion.closeHRegion(r); - final int countPerFamily = 10; - final MutableBoolean goslow = new MutableBoolean(false); - // Bypass factory so I can subclass and doctor a method. - FSHLog wal = new FSHLog(FileSystem.get(conf), TEST_UTIL.getDefaultRootDirPath(), - testName, conf) { - @Override - void atHeadOfRingBufferEventHandlerAppend() { - if (goslow.isTrue()) { - Threads.sleep(100); - LOG.debug("Sleeping before appending 100ms"); - } - super.atHeadOfRingBufferEventHandlerAppend(); - } - }; - HRegion region = HRegion.openHRegion(TEST_UTIL.getConfiguration(), - TEST_UTIL.getTestFileSystem(), TEST_UTIL.getDefaultRootDirPath(), hri, htd, wal); - EnvironmentEdge ee = EnvironmentEdgeManager.getDelegate(); - try { - List puts = null; - for (HColumnDescriptor hcd: htd.getFamilies()) { - puts = - TestWALReplay.addRegionEdits(rowName, hcd.getName(), countPerFamily, ee, region, "x"); - } - - // Now assert edits made it in. - final Get g = new Get(rowName); - Result result = region.get(g); - assertEquals(countPerFamily * htd.getFamilies().size(), result.size()); - - // Construct a WALEdit and add it a few times to the WAL. - WALEdit edits = new WALEdit(); - for (Put p: puts) { - CellScanner cs = p.cellScanner(); - while (cs.advance()) { - edits.add(cs.current()); - } - } - // Add any old cluster id. - List clusterIds = new ArrayList(); - clusterIds.add(UUID.randomUUID()); - // Now make appends run slow. - goslow.setValue(true); - for (int i = 0; i < countPerFamily; i++) { - wal.appendNoSync(region.getRegionInfo(), tableName, edits, - clusterIds, System.currentTimeMillis(), htd, region.getSequenceId(), true, -1, -1); - } - region.flushcache(); - // FlushResult.flushSequenceId is not visible here so go get the current sequence id. - long currentSequenceId = region.getSequenceId().get(); - // Now release the appends - goslow.setValue(false); - synchronized (goslow) { - goslow.notifyAll(); - } - assertTrue(currentSequenceId >= region.getSequenceId().get()); - } finally { - region.close(true); - wal.close(); - } - } - - /** - * Write to a log file with three concurrent threads and verifying all data is written. - * @throws Exception - */ - @Test - public void testConcurrentWrites() throws Exception { - // Run the HPE tool with three threads writing 3000 edits each concurrently. - // When done, verify that all edits were written. - int errCode = HLogPerformanceEvaluation. - innerMain(new Configuration(TEST_UTIL.getConfiguration()), - new String [] {"-threads", "3", "-verify", "-noclosefs", "-iterations", "3000"}); - assertEquals(0, errCode); - } - - /** - * Just write multiple logs then split. Before fix for HADOOP-2283, this - * would fail. - * @throws IOException - */ - @Test - public void testSplit() throws IOException { - final TableName tableName = - TableName.valueOf(getName()); - final byte [] rowName = tableName.getName(); - Path logdir = new Path(hbaseDir, HConstants.HREGION_LOGDIR_NAME); - HLog log = HLogFactory.createHLog(fs, hbaseDir, HConstants.HREGION_LOGDIR_NAME, conf); - final int howmany = 3; - HRegionInfo[] infos = new HRegionInfo[3]; - Path tabledir = FSUtils.getTableDir(hbaseDir, tableName); - fs.mkdirs(tabledir); - for(int i = 0; i < howmany; i++) { - infos[i] = new HRegionInfo(tableName, - Bytes.toBytes("" + i), Bytes.toBytes("" + (i+1)), false); - fs.mkdirs(new Path(tabledir, infos[i].getEncodedName())); - LOG.info("allo " + new Path(tabledir, infos[i].getEncodedName()).toString()); - } - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor("column")); - - // Add edits for three regions. - final AtomicLong sequenceId = new AtomicLong(1); - try { - for (int ii = 0; ii < howmany; ii++) { - for (int i = 0; i < howmany; i++) { - - for (int j = 0; j < howmany; j++) { - WALEdit edit = new WALEdit(); - byte [] family = Bytes.toBytes("column"); - byte [] qualifier = Bytes.toBytes(Integer.toString(j)); - byte [] column = Bytes.toBytes("column:" + Integer.toString(j)); - edit.add(new KeyValue(rowName, family, qualifier, - System.currentTimeMillis(), column)); - LOG.info("Region " + i + ": " + edit); - log.append(infos[i], tableName, edit, - System.currentTimeMillis(), htd, sequenceId); - } - } - log.rollWriter(); - } - log.close(); - List splits = HLogSplitter.split(hbaseDir, logdir, oldLogDir, fs, conf); - verifySplits(splits, howmany); - log = null; - } finally { - if (log != null) { - log.closeAndDelete(); - } - } - } - - /** - * Test new HDFS-265 sync. - * @throws Exception - */ - @Test - public void Broken_testSync() throws Exception { - TableName tableName = - TableName.valueOf(getName()); - // First verify that using streams all works. - Path p = new Path(dir, getName() + ".fsdos"); - FSDataOutputStream out = fs.create(p); - out.write(tableName.getName()); - Method syncMethod = null; - try { - syncMethod = out.getClass().getMethod("hflush", new Class []{}); - } catch (NoSuchMethodException e) { - try { - syncMethod = out.getClass().getMethod("sync", new Class []{}); - } catch (NoSuchMethodException ex) { - fail("This version of Hadoop supports neither Syncable.sync() " + - "nor Syncable.hflush()."); - } - } - syncMethod.invoke(out, new Object[]{}); - FSDataInputStream in = fs.open(p); - assertTrue(in.available() > 0); - byte [] buffer = new byte [1024]; - int read = in.read(buffer); - assertEquals(tableName.getName().length, read); - out.close(); - in.close(); - - HLog wal = HLogFactory.createHLog(fs, dir, "hlogdir", conf); - final AtomicLong sequenceId = new AtomicLong(1); - final int total = 20; - HLog.Reader reader = null; - - try { - HRegionInfo info = new HRegionInfo(tableName, - null,null, false); - HTableDescriptor htd = new HTableDescriptor(); - htd.addFamily(new HColumnDescriptor(tableName.getName())); - - for (int i = 0; i < total; i++) { - WALEdit kvs = new WALEdit(); - kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName())); - wal.append(info, tableName, kvs, System.currentTimeMillis(), htd, sequenceId); - } - // Now call sync and try reading. Opening a Reader before you sync just - // gives you EOFE. - wal.sync(); - // Open a Reader. - Path walPath = ((FSHLog) wal).computeFilename(); - reader = HLogFactory.createReader(fs, walPath, conf); - int count = 0; - HLog.Entry entry = new HLog.Entry(); - while ((entry = reader.next(entry)) != null) count++; - assertEquals(total, count); - reader.close(); - // Add test that checks to see that an open of a Reader works on a file - // that has had a sync done on it. - for (int i = 0; i < total; i++) { - WALEdit kvs = new WALEdit(); - kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName())); - wal.append(info, tableName, kvs, System.currentTimeMillis(), htd, sequenceId); - } - reader = HLogFactory.createReader(fs, walPath, conf); - count = 0; - while((entry = reader.next(entry)) != null) count++; - assertTrue(count >= total); - reader.close(); - // If I sync, should see double the edits. - wal.sync(); - reader = HLogFactory.createReader(fs, walPath, conf); - count = 0; - while((entry = reader.next(entry)) != null) count++; - assertEquals(total * 2, count); - // Now do a test that ensures stuff works when we go over block boundary, - // especially that we return good length on file. - final byte [] value = new byte[1025 * 1024]; // Make a 1M value. - for (int i = 0; i < total; i++) { - WALEdit kvs = new WALEdit(); - kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), value)); - wal.append(info, tableName, kvs, System.currentTimeMillis(), htd, sequenceId); - } - // Now I should have written out lots of blocks. Sync then read. - wal.sync(); - reader = HLogFactory.createReader(fs, walPath, conf); - count = 0; - while((entry = reader.next(entry)) != null) count++; - assertEquals(total * 3, count); - reader.close(); - // Close it and ensure that closed, Reader gets right length also. - wal.close(); - reader = HLogFactory.createReader(fs, walPath, conf); - count = 0; - while((entry = reader.next(entry)) != null) count++; - assertEquals(total * 3, count); - reader.close(); - } finally { - if (wal != null) wal.closeAndDelete(); - if (reader != null) reader.close(); - } - } - - private void verifySplits(List splits, final int howmany) - throws IOException { - 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); - try { - int count = 0; - String previousRegion = null; - long seqno = -1; - HLog.Entry entry = new HLog.Entry(); - while((entry = reader.next(entry)) != null) { - HLogKey key = entry.getKey(); - String region = Bytes.toString(key.getEncodedRegionName()); - // Assert that all edits are for same region. - if (previousRegion != null) { - assertEquals(previousRegion, region); - } - LOG.info("oldseqno=" + seqno + ", newseqno=" + key.getLogSeqNum()); - assertTrue(seqno < key.getLogSeqNum()); - seqno = key.getLogSeqNum(); - previousRegion = region; - count++; - } - assertEquals(howmany, count); - } finally { - reader.close(); - } - } - } - - /* - * We pass different values to recoverFileLease() so that different code paths are covered - * - * For this test to pass, requires: - * 1. HDFS-200 (append support) - * 2. HDFS-988 (SafeMode should freeze file operations - * [FSNamesystem.nextGenerationStampForBlock]) - * 3. HDFS-142 (on restart, maintain pendingCreates) - */ - @Test (timeout=300000) - public void testAppendClose() throws Exception { - TableName tableName = - TableName.valueOf(getName()); - HRegionInfo regioninfo = new HRegionInfo(tableName, - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false); - - HLog wal = HLogFactory.createHLog(fs, dir, "hlogdir", - "hlogdir_archive", conf); - final AtomicLong sequenceId = new AtomicLong(1); - final int total = 20; - - HTableDescriptor htd = new HTableDescriptor(); - htd.addFamily(new HColumnDescriptor(tableName.getName())); - - for (int i = 0; i < total; i++) { - WALEdit kvs = new WALEdit(); - kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName())); - wal.append(regioninfo, tableName, kvs, System.currentTimeMillis(), htd, sequenceId); - } - // Now call sync to send the data to HDFS datanodes - wal.sync(); - int namenodePort = cluster.getNameNodePort(); - final Path walPath = ((FSHLog) wal).computeFilename(); - - - // Stop the cluster. (ensure restart since we're sharing MiniDFSCluster) - try { - DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem(); - dfs.setSafeMode(FSConstants.SafeModeAction.SAFEMODE_ENTER); - TEST_UTIL.shutdownMiniDFSCluster(); - try { - // wal.writer.close() will throw an exception, - // but still call this since it closes the LogSyncer thread first - wal.close(); - } catch (IOException e) { - LOG.info(e); - } - fs.close(); // closing FS last so DFSOutputStream can't call close - LOG.info("STOPPED first instance of the cluster"); - } finally { - // Restart the cluster - while (cluster.isClusterUp()){ - LOG.error("Waiting for cluster to go down"); - Thread.sleep(1000); - } - assertFalse(cluster.isClusterUp()); - cluster = null; - for (int i = 0; i < 100; i++) { - try { - cluster = TEST_UTIL.startMiniDFSClusterForTestHLog(namenodePort); - break; - } catch (BindException e) { - LOG.info("Sleeping. BindException bringing up new cluster"); - Threads.sleep(1000); - } - } - cluster.waitActive(); - fs = cluster.getFileSystem(); - LOG.info("STARTED second instance."); - } - - // set the lease period to be 1 second so that the - // namenode triggers lease recovery upon append request - Method setLeasePeriod = cluster.getClass() - .getDeclaredMethod("setLeasePeriod", new Class[]{Long.TYPE, Long.TYPE}); - setLeasePeriod.setAccessible(true); - setLeasePeriod.invoke(cluster, 1000L, 1000L); - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - LOG.info(e); - } - - // Now try recovering the log, like the HMaster would do - final FileSystem recoveredFs = fs; - final Configuration rlConf = conf; - - class RecoverLogThread extends Thread { - public Exception exception = null; - public void run() { - try { - FSUtils.getInstance(fs, rlConf) - .recoverFileLease(recoveredFs, walPath, rlConf, null); - } catch (IOException e) { - exception = e; - } - } - } - - RecoverLogThread t = new RecoverLogThread(); - t.start(); - // Timeout after 60 sec. Without correct patches, would be an infinite loop - t.join(60 * 1000); - if(t.isAlive()) { - t.interrupt(); - throw new Exception("Timed out waiting for HLog.recoverLog()"); - } - - if (t.exception != null) - throw t.exception; - - // Make sure you can read all the content - HLog.Reader reader = HLogFactory.createReader(fs, walPath, conf); - int count = 0; - HLog.Entry entry = new HLog.Entry(); - while (reader.next(entry) != null) { - count++; - assertTrue("Should be one KeyValue per WALEdit", - entry.getEdit().getCells().size() == 1); - } - assertEquals(total, count); - reader.close(); - - // Reset the lease period - setLeasePeriod.invoke(cluster, new Object[]{new Long(60000), new Long(3600000)}); - } - - /** - * Tests that we can write out an edit, close, and then read it back in again. - * @throws IOException - */ - @Test - public void testEditAdd() throws IOException { - final int COL_COUNT = 10; - final TableName tableName = - TableName.valueOf("tablename"); - final byte [] row = Bytes.toBytes("row"); - HLog.Reader reader = null; - HLog log = null; - try { - log = HLogFactory.createHLog(fs, hbaseDir, getName(), conf); - final AtomicLong sequenceId = new AtomicLong(1); - - // Write columns named 1, 2, 3, etc. and then values of single byte - // 1, 2, 3... - long timestamp = System.currentTimeMillis(); - WALEdit cols = new WALEdit(); - for (int i = 0; i < COL_COUNT; i++) { - cols.add(new KeyValue(row, Bytes.toBytes("column"), - Bytes.toBytes(Integer.toString(i)), - timestamp, new byte[] { (byte)(i + '0') })); - } - HRegionInfo info = new HRegionInfo(tableName, - row,Bytes.toBytes(Bytes.toString(row) + "1"), false); - HTableDescriptor htd = new HTableDescriptor(); - htd.addFamily(new HColumnDescriptor("column")); - - log.append(info, tableName, cols, System.currentTimeMillis(), htd, sequenceId); - log.startCacheFlush(info.getEncodedNameAsBytes()); - log.completeCacheFlush(info.getEncodedNameAsBytes()); - log.close(); - Path filename = ((FSHLog) log).computeFilename(); - 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); - if (entry == null) break; - HLogKey key = entry.getKey(); - WALEdit val = entry.getEdit(); - assertTrue(Bytes.equals(info.getEncodedNameAsBytes(), key.getEncodedRegionName())); - assertTrue(tableName.equals(key.getTablename())); - Cell cell = val.getCells().get(0); - assertTrue(Bytes.equals(row, cell.getRow())); - assertEquals((byte)(i + '0'), cell.getValue()[0]); - System.out.println(key + " " + val); - } - } finally { - if (log != null) { - log.closeAndDelete(); - } - if (reader != null) { - reader.close(); - } - } - } - - /** - * @throws IOException - */ - @Test - public void testAppend() throws IOException { - final int COL_COUNT = 10; - final TableName tableName = - TableName.valueOf("tablename"); - final byte [] row = Bytes.toBytes("row"); - Reader reader = null; - HLog 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 - // 1, 2, 3... - long timestamp = System.currentTimeMillis(); - WALEdit cols = new WALEdit(); - for (int i = 0; i < COL_COUNT; i++) { - cols.add(new KeyValue(row, Bytes.toBytes("column"), - Bytes.toBytes(Integer.toString(i)), - timestamp, new byte[] { (byte)(i + '0') })); - } - HRegionInfo hri = new HRegionInfo(tableName, - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - HTableDescriptor htd = new HTableDescriptor(); - htd.addFamily(new HColumnDescriptor("column")); - log.append(hri, tableName, cols, System.currentTimeMillis(), htd, sequenceId); - log.startCacheFlush(hri.getEncodedNameAsBytes()); - log.completeCacheFlush(hri.getEncodedNameAsBytes()); - log.close(); - Path filename = ((FSHLog) log).computeFilename(); - log = null; - // Now open a reader on the log and assert append worked. - reader = HLogFactory.createReader(fs, filename, conf); - HLog.Entry entry = reader.next(); - assertEquals(COL_COUNT, entry.getEdit().size()); - int idx = 0; - for (Cell val : entry.getEdit().getCells()) { - assertTrue(Bytes.equals(hri.getEncodedNameAsBytes(), - entry.getKey().getEncodedRegionName())); - assertTrue(tableName.equals(entry.getKey().getTablename())); - assertTrue(Bytes.equals(row, val.getRow())); - assertEquals((byte)(idx + '0'), val.getValue()[0]); - System.out.println(entry.getKey() + " " + val); - idx++; - } - } finally { - if (log != null) { - log.closeAndDelete(); - } - if (reader != null) { - reader.close(); - } - } - } - - /** - * Test that we can visit entries before they are appended - * @throws Exception - */ - @Test - public void testVisitors() throws Exception { - final int COL_COUNT = 10; - final TableName tableName = - TableName.valueOf("tablename"); - final byte [] row = Bytes.toBytes("row"); - HLog log = HLogFactory.createHLog(fs, hbaseDir, getName(), conf); - final AtomicLong sequenceId = new AtomicLong(1); - try { - DumbWALActionsListener visitor = new DumbWALActionsListener(); - log.registerWALActionsListener(visitor); - long timestamp = System.currentTimeMillis(); - HTableDescriptor htd = new HTableDescriptor(); - htd.addFamily(new HColumnDescriptor("column")); - - HRegionInfo hri = new HRegionInfo(tableName, - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - for (int i = 0; i < COL_COUNT; i++) { - WALEdit cols = new WALEdit(); - cols.add(new KeyValue(row, Bytes.toBytes("column"), - Bytes.toBytes(Integer.toString(i)), - timestamp, new byte[]{(byte) (i + '0')})); - log.append(hri, tableName, cols, System.currentTimeMillis(), htd, sequenceId); - } - assertEquals(COL_COUNT, visitor.increments); - log.unregisterWALActionsListener(visitor); - WALEdit cols = new WALEdit(); - cols.add(new KeyValue(row, Bytes.toBytes("column"), - Bytes.toBytes(Integer.toString(11)), - timestamp, new byte[]{(byte) (11 + '0')})); - log.append(hri, tableName, cols, System.currentTimeMillis(), htd, sequenceId); - assertEquals(COL_COUNT, visitor.increments); - } finally { - if (log != null) log.closeAndDelete(); - } - } - - @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(expected=IOException.class) - 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); - HLogFactory.createWALWriter(fs, path, conf); - fail("It should fail to create the new WAL"); - } - - @Test - public void testGetServerNameFromHLogDirectoryName() throws IOException { - ServerName sn = ServerName.valueOf("hn", 450, 1398); - String hl = FSUtils.getRootDir(conf) + "/" + HLogUtil.getHLogDirectoryName(sn.toString()); - - // Must not throw exception - Assert.assertNull(HLogUtil.getServerNameFromHLogDirectoryName(conf, null)); - Assert.assertNull(HLogUtil.getServerNameFromHLogDirectoryName(conf, - FSUtils.getRootDir(conf).toUri().toString())); - Assert.assertNull(HLogUtil.getServerNameFromHLogDirectoryName(conf, "")); - Assert.assertNull(HLogUtil.getServerNameFromHLogDirectoryName(conf, " ")); - Assert.assertNull(HLogUtil.getServerNameFromHLogDirectoryName(conf, hl)); - Assert.assertNull(HLogUtil.getServerNameFromHLogDirectoryName(conf, hl + "qdf")); - Assert.assertNull(HLogUtil.getServerNameFromHLogDirectoryName(conf, "sfqf" + hl + "qdf")); - - final String wals = "/WALs/"; - ServerName parsed = HLogUtil.getServerNameFromHLogDirectoryName(conf, - FSUtils.getRootDir(conf).toUri().toString() + wals + sn + - "/localhost%2C32984%2C1343316388997.1343316390417"); - Assert.assertEquals("standard", sn, parsed); - - parsed = HLogUtil.getServerNameFromHLogDirectoryName(conf, hl + "/qdf"); - Assert.assertEquals("subdir", sn, parsed); - - parsed = HLogUtil.getServerNameFromHLogDirectoryName(conf, - FSUtils.getRootDir(conf).toUri().toString() + wals + sn + - "-splitting/localhost%3A57020.1340474893931"); - Assert.assertEquals("split", sn, parsed); - } - - /** - * A loaded WAL coprocessor won't break existing HLog test cases. - */ - @Test - public void testWALCoprocessorLoaded() throws Exception { - // test to see whether the coprocessor is loaded or not. - HLog log = HLogFactory.createHLog(fs, hbaseDir, - getName(), conf); - try { - WALCoprocessorHost host = log.getCoprocessorHost(); - Coprocessor c = host.findCoprocessor(SampleRegionWALObserver.class.getName()); - assertNotNull(c); - } finally { - if (log != null) log.closeAndDelete(); - } - } - - private void addEdits(HLog log, HRegionInfo hri, TableName tableName, - int times, AtomicLong sequenceId) throws IOException { - HTableDescriptor htd = new HTableDescriptor(); - htd.addFamily(new HColumnDescriptor("row")); - - final byte [] row = Bytes.toBytes("row"); - for (int i = 0; i < times; i++) { - long timestamp = System.currentTimeMillis(); - WALEdit cols = new WALEdit(); - cols.add(new KeyValue(row, row, row, timestamp, row)); - log.append(hri, tableName, cols, timestamp, htd, sequenceId); - } - } - - - /** - * @throws IOException - */ - @Test - public void testReadLegacyLog() throws IOException { - final int columnCount = 5; - final int recordCount = 5; - final TableName tableName = - TableName.valueOf("tablename"); - final byte[] row = Bytes.toBytes("row"); - long timestamp = System.currentTimeMillis(); - Path path = new Path(dir, "temphlog"); - SequenceFileLogWriter sflw = null; - HLog.Reader reader = null; - try { - HRegionInfo hri = new HRegionInfo(tableName, - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - HTableDescriptor htd = new HTableDescriptor(tableName); - fs.mkdirs(dir); - // Write log in pre-PB format. - sflw = new SequenceFileLogWriter(); - sflw.init(fs, path, conf, false); - for (int i = 0; i < recordCount; ++i) { - HLogKey key = new HLogKey( - hri.getEncodedNameAsBytes(), tableName, i, timestamp, HConstants.DEFAULT_CLUSTER_ID); - WALEdit edit = new WALEdit(); - for (int j = 0; j < columnCount; ++j) { - if (i == 0) { - htd.addFamily(new HColumnDescriptor("column" + j)); - } - String value = i + "" + j; - edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value))); - } - sflw.append(new HLog.Entry(key, edit)); - } - sflw.sync(); - sflw.close(); - - // Now read the log using standard means. - reader = HLogFactory.createReader(fs, path, conf); - assertTrue(reader instanceof SequenceFileLogReader); - for (int i = 0; i < recordCount; ++i) { - HLog.Entry entry = reader.next(); - assertNotNull(entry); - assertEquals(columnCount, entry.getEdit().size()); - assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName()); - assertEquals(tableName, entry.getKey().getTablename()); - int idx = 0; - for (Cell val : entry.getEdit().getCells()) { - assertTrue(Bytes.equals(row, val.getRow())); - String value = i + "" + idx; - assertArrayEquals(Bytes.toBytes(value), val.getValue()); - idx++; - } - } - HLog.Entry entry = reader.next(); - assertNull(entry); - } finally { - if (sflw != null) { - sflw.close(); - } - if (reader != null) { - reader.close(); - } - } - } - - /** - * Reads the WAL with and without WALTrailer. - * @throws IOException - */ - @Test - public void testWALTrailer() throws IOException { - // read With trailer. - doRead(true); - // read without trailer - doRead(false); - } - - /** - * Appends entries in the WAL and reads it. - * @param withTrailer If 'withTrailer' is true, it calls a close on the WALwriter before reading - * so that a trailer is appended to the WAL. Otherwise, it starts reading after the sync - * call. This means that reader is not aware of the trailer. In this scenario, if the - * reader tries to read the trailer in its next() call, it returns false from - * ProtoBufLogReader. - * @throws IOException - */ - private void doRead(boolean withTrailer) throws IOException { - final int columnCount = 5; - final int recordCount = 5; - final TableName tableName = - TableName.valueOf("tablename"); - final byte[] row = Bytes.toBytes("row"); - long timestamp = System.currentTimeMillis(); - 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; - try { - HRegionInfo hri = new HRegionInfo(tableName, - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - HTableDescriptor htd = new HTableDescriptor(tableName); - fs.mkdirs(dir); - // Write log in pb format. - writer = HLogFactory.createWALWriter(fs, path, conf); - for (int i = 0; i < recordCount; ++i) { - HLogKey key = new HLogKey( - hri.getEncodedNameAsBytes(), tableName, i, timestamp, HConstants.DEFAULT_CLUSTER_ID); - WALEdit edit = new WALEdit(); - for (int j = 0; j < columnCount; ++j) { - if (i == 0) { - htd.addFamily(new HColumnDescriptor("column" + j)); - } - String value = i + "" + j; - edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value))); - } - writer.append(new HLog.Entry(key, edit)); - } - writer.sync(); - if (withTrailer) writer.close(); - - // Now read the log using standard means. - reader = HLogFactory.createReader(fs, path, conf); - assertTrue(reader instanceof ProtobufLogReader); - if (withTrailer) { - assertNotNull(reader.getWALTrailer()); - } else { - assertNull(reader.getWALTrailer()); - } - for (int i = 0; i < recordCount; ++i) { - HLog.Entry entry = reader.next(); - assertNotNull(entry); - assertEquals(columnCount, entry.getEdit().size()); - assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName()); - assertEquals(tableName, entry.getKey().getTablename()); - int idx = 0; - for (Cell val : entry.getEdit().getCells()) { - assertTrue(Bytes.equals(row, val.getRow())); - String value = i + "" + idx; - assertArrayEquals(Bytes.toBytes(value), val.getValue()); - idx++; - } - } - HLog.Entry entry = reader.next(); - assertNull(entry); - } finally { - if (writer != null) { - writer.close(); - } - if (reader != null) { - reader.close(); - } - } - } - - /** - * 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 - 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(); - } - } - - /** - * 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. - *

    - * @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(); - } - } - - /** - * 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 - */ - @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(); - } - } - - /** - * 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)); - } - - /** - * helper method to simulate region flush for a WAL. - * @param hlog - * @param regionEncodedName - */ - private void flushRegion(HLog hlog, byte[] regionEncodedName) { - hlog.startCacheFlush(regionEncodedName); - hlog.completeCacheFlush(regionEncodedName); - } - - static class DumbWALActionsListener implements WALActionsListener { - int increments = 0; - - @Override - public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey, - WALEdit logEdit) { - increments++; - } - - @Override - public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, WALEdit logEdit) { - //To change body of implemented methods use File | Settings | File Templates. - increments++; - } - - @Override - public void preLogRoll(Path oldFile, Path newFile) { - // TODO Auto-generated method stub - } - - @Override - public void postLogRoll(Path oldFile, Path newFile) { - // TODO Auto-generated method stub - } - - @Override - public void preLogArchive(Path oldFile, Path newFile) { - // TODO Auto-generated method stub - } - - @Override - public void postLogArchive(Path oldFile, Path newFile) { - // TODO Auto-generated method stub - } - - @Override - public void logRollRequested() { - // TODO Auto-generated method stub - - } - - @Override - public void logCloseRequested() { - // not interested - } - } - -} - diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogFiltering.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogFiltering.java deleted file mode 100644 index 1e373775c3c..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogFiltering.java +++ /dev/null @@ -1,154 +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 static org.junit.Assert.assertEquals; - -import java.io.IOException; -import java.util.List; -import java.util.Random; -import java.util.SortedMap; -import java.util.TreeMap; - -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.master.HMaster; -import org.apache.hadoop.hbase.protobuf.RequestConverter; -import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest; -import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.util.Bytes; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -import com.google.common.collect.Lists; -import com.google.protobuf.ServiceException; - -@Category({RegionServerTests.class, MediumTests.class}) -public class TestHLogFiltering { - private static final int NUM_MASTERS = 1; - private static final int NUM_RS = 4; - - private static final TableName TABLE_NAME = - TableName.valueOf("TestHLogFiltering"); - private static final byte[] CF1 = Bytes.toBytes("MyCF1"); - private static final byte[] CF2 = Bytes.toBytes("MyCF2"); - private static final byte[][] FAMILIES = { CF1, CF2 }; - - private HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - - @Before - public void setUp() throws Exception { - TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS); - fillTable(); - } - - @After - public void tearDown() throws Exception { - TEST_UTIL.shutdownMiniCluster(); - } - - private void fillTable() throws IOException, InterruptedException { - Table table = TEST_UTIL.createTable(TABLE_NAME, FAMILIES, 3, - Bytes.toBytes("row0"), Bytes.toBytes("row99"), NUM_RS); - Random rand = new Random(19387129L); - for (int iStoreFile = 0; iStoreFile < 4; ++iStoreFile) { - for (int iRow = 0; iRow < 100; ++iRow) { - final byte[] row = Bytes.toBytes("row" + iRow); - Put put = new Put(row); - Delete del = new Delete(row); - for (int iCol = 0; iCol < 10; ++iCol) { - final byte[] cf = rand.nextBoolean() ? CF1 : CF2; - final long ts = Math.abs(rand.nextInt()); - final byte[] qual = Bytes.toBytes("col" + iCol); - if (rand.nextBoolean()) { - final byte[] value = Bytes.toBytes("value_for_row_" + iRow + - "_cf_" + Bytes.toStringBinary(cf) + "_col_" + iCol + "_ts_" + - ts + "_random_" + rand.nextLong()); - put.add(cf, qual, ts, value); - } else if (rand.nextDouble() < 0.8) { - del.deleteColumn(cf, qual, ts); - } else { - del.deleteColumns(cf, qual, ts); - } - } - table.put(put); - table.delete(del); - table.flushCommits(); - } - } - TEST_UTIL.waitUntilAllRegionsAssigned(TABLE_NAME); - } - - @Test - public void testFlushedSequenceIdsSentToHMaster() - throws IOException, InterruptedException, ServiceException { - SortedMap allFlushedSequenceIds = - new TreeMap(Bytes.BYTES_COMPARATOR); - for (int i = 0; i < NUM_RS; ++i) { - flushAllRegions(i); - } - Thread.sleep(10000); - HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster(); - for (int i = 0; i < NUM_RS; ++i) { - for (byte[] regionName : getRegionsByServer(i)) { - if (allFlushedSequenceIds.containsKey(regionName)) { - GetLastFlushedSequenceIdRequest req = - RequestConverter.buildGetLastFlushedSequenceIdRequest(regionName); - - assertEquals((long)allFlushedSequenceIds.get(regionName), - master.getMasterRpcServices().getLastFlushedSequenceId( - null, req).getLastFlushedSequenceId()); - } - } - } - } - - private List getRegionsByServer(int rsId) throws IOException { - List regionNames = Lists.newArrayList(); - HRegionServer hrs = getRegionServer(rsId); - for (HRegion r : hrs.getOnlineRegions(TABLE_NAME)) { - regionNames.add(r.getRegionName()); - } - return regionNames; - } - - private HRegionServer getRegionServer(int rsId) { - return TEST_UTIL.getMiniHBaseCluster().getRegionServer(rsId); - } - - private void flushAllRegions(int rsId) - throws ServiceException, IOException { - HRegionServer hrs = getRegionServer(rsId); - for (byte[] regionName : getRegionsByServer(rsId)) { - FlushRegionRequest request = - RequestConverter.buildFlushRegionRequest(regionName); - hrs.getRSRpcServices().flushRegion(null, request); - } - } - -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogMethods.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogMethods.java deleted file mode 100644 index b1b555054b9..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogMethods.java +++ /dev/null @@ -1,171 +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 static org.junit.Assert.*; - -import java.io.IOException; -import java.util.NavigableSet; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.*; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; -import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.EntryBuffers; -import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.PipelineController; -import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.RegionEntryBuffer; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -/** - * Simple testing of a few HLog methods. - */ -@Category({RegionServerTests.class, SmallTests.class}) -public class TestHLogMethods { - private static final byte[] TEST_REGION = Bytes.toBytes("test_region");; - private static final TableName TEST_TABLE = - TableName.valueOf("test_table"); - - private final HBaseTestingUtility util = new HBaseTestingUtility(); - - /** - * Assert that getSplitEditFilesSorted returns files in expected order and - * that it skips moved-aside files. - * @throws IOException - */ - @Test public void testGetSplitEditFilesSorted() throws IOException { - FileSystem fs = FileSystem.get(util.getConfiguration()); - Path regiondir = util.getDataTestDir("regiondir"); - fs.delete(regiondir, true); - fs.mkdirs(regiondir); - Path recoverededits = HLogUtil.getRegionDirRecoveredEditsDir(regiondir); - String first = HLogSplitter.formatRecoveredEditsFileName(-1); - createFile(fs, recoverededits, first); - createFile(fs, recoverededits, HLogSplitter.formatRecoveredEditsFileName(0)); - createFile(fs, recoverededits, HLogSplitter.formatRecoveredEditsFileName(1)); - createFile(fs, recoverededits, HLogSplitter - .formatRecoveredEditsFileName(11)); - createFile(fs, recoverededits, HLogSplitter.formatRecoveredEditsFileName(2)); - createFile(fs, recoverededits, HLogSplitter - .formatRecoveredEditsFileName(50)); - String last = HLogSplitter.formatRecoveredEditsFileName(Long.MAX_VALUE); - createFile(fs, recoverededits, last); - createFile(fs, recoverededits, - Long.toString(Long.MAX_VALUE) + "." + System.currentTimeMillis()); - - HLogFactory.createHLog(fs, regiondir, "dummyLogName", util.getConfiguration()); - NavigableSet files = HLogUtil.getSplitEditFilesSorted(fs, regiondir); - assertEquals(7, files.size()); - assertEquals(files.pollFirst().getName(), first); - assertEquals(files.pollLast().getName(), last); - assertEquals(files.pollFirst().getName(), - HLogSplitter - .formatRecoveredEditsFileName(0)); - assertEquals(files.pollFirst().getName(), - HLogSplitter - .formatRecoveredEditsFileName(1)); - assertEquals(files.pollFirst().getName(), - HLogSplitter - .formatRecoveredEditsFileName(2)); - assertEquals(files.pollFirst().getName(), - HLogSplitter - .formatRecoveredEditsFileName(11)); - } - - private void createFile(final FileSystem fs, final Path testdir, - final String name) - throws IOException { - FSDataOutputStream fdos = fs.create(new Path(testdir, name), true); - fdos.close(); - } - - @Test - public void testRegionEntryBuffer() throws Exception { - HLogSplitter.RegionEntryBuffer reb = new HLogSplitter.RegionEntryBuffer( - TEST_TABLE, TEST_REGION); - assertEquals(0, reb.heapSize()); - - reb.appendEntry(createTestLogEntry(1)); - assertTrue(reb.heapSize() > 0); - } - - @Test - public void testEntrySink() throws Exception { - Configuration conf = new Configuration(); - RecoveryMode mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ? - RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING); - - EntryBuffers sink = new EntryBuffers(new PipelineController(), 1*1024*1024); - for (int i = 0; i < 1000; i++) { - HLog.Entry entry = createTestLogEntry(i); - sink.appendEntry(entry); - } - - assertTrue(sink.totalBuffered > 0); - long amountInChunk = sink.totalBuffered; - // Get a chunk - RegionEntryBuffer chunk = sink.getChunkToWrite(); - assertEquals(chunk.heapSize(), amountInChunk); - - // Make sure it got marked that a thread is "working on this" - assertTrue(sink.isRegionCurrentlyWriting(TEST_REGION)); - - // Insert some more entries - for (int i = 0; i < 500; i++) { - HLog.Entry entry = createTestLogEntry(i); - sink.appendEntry(entry); - } - // Asking for another chunk shouldn't work since the first one - // is still writing - assertNull(sink.getChunkToWrite()); - - // If we say we're done writing the first chunk, then we should be able - // to get the second - sink.doneWriting(chunk); - - RegionEntryBuffer chunk2 = sink.getChunkToWrite(); - assertNotNull(chunk2); - assertNotSame(chunk, chunk2); - long amountInChunk2 = sink.totalBuffered; - // The second chunk had fewer rows than the first - assertTrue(amountInChunk2 < amountInChunk); - - sink.doneWriting(chunk2); - assertEquals(0, sink.totalBuffered); - } - - private HLog.Entry createTestLogEntry(int i) { - long seq = i; - long now = i * 1000; - - WALEdit edit = new WALEdit(); - 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); - return entry; - } - -} - diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogReaderOnSecureHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogReaderOnSecureHLog.java deleted file mode 100644 index 4a9a395b43a..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogReaderOnSecureHLog.java +++ /dev/null @@ -1,198 +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 static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.commons.io.IOUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.commons.logging.impl.Log4JLogger; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; -import org.apache.log4j.Level; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -/* - * Test that verifies WAL written by SecureProtobufLogWriter is not readable by ProtobufLogReader - */ -@Category({RegionServerTests.class, MediumTests.class}) -public class TestHLogReaderOnSecureHLog { - static final Log LOG = LogFactory.getLog(TestHLogReaderOnSecureHLog.class); - static { - ((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hbase.regionserver.wal")) - .getLogger().setLevel(Level.ALL); - }; - static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - final byte[] value = Bytes.toBytes("Test value"); - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - Configuration conf = TEST_UTIL.getConfiguration(); - conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName()); - conf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase"); - conf.setBoolean("hbase.hlog.split.skip.errors", true); - conf.setBoolean(HConstants.ENABLE_WAL_ENCRYPTION, true); - } - - private Path writeWAL(String tblName, boolean encrypt) throws IOException { - Configuration conf = TEST_UTIL.getConfiguration(); - String clsName = conf.get(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, WALCellCodec.class.getName()); - conf.setClass(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, SecureWALCellCodec.class, - WALCellCodec.class); - if (encrypt) { - conf.set("hbase.regionserver.wal.encryption", "true"); - } else { - conf.set("hbase.regionserver.wal.encryption", "false"); - } - TableName tableName = TableName.valueOf(tblName); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(tableName.getName())); - HRegionInfo regioninfo = new HRegionInfo(tableName, - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false); - final int total = 10; - final byte[] row = Bytes.toBytes("row"); - final byte[] family = Bytes.toBytes("family"); - FileSystem fs = TEST_UTIL.getTestFileSystem(); - Path logDir = TEST_UTIL.getDataTestDir(tblName); - final AtomicLong sequenceId = new AtomicLong(1); - - // Write the WAL - FSHLog wal = new FSHLog(fs, TEST_UTIL.getDataTestDir(), logDir.toString(), conf); - 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(); - wal.close(); - // restore the cell codec class - conf.set(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, clsName); - - return walPath; - } - - @Test() - public void testHLogReaderOnSecureHLog() throws Exception { - Configuration conf = TEST_UTIL.getConfiguration(); - HLogFactory.resetLogReaderClass(); - HLogFactory.resetLogWriterClass(); - conf.setClass("hbase.regionserver.hlog.reader.impl", ProtobufLogReader.class, - HLog.Reader.class); - conf.setClass("hbase.regionserver.hlog.writer.impl", SecureProtobufLogWriter.class, - HLog.Writer.class); - FileSystem fs = TEST_UTIL.getTestFileSystem(); - Path walPath = writeWAL("testHLogReaderOnSecureHLog", true); - - // Insure edits are not plaintext - long length = fs.getFileStatus(walPath).getLen(); - FSDataInputStream in = fs.open(walPath); - byte[] fileData = new byte[(int)length]; - IOUtils.readFully(in, fileData); - in.close(); - assertFalse("Cells appear to be plaintext", Bytes.contains(fileData, value)); - - // Confirm the WAL cannot be read back by ProtobufLogReader - try { - HLog.Reader reader = HLogFactory.createReader(TEST_UTIL.getTestFileSystem(), walPath, conf); - assertFalse(true); - } catch (IOException ioe) { - // expected IOE - } - - FileStatus[] listStatus = fs.listStatus(walPath.getParent()); - RecoveryMode mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ? - RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING); - Path rootdir = FSUtils.getRootDir(conf); - try { - HLogSplitter s = new HLogSplitter(conf, rootdir, fs, null, null, mode); - s.splitLogFile(listStatus[0], null); - Path file = new Path(ZKSplitLog.getSplitLogDir(rootdir, listStatus[0].getPath().getName()), - "corrupt"); - assertTrue(fs.exists(file)); - // assertFalse("log splitting should have failed", true); - } catch (IOException ioe) { - assertTrue("WAL should have been sidelined", false); - } - } - - @Test() - public void testSecureHLogReaderOnHLog() throws Exception { - Configuration conf = TEST_UTIL.getConfiguration(); - HLogFactory.resetLogReaderClass(); - HLogFactory.resetLogWriterClass(); - conf.setClass("hbase.regionserver.hlog.reader.impl", SecureProtobufLogReader.class, - HLog.Reader.class); - conf.setClass("hbase.regionserver.hlog.writer.impl", ProtobufLogWriter.class, - HLog.Writer.class); - FileSystem fs = TEST_UTIL.getTestFileSystem(); - Path walPath = writeWAL("testSecureHLogReaderOnHLog", false); - - // Ensure edits are plaintext - long length = fs.getFileStatus(walPath).getLen(); - FSDataInputStream in = fs.open(walPath); - byte[] fileData = new byte[(int)length]; - IOUtils.readFully(in, fileData); - in.close(); - assertTrue("Cells should be plaintext", Bytes.contains(fileData, value)); - - // Confirm the WAL can be read back by SecureProtobufLogReader - try { - HLog.Reader reader = HLogFactory.createReader(TEST_UTIL.getTestFileSystem(), walPath, conf); - } catch (IOException ioe) { - assertFalse(true); - } - - FileStatus[] listStatus = fs.listStatus(walPath.getParent()); - RecoveryMode mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ? - RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING); - Path rootdir = FSUtils.getRootDir(conf); - try { - HLogSplitter s = new HLogSplitter(conf, rootdir, fs, null, null, mode); - s.splitLogFile(listStatus[0], null); - Path file = new Path(ZKSplitLog.getSplitLogDir(rootdir, listStatus[0].getPath().getName()), - "corrupt"); - assertTrue(!fs.exists(file)); - } catch (IOException ioe) { - assertTrue("WAL should have been processed", false); - } - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java deleted file mode 100644 index fb340932f01..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java +++ /dev/null @@ -1,1454 +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 static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.lang.reflect.Method; -import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.NavigableSet; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.commons.logging.impl.Log4JLogger; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.TableName; -import org.apache.log4j.Level; -import org.apache.hadoop.hdfs.server.datanode.DataNode; -import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; -import org.apache.hadoop.hdfs.server.namenode.LeaseManager; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; -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.HLogSplitter.CorruptedLogFileException; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.CancelableProgressable; -import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.hdfs.DFSTestUtil; -import org.apache.hadoop.hdfs.DistributedFileSystem; -import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException; -import org.apache.hadoop.ipc.RemoteException; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -import com.google.common.base.Joiner; -import com.google.common.collect.ImmutableList; - -/** - * Testing {@link HLog} splitting code. - */ -@Category({RegionServerTests.class, LargeTests.class}) -public class TestHLogSplit { - { - // Uncomment the following lines if more verbosity is needed for - // debugging (see HBASE-12285 for details). - //((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL); - //((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL); - //((Log4JLogger)FSNamesystem.LOG).getLogger().setLevel(Level.ALL); - } - private final static Log LOG = LogFactory.getLog(TestHLogSplit.class); - - private Configuration conf; - private FileSystem fs; - - protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - - private static final Path HBASEDIR = new Path("/hbase"); - private static final Path HLOGDIR = new Path(HBASEDIR, "hlog"); - private static final Path OLDLOGDIR = new Path(HBASEDIR, "hlog.old"); - private static final Path CORRUPTDIR = new Path(HBASEDIR, HConstants.CORRUPT_DIR_NAME); - - private static final int NUM_WRITERS = 10; - private static final int ENTRIES = 10; // entries per writer per region - - private static final TableName TABLE_NAME = - TableName.valueOf("t1"); - private static final byte[] FAMILY = "f1".getBytes(); - private static final byte[] QUALIFIER = "q1".getBytes(); - private static final byte[] VALUE = "v1".getBytes(); - private static final String HLOG_FILE_PREFIX = "hlog.dat."; - private static List REGIONS = new ArrayList(); - private static final String HBASE_SKIP_ERRORS = "hbase.hlog.split.skip.errors"; - private static final Path TABLEDIR = FSUtils.getTableDir(HBASEDIR, TABLE_NAME); - private static String ROBBER; - private static String ZOMBIE; - private static String [] GROUP = new String [] {"supergroup"}; - private RecoveryMode mode; - - static enum Corruptions { - INSERT_GARBAGE_ON_FIRST_LINE, - INSERT_GARBAGE_IN_THE_MIDDLE, - APPEND_GARBAGE, - TRUNCATE, - TRUNCATE_TRAILER - } - - @BeforeClass - 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); - 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. - System.setProperty("hbase.tests.use.shortcircuit.reads", "false"); - // Create fake maping user to group and set it to the conf. - Map u2g_map = new HashMap(2); - ROBBER = User.getCurrent().getName() + "-robber"; - ZOMBIE = User.getCurrent().getName() + "-zombie"; - u2g_map.put(ROBBER, GROUP); - u2g_map.put(ZOMBIE, GROUP); - DFSTestUtil.updateConfWithFakeGroupMapping(TEST_UTIL.getConfiguration(), u2g_map); - TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1); - TEST_UTIL.startMiniDFSCluster(2); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - TEST_UTIL.shutdownMiniDFSCluster(); - } - - @Before - public void setUp() throws Exception { - flushToConsole("Cleaning up cluster for new test\n" - + "--------------------------"); - conf = TEST_UTIL.getConfiguration(); - fs = TEST_UTIL.getDFSCluster().getFileSystem(); - FileStatus[] entries = fs.listStatus(new Path("/")); - flushToConsole("Num entries in /:" + entries.length); - for (FileStatus dir : entries){ - assertTrue("Deleting " + dir.getPath(), fs.delete(dir.getPath(), true)); - } - // create the HLog directory because recursive log creates are not allowed - fs.mkdirs(HLOGDIR); - REGIONS.clear(); - Collections.addAll(REGIONS, "bbb", "ccc"); - InstrumentedSequenceFileLogWriter.activateFailure = false; - this.mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ? - RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING); - } - - @After - public void tearDown() throws Exception { - } - - /** - * Simulates splitting a WAL out from under a regionserver that is still trying to write it. Ensures we do not - * lose edits. - * @throws IOException - * @throws InterruptedException - */ - @Test (timeout=300000) - public void testLogCannotBeWrittenOnceParsed() throws IOException, InterruptedException { - final AtomicLong counter = new AtomicLong(0); - AtomicBoolean stop = new AtomicBoolean(false); - // Region we'll write edits too and then later examine to make sure they all made it in. - final String region = REGIONS.get(0); - Thread zombie = new ZombieLastLogWriterRegionServer(this.conf, counter, stop, region); - try { - long startCount = counter.get(); - zombie.start(); - // Wait till writer starts going. - while (startCount == counter.get()) Threads.sleep(1); - // Give it a second to write a few appends. - Threads.sleep(1000); - final Configuration conf2 = HBaseConfiguration.create(this.conf); - final User robber = User.createUserForTesting(conf2, ROBBER, GROUP); - int count = robber.runAs(new PrivilegedExceptionAction() { - @Override - public Integer run() throws Exception { - FileSystem fs = FileSystem.get(conf2); - int expectedFiles = fs.listStatus(HLOGDIR).length; - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf2); - Path[] logfiles = getLogForRegion(HBASEDIR, TABLE_NAME, region); - assertEquals(expectedFiles, logfiles.length); - int count = 0; - for (Path logfile: logfiles) { - count += countHLog(logfile, fs, conf2); - } - return count; - } - }); - LOG.info("zombie=" + counter.get() + ", robber=" + count); - assertTrue("The log file could have at most 1 extra log entry, but can't have less. Zombie could write " + - counter.get() + " and logfile had only " + count, - counter.get() == count || counter.get() + 1 == count); - } finally { - stop.set(true); - zombie.interrupt(); - Threads.threadDumpingIsAlive(zombie); - } - } - - /** - * This thread will keep writing to a 'wal' file even after the split process has started. - * It simulates a region server that was considered dead but woke up and wrote some more to he last log entry. - * Does its writing as an alternate user in another filesystem instance to simulate better it being a regionserver. - */ - static class ZombieLastLogWriterRegionServer extends Thread { - final AtomicLong editsCount; - final AtomicBoolean stop; - // final User user; - /** - * Region to write edits for. - */ - final String region; - final Configuration conf; - final User user; - - public ZombieLastLogWriterRegionServer(final Configuration conf, AtomicLong counter, AtomicBoolean stop, - final String region) - throws IOException, InterruptedException { - super("ZombieLastLogWriterRegionServer"); - setDaemon(true); - this.stop = stop; - this.editsCount = counter; - this.region = region; - this.conf = HBaseConfiguration.create(conf); - this.user = User.createUserForTesting(this.conf, ZOMBIE, GROUP); - } - - @Override - public void run() { - try { - doWriting(); - } catch (IOException e) { - LOG.warn(getName() + " Writer exiting " + e); - } catch (InterruptedException e) { - LOG.warn(getName() + " Writer exiting " + e); - } - } - - private void doWriting() throws IOException, InterruptedException { - this.user.runAs(new PrivilegedExceptionAction() { - @Override - public Object run() throws Exception { - // Index of the WAL we want to keep open. generateHLogs will leave open the WAL whose index we supply here. - int walToKeepOpen = 2; - // How many files to write. - 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; - try { - DistributedFileSystem dfs = (DistributedFileSystem)FileSystem.get(conf); - writers = generateHLogs(dfs, numOfWriters, ENTRIES, walToKeepOpen); - } catch (IOException e1) { - throw new RuntimeException("Failed", e1); - } - // 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]; - loop(writer); - return null; - } - }); - } - - private void loop(final HLog.Writer writer) { - byte [] regionBytes = Bytes.toBytes(this.region); - while (true) { - try { - long seq = appendEntry(writer, TABLE_NAME, regionBytes, ("r" + editsCount.get()).getBytes(), - regionBytes, QUALIFIER, VALUE, 0); - long count = editsCount.incrementAndGet(); - flushToConsole(getName() + " sync count=" + count + ", seq=" + seq); - try { - Thread.sleep(1); - } catch (InterruptedException e) { - // - } - } catch (IOException ex) { - flushToConsole(getName() + " ex " + ex.toString()); - if (ex instanceof RemoteException) { - flushToConsole("Juliet: got RemoteException " + ex.getMessage() + - " while writing " + (editsCount.get() + 1)); - } else { - flushToConsole(getName() + " failed to write....at " + editsCount.get()); - assertTrue("Failed to write " + editsCount.get(), false); - } - break; - } catch (Throwable t) { - flushToConsole(getName() + " HOW? " + t); - t.printStackTrace(); - break; - } - } - flushToConsole(getName() + " Writer exiting"); - } - } - - /** - * @throws IOException - * @see https://issues.apache.org/jira/browse/HBASE-3020 - */ - @Test (timeout=300000) - public void testRecoveredEditsPathForMeta() throws IOException { - FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration()); - byte [] encoded = HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(); - Path tdir = FSUtils.getTableDir(HBASEDIR, TableName.META_TABLE_NAME); - Path regiondir = new Path(tdir, - HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); - fs.mkdirs(regiondir); - long now = System.currentTimeMillis(); - HLog.Entry entry = - new HLog.Entry(new HLogKey(encoded, - TableName.META_TABLE_NAME, 1, now, HConstants.DEFAULT_CLUSTER_ID), - new WALEdit()); - Path p = HLogSplitter.getRegionSplitEditsPath(fs, entry, HBASEDIR, true); - String parentOfParent = p.getParent().getParent().getName(); - assertEquals(parentOfParent, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); - } - - /** - * Test old recovered edits file doesn't break HLogSplitter. - * This is useful in upgrading old instances. - */ - @Test (timeout=300000) - public void testOldRecoveredEditsFileSidelined() throws IOException { - FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration()); - byte [] encoded = HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(); - Path tdir = FSUtils.getTableDir(HBASEDIR, TableName.META_TABLE_NAME); - Path regiondir = new Path(tdir, - HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); - fs.mkdirs(regiondir); - long now = System.currentTimeMillis(); - HLog.Entry entry = - new HLog.Entry(new HLogKey(encoded, - TableName.META_TABLE_NAME, 1, now, HConstants.DEFAULT_CLUSTER_ID), - new WALEdit()); - Path parent = HLogUtil.getRegionDirRecoveredEditsDir(regiondir); - assertEquals(parent.getName(), HConstants.RECOVERED_EDITS_DIR); - fs.createNewFile(parent); // create a recovered.edits file - - Path p = HLogSplitter.getRegionSplitEditsPath(fs, entry, HBASEDIR, true); - String parentOfParent = p.getParent().getParent().getName(); - assertEquals(parentOfParent, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); - HLogFactory.createRecoveredEditsWriter(fs, p, conf).close(); - } - - @Test (timeout=300000) - public void testSplitPreservesEdits() throws IOException{ - final String REGION = "region__1"; - REGIONS.removeAll(REGIONS); - REGIONS.add(REGION); - - generateHLogs(1, 10, -1); - fs.initialize(fs.getUri(), conf); - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - Path originalLog = (fs.listStatus(OLDLOGDIR))[0].getPath(); - Path[] splitLog = getLogForRegion(HBASEDIR, TABLE_NAME, REGION); - assertEquals(1, splitLog.length); - - assertEquals("edits differ after split", true, logsAreEqual(originalLog, splitLog[0])); - } - - - @Test (timeout=300000) - public void testEmptyLogFiles() throws IOException { - - injectEmptyFile(".empty", true); - generateHLogs(Integer.MAX_VALUE); - injectEmptyFile("empty", true); - - // make fs act as a different client now - // initialize will create a new DFSClient with a new client ID - fs.initialize(fs.getUri(), conf); - - int expectedFiles = fs.listStatus(HLOGDIR).length - 2; // less 2 empty files - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - for (String region : REGIONS) { - Path[] logfiles = getLogForRegion(HBASEDIR, TABLE_NAME, region); - assertEquals(expectedFiles, logfiles.length); - int count = 0; - for (Path logfile: logfiles) { - count += countHLog(logfile, fs, conf); - } - assertEquals(NUM_WRITERS * ENTRIES, count); - } - } - - - @Test (timeout=300000) - public void testEmptyOpenLogFiles() throws IOException { - injectEmptyFile(".empty", false); - generateHLogs(Integer.MAX_VALUE); - injectEmptyFile("empty", false); - - // make fs act as a different client now - // initialize will create a new DFSClient with a new client ID - fs.initialize(fs.getUri(), conf); - - int expectedFiles = fs.listStatus(HLOGDIR).length - 2 ; // less 2 empty files - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - for (String region : REGIONS) { - Path[] logfiles = getLogForRegion(HBASEDIR, TABLE_NAME, region); - assertEquals(expectedFiles, logfiles.length); - int count = 0; - for (Path logfile: logfiles) { - count += countHLog(logfile, fs, conf); - } - assertEquals(NUM_WRITERS * ENTRIES, count); - } - } - - @Test (timeout=300000) - public void testOpenZeroLengthReportedFileButWithDataGetsSplit() throws IOException { - // generate logs but leave hlog.dat.5 open. - generateHLogs(5); - - fs.initialize(fs.getUri(), conf); - - int expectedFiles = fs.listStatus(HLOGDIR).length; - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - for (String region : REGIONS) { - Path[] logfiles = getLogForRegion(HBASEDIR, TABLE_NAME, region); - assertEquals(expectedFiles, logfiles.length); - int count = 0; - for (Path logfile: logfiles) { - count += countHLog(logfile, fs, conf); - } - assertEquals(NUM_WRITERS * ENTRIES, count); - } - } - - - @Test (timeout=300000) - public void testTralingGarbageCorruptionFileSkipErrorsPasses() throws IOException { - conf.setBoolean(HBASE_SKIP_ERRORS, true); - generateHLogs(Integer.MAX_VALUE); - corruptHLog(new Path(HLOGDIR, HLOG_FILE_PREFIX + "5"), - Corruptions.APPEND_GARBAGE, true, fs); - fs.initialize(fs.getUri(), conf); - - int expectedFiles = fs.listStatus(HLOGDIR).length; - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - for (String region : REGIONS) { - Path[] logfiles = getLogForRegion(HBASEDIR, TABLE_NAME, region); - assertEquals(expectedFiles, logfiles.length); - int count = 0; - for (Path logfile: logfiles) { - count += countHLog(logfile, fs, conf); - } - assertEquals(NUM_WRITERS * ENTRIES, count); - } - } - - @Test (timeout=300000) - public void testFirstLineCorruptionLogFileSkipErrorsPasses() throws IOException { - conf.setBoolean(HBASE_SKIP_ERRORS, true); - generateHLogs(Integer.MAX_VALUE); - corruptHLog(new Path(HLOGDIR, HLOG_FILE_PREFIX + "5"), - Corruptions.INSERT_GARBAGE_ON_FIRST_LINE, true, fs); - fs.initialize(fs.getUri(), conf); - - int expectedFiles = fs.listStatus(HLOGDIR).length - 1; // less 1 corrupted file - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - for (String region : REGIONS) { - Path[] logfiles = getLogForRegion(HBASEDIR, TABLE_NAME, region); - assertEquals(expectedFiles, logfiles.length); - int count = 0; - for (Path logfile: logfiles) { - count += countHLog(logfile, fs, conf); - } - assertEquals((NUM_WRITERS - 1) * ENTRIES, count); - } - } - - @Test (timeout=300000) - public void testMiddleGarbageCorruptionSkipErrorsReadsHalfOfFile() throws IOException { - conf.setBoolean(HBASE_SKIP_ERRORS, true); - generateHLogs(Integer.MAX_VALUE); - corruptHLog(new Path(HLOGDIR, HLOG_FILE_PREFIX + "5"), - Corruptions.INSERT_GARBAGE_IN_THE_MIDDLE, false, fs); - fs.initialize(fs.getUri(), conf); - - int expectedFiles = fs.listStatus(HLOGDIR).length; - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - for (String region : REGIONS) { - Path[] logfiles = getLogForRegion(HBASEDIR, TABLE_NAME, region); - assertEquals(expectedFiles, logfiles.length); - int count = 0; - for (Path logfile: logfiles) { - count += countHLog(logfile, fs, conf); - } - // the entries in the original logs are alternating regions - // considering the sequence file header, the middle corruption should - // affect at least half of the entries - int goodEntries = (NUM_WRITERS - 1) * ENTRIES; - int firstHalfEntries = (int) Math.ceil(ENTRIES / 2) - 1; - assertTrue("The file up to the corrupted area hasn't been parsed", - goodEntries + firstHalfEntries <= count); - } - } - - @Test (timeout=300000) - public void testCorruptedFileGetsArchivedIfSkipErrors() throws IOException { - conf.setBoolean(HBASE_SKIP_ERRORS, true); - Class backupClass = conf.getClass("hbase.regionserver.hlog.reader.impl", - Reader.class); - InstrumentedSequenceFileLogWriter.activateFailure = false; - HLogFactory.resetLogReaderClass(); - - try { - Path c1 = new Path(HLOGDIR, HLOG_FILE_PREFIX + "0"); - conf.setClass("hbase.regionserver.hlog.reader.impl", - FaultySequenceFileLogReader.class, HLog.Reader.class); - for (FaultySequenceFileLogReader.FailureType failureType : FaultySequenceFileLogReader.FailureType.values()) { - conf.set("faultysequencefilelogreader.failuretype", failureType.name()); - generateHLogs(1, ENTRIES, -1); - fs.initialize(fs.getUri(), conf); - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - FileStatus[] archivedLogs = fs.listStatus(CORRUPTDIR); - assertEquals("expected a different file", c1.getName(), archivedLogs[0] - .getPath().getName()); - assertEquals(archivedLogs.length, 1); - fs.delete(new Path(OLDLOGDIR, HLOG_FILE_PREFIX + "0"), false); - } - } finally { - conf.setClass("hbase.regionserver.hlog.reader.impl", backupClass, - Reader.class); - HLogFactory.resetLogReaderClass(); - } - } - - @Test (timeout=300000, expected = IOException.class) - public void testTrailingGarbageCorruptionLogFileSkipErrorsFalseThrows() - throws IOException { - conf.setBoolean(HBASE_SKIP_ERRORS, false); - Class backupClass = conf.getClass("hbase.regionserver.hlog.reader.impl", - Reader.class); - InstrumentedSequenceFileLogWriter.activateFailure = false; - HLogFactory.resetLogReaderClass(); - - try { - conf.setClass("hbase.regionserver.hlog.reader.impl", - FaultySequenceFileLogReader.class, HLog.Reader.class); - conf.set("faultysequencefilelogreader.failuretype", FaultySequenceFileLogReader.FailureType.BEGINNING.name()); - generateHLogs(Integer.MAX_VALUE); - fs.initialize(fs.getUri(), conf); - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - } finally { - conf.setClass("hbase.regionserver.hlog.reader.impl", backupClass, - Reader.class); - HLogFactory.resetLogReaderClass(); - } - } - - @Test (timeout=300000) - public void testCorruptedLogFilesSkipErrorsFalseDoesNotTouchLogs() - throws IOException { - conf.setBoolean(HBASE_SKIP_ERRORS, false); - Class backupClass = conf.getClass("hbase.regionserver.hlog.reader.impl", - Reader.class); - InstrumentedSequenceFileLogWriter.activateFailure = false; - HLogFactory.resetLogReaderClass(); - - try { - conf.setClass("hbase.regionserver.hlog.reader.impl", - FaultySequenceFileLogReader.class, HLog.Reader.class); - conf.set("faultysequencefilelogreader.failuretype", FaultySequenceFileLogReader.FailureType.BEGINNING.name()); - generateHLogs(-1); - fs.initialize(fs.getUri(), conf); - try { - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - } catch (IOException e) { - assertEquals( - "if skip.errors is false all files should remain in place", - NUM_WRITERS, fs.listStatus(HLOGDIR).length); - } - } finally { - conf.setClass("hbase.regionserver.hlog.reader.impl", backupClass, - Reader.class); - HLogFactory.resetLogReaderClass(); - } - } - - @Test (timeout=300000) - public void testEOFisIgnored() throws IOException { - conf.setBoolean(HBASE_SKIP_ERRORS, false); - - final String REGION = "region__1"; - REGIONS.removeAll(REGIONS); - REGIONS.add(REGION); - - int entryCount = 10; - Path c1 = new Path(HLOGDIR, HLOG_FILE_PREFIX + "0"); - generateHLogs(1, entryCount, -1); - corruptHLog(c1, Corruptions.TRUNCATE, true, fs); - - fs.initialize(fs.getUri(), conf); - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - - Path[] splitLog = getLogForRegion(HBASEDIR, TABLE_NAME, REGION); - assertEquals(1, splitLog.length); - - int actualCount = 0; - HLog.Reader in = HLogFactory.createReader(fs, splitLog[0], conf); - @SuppressWarnings("unused") - HLog.Entry entry; - while ((entry = in.next()) != null) ++actualCount; - assertEquals(entryCount-1, actualCount); - - // should not have stored the EOF files as corrupt - FileStatus[] archivedLogs = fs.listStatus(CORRUPTDIR); - assertEquals(archivedLogs.length, 0); - } - - @Test (timeout=300000) - public void testCorruptWALTrailer() throws IOException { - conf.setBoolean(HBASE_SKIP_ERRORS, false); - - final String REGION = "region__1"; - REGIONS.removeAll(REGIONS); - REGIONS.add(REGION); - - int entryCount = 10; - Path c1 = new Path(HLOGDIR, HLOG_FILE_PREFIX + "0"); - generateHLogs(1, entryCount, -1); - corruptHLog(c1, Corruptions.TRUNCATE_TRAILER, true, fs); - - fs.initialize(fs.getUri(), conf); - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - - Path[] splitLog = getLogForRegion(HBASEDIR, TABLE_NAME, REGION); - assertEquals(1, splitLog.length); - - int actualCount = 0; - HLog.Reader in = HLogFactory.createReader(fs, splitLog[0], conf); - @SuppressWarnings("unused") - HLog.Entry entry; - while ((entry = in.next()) != null) ++actualCount; - assertEquals(entryCount, actualCount); - - // should not have stored the EOF files as corrupt - FileStatus[] archivedLogs = fs.listStatus(CORRUPTDIR); - assertEquals(archivedLogs.length, 0); - } - - @Test (timeout=300000) - public void testLogsGetArchivedAfterSplit() throws IOException { - conf.setBoolean(HBASE_SKIP_ERRORS, false); - generateHLogs(-1); - fs.initialize(fs.getUri(), conf); - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - FileStatus[] archivedLogs = fs.listStatus(OLDLOGDIR); - assertEquals("wrong number of files in the archive log", NUM_WRITERS, archivedLogs.length); - } - - @Test (timeout=300000) - public void testSplit() throws IOException { - generateHLogs(-1); - fs.initialize(fs.getUri(), conf); - - int expectedFiles = fs.listStatus(HLOGDIR).length; - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - for (String region : REGIONS) { - Path[] logfiles = getLogForRegion(HBASEDIR, TABLE_NAME, region); - assertEquals(expectedFiles, logfiles.length); - int count = 0; - for (Path logfile: logfiles) { - count += countHLog(logfile, fs, conf); - } - assertEquals(NUM_WRITERS * ENTRIES, count); - } - } - - @Test (timeout=300000) - public void testLogDirectoryShouldBeDeletedAfterSuccessfulSplit() - throws IOException { - generateHLogs(-1); - fs.initialize(fs.getUri(), conf); - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - FileStatus [] statuses = null; - try { - statuses = fs.listStatus(HLOGDIR); - if (statuses != null) { - Assert.fail("Files left in log dir: " + - Joiner.on(",").join(FileUtil.stat2Paths(statuses))); - } - } catch (FileNotFoundException e) { - // hadoop 0.21 throws FNFE whereas hadoop 0.20 returns null - } - } - - @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); - - fs.initialize(fs.getUri(), conf); - - String region = "break"; - Path regiondir = new Path(TABLEDIR, region); - fs.mkdirs(regiondir); - - InstrumentedSequenceFileLogWriter.activateFailure = false; - appendEntry(writer[4], TABLE_NAME, Bytes.toBytes(region), - ("r" + 999).getBytes(), FAMILY, QUALIFIER, VALUE, 0); - writer[4].close(); - - try { - InstrumentedSequenceFileLogWriter.activateFailure = true; - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - } catch (IOException e) { - assertTrue(e.getMessage(). - contains("This exception is instrumented and should only be thrown for testing")); - throw e; - } finally { - InstrumentedSequenceFileLogWriter.activateFailure = false; - } - } - - - // @Test TODO this test has been disabled since it was created! - // It currently fails because the second split doesn't output anything - // -- because there are no region dirs after we move aside the first - // split result - public void testSplittingLargeNumberOfRegionsConsistency() throws IOException { - - REGIONS.removeAll(REGIONS); - for (int i=0; i<100; i++) { - REGIONS.add("region__"+i); - } - - generateHLogs(1, 100, -1); - fs.initialize(fs.getUri(), conf); - - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - fs.rename(OLDLOGDIR, HLOGDIR); - Path firstSplitPath = new Path(HBASEDIR, TABLE_NAME+ ".first"); - Path splitPath = new Path(HBASEDIR, TABLE_NAME.getNameAsString()); - fs.rename(splitPath, - firstSplitPath); - - fs.initialize(fs.getUri(), conf); - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - assertEquals(0, compareHLogSplitDirs(firstSplitPath, splitPath)); - } - - @Test (timeout=300000) - public void testSplitDeletedRegion() throws IOException { - REGIONS.removeAll(REGIONS); - String region = "region_that_splits"; - REGIONS.add(region); - - generateHLogs(1); - fs.initialize(fs.getUri(), conf); - - Path regiondir = new Path(TABLEDIR, region); - fs.delete(regiondir, true); - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - assertFalse(fs.exists(regiondir)); - } - - @Test (timeout=300000) - public void testIOEOnOutputThread() throws Exception { - conf.setBoolean(HBASE_SKIP_ERRORS, false); - - generateHLogs(-1); - fs.initialize(fs.getUri(), conf); - FileStatus[] logfiles = fs.listStatus(HLOGDIR); - assertTrue("There should be some log file", - logfiles != null && logfiles.length > 0); - // Set up a splitter that will throw an IOE on the output side - HLogSplitter logSplitter = new HLogSplitter( - conf, HBASEDIR, fs, null, null, this.mode) { - protected HLog.Writer createWriter(FileSystem fs, - Path logfile, Configuration conf) throws IOException { - HLog.Writer mockWriter = Mockito.mock(HLog.Writer.class); - Mockito.doThrow(new IOException("Injected")).when( - mockWriter).append(Mockito.any()); - return mockWriter; - } - }; - // Set up a background thread dumper. Needs a thread to depend on and then we need to run - // the thread dumping in a background thread so it does not hold up the test. - final AtomicBoolean stop = new AtomicBoolean(false); - final Thread someOldThread = new Thread("Some-old-thread") { - @Override - public void run() { - while(!stop.get()) Threads.sleep(10); - } - }; - someOldThread.setDaemon(true); - someOldThread.start(); - final Thread t = new Thread("Background-thread-dumper") { - public void run() { - try { - Threads.threadDumpingIsAlive(someOldThread); - } catch (InterruptedException e) { - e.printStackTrace(); - } - } - }; - t.setDaemon(true); - t.start(); - try { - logSplitter.splitLogFile(logfiles[0], null); - fail("Didn't throw!"); - } catch (IOException ioe) { - assertTrue(ioe.toString().contains("Injected")); - } finally { - // Setting this to true will turn off the background thread dumper. - stop.set(true); - } - } - - // Test for HBASE-3412 - @Test (timeout=300000) - public void testMovedHLogDuringRecovery() throws Exception { - generateHLogs(-1); - - fs.initialize(fs.getUri(), conf); - - // This partial mock will throw LEE for every file simulating - // files that were moved - FileSystem spiedFs = Mockito.spy(fs); - // The "File does not exist" part is very important, - // that's how it comes out of HDFS - Mockito.doThrow(new LeaseExpiredException("Injected: File does not exist")). - when(spiedFs).append(Mockito.any()); - - try { - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, spiedFs, conf); - assertEquals(NUM_WRITERS, fs.listStatus(OLDLOGDIR).length); - assertFalse(fs.exists(HLOGDIR)); - } catch (IOException e) { - fail("There shouldn't be any exception but: " + e.toString()); - } - } - - @Test (timeout=300000) - public void testRetryOpenDuringRecovery() throws Exception { - generateHLogs(-1); - - fs.initialize(fs.getUri(), conf); - - FileSystem spiedFs = Mockito.spy(fs); - // The "Cannot obtain block length", "Could not obtain the last block", - // and "Blocklist for [^ ]* has changed.*" part is very important, - // that's how it comes out of HDFS. If HDFS changes the exception - // message, this test needs to be adjusted accordingly. - // - // When DFSClient tries to open a file, HDFS needs to locate - // the last block of the file and get its length. However, if the - // last block is under recovery, HDFS may have problem to obtain - // the block length, in which case, retry may help. - Mockito.doAnswer(new Answer() { - private final String[] errors = new String[] { - "Cannot obtain block length", "Could not obtain the last block", - "Blocklist for " + OLDLOGDIR + " has changed"}; - private int count = 0; - - public FSDataInputStream answer(InvocationOnMock invocation) throws Throwable { - if (count < 3) { - throw new IOException(errors[count++]); - } - return (FSDataInputStream)invocation.callRealMethod(); - } - }).when(spiedFs).open(Mockito.any(), Mockito.anyInt()); - - try { - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, spiedFs, conf); - assertEquals(NUM_WRITERS, fs.listStatus(OLDLOGDIR).length); - assertFalse(fs.exists(HLOGDIR)); - } catch (IOException e) { - fail("There shouldn't be any exception but: " + e.toString()); - } - } - - @Test (timeout=300000) - public void testTerminationAskedByReporter() throws IOException, CorruptedLogFileException { - generateHLogs(1, 10, -1); - FileStatus logfile = fs.listStatus(HLOGDIR)[0]; - fs.initialize(fs.getUri(), conf); - - final AtomicInteger count = new AtomicInteger(); - - CancelableProgressable localReporter - = new CancelableProgressable() { - @Override - public boolean progress() { - count.getAndIncrement(); - return false; - } - }; - - FileSystem spiedFs = Mockito.spy(fs); - Mockito.doAnswer(new Answer() { - public FSDataInputStream answer(InvocationOnMock invocation) throws Throwable { - Thread.sleep(1500); // Sleep a while and wait report status invoked - return (FSDataInputStream)invocation.callRealMethod(); - } - }).when(spiedFs).open(Mockito.any(), Mockito.anyInt()); - - try { - conf.setInt("hbase.splitlog.report.period", 1000); - boolean ret = HLogSplitter.splitLogFile( - HBASEDIR, logfile, spiedFs, conf, localReporter, null, null, this.mode); - assertFalse("Log splitting should failed", ret); - assertTrue(count.get() > 0); - } catch (IOException e) { - fail("There shouldn't be any exception but: " + e.toString()); - } finally { - // reset it back to its default value - conf.setInt("hbase.splitlog.report.period", 59000); - } - } - - /** - * Test log split process with fake data and lots of edits to trigger threading - * issues. - */ - @Test (timeout=300000) - public void testThreading() throws Exception { - doTestThreading(20000, 128*1024*1024, 0); - } - - /** - * Test blocking behavior of the log split process if writers are writing slower - * than the reader is reading. - */ - @Test (timeout=300000) - public void testThreadingSlowWriterSmallBuffer() throws Exception { - doTestThreading(200, 1024, 50); - } - - /** - * Sets up a log splitter with a mock reader and writer. The mock reader generates - * a specified number of edits spread across 5 regions. The mock writer optionally - * sleeps for each edit it is fed. - * * - * After the split is complete, verifies that the statistics show the correct number - * of edits output into each region. - * - * @param numFakeEdits number of fake edits to push through pipeline - * @param bufferSize size of in-memory buffer - * @param writerSlowness writer threads will sleep this many ms per edit - */ - private void doTestThreading(final int numFakeEdits, - final int bufferSize, - final int writerSlowness) throws Exception { - - Configuration localConf = new Configuration(conf); - localConf.setInt("hbase.regionserver.hlog.splitlog.buffersize", bufferSize); - - // Create a fake log file (we'll override the reader to produce a stream of edits) - Path logPath = new Path(HLOGDIR, HLOG_FILE_PREFIX + ".fake"); - FSDataOutputStream out = fs.create(logPath); - out.close(); - - // Make region dirs for our destination regions so the output doesn't get skipped - final List regions = ImmutableList.of("r0", "r1", "r2", "r3", "r4"); - makeRegionDirs(fs, regions); - - // Create a splitter that reads and writes the data without touching disk - HLogSplitter logSplitter = new HLogSplitter( - localConf, HBASEDIR, fs, null, null, this.mode) { - - /* Produce a mock writer that doesn't write anywhere */ - protected HLog.Writer createWriter(FileSystem fs, Path logfile, Configuration conf) - throws IOException { - HLog.Writer mockWriter = Mockito.mock(HLog.Writer.class); - Mockito.doAnswer(new Answer() { - int expectedIndex = 0; - - @Override - public Void answer(InvocationOnMock invocation) { - if (writerSlowness > 0) { - try { - Thread.sleep(writerSlowness); - } catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - } - } - HLog.Entry entry = (Entry) invocation.getArguments()[0]; - WALEdit edit = entry.getEdit(); - List cells = edit.getCells(); - assertEquals(1, cells.size()); - Cell cell = cells.get(0); - - // Check that the edits come in the right order. - assertEquals(expectedIndex, Bytes.toInt(cell.getRow())); - expectedIndex++; - return null; - } - }).when(mockWriter).append(Mockito.any()); - return mockWriter; - } - - /* Produce a mock reader that generates fake entries */ - protected Reader getReader(FileSystem fs, Path curLogFile, - Configuration conf, CancelableProgressable reporter) throws IOException { - Reader mockReader = Mockito.mock(Reader.class); - Mockito.doAnswer(new Answer() { - int index = 0; - - @Override - public HLog.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, - Bytes.toBytes((int)(index / regions.size())), - FAMILY, QUALIFIER, VALUE, index); - index++; - return ret; - } - }).when(mockReader).next(); - return mockReader; - } - }; - - logSplitter.splitLogFile(fs.getFileStatus(logPath), null); - - // Verify number of written edits per region - Map outputCounts = logSplitter.outputSink.getOutputCounts(); - for (Map.Entry entry : outputCounts.entrySet()) { - LOG.info("Got " + entry.getValue() + " output edits for region " + - Bytes.toString(entry.getKey())); - assertEquals((long)entry.getValue(), numFakeEdits / regions.size()); - } - assertEquals(regions.size(), outputCounts.size()); - } - - @Test (timeout=300000) - public void testSplitLogFileWithOneRegion() throws IOException { - LOG.info("testSplitLogFileWithOneRegion"); - final String REGION = "region__1"; - REGIONS.removeAll(REGIONS); - REGIONS.add(REGION); - - generateHLogs(1, 10, -1); - fs.initialize(fs.getUri(), conf); - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - - Path originalLog = (fs.listStatus(OLDLOGDIR))[0].getPath(); - Path[] splitLog = getLogForRegion(HBASEDIR, TABLE_NAME, REGION); - assertEquals(1, splitLog.length); - - assertEquals(true, logsAreEqual(originalLog, splitLog[0])); - } - - @Test (timeout=300000) - public void testSplitLogFileDeletedRegionDir() throws IOException { - LOG.info("testSplitLogFileDeletedRegionDir"); - final String REGION = "region__1"; - REGIONS.removeAll(REGIONS); - REGIONS.add(REGION); - - generateHLogs(1, 10, -1); - fs.initialize(fs.getUri(), conf); - - Path regiondir = new Path(TABLEDIR, REGION); - LOG.info("Region directory is" + regiondir); - fs.delete(regiondir, true); - - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - - assertTrue(!fs.exists(regiondir)); - assertTrue(true); - } - - @Test (timeout=300000) - public void testSplitLogFileEmpty() throws IOException { - LOG.info("testSplitLogFileEmpty"); - injectEmptyFile(".empty", true); - - fs.initialize(fs.getUri(), conf); - - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - Path tdir = FSUtils.getTableDir(HBASEDIR, TABLE_NAME); - assertFalse(fs.exists(tdir)); - - assertEquals(0, countHLog(fs.listStatus(OLDLOGDIR)[0].getPath(), fs, conf)); - } - - @Test (timeout=300000) - public void testSplitLogFileMultipleRegions() throws IOException { - LOG.info("testSplitLogFileMultipleRegions"); - generateHLogs(1, 10, -1); - fs.initialize(fs.getUri(), conf); - - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - for (String region : REGIONS) { - Path[] recovered = getLogForRegion(HBASEDIR, TABLE_NAME, region); - assertEquals(1, recovered.length); - assertEquals(10, countHLog(recovered[0], fs, conf)); - } - } - - @Test (timeout=300000) - public void testSplitLogFileFirstLineCorruptionLog() - throws IOException { - conf.setBoolean(HBASE_SKIP_ERRORS, true); - generateHLogs(1, 10, -1); - FileStatus logfile = fs.listStatus(HLOGDIR)[0]; - - corruptHLog(logfile.getPath(), - Corruptions.INSERT_GARBAGE_ON_FIRST_LINE, true, fs); - - fs.initialize(fs.getUri(), conf); - HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf); - - final Path corruptDir = new Path(FSUtils.getRootDir(conf), conf.get( - "hbase.regionserver.hlog.splitlog.corrupt.dir", HConstants.CORRUPT_DIR_NAME)); - assertEquals(1, fs.listStatus(corruptDir).length); - } - - /** - * @throws IOException - * @see https://issues.apache.org/jira/browse/HBASE-4862 - */ - @Test (timeout=300000) - public void testConcurrentSplitLogAndReplayRecoverEdit() throws IOException { - LOG.info("testConcurrentSplitLogAndReplayRecoverEdit"); - // Generate hlogs for our destination region - String regionName = "r0"; - final Path regiondir = new Path(TABLEDIR, regionName); - REGIONS = new ArrayList(); - REGIONS.add(regionName); - generateHLogs(-1); - - HLogFactory.createHLog(fs, regiondir, regionName, conf); - FileStatus[] logfiles = fs.listStatus(HLOGDIR); - assertTrue("There should be some log file", - logfiles != null && logfiles.length > 0); - - HLogSplitter logSplitter = new HLogSplitter( - conf, HBASEDIR, fs, null, null, this.mode) { - protected HLog.Writer createWriter(FileSystem fs, Path logfile, Configuration conf) - throws IOException { - HLog.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. - NavigableSet files = HLogUtil.getSplitEditFilesSorted(fs, regiondir); - if (files != null && !files.isEmpty()) { - for (Path file : files) { - if (!this.fs.delete(file, false)) { - LOG.error("Failed delete of " + file); - } else { - LOG.debug("Deleted recovered.edits file=" + file); - } - } - } - return writer; - } - }; - try{ - logSplitter.splitLogFile(logfiles[0], null); - } catch (IOException e) { - LOG.info(e); - Assert.fail("Throws IOException when spliting " - + "log, it is most likely because writing file does not " - + "exist which is caused by concurrent replayRecoveredEditsIfAny()"); - } - if (fs.exists(CORRUPTDIR)) { - if (fs.listStatus(CORRUPTDIR).length > 0) { - Assert.fail("There are some corrupt logs, " - + "it is most likely caused by concurrent replayRecoveredEditsIfAny()"); - } - } - } - - private static void flushToConsole(String s) { - System.out.println(s); - System.out.flush(); - } - - - private HLog.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 static void makeRegionDirs(FileSystem fs, List regions) throws IOException { - for (String region : regions) { - flushToConsole("Creating dir for region " + region); - fs.mkdirs(new Path(TABLEDIR, region)); - } - } - - private static HLog.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]; - int seq = 0; - for (int i = 0; i < writers; i++) { - ws[i] = HLogFactory.createWALWriter(dfs, new Path(HLOGDIR, HLOG_FILE_PREFIX + i), dfs.getConf()); - for (int j = 0; j < entries; j++) { - int prefix = 0; - for (String region : REGIONS) { - String row_key = region + prefix++ + i + j; - appendEntry(ws[i], TABLE_NAME, region.getBytes(), row_key.getBytes(), FAMILY, QUALIFIER, VALUE, seq++); - } - } - if (i != leaveOpen) { - ws[i].close(); - LOG.info("Closing writer " + i); - } - } - return ws; - } - - private Path[] getLogForRegion(Path rootdir, TableName table, String region) - throws IOException { - Path tdir = FSUtils.getTableDir(rootdir, table); - @SuppressWarnings("deprecation") - Path editsdir = HLogUtil.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, - Bytes.toString(region.getBytes()))); - FileStatus [] files = this.fs.listStatus(editsdir); - Path[] paths = new Path[files.length]; - for (int i = 0; i < files.length; i++) { - paths[i] = files[i].getPath(); - } - return paths; - } - - private void corruptHLog(Path path, Corruptions corruption, boolean close, - FileSystem fs) throws IOException { - - FSDataOutputStream out; - int fileSize = (int) fs.listStatus(path)[0].getLen(); - - FSDataInputStream in = fs.open(path); - byte[] corrupted_bytes = new byte[fileSize]; - in.readFully(0, corrupted_bytes, 0, fileSize); - in.close(); - - switch (corruption) { - case APPEND_GARBAGE: - fs.delete(path, false); - out = fs.create(path); - out.write(corrupted_bytes); - out.write("-----".getBytes()); - closeOrFlush(close, out); - break; - - case INSERT_GARBAGE_ON_FIRST_LINE: - fs.delete(path, false); - out = fs.create(path); - out.write(0); - out.write(corrupted_bytes); - closeOrFlush(close, out); - break; - - case INSERT_GARBAGE_IN_THE_MIDDLE: - fs.delete(path, false); - out = fs.create(path); - int middle = (int) Math.floor(corrupted_bytes.length / 2); - out.write(corrupted_bytes, 0, middle); - out.write(0); - out.write(corrupted_bytes, middle, corrupted_bytes.length - middle); - closeOrFlush(close, out); - break; - - case TRUNCATE: - fs.delete(path, false); - out = fs.create(path); - out.write(corrupted_bytes, 0, fileSize - - (32 + ProtobufLogReader.PB_WAL_COMPLETE_MAGIC.length + Bytes.SIZEOF_INT)); - closeOrFlush(close, out); - break; - - case TRUNCATE_TRAILER: - fs.delete(path, false); - out = fs.create(path); - out.write(corrupted_bytes, 0, fileSize - Bytes.SIZEOF_INT);// trailer is truncated. - closeOrFlush(close, out); - break; - } - } - - private void closeOrFlush(boolean close, FSDataOutputStream out) - throws IOException { - if (close) { - out.close(); - } else { - Method syncMethod = null; - try { - syncMethod = out.getClass().getMethod("hflush", new Class []{}); - } catch (NoSuchMethodException e) { - try { - syncMethod = out.getClass().getMethod("sync", new Class []{}); - } catch (NoSuchMethodException ex) { - throw new IOException("This version of Hadoop supports " + - "neither Syncable.sync() nor Syncable.hflush()."); - } - } - try { - syncMethod.invoke(out, new Object[]{}); - } catch (Exception e) { - throw new IOException(e); - } - // Not in 0out.hflush(); - } - } - - @SuppressWarnings("unused") - private void dumpHLog(Path log, FileSystem fs, Configuration conf) throws IOException { - HLog.Entry entry; - HLog.Reader in = HLogFactory.createReader(fs, log, conf); - while ((entry = in.next()) != null) { - System.out.println(entry); - } - } - - private int countHLog(Path log, FileSystem fs, Configuration conf) throws IOException { - int count = 0; - HLog.Reader in = HLogFactory.createReader(fs, log, conf); - while (in.next() != null) { - count++; - } - return count; - } - - - public static long appendEntry(HLog.Writer writer, TableName table, byte[] region, - byte[] row, byte[] family, byte[] qualifier, - byte[] value, long seq) - throws IOException { - LOG.info(Thread.currentThread().getName() + " append"); - writer.append(createTestEntry(table, region, row, family, qualifier, value, seq)); - LOG.info(Thread.currentThread().getName() + " sync"); - writer.sync(); - return seq; - } - - private static HLog.Entry createTestEntry( - TableName table, byte[] region, - byte[] row, byte[] family, byte[] qualifier, - byte[] value, long seq) { - long time = System.nanoTime(); - 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, - HConstants.DEFAULT_CLUSTER_ID), edit); - } - - - private void injectEmptyFile(String suffix, boolean closeFile) - throws IOException { - HLog.Writer writer = HLogFactory.createWALWriter( - fs, new Path(HLOGDIR, HLOG_FILE_PREFIX + suffix), conf); - if (closeFile) writer.close(); - } - - @SuppressWarnings("unused") - private void listLogs(FileSystem fs, Path dir) throws IOException { - for (FileStatus file : fs.listStatus(dir)) { - System.out.println(file.getPath()); - } - - } - - private int compareHLogSplitDirs(Path p1, Path p2) throws IOException { - FileStatus[] f1 = fs.listStatus(p1); - FileStatus[] f2 = fs.listStatus(p2); - assertNotNull("Path " + p1 + " doesn't exist", f1); - assertNotNull("Path " + p2 + " doesn't exist", f2); - - System.out.println("Files in " + p1 + ": " + - Joiner.on(",").join(FileUtil.stat2Paths(f1))); - System.out.println("Files in " + p2 + ": " + - Joiner.on(",").join(FileUtil.stat2Paths(f2))); - assertEquals(f1.length, f2.length); - - for (int i = 0; i < f1.length; i++) { - // Regions now have a directory named RECOVERED_EDITS_DIR and in here - // are split edit files. In below presume only 1. - Path rd1 = HLogUtil.getRegionDirRecoveredEditsDir(f1[i].getPath()); - FileStatus[] rd1fs = fs.listStatus(rd1); - assertEquals(1, rd1fs.length); - Path rd2 = HLogUtil.getRegionDirRecoveredEditsDir(f2[i].getPath()); - FileStatus[] rd2fs = fs.listStatus(rd2); - assertEquals(1, rd2fs.length); - if (!logsAreEqual(rd1fs[0].getPath(), rd2fs[0].getPath())) { - return -1; - } - } - return 0; - } - - private boolean logsAreEqual(Path p1, Path p2) throws IOException { - HLog.Reader in1, in2; - in1 = HLogFactory.createReader(fs, p1, conf); - in2 = HLogFactory.createReader(fs, p2, conf); - HLog.Entry entry1; - HLog.Entry entry2; - while ((entry1 = in1.next()) != null) { - entry2 = in2.next(); - if ((entry1.getKey().compareTo(entry2.getKey()) != 0) || - (!entry1.getEdit().toString().equals(entry2.getEdit().toString()))) { - return false; - } - } - return true; - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplitCompressed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplitCompressed.java deleted file mode 100644 index 67d936118bc..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplitCompressed.java +++ /dev/null @@ -1,36 +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 org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.junit.BeforeClass; -import org.junit.experimental.categories.Category; - -@Category({RegionServerTests.class, LargeTests.class}) -public class TestHLogSplitCompressed extends TestHLogSplit { - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - TestHLogSplit.setUpBeforeClass(); - TEST_UTIL.getConfiguration().setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true); - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java index 9bcdd82be47..b4cb213e906 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java @@ -45,6 +45,11 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.After; import org.junit.Before; @@ -54,7 +59,7 @@ import org.junit.experimental.categories.Category; /** * Tests for conditions that should trigger RegionServer aborts when - * rolling the current HLog fails. + * rolling the current WAL fails. */ @Category({RegionServerTests.class, MediumTests.class}) public class TestLogRollAbort { @@ -66,7 +71,7 @@ public class TestLogRollAbort { /* For the split-then-roll test */ private static final Path HBASEDIR = new Path("/hbase"); - private static final Path OLDLOGDIR = new Path(HBASEDIR, "hlog.old"); + private static final Path OLDLOGDIR = new Path(HBASEDIR, HConstants.HREGION_OLDLOGDIR_NAME); // Need to override this setup so we can edit the config before it gets sent // to the HDFS & HBase cluster startup. @@ -106,6 +111,7 @@ public class TestLogRollAbort { // disable region rebalancing (interferes with log watching) cluster.getMaster().balanceSwitch(false); + FSUtils.setRootDir(conf, HBASEDIR); } @After @@ -135,9 +141,8 @@ public class TestLogRollAbort { try { HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(tableName); - HLog log = server.getWAL(); + WAL log = server.getWAL(null); - 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())); @@ -177,17 +182,18 @@ public class TestLogRollAbort { @Test (timeout=300000) public void testLogRollAfterSplitStart() throws IOException { LOG.info("Verify wal roll after split starts will fail."); - HLog log = null; String logName = "testLogRollAfterSplitStart"; - Path thisTestsDir = new Path(HBASEDIR, logName); + Path thisTestsDir = new Path(HBASEDIR, DefaultWALProvider.getWALDirectoryName(logName)); + final WALFactory wals = new WALFactory(conf, null, logName); try { - // put some entries in an HLog + // put some entries in an WAL TableName tableName = 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); + final WAL log = wals.getWAL(regioninfo.getEncodedNameAsBytes()); + final AtomicLong sequenceId = new AtomicLong(1); final int total = 20; @@ -196,7 +202,8 @@ public class TestLogRollAbort { kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName())); HTableDescriptor htd = new HTableDescriptor(tableName); htd.addFamily(new HColumnDescriptor("column")); - log.append(regioninfo, tableName, kvs, System.currentTimeMillis(), htd, sequenceId); + log.append(htd, regioninfo, new WALKey(regioninfo.getEncodedNameAsBytes(), tableName, + System.currentTimeMillis()), kvs, sequenceId, true, null); } // Send the data to HDFS datanodes and close the HDFS writer log.sync(); @@ -205,17 +212,17 @@ public class TestLogRollAbort { /* code taken from MasterFileSystem.getLogDirs(), which is called from MasterFileSystem.splitLog() * handles RS shutdowns (as observed by the splitting process) */ - // rename the directory so a rogue RS doesn't create more HLogs - Path rsSplitDir = thisTestsDir.suffix(HLog.SPLITTING_EXT); + // rename the directory so a rogue RS doesn't create more WALs + Path rsSplitDir = thisTestsDir.suffix(DefaultWALProvider.SPLITTING_EXT); if (!fs.rename(thisTestsDir, rsSplitDir)) { throw new IOException("Failed fs.rename for log split: " + thisTestsDir); } LOG.debug("Renamed region directory: " + rsSplitDir); LOG.debug("Processing the old log files."); - HLogSplitter.split(HBASEDIR, rsSplitDir, OLDLOGDIR, fs, conf); + WALSplitter.split(HBASEDIR, rsSplitDir, OLDLOGDIR, fs, conf, wals); - LOG.debug("Trying to roll the HLog."); + LOG.debug("Trying to roll the WAL."); try { log.rollWriter(); Assert.fail("rollWriter() did not throw any exception."); @@ -227,9 +234,7 @@ public class TestLogRollAbort { } } } finally { - if (log != null) { - log.close(); - } + wals.close(); if (fs.exists(thisTestsDir)) { fs.delete(thisTestsDir, true); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java index 3357e08f7df..cdbdf6fddd3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -79,7 +80,7 @@ public class TestLogRollPeriod { Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); try { HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(tableName); - HLog log = server.getWAL(); + WAL log = server.getWAL(null); checkMinLogRolls(log, 5); } finally { table.close(); @@ -100,7 +101,7 @@ public class TestLogRollPeriod { TEST_UTIL.createTable(tableName, family); try { HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(tableName); - HLog log = server.getWAL(); + WAL log = server.getWAL(null); final Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); Thread writerThread = new Thread("writer") { @@ -135,29 +136,15 @@ public class TestLogRollPeriod { } } - private void checkMinLogRolls(final HLog log, final int minRolls) + private void checkMinLogRolls(final WAL log, final int minRolls) throws Exception { final List paths = new ArrayList(); - log.registerWALActionsListener(new WALActionsListener() { - @Override - public void preLogRoll(Path oldFile, Path newFile) {} + log.registerWALActionsListener(new WALActionsListener.Base() { @Override public void postLogRoll(Path oldFile, Path newFile) { LOG.debug("postLogRoll: oldFile="+oldFile+" newFile="+newFile); paths.add(newFile); } - @Override - public void preLogArchive(Path oldFile, Path newFile) {} - @Override - public void postLogArchive(Path oldFile, Path newFile) {} - @Override - public void logRollRequested() {} - @Override - public void logCloseRequested() {} - @Override - public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit) {} - @Override - public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, WALEdit logEdit) {} }); // Sleep until we should get at least min-LogRoll events diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java index a3349a15dae..0c9069d674b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java @@ -37,14 +37,15 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Get; @@ -64,6 +65,9 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.server.datanode.DataNode; @@ -81,7 +85,6 @@ import org.junit.experimental.categories.Category; public class TestLogRolling { private static final Log LOG = LogFactory.getLog(TestLogRolling.class); private HRegionServer server; - private HLog log; private String tableName; private byte[] value; private FileSystem fs; @@ -96,7 +99,6 @@ public class TestLogRolling { */ public TestLogRolling() { this.server = null; - this.log = null; this.tableName = null; String className = this.getClass().getName(); @@ -175,12 +177,10 @@ 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(); Table table = createTestTable(this.tableName); server = TEST_UTIL.getRSForFirstRegionInTable(table.getName()); - this.log = server.getWAL(); for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls doPut(table, i); if (i % 32 == 0) { @@ -199,15 +199,15 @@ public class TestLogRolling { */ @Test(timeout=120000) public void testLogRollOnNothingWritten() throws Exception { - Configuration conf = TEST_UTIL.getConfiguration(); - HFileSystem fs = new HFileSystem(conf, false); - HLog newLog = HLogFactory.createHLog(fs.getBackingFs(), - FSUtils.getRootDir(conf), "test", conf, null, "test.com:8080:1"); + final Configuration conf = TEST_UTIL.getConfiguration(); + final WALFactory wals = new WALFactory(conf, null, + ServerName.valueOf("test.com",8080, 1).toString()); + final WAL newLog = wals.getWAL(new byte[]{}); try { // Now roll the log before we write anything. newLog.rollWriter(true); } finally { - newLog.closeAndDelete(); + wals.close(); } } @@ -221,7 +221,9 @@ public class TestLogRolling { this.tableName = getName(); // TODO: Why does this write data take for ever? startAndWriteData(); - LOG.info("after writing there are " + ((FSHLog) log).getNumRolledLogFiles() + " log files"); + final WAL log = server.getWAL(null); + LOG.info("after writing there are " + DefaultWALProvider.getNumRolledLogFiles(log) + + " log files"); // flush all regions @@ -234,9 +236,8 @@ public class TestLogRolling { // Now roll the log log.rollWriter(); - int count = ((FSHLog) log).getNumRolledLogFiles(); - LOG.info("after flushing all regions and rolling logs there are " + - ((FSHLog) log).getNumRolledLogFiles() + " log files"); + int count = DefaultWALProvider.getNumRolledLogFiles(log); + LOG.info("after flushing all regions and rolling logs there are " + count + " log files"); assertTrue(("actual count: " + count), count <= 2); } @@ -266,7 +267,7 @@ public class TestLogRolling { LOG.info("Validated row " + row); } - void batchWriteAndWait(Table table, int start, boolean expect, int timeout) + void batchWriteAndWait(Table table, final FSHLog log, int start, boolean expect, int timeout) throws IOException { for (int i = 0; i < 10; i++) { Put put = new Put(Bytes.toBytes("row" @@ -294,28 +295,6 @@ public class TestLogRolling { } } - /** - * Give me the HDFS pipeline for this log file - */ - DatanodeInfo[] getPipeline(HLog log) throws IllegalArgumentException, - IllegalAccessException, InvocationTargetException { - OutputStream stm = ((FSHLog) log).getOutputStream(); - Method getPipeline = null; - for (Method m : stm.getClass().getDeclaredMethods()) { - if (m.getName().endsWith("getPipeline")) { - getPipeline = m; - getPipeline.setAccessible(true); - break; - } - } - - assertTrue("Need DFSOutputStream.getPipeline() for this test", - null != getPipeline); - Object repl = getPipeline.invoke(stm, new Object[] {} /* NO_ARGS */); - return (DatanodeInfo[]) repl; - } - - /** * Tests that logs are rolled upon detecting datanode death * Requires an HDFS jar with HDFS-826 & syncFs() support (HDFS-200) @@ -323,13 +302,12 @@ public class TestLogRolling { @Test public void testLogRollOnDatanodeDeath() throws Exception { TEST_UTIL.ensureSomeRegionServersAvailable(2); - assertTrue("This test requires HLog file replication set to 2.", + assertTrue("This test requires WAL file replication set to 2.", fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) == 2); LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS())); this.server = cluster.getRegionServer(0); - this.log = server.getWAL(); // Create the test table and open it HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(getName())); @@ -340,9 +318,8 @@ public class TestLogRolling { assertTrue(table.isAutoFlush()); server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName()); - this.log = server.getWAL(); + final FSHLog log = (FSHLog) server.getWAL(null); - 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())); @@ -371,13 +348,14 @@ public class TestLogRolling { writeData(table, 2); long curTime = System.currentTimeMillis(); - long oldFilenum = ((FSHLog) log).getFilenum(); + LOG.info("log.getCurrentFileName(): " + log.getCurrentFileName()); + long oldFilenum = DefaultWALProvider.extractFileNumFromWAL(log); 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()); - final DatanodeInfo[] pipeline = getPipeline(log); + oldFilenum == DefaultWALProvider.extractFileNumFromWAL(log)); + final DatanodeInfo[] pipeline = log.getPipeLine(); assertTrue(pipeline.length == fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS())); @@ -387,7 +365,7 @@ public class TestLogRolling { // this write should succeed, but trigger a log roll writeData(table, 2); - long newFilenum = ((FSHLog) log).getFilenum(); + long newFilenum = DefaultWALProvider.extractFileNumFromWAL(log); assertTrue("Missing datanode should've triggered a log roll", newFilenum > oldFilenum && newFilenum > curTime); @@ -395,15 +373,15 @@ 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); + DefaultWALProvider.extractFileNumFromWAL(log) == 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); - batchWriteAndWait(table, 3, false, 14000); + batchWriteAndWait(table, log, 3, false, 14000); + int replication = log.getLogReplication(); assertTrue("LowReplication Roller should've been disabled, current replication=" - + ((FSHLog) log).getLogReplication(), - !log.isLowReplicationRollEnabled()); + + replication, !log.isLowReplicationRollEnabled()); dfsCluster .startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null); @@ -411,24 +389,23 @@ public class TestLogRolling { // Force roll writer. The new log file will have the default replications, // and the LowReplication Roller will be enabled. log.rollWriter(true); - batchWriteAndWait(table, 13, true, 10000); + batchWriteAndWait(table, log, 13, true, 10000); + replication = log.getLogReplication(); assertTrue("New log file should have the default replication instead of " + - ((FSHLog) log).getLogReplication(), - ((FSHLog) log).getLogReplication() == - fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS())); - assertTrue("LowReplication Roller should've been enabled", - log.isLowReplicationRollEnabled()); + replication, + replication == fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS())); + assertTrue("LowReplication Roller should've been enabled", log.isLowReplicationRollEnabled()); } /** - * Test that HLog is rolled when all data nodes in the pipeline have been + * Test that WAL is rolled when all data nodes in the pipeline have been * restarted. * @throws Exception */ @Test public void testLogRollOnPipelineRestart() throws Exception { LOG.info("Starting testLogRollOnPipelineRestart"); - assertTrue("This test requires HLog file replication.", + assertTrue("This test requires WAL file replication.", fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) > 1); LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS())); @@ -436,7 +413,6 @@ public class TestLogRolling { Table t = new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME); try { this.server = cluster.getRegionServer(0); - this.log = server.getWAL(); // Create the test table and open it HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(getName())); @@ -446,11 +422,11 @@ public class TestLogRolling { HTable table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName()); server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName()); - this.log = server.getWAL(); + final WAL log = server.getWAL(null); final List paths = new ArrayList(); final List preLogRolledCalled = new ArrayList(); - paths.add(((FSHLog) log).computeFilename()); - log.registerWALActionsListener(new WALActionsListener() { + paths.add(DefaultWALProvider.getCurrentFileName(log)); + log.registerWALActionsListener(new WALActionsListener.Base() { @Override public void preLogRoll(Path oldFile, Path newFile) { LOG.debug("preLogRoll: oldFile="+oldFile+" newFile="+newFile); @@ -460,23 +436,8 @@ public class TestLogRolling { public void postLogRoll(Path oldFile, Path newFile) { paths.add(newFile); } - @Override - public void preLogArchive(Path oldFile, Path newFile) {} - @Override - public void postLogArchive(Path oldFile, Path newFile) {} - @Override - public void logRollRequested() {} - @Override - public void logCloseRequested() {} - @Override - public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey, - WALEdit logEdit) {} - @Override - public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, - 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())); @@ -486,11 +447,13 @@ public class TestLogRolling { table.setAutoFlush(true, true); long curTime = System.currentTimeMillis(); - long oldFilenum = log.getFilenum(); + LOG.info("log.getCurrentFileName()): " + DefaultWALProvider.getCurrentFileName(log)); + long oldFilenum = DefaultWALProvider.extractFileNumFromWAL(log); 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 == + DefaultWALProvider.extractFileNumFromWAL(log)); // roll all datanodes in the pipeline dfsCluster.restartDataNodes(); @@ -501,7 +464,7 @@ public class TestLogRolling { // this write should succeed, but trigger a log roll writeData(table, 1003); - long newFilenum = log.getFilenum(); + long newFilenum = DefaultWALProvider.extractFileNumFromWAL(log); assertTrue("Missing datanode should've triggered a log roll", newFilenum > oldFilenum && newFilenum > curTime); @@ -532,12 +495,11 @@ public class TestLogRolling { fsUtils.recoverFileLease(((HFileSystem)fs).getBackingFs(), p, TEST_UTIL.getConfiguration(), null); - LOG.debug("Reading HLog "+FSUtils.getPath(p)); - HLog.Reader reader = null; + LOG.debug("Reading WAL "+FSUtils.getPath(p)); + WAL.Reader reader = null; try { - reader = HLogFactory.createReader(fs, p, - TEST_UTIL.getConfiguration()); - HLog.Entry entry; + reader = WALFactory.createReader(fs, p, TEST_UTIL.getConfiguration()); + WAL.Entry entry; while ((entry = reader.next()) != null) { LOG.debug("#"+entry.getKey().getLogSeqNum()+": "+entry.getEdit().getCells()); for (Cell cell : entry.getEdit().getCells()) { @@ -601,8 +563,7 @@ public class TestLogRolling { table2 = createTestTable(getName() + "1"); server = TEST_UTIL.getRSForFirstRegionInTable(table.getName()); - this.log = server.getWAL(); - FSHLog fshLog = (FSHLog)log; + final WAL log = server.getWAL(null); HRegion region = server.getOnlineRegions(table2.getName()).get(0); Store s = region.getStore(HConstants.CATALOG_FAMILY); @@ -615,12 +576,14 @@ public class TestLogRolling { admin.flush(table2.getName()); } 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, + DefaultWALProvider.getNumRolledLogFiles(log)); 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, + DefaultWALProvider.getNumRolledLogFiles(log)); admin.flush(table2.getName()); region.compactStores(); // Wait for compaction in case if flush triggered it before us. @@ -632,14 +595,16 @@ 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, + DefaultWALProvider.getNumRolledLogFiles(log)); // Flush table to make latest WAL obsolete; write another record, and roll again. admin.flush(table.getName()); 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, + DefaultWALProvider.getNumRolledLogFiles(log)); } finally { if (t != null) t.close(); if (table != null) table.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java index fb45c00a197..41e05aee1a7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java @@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; @@ -35,11 +36,15 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKey; import org.junit.Test; import org.junit.experimental.categories.Category; /** - * Test many concurrent appenders to an {@link #HLog} while rolling the log. + * Test many concurrent appenders to an {@link #WAL} while rolling the log. */ @Category({RegionServerTests.class, SmallTests.class}) public class TestLogRollingNoCluster { @@ -59,7 +64,10 @@ public class TestLogRollingNoCluster { Path dir = TEST_UTIL.getDataTestDir(); // The implementation needs to know the 'handler' count. TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, THREAD_COUNT); - HLog wal = HLogFactory.createHLog(fs, dir, "logs", TEST_UTIL.getConfiguration()); + final Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); + FSUtils.setRootDir(conf, dir); + final WALFactory wals = new WALFactory(conf, null, TestLogRollingNoCluster.class.getName()); + final WAL wal = wals.getWAL(new byte[]{}); Appender [] appenders = null; @@ -78,7 +86,7 @@ public class TestLogRollingNoCluster { appenders[i].join(); } } finally { - wal.close(); + wals.close(); } for (int i = 0; i < count; i++) { assertFalse(appenders[i].isException()); @@ -90,11 +98,11 @@ public class TestLogRollingNoCluster { */ static class Appender extends Thread { private final Log log; - private final HLog wal; + private final WAL wal; private final int count; private Exception e = null; - Appender(final HLog wal, final int index, final int count) { + Appender(final WAL wal, final int index, final int count) { super("" + index); this.wal = wal; this.count = count; @@ -119,16 +127,18 @@ public class TestLogRollingNoCluster { try { 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) { + // Roll every ten edits + if (i % 10 == 0) { this.wal.rollWriter(); } WALEdit edit = new WALEdit(); byte[] bytes = Bytes.toBytes(i); edit.add(new KeyValue(bytes, bytes, bytes, now, EMPTY_1K_ARRAY)); - this.wal.append(HRegionInfo.FIRST_META_REGIONINFO, - TableName.META_TABLE_NAME, - edit, now, TEST_UTIL.getMetaTableDescriptor(), sequenceId); + final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO; + final HTableDescriptor htd = TEST_UTIL.getMetaTableDescriptor(); + final long txid = wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), + TableName.META_TABLE_NAME, now), edit, sequenceId, true, null); + wal.sync(txid); } String msg = getName() + " finished"; if (isException()) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadOldRootAndMetaEdits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadOldRootAndMetaEdits.java index 66a231d0b91..b256651cd32 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadOldRootAndMetaEdits.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadOldRootAndMetaEdits.java @@ -41,6 +41,10 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.hbase.wal.WALProvider; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -55,13 +59,15 @@ public class TestReadOldRootAndMetaEdits { private final static Log LOG = LogFactory.getLog(TestReadOldRootAndMetaEdits.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static Configuration conf; private static FileSystem fs; private static Path dir; @BeforeClass public static void setupBeforeClass() throws Exception { - TEST_UTIL.getConfiguration().setClass("hbase.regionserver.hlog.writer.impl", - SequenceFileLogWriter.class, HLog.Writer.class); + conf = TEST_UTIL.getConfiguration(); + conf.setClass("hbase.regionserver.hlog.writer.impl", + SequenceFileLogWriter.class, WALProvider.Writer.class); fs = TEST_UTIL.getTestFileSystem(); dir = new Path(TEST_UTIL.createRootDir(), "testReadOldRootAndMetaEdits"); fs.mkdirs(dir); @@ -80,17 +86,14 @@ public class TestReadOldRootAndMetaEdits { @Test public void testReadOldRootAndMetaEdits() throws IOException { LOG.debug("testReadOldRootAndMetaEdits"); - Configuration conf = HBaseConfiguration.create(); - conf.setClass("hbase.regionserver.hlog.writer.impl", SequenceFileLogWriter.class, - HLog.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; + WALProvider.Writer writer = null; + WAL.Reader reader = null; // a regular table TableName t = TableName.valueOf("t"); HRegionInfo tRegionInfo = null; @@ -99,21 +102,21 @@ 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); // write above entries - writer = HLogFactory.createWALWriter(fs, path, conf); + writer = WALFactory.createWALWriter(fs, path, conf); writer.append(tEntry); writer.append(rootEntry); writer.append(oldMetaEntry); @@ -123,8 +126,8 @@ public class TestReadOldRootAndMetaEdits { writer.close(); // read the log and see things are okay. - reader = HLogFactory.createReader(fs, path, conf); - HLog.Entry entry = reader.next(); + reader = WALFactory.createReader(fs, path, conf); + WAL.Entry entry = reader.next(); assertNotNull(entry); assertTrue(entry.getKey().getTablename().equals(t)); assertEquals(Bytes.toString(entry.getKey().getEncodedRegionName()), @@ -145,15 +148,15 @@ public class TestReadOldRootAndMetaEdits { } } /** - * Creates a WALEdit for the passed KeyValues and returns a HLog.Entry instance composed of - * the WALEdit and passed HLogKey. - * @return HLog.Entry instance for the passed HLogKey and KeyValues + * Creates a WALEdit for the passed KeyValues and returns a WALProvider.Entry instance composed of + * the WALEdit and passed WALKey. + * @return WAL.Entry instance for the passed WALKey and KeyValues */ - private HLog.Entry createAEntry(HLogKey hlogKey, List kvs) { + private WAL.Entry createAEntry(WALKey walKey, List kvs) { WALEdit edit = new WALEdit(); for (KeyValue kv : kvs ) edit.add(kv); - return new HLog.Entry(hlogKey, edit); + return new WAL.Entry(walKey, edit); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureHLog.java deleted file mode 100644 index 4dccccb2d42..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureHLog.java +++ /dev/null @@ -1,130 +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 static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import java.util.List; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.commons.io.IOUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.commons.logging.impl.Log4JLogger; -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.Cell; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.log4j.Level; - -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -@Category({RegionServerTests.class, MediumTests.class}) -public class TestSecureHLog { - static final Log LOG = LogFactory.getLog(TestSecureHLog.class); - static { - ((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hbase.regionserver.wal")) - .getLogger().setLevel(Level.ALL); - }; - static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - Configuration conf = TEST_UTIL.getConfiguration(); - 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); - conf.setClass("hbase.regionserver.hlog.writer.impl", SecureProtobufLogWriter.class, - HLog.Writer.class); - conf.setBoolean(HConstants.ENABLE_WAL_ENCRYPTION, true); - } - - @Test - public void testSecureHLog() throws Exception { - TableName tableName = TableName.valueOf("TestSecureHLog"); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(tableName.getName())); - HRegionInfo regioninfo = new HRegionInfo(tableName, - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false); - final int total = 10; - final byte[] row = Bytes.toBytes("row"); - final byte[] family = Bytes.toBytes("family"); - final byte[] value = Bytes.toBytes("Test value"); - FileSystem fs = TEST_UTIL.getTestFileSystem(); - Path logDir = TEST_UTIL.getDataTestDir("log"); - final AtomicLong sequenceId = new AtomicLong(1); - - // Write the WAL - HLog wal = new FSHLog(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(); - wal.close(); - - // Insure edits are not plaintext - long length = fs.getFileStatus(walPath).getLen(); - FSDataInputStream in = fs.open(walPath); - byte[] fileData = new byte[(int)length]; - IOUtils.readFully(in, fileData); - in.close(); - 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, - TEST_UTIL.getConfiguration()); - int count = 0; - HLog.Entry entry = new HLog.Entry(); - while (reader.next(entry) != null) { - count++; - List cells = entry.getEdit().getCells(); - assertTrue("Should be one KV per WALEdit", cells.size() == 1); - for (Cell cell: cells) { - byte[] thisRow = cell.getRow(); - assertTrue("Incorrect row", Bytes.equals(thisRow, row)); - byte[] thisFamily = cell.getFamily(); - assertTrue("Incorrect family", Bytes.equals(thisFamily, family)); - byte[] thisValue = cell.getValue(); - assertTrue("Incorrect value", Bytes.equals(thisValue, value)); - } - } - assertEquals("Should have read back as many KVs as written", total, count); - reader.close(); - } - -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java index 4cea1205ab5..be5d9518a59 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java @@ -22,6 +22,8 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; +import org.apache.hadoop.hbase.wal.WAL.Reader; +import org.apache.hadoop.hbase.wal.WALProvider.Writer; import org.junit.BeforeClass; import org.junit.experimental.categories.Category; @@ -35,9 +37,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); + Reader.class); conf.setClass("hbase.regionserver.hlog.writer.impl", SecureProtobufLogWriter.class, - HLog.Writer.class); + Writer.class); conf.setBoolean(HConstants.ENABLE_WAL_ENCRYPTION, true); TestWALReplay.setUpBeforeClass(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java index 35498c1346a..c8629d06f8a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java @@ -31,6 +31,10 @@ import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALKey; import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; @@ -40,7 +44,7 @@ import org.junit.experimental.categories.Category; import static org.junit.Assert.*; /** - * Test that the actions are called while playing with an HLog + * Test that the actions are called while playing with an WAL */ @Category({RegionServerTests.class, SmallTests.class}) public class TestWALActionsListener { @@ -51,9 +55,6 @@ public class TestWALActionsListener { private final static byte[] SOME_BYTES = Bytes.toBytes("t"); private static FileSystem fs; - private static Path oldLogDir; - private static Path logDir; - private static String logName; private static Configuration conf; @BeforeClass @@ -61,17 +62,13 @@ public class TestWALActionsListener { conf = TEST_UTIL.getConfiguration(); conf.setInt("hbase.regionserver.maxlogs", 5); fs = FileSystem.get(conf); - oldLogDir = new Path(TEST_UTIL.getDataTestDir(), - HConstants.HREGION_OLDLOGDIR_NAME); - logName = HConstants.HREGION_LOGDIR_NAME; - logDir = new Path(TEST_UTIL.getDataTestDir(), - logName); + FSUtils.setRootDir(conf, TEST_UTIL.getDataTestDir()); } @Before public void setUp() throws Exception { - fs.delete(logDir, true); - fs.delete(oldLogDir, true); + fs.delete(new Path(TEST_UTIL.getDataTestDir(), HConstants.HREGION_LOGDIR_NAME), true); + fs.delete(new Path(TEST_UTIL.getDataTestDir(), HConstants.HREGION_OLDLOGDIR_NAME), true); } @After @@ -89,12 +86,12 @@ public class TestWALActionsListener { DummyWALActionsListener observer = new DummyWALActionsListener(); List list = new ArrayList(); list.add(observer); + final WALFactory wals = new WALFactory(conf, list, "testActionListener"); DummyWALActionsListener laterobserver = new DummyWALActionsListener(); - HLog 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), SOME_BYTES, SOME_BYTES, false); + final WAL wal = wals.getWAL(hri.getEncodedNameAsBytes()); for (int i = 0; i < 20; i++) { byte[] b = Bytes.toBytes(i+""); @@ -104,17 +101,18 @@ public class TestWALActionsListener { HTableDescriptor htd = new HTableDescriptor(); htd.addFamily(new HColumnDescriptor(b)); - hlog.append(hri, TableName.valueOf(b), edit, 0, htd, sequenceId); + final long txid = wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), + TableName.valueOf(b), 0), edit, sequenceId, true, null); + wal.sync(txid); if (i == 10) { - hlog.registerWALActionsListener(laterobserver); + wal.registerWALActionsListener(laterobserver); } if (i % 2 == 0) { - hlog.rollWriter(); + wal.rollWriter(); } } - hlog.close(); - hlog.closeAndDelete(); + wal.close(); assertEquals(11, observer.preLogRollCounter); assertEquals(11, observer.postLogRollCounter); @@ -127,7 +125,7 @@ public class TestWALActionsListener { /** * Just counts when methods are called */ - static class DummyWALActionsListener implements WALActionsListener { + static class DummyWALActionsListener extends WALActionsListener.Base { public int preLogRollCounter = 0; public int postLogRollCounter = 0; public int closedCount = 0; @@ -142,37 +140,10 @@ public class TestWALActionsListener { postLogRollCounter++; } - @Override - public void preLogArchive(Path oldFile, Path newFile) { - // Not interested - } - - @Override - public void postLogArchive(Path oldFile, Path newFile) { - // Not interested - } - - @Override - public void logRollRequested() { - // Not interested - } - - @Override - public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey, - WALEdit logEdit) { - // Not interested - - } - @Override public void logCloseRequested() { closedCount++; } - - public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, WALEdit logEdit) { - //To change body of implemented methods use File | Settings | File Templates. - } - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java index 1f6bd1c425c..f3651ae0df3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver.wal; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Mockito.when; @@ -80,11 +81,18 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HFileTestUtil; import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALSplitter; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; +import org.junit.Rule; +import org.junit.rules.TestName; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -103,6 +111,10 @@ public class TestWALReplay { private FileSystem fs; private Configuration conf; private RecoveryMode mode; + private WALFactory wals; + + @Rule + public final TestName currentTest = new TestName(); @BeforeClass @@ -129,17 +141,19 @@ public class TestWALReplay { this.fs = TEST_UTIL.getDFSCluster().getFileSystem(); this.hbaseRootDir = FSUtils.getRootDir(this.conf); this.oldLogDir = new Path(this.hbaseRootDir, HConstants.HREGION_OLDLOGDIR_NAME); - this.logName = HConstants.HREGION_LOGDIR_NAME; + this.logName = DefaultWALProvider.getWALDirectoryName(currentTest.getMethodName() + "-manual"); this.logDir = new Path(this.hbaseRootDir, logName); if (TEST_UTIL.getDFSCluster().getFileSystem().exists(this.hbaseRootDir)) { TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true); } this.mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ? RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING); + this.wals = new WALFactory(conf, null, currentTest.getMethodName()); } @After public void tearDown() throws Exception { + this.wals.close(); TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true); } @@ -273,7 +287,7 @@ public class TestWALReplay { HRegion.closeHRegion(region2); final byte [] rowName = tableName.getName(); - HLog wal1 = createWAL(this.conf); + WAL wal1 = createWAL(this.conf); // Add 1k to each family. final int countPerFamily = 1000; final AtomicLong sequenceId = new AtomicLong(1); @@ -281,19 +295,19 @@ public class TestWALReplay { addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee, wal1, htd, sequenceId); } - wal1.close(); + wal1.shutdown(); runWALSplit(this.conf); - HLog wal2 = createWAL(this.conf); + WAL wal2 = createWAL(this.conf); // Add 1k to each family. for (HColumnDescriptor hcd: htd.getFamilies()) { addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee, wal2, htd, sequenceId); } - wal2.close(); + wal2.shutdown(); runWALSplit(this.conf); - HLog wal3 = createWAL(this.conf); + WAL wal3 = createWAL(this.conf); try { HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal3); long seqid = region.getOpenSeqNum(); @@ -307,7 +321,7 @@ public class TestWALReplay { // TODO: Scan all. region.close(); } finally { - wal3.closeAndDelete(); + wal3.close(); } } @@ -333,7 +347,7 @@ public class TestWALReplay { HRegion region2 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd); HRegion.closeHRegion(region2); - HLog wal = createWAL(this.conf); + WAL wal = createWAL(this.conf); HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf); byte [] family = htd.getFamilies().iterator().next().getName(); @@ -360,7 +374,7 @@ public class TestWALReplay { @Override public Object run() throws Exception { runWALSplit(newConf); - HLog wal2 = createWAL(newConf); + WAL wal2 = createWAL(newConf); HRegion region2 = HRegion.openHRegion(newConf, FileSystem.get(newConf), hbaseRootDir, hri, htd, wal2); @@ -370,7 +384,7 @@ public class TestWALReplay { // I can't close wal1. Its been appropriated when we split. region2.close(); - wal2.closeAndDelete(); + wal2.close(); return null; } }); @@ -400,7 +414,7 @@ public class TestWALReplay { HRegion region2 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd); HRegion.closeHRegion(region2); - HLog wal = createWAL(this.conf); + WAL wal = createWAL(this.conf); HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf); // Add an edit so something in the WAL @@ -432,7 +446,7 @@ public class TestWALReplay { @Override public Object run() throws Exception { runWALSplit(newConf); - HLog wal2 = createWAL(newConf); + WAL wal2 = createWAL(newConf); HRegion region2 = HRegion.openHRegion(newConf, FileSystem.get(newConf), hbaseRootDir, hri, htd, wal2); @@ -442,7 +456,7 @@ public class TestWALReplay { // I can't close wal1. Its been appropriated when we split. region2.close(); - wal2.closeAndDelete(); + wal2.close(); return null; } }); @@ -476,7 +490,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); + WAL wal = createWAL(this.conf); HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal); long seqid = region.getOpenSeqNum(); boolean first = true; @@ -497,9 +511,9 @@ public class TestWALReplay { // replay of log has the correct effect, that our seqids are calculated correctly so // all edits in logs are seen as 'stale'/old. region.close(true); - wal.close(); + wal.shutdown(); runWALSplit(this.conf); - HLog wal2 = createWAL(this.conf); + WAL wal2 = createWAL(this.conf); HRegion region2 = HRegion.openHRegion(conf, this.fs, hbaseRootDir, hri, htd, wal2); long seqid2 = region2.getOpenSeqNum(); assertTrue(seqid + result.size() < seqid2); @@ -516,9 +530,6 @@ public class TestWALReplay { final Result result2 = region2.get(g); assertEquals(2 * result.size(), result2.size()); wal2.sync(); - // Set down maximum recovery so we dfsclient doesn't linger retrying something - // long gone. - HBaseTestingUtility.setMaxRecoveryErrorCount(((FSHLog) wal2).getOutputStream(), 1); final Configuration newConf = HBaseConfiguration.create(this.conf); User user = HBaseTestingUtility.getDifferentUser(newConf, tableName.getNameAsString()); @@ -528,7 +539,7 @@ public class TestWALReplay { runWALSplit(newConf); FileSystem newFS = FileSystem.get(newConf); // Make a new wal for new region open. - HLog wal3 = createWAL(newConf); + WAL wal3 = createWAL(newConf); final AtomicInteger countOfRestoredEdits = new AtomicInteger(0); HRegion region3 = new HRegion(basedir, wal3, newFS, newConf, hri, htd, null) { @Override @@ -547,7 +558,7 @@ public class TestWALReplay { // I can't close wal1. Its been appropriated when we split. region3.close(); - wal3.closeAndDelete(); + wal3.close(); return null; } }); @@ -589,7 +600,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); + WAL 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()) { @@ -605,7 +616,7 @@ public class TestWALReplay { // Let us flush the region region.flushcache(); region.close(true); - wal.close(); + wal.shutdown(); // delete the store files in the second column family to simulate a failure // in between the flushcache(); @@ -622,7 +633,7 @@ public class TestWALReplay { // Let us try to split and recover runWALSplit(this.conf); - HLog wal2 = createWAL(this.conf); + WAL 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); @@ -668,11 +679,11 @@ public class TestWALReplay { final HTableDescriptor htd = createBasic3FamilyHTD(tableName); HRegion region3 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd); region3.close(); - region3.getLog().closeAndDelete(); + region3.getWAL().close(); // 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); + WAL wal = createWAL(this.conf); RegionServerServices rsServices = Mockito.mock(RegionServerServices.class); Mockito.doReturn(false).when(rsServices).isAborted(); when(rsServices.getServerName()).thenReturn(ServerName.valueOf("foo", 10, 10)); @@ -725,11 +736,11 @@ public class TestWALReplay { } region.close(true); - wal.close(); + wal.shutdown(); // Let us try to split and recover runWALSplit(this.conf); - HLog wal2 = createWAL(this.conf); + WAL wal2 = createWAL(this.conf); Mockito.doReturn(false).when(rsServices).isAborted(); HRegion region2 = HRegion.openHRegion(this.hbaseRootDir, hri, htd, wal2, this.conf, rsServices, null); @@ -752,7 +763,7 @@ public class TestWALReplay { } /** - * Create an HRegion with the result of a HLog split and test we only see the + * Create an HRegion with the result of a WAL split and test we only see the * good edits * @throws Exception */ @@ -768,7 +779,7 @@ public class TestWALReplay { HRegion region2 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd); HRegion.closeHRegion(region2); - final HLog wal = createWAL(this.conf); + final WAL wal = createWAL(this.conf); final byte[] rowName = tableName.getName(); final byte[] regionName = hri.getEncodedNameAsBytes(); final AtomicLong sequenceId = new AtomicLong(1); @@ -789,20 +800,19 @@ public class TestWALReplay { long now = ee.currentTime(); edit.add(new KeyValue(rowName, Bytes.toBytes("another family"), rowName, now, rowName)); - wal.append(hri, tableName, edit, now, htd, sequenceId); + wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now), edit, sequenceId, + true, null); // Delete the c family to verify deletes make it over. edit = new WALEdit(); now = ee.currentTime(); edit.add(new KeyValue(rowName, Bytes.toBytes("c"), null, now, KeyValue.Type.DeleteFamily)); - wal.append(hri, tableName, edit, now, htd, sequenceId); + wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now), edit, sequenceId, + true, null); // Sync. wal.sync(); - // Set down maximum recovery so we dfsclient doesn't linger retrying something - // long gone. - HBaseTestingUtility.setMaxRecoveryErrorCount(((FSHLog) 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); @@ -816,14 +826,14 @@ 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); + WAL newWal = createWAL(newConf); final AtomicInteger flushcount = new AtomicInteger(0); try { final HRegion region = new HRegion(basedir, newWal, newFS, newConf, hri, htd, null) { @Override protected FlushResult internalFlushcache( - final HLog wal, final long myseqid, MonitoredTask status) + final WAL wal, final long myseqid, MonitoredTask status) throws IOException { LOG.info("InternalFlushCache Invoked"); FlushResult fs = super.internalFlushcache(wal, myseqid, @@ -844,7 +854,7 @@ public class TestWALReplay { result.size()); region.close(); } finally { - newWal.closeAndDelete(); + newWal.close(); } return null; } @@ -854,8 +864,7 @@ public class TestWALReplay { @Test // the following test is for HBASE-6065 public void testSequentialEditLogSeqNum() throws IOException { - final TableName tableName = - TableName.valueOf("testSequentialEditLogSeqNum"); + final TableName tableName = TableName.valueOf(currentTest.getMethodName()); final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName); final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName); @@ -864,8 +873,8 @@ public class TestWALReplay { final int countPerFamily = 10; final HTableDescriptor htd = createBasic1FamilyHTD(tableName); - // Mock the HLog - MockHLog wal = createMockWAL(this.conf); + // Mock the WAL + MockWAL wal = createMockWAL(); HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal); for (HColumnDescriptor hcd : htd.getFamilies()) { @@ -884,10 +893,12 @@ public class TestWALReplay { // allow complete cache flush with the previous seq number got after first // set of edits. wal.completeCacheFlush(hri.getEncodedNameAsBytes()); - wal.close(); - FileStatus[] listStatus = this.fs.listStatus(wal.getDir()); - HLogSplitter.splitLogFile(hbaseRootDir, listStatus[0], - this.fs, this.conf, null, null, null, mode); + wal.shutdown(); + FileStatus[] listStatus = wal.getFiles(); + assertNotNull(listStatus); + assertTrue(listStatus.length > 0); + WALSplitter.splitLogFile(hbaseRootDir, listStatus[0], + this.fs, this.conf, null, null, null, mode, wals); FileStatus[] listStatus1 = this.fs.listStatus( new Path(FSUtils.getTableDir(hbaseRootDir, tableName), new Path(hri.getEncodedName(), "recovered.edits"))); @@ -901,11 +912,12 @@ public class TestWALReplay { lastestSeqNumber, editCount); } - static class MockHLog extends FSHLog { + static class MockWAL extends FSHLog { boolean doCompleteCacheFlush = false; - public MockHLog(FileSystem fs, Path rootDir, String logName, Configuration conf) throws IOException { - super(fs, rootDir, logName, conf); + public MockWAL(FileSystem fs, Path rootDir, String logName, Configuration conf) + throws IOException { + super(fs, rootDir, logName, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null); } @Override @@ -924,11 +936,11 @@ public class TestWALReplay { return htd; } - private MockHLog createMockWAL(Configuration conf) throws IOException { - MockHLog wal = new MockHLog(FileSystem.get(conf), hbaseRootDir, logName, conf); + private MockWAL createMockWAL() throws IOException { + MockWAL wal = new MockWAL(fs, 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(wal.getOutputStream(), 1); return wal; } @@ -969,7 +981,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 WAL wal, final HTableDescriptor htd, final AtomicLong sequenceId) throws IOException { String familyStr = Bytes.toString(family); @@ -979,8 +991,10 @@ public class TestWALReplay { WALEdit edit = new WALEdit(); edit.add(new KeyValue(rowName, family, qualifierBytes, ee.currentTime(), columnBytes)); - wal.append(hri, tableName, edit, ee.currentTime(), htd, sequenceId); + wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, ee.currentTime()), + edit, sequenceId, true, null); } + wal.sync(); } static List addRegionEdits (final byte [] rowName, final byte [] family, @@ -1014,8 +1028,8 @@ public class TestWALReplay { * @throws IOException */ private Path runWALSplit(final Configuration c) throws IOException { - List splits = HLogSplitter.split( - hbaseRootDir, logDir, oldLogDir, FileSystem.get(c), c); + List splits = WALSplitter.split( + hbaseRootDir, logDir, oldLogDir, FileSystem.get(c), c, wals); // Split should generate only 1 file since there's only 1 region assertEquals("splits=" + splits, 1, splits.size()); // Make sure the file exists @@ -1029,12 +1043,11 @@ 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), - hbaseRootDir, logName, c); + private WAL createWAL(final Configuration c) throws IOException { + FSHLog wal = new FSHLog(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(wal.getOutputStream(), 1); return wal; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java index 6b241b948d1..f9aeb6f5cfb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java @@ -21,16 +21,21 @@ package org.apache.hadoop.hbase.replication; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.fail; import java.io.IOException; +import java.util.Arrays; +import java.util.concurrent.CountDownLatch; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; @@ -38,6 +43,8 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; @@ -102,7 +109,7 @@ public class TestMultiSlaveReplication { utility2 = new HBaseTestingUtility(conf2); utility2.setZkCluster(miniZK); - new ZooKeeperWatcher(conf2, "cluster3", null, true); + new ZooKeeperWatcher(conf2, "cluster2", null, true); utility3 = new HBaseTestingUtility(conf3); utility3.setZkCluster(miniZK); @@ -145,7 +152,8 @@ public class TestMultiSlaveReplication { putAndWait(row2, famName, htable1, htable2); // now roll the region server's logs - new HBaseAdmin(conf1).rollHLogWriter(master.getRegionServer(0).getServerName().toString()); + rollWALAndWait(utility1, htable1.getName(), row2); + // after the log was rolled put a new row putAndWait(row3, famName, htable1, htable2); @@ -168,8 +176,7 @@ public class TestMultiSlaveReplication { p.add(famName, row, row); htable1.put(p); // now roll the logs again - new HBaseAdmin(conf1).rollHLogWriter(master.getRegionServer(0) - .getServerName().toString()); + rollWALAndWait(utility1, htable1.getName(), row); // cleanup "row2", also conveniently use this to wait replication // to finish @@ -189,6 +196,48 @@ public class TestMultiSlaveReplication { utility2.shutdownMiniCluster(); utility1.shutdownMiniCluster(); } + + private void rollWALAndWait(final HBaseTestingUtility utility, final TableName table, + final byte[] row) throws IOException { + final Admin admin = utility.getHBaseAdmin(); + final MiniHBaseCluster cluster = utility.getMiniHBaseCluster(); + + // find the region that corresponds to the given row. + HRegion region = null; + for (HRegion candidate : cluster.getRegions(table)) { + if (HRegion.rowIsInRange(candidate.getRegionInfo(), row)) { + region = candidate; + break; + } + } + assertNotNull("Couldn't find the region for row '" + Arrays.toString(row) + "'", region); + + final CountDownLatch latch = new CountDownLatch(1); + + // listen for successful log rolls + final WALActionsListener listener = new WALActionsListener.Base() { + @Override + public void postLogRoll(final Path oldPath, final Path newPath) throws IOException { + latch.countDown(); + } + }; + region.getWAL().registerWALActionsListener(listener); + + // request a roll + admin.rollWALWriter(cluster.getServerHoldingRegion(region.getTableDesc().getTableName(), + region.getRegionName())); + + // wait + try { + latch.await(); + } catch (InterruptedException exception) { + LOG.warn("Interrupted while waiting for the wal of '" + region + "' to roll. If later " + + "replication tests fail, it's probably because we should still be waiting."); + Thread.currentThread().interrupt(); + } + region.getWAL().unregisterWALActionsListener(listener); + } + private void checkWithWait(byte[] row, int count, Table table) throws Exception { Get get = new Get(row); @@ -201,7 +250,8 @@ public class TestMultiSlaveReplication { if (res.size() >= 1) { LOG.info("Row is replicated"); rowReplicated = true; - assertEquals(count, res.size()); + assertEquals("Table '" + table + "' did not have the expected number of results.", + count, res.size()); break; } if (rowReplicated) { @@ -216,7 +266,8 @@ public class TestMultiSlaveReplication { Get get = new Get(row); for (Table table : tables) { Result res = table.get(get); - assertEquals(count, res.size()); + assertEquals("Table '" + table + "' did not have the expected number of results.", + count, res.size()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java index fcb3cdabbfb..6dc354858a8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java @@ -59,7 +59,7 @@ public class TestReplicationChangingPeerRegionservers extends TestReplicationBas // rolling like this makes sure the most recent one gets added to the queue for (JVMClusterUtil.RegionServerThread r : utility1.getHBaseCluster().getRegionServerThreads()) { - r.getRegionServer().getWAL().rollWriter(); + utility1.getHBaseAdmin().rollWALWriter(r.getRegionServer().getServerName()); } utility1.deleteTableData(tableName); // truncating the table will send one Delete per row to the slave cluster diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java index 6ef0dd04160..456a0867b94 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java @@ -31,8 +31,7 @@ import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry; +import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; @@ -75,7 +74,7 @@ public class TestReplicationEndpoint extends TestReplicationBase { ReplicationEndpointForTest.replicateCount.set(0); ReplicationEndpointForTest.lastEntries = null; for (RegionServerThread rs : utility1.getMiniHBaseCluster().getRegionServerThreads()) { - utility1.getHBaseAdmin().rollHLogWriter(rs.getRegionServer().getServerName().toString()); + utility1.getHBaseAdmin().rollWALWriter(rs.getRegionServer().getServerName()); } } @@ -186,7 +185,7 @@ public class TestReplicationEndpoint extends TestReplicationBase { static AtomicInteger startedCount = new AtomicInteger(); static AtomicInteger stoppedCount = new AtomicInteger(); static AtomicInteger replicateCount = new AtomicInteger(); - static volatile List lastEntries = null; + static volatile List lastEntries = null; public ReplicationEndpointForTest() { contructedCount.incrementAndGet(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java index 0975a66de4a..8deffd98394 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java @@ -25,7 +25,7 @@ import org.junit.BeforeClass; import org.junit.experimental.categories.Category; /** - * Run the same test as TestReplicationKillMasterRS but with HLog compression enabled + * Run the same test as TestReplicationKillMasterRS but with WAL compression enabled * Do not add other tests in this class. */ @Category({ReplicationTests.class, LargeTests.class}) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java index b61b19ad630..3ecec9122df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java @@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; import org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication; import org.apache.hadoop.hbase.protobuf.generated.WALProtos; -import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.replication.regionserver.Replication; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -74,7 +74,7 @@ public class TestReplicationSmallTests extends TestReplicationBase { // rolling like this makes sure the most recent one gets added to the queue for ( JVMClusterUtil.RegionServerThread r : utility1.getHBaseCluster().getRegionServerThreads()) { - r.getRegionServer().getWAL().rollWriter(); + utility1.getHBaseAdmin().rollWALWriter(r.getRegionServer().getServerName()); } utility1.deleteTableData(tableName); // truncating the table will send one Delete per row to the slave cluster @@ -380,7 +380,7 @@ public class TestReplicationSmallTests extends TestReplicationBase { /** * Do a more intense version testSmallBatch, one that will trigger - * hlog rolling and other non-trivial code paths + * wal rolling and other non-trivial code paths * @throws Exception */ @Test(timeout=300000) @@ -499,7 +499,7 @@ public class TestReplicationSmallTests extends TestReplicationBase { HRegionInfo hri = new HRegionInfo(htable1.getName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); WALEdit edit = WALEdit.createCompaction(hri, compactionDescriptor); - Replication.scopeWALEdits(htable1.getTableDescriptor(), new HLogKey(), edit); + Replication.scopeWALEdits(htable1.getTableDescriptor(), new WALKey(), edit); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java index 9bdee7c5bde..7700e317fa0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java @@ -31,13 +31,15 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; -import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALProvider; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -61,6 +63,7 @@ public class TestReplicationSource { public static void setUpBeforeClass() throws Exception { TEST_UTIL.startMiniDFSCluster(1); FS = TEST_UTIL.getDFSCluster().getFileSystem(); + FSUtils.setRootDir(TEST_UTIL.getConfiguration(), FS.getHomeDirectory()); oldLogDir = new Path(FS.getHomeDirectory(), HConstants.HREGION_OLDLOGDIR_NAME); if (FS.exists(oldLogDir)) FS.delete(oldLogDir, true); @@ -80,23 +83,22 @@ 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, - logPath, conf); + WALProvider.Writer writer = WALFactory.createWALWriter(FS, logPath, + TEST_UTIL.getConfiguration()); for(int i = 0; i < 3; i++) { byte[] b = Bytes.toBytes(Integer.toString(i)); KeyValue kv = new KeyValue(b,b,b); WALEdit edit = new WALEdit(); edit.add(kv); - HLogKey key = new HLogKey(b, TableName.valueOf(b), 0, 0, + WALKey key = new WALKey(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, - logPath, conf); - HLog.Entry entry = reader.next(); + WAL.Reader reader = WALFactory.createReader(FS, logPath, TEST_UTIL.getConfiguration()); + WAL.Entry entry = reader.next(); assertNotNull(entry); Path oldLogPath = new Path(oldLogDir, "log"); @@ -109,7 +111,7 @@ public class TestReplicationSource { entry = reader.next(); assertNull(entry); - + reader.close(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java index 407f4e2873c..3710fd6b853 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java @@ -31,10 +31,9 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; -import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; import org.junit.Test; @@ -58,21 +57,21 @@ public class TestReplicationWALEntryFilters { SystemTableWALEntryFilter filter = new SystemTableWALEntryFilter(); // meta - HLogKey key1 = new HLogKey( HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(), + WALKey key1 = new WALKey( HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(), TableName.META_TABLE_NAME); - HLog.Entry metaEntry = new Entry(key1, null); + Entry metaEntry = new Entry(key1, null); assertNull(filter.filter(metaEntry)); // ns table - HLogKey key2 = new HLogKey(new byte[] {}, TableName.NAMESPACE_TABLE_NAME); - HLog.Entry nsEntry = new Entry(key2, null); + WALKey key2 = new WALKey(new byte[] {}, TableName.NAMESPACE_TABLE_NAME); + Entry nsEntry = new Entry(key2, null); assertNull(filter.filter(nsEntry)); // user table - HLogKey key3 = new HLogKey(new byte[] {}, TableName.valueOf("foo")); - HLog.Entry userEntry = new Entry(key3, null); + WALKey key3 = new WALKey(new byte[] {}, TableName.valueOf("foo")); + Entry userEntry = new Entry(key3, null); assertEquals(userEntry, filter.filter(userEntry)); } @@ -81,10 +80,10 @@ public class TestReplicationWALEntryFilters { public void testScopeWALEntryFilter() { ScopeWALEntryFilter filter = new ScopeWALEntryFilter(); - HLog.Entry userEntry = createEntry(a, b); - HLog.Entry userEntryA = createEntry(a); - HLog.Entry userEntryB = createEntry(b); - HLog.Entry userEntryEmpty = createEntry(); + Entry userEntry = createEntry(a, b); + Entry userEntryA = createEntry(a); + Entry userEntryB = createEntry(b); + Entry userEntryEmpty = createEntry(); // no scopes assertEquals(null, filter.filter(userEntry)); @@ -156,7 +155,7 @@ public class TestReplicationWALEntryFilters { @Test public void testChainWALEntryFilter() { - HLog.Entry userEntry = createEntry(a, b, c); + Entry userEntry = createEntry(a, b, c); ChainWALEntryFilter filter = new ChainWALEntryFilter(passFilter); assertEquals(createEntry(a,b,c), filter.filter(userEntry)); @@ -207,7 +206,7 @@ public class TestReplicationWALEntryFilters { ReplicationPeer peer = mock(ReplicationPeer.class); when(peer.getTableCFs()).thenReturn(null); - HLog.Entry userEntry = createEntry(a, b, c); + Entry userEntry = createEntry(a, b, c); TableCfWALEntryFilter filter = new TableCfWALEntryFilter(peer); assertEquals(createEntry(a,b,c), filter.filter(userEntry)); @@ -243,24 +242,24 @@ public class TestReplicationWALEntryFilters { assertEquals(createEntry(a,c), filter.filter(userEntry)); } - private HLog.Entry createEntry(byte[]... kvs) { - HLogKey key1 = new HLogKey(new byte[] {}, TableName.valueOf("foo")); + private Entry createEntry(byte[]... kvs) { + WALKey key1 = new WALKey(new byte[] {}, TableName.valueOf("foo")); WALEdit edit1 = new WALEdit(); for (byte[] kv : kvs) { edit1.add(new KeyValue(kv, kv, kv)); } - return new HLog.Entry(key1, edit1); + return new Entry(key1, edit1); } - private void assertEquals(HLog.Entry e1, HLog.Entry e2) { + private void assertEquals(Entry e1, Entry e2) { Assert.assertEquals(e1 == null, e2 == null); if (e1 == null) { return; } - // do not compare HLogKeys + // do not compare WALKeys // compare kvs Assert.assertEquals(e1.getEdit() == null, e2.getEdit() == null); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java index 928a8135b5b..f5380efb43f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java @@ -46,8 +46,8 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; 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.HLogKey; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; @@ -322,8 +322,8 @@ public class TestRegionReplicaReplicationEndpoint { HRegionLocation hrl = connection.locateRegion(toBeDisabledTable, HConstants.EMPTY_BYTE_ARRAY); byte[] encodedRegionName = hrl.getRegionInfo().getEncodedNameAsBytes(); - HLog.Entry entry = new HLog.Entry( - new HLogKey(encodedRegionName, toBeDisabledTable, 1), + Entry entry = new Entry( + new WALKey(encodedRegionName, toBeDisabledTable, 1), new WALEdit()); HTU.getHBaseAdmin().disableTable(toBeDisabledTable); // disable the table diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java index bf7852c505f..a191bdd79f2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java @@ -49,8 +49,8 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryR import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationEndpoint.ReplicateContext; @@ -141,18 +141,18 @@ public class TestRegionReplicaReplicationEndpointNoMaster { public void after() throws Exception { } - static ConcurrentLinkedQueue entries = new ConcurrentLinkedQueue(); + static ConcurrentLinkedQueue entries = new ConcurrentLinkedQueue(); public static class WALEditCopro extends BaseWALObserver { public WALEditCopro() { entries.clear(); } @Override - public void postWALWrite(ObserverContext ctx, HRegionInfo info, - HLogKey logKey, WALEdit logEdit) throws IOException { + public void postWALWrite(ObserverContext ctx, + HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException { // only keep primary region's edits if (logKey.getTablename().equals(tableName) && info.getReplicaId() == 0) { - entries.add(new HLog.Entry(logKey, logEdit)); + entries.add(new Entry(logKey, logEdit)); } } } @@ -179,9 +179,9 @@ public class TestRegionReplicaReplicationEndpointNoMaster { connection.close(); } - private void replicateUsingCallable(ClusterConnection connection, Queue entries) + private void replicateUsingCallable(ClusterConnection connection, Queue entries) throws IOException, RuntimeException { - HLog.Entry entry; + Entry entry; while ((entry = entries.poll()) != null) { byte[] row = entry.getEdit().getCells().get(0).getRow(); RegionLocations locations = connection.locateRegion(tableName, row, true, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationHLogReaderManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationHLogReaderManager.java deleted file mode 100644 index 2b0f34b58b1..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationHLogReaderManager.java +++ /dev/null @@ -1,239 +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.replication.regionserver; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; -import org.apache.hadoop.hbase.regionserver.wal.HLogKey; -import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; -import org.apache.hadoop.hbase.regionserver.wal.WALEdit; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.testclassification.ReplicationTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -import static org.junit.Assert.*; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.atomic.AtomicLong; - -@Category({ReplicationTests.class, LargeTests.class}) -@RunWith(Parameterized.class) -public class TestReplicationHLogReaderManager { - - private static HBaseTestingUtility TEST_UTIL; - private static Configuration conf; - private static Path hbaseDir; - private static FileSystem fs; - private static MiniDFSCluster cluster; - private static final TableName tableName = TableName.valueOf("tablename"); - private static final byte [] family = Bytes.toBytes("column"); - private static final byte [] qualifier = Bytes.toBytes("qualifier"); - private static final HRegionInfo info = new HRegionInfo(tableName, - HConstants.EMPTY_START_ROW, HConstants.LAST_ROW, false); - private static final HTableDescriptor htd = new HTableDescriptor(tableName); - - private HLog log; - private ReplicationHLogReaderManager logManager; - private PathWatcher pathWatcher; - private int nbRows; - private int walEditKVs; - private final AtomicLong sequenceId = new AtomicLong(1); - - @Parameters - public static Collection parameters() { - // Try out different combinations of row count and KeyValue count - int[] NB_ROWS = { 1500, 60000 }; - int[] NB_KVS = { 1, 100 }; - // whether compression is used - Boolean[] BOOL_VALS = { false, true }; - List parameters = new ArrayList(); - for (int nbRows : NB_ROWS) { - for (int walEditKVs : NB_KVS) { - for (boolean b : BOOL_VALS) { - Object[] arr = new Object[3]; - arr[0] = nbRows; - arr[1] = walEditKVs; - arr[2] = b; - parameters.add(arr); - } - } - } - return parameters; - } - - public TestReplicationHLogReaderManager(int nbRows, int walEditKVs, boolean enableCompression) { - this.nbRows = nbRows; - this.walEditKVs = walEditKVs; - TEST_UTIL.getConfiguration().setBoolean(HConstants.ENABLE_WAL_COMPRESSION, - enableCompression); - } - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - TEST_UTIL = new HBaseTestingUtility(); - conf = TEST_UTIL.getConfiguration(); - TEST_UTIL.startMiniDFSCluster(3); - - hbaseDir = TEST_UTIL.createRootDir(); - cluster = TEST_UTIL.getDFSCluster(); - fs = cluster.getFileSystem(); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - TEST_UTIL.shutdownMiniCluster(); - } - - @Before - public void setUp() throws Exception { - logManager = new ReplicationHLogReaderManager(fs, conf); - List listeners = new ArrayList(); - pathWatcher = new PathWatcher(); - listeners.add(pathWatcher); - log = HLogFactory.createHLog(fs, hbaseDir, "test", conf, listeners, "some server"); - } - - @After - public void tearDown() throws Exception { - log.closeAndDelete(); - } - - @Test - public void test() throws Exception { - // Grab the path that was generated when the log rolled as part of its creation - Path path = pathWatcher.currentPath; - - assertEquals(0, logManager.getPosition()); - - appendToLog(); - - // 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(); - assertNotNull(entry); - entry = logManager.readNextAndSetPosition(); - assertNull(entry); - logManager.closeReader(); - long oldPos = logManager.getPosition(); - - appendToLog(); - - // Read the newly added entry, make sure we made progress - assertNotNull(logManager.openReader(path)); - logManager.seek(); - entry = logManager.readNextAndSetPosition(); - assertNotEquals(oldPos, logManager.getPosition()); - assertNotNull(entry); - logManager.closeReader(); - oldPos = logManager.getPosition(); - - log.rollWriter(); - - // We rolled but we still should see the end of the first log and not get data - assertNotNull(logManager.openReader(path)); - logManager.seek(); - entry = logManager.readNextAndSetPosition(); - assertEquals(oldPos, logManager.getPosition()); - assertNull(entry); - logManager.finishCurrentFile(); - - path = pathWatcher.currentPath; - - for (int i = 0; i < nbRows; i++) { appendToLogPlus(walEditKVs); } - log.rollWriter(); - logManager.openReader(path); - logManager.seek(); - for (int i = 0; i < nbRows; i++) { - HLog.Entry e = logManager.readNextAndSetPosition(); - if (e == null) { - fail("Should have enough entries"); - } - } - } - - private void appendToLog() throws IOException { - appendToLogPlus(1); - } - - private void appendToLogPlus(int count) throws IOException { - log.append(info, tableName, getWALEdits(count), System.currentTimeMillis(), htd, sequenceId); - } - - private WALEdit getWALEdits(int count) { - WALEdit edit = new WALEdit(); - for (int i = 0; i < count; i++) { - edit.add(new KeyValue(Bytes.toBytes(System.currentTimeMillis()), family, qualifier, - System.currentTimeMillis(), qualifier)); - } - return edit; - } - - class PathWatcher implements WALActionsListener { - - Path currentPath; - - @Override - public void preLogRoll(Path oldPath, Path newPath) throws IOException { - currentPath = newPath; - } - - @Override - public void postLogRoll(Path oldPath, Path newPath) throws IOException {} - - @Override - public void preLogArchive(Path oldPath, Path newPath) throws IOException {} - - @Override - public void postLogArchive(Path oldPath, Path newPath) throws IOException {} - - @Override - public void logRollRequested() {} - - @Override - public void logCloseRequested() {} - - @Override - public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit) {} - - @Override - public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, WALEdit logEdit) {} - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java index 43b37d01f08..ff6a079654a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java @@ -50,11 +50,11 @@ import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.client.HConnection; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; -import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.replication.ReplicationFactory; import org.apache.hadoop.hbase.replication.ReplicationPeers; import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; @@ -65,6 +65,7 @@ import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKClusterId; import org.apache.hadoop.hbase.zookeeper.ZKUtil; @@ -111,8 +112,6 @@ public class TestReplicationSourceManager { private static FileSystem fs; - private static String logName; - private static Path oldLogDir; private static Path logDir; @@ -147,6 +146,7 @@ public class TestReplicationSourceManager { ZKUtil.setData(zkw, "/hbase/replication/state", ReplicationStateZKBase.ENABLED_ZNODE_BYTES); ZKClusterId.setClusterId(zkw, new ClusterId()); + FSUtils.setRootDir(utility.getConfiguration(), utility.getDataTestDir()); fs = FileSystem.get(conf); oldLogDir = new Path(utility.getDataTestDir(), HConstants.HREGION_OLDLOGDIR_NAME); @@ -155,8 +155,6 @@ public class TestReplicationSourceManager { replication = new Replication(new DummyServer(), fs, logDir, oldLogDir); manager = replication.getReplicationManager(); - logName = HConstants.HREGION_LOGDIR_NAME; - manager.addSource(slaveId); htd = new HTableDescriptor(test); @@ -198,8 +196,9 @@ public class TestReplicationSourceManager { List listeners = new ArrayList(); listeners.add(replication); - HLog hlog = HLogFactory.createHLog(fs, utility.getDataTestDir(), logName, - conf, listeners, URLEncoder.encode("regionserver:60020", "UTF8")); + final WALFactory wals = new WALFactory(utility.getConfiguration(), listeners, + URLEncoder.encode("regionserver:60020", "UTF8")); + final WAL wal = wals.getWAL(hri.getEncodedNameAsBytes()); final AtomicLong sequenceId = new AtomicLong(1); manager.init(); HTableDescriptor htd = new HTableDescriptor(); @@ -207,12 +206,12 @@ public class TestReplicationSourceManager { // Testing normal log rolling every 20 for(long i = 1; i < 101; i++) { if(i > 1 && i % 20 == 0) { - hlog.rollWriter(); + wal.rollWriter(); } LOG.info(i); - HLogKey key = new HLogKey(hri.getRegionName(), test, seq++, - System.currentTimeMillis(), HConstants.DEFAULT_CLUSTER_ID); - hlog.append(hri, test, edit, System.currentTimeMillis(), htd, sequenceId); + final long txid = wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), test, + System.currentTimeMillis()), edit, sequenceId, true ,null); + wal.sync(txid); } // Simulate a rapid insert that's followed @@ -223,22 +222,26 @@ public class TestReplicationSourceManager { LOG.info(baseline + " and " + time); for (int i = 0; i < 3; i++) { - hlog.append(hri, test, edit, System.currentTimeMillis(), htd, sequenceId); + wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), test, + System.currentTimeMillis()), edit, sequenceId, true, null); } + wal.sync(); - assertEquals(6, manager.getHLogs().get(slaveId).size()); + assertEquals(6, manager.getWALs().get(slaveId).size()); - hlog.rollWriter(); + wal.rollWriter(); manager.logPositionAndCleanOldLogs(manager.getSources().get(0).getCurrentPath(), "1", 0, false, false); - hlog.append(hri, test, edit, System.currentTimeMillis(), htd, sequenceId); + wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), test, + System.currentTimeMillis()), edit, sequenceId, true, null); + wal.sync(); - assertEquals(1, manager.getHLogs().size()); + assertEquals(1, manager.getWALs().size()); - // TODO Need a case with only 2 HLogs and we only want to delete the first one + // TODO Need a case with only 2 WALs and we only want to delete the first one } @Test @@ -310,12 +313,12 @@ public class TestReplicationSourceManager { new Long(1), new Long(2))); w1.start(); w1.join(5000); - assertEquals(1, manager.getHlogsByIdRecoveredQueues().size()); + assertEquals(1, manager.getWalsByIdRecoveredQueues().size()); String id = "1-" + server.getServerName().getServerName(); - assertEquals(files, manager.getHlogsByIdRecoveredQueues().get(id)); + assertEquals(files, manager.getWalsByIdRecoveredQueues().get(id)); manager.cleanOldLogs("log2", id, true); // log1 should be deleted - assertEquals(Sets.newHashSet("log2"), manager.getHlogsByIdRecoveredQueues().get(id)); + assertEquals(Sets.newHashSet("log2"), manager.getWalsByIdRecoveredQueues().get(id)); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java index 1f7cab763ca..d1516ca45b5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java @@ -32,7 +32,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.junit.*; @@ -174,7 +174,7 @@ public class TestFSVisitor { private void createRecoverEdits(final Path tableDir, final Set tableRegions, final Set recoverEdits) throws IOException { for (String region: tableRegions) { - Path regionEditsDir = HLogUtil.getRegionDirRecoveredEditsDir(new Path(tableDir, region)); + Path regionEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(new Path(tableDir, region)); long seqId = System.currentTimeMillis(); for (int i = 0; i < 3; ++i) { String editName = String.format("%019d", seqId + i); @@ -187,12 +187,20 @@ public class TestFSVisitor { } /* + * Old style * |-.logs/ * |----server5,5,1351969633508/ * |-------server5,5,1351969633508.0 * |----server6,6,1351969633512/ * |-------server6,6,1351969633512.0 * |-------server6,6,1351969633512.3 + * New style + * |-.logs/ + * |----server3,5,1351969633508/ + * |-------server3,5,1351969633508.default.0 + * |----server4,6,1351969633512/ + * |-------server4,6,1351969633512.default.0 + * |-------server4,6,1351969633512.some_provider.3 */ private void createLogs(final Path logDir, final Set servers, final Set logs) throws IOException { @@ -200,6 +208,13 @@ public class TestFSVisitor { String server = String.format("server%d,%d,%d", s, s, System.currentTimeMillis()); servers.add(server); Path serverLogDir = new Path(logDir, server); + if (s % 2 == 0) { + if (s % 3 == 0) { + server += ".default"; + } else { + server += "." + s; + } + } fs.mkdirs(serverLogDir); for (int i = 0; i < 5; ++i) { String logfile = server + '.' + i; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java index 2f7051ebe1b..e756f483cfc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java @@ -1230,7 +1230,7 @@ public class TestHBaseFsck { setupTable(table); assertEquals(ROWKEYS.length, countRows()); - // make sure data in regions, if in hlog only there is no data loss + // make sure data in regions, if in wal only there is no data loss TEST_UTIL.getHBaseAdmin().flush(table); // Mess it up by leaving a hole in the hdfs data @@ -1336,7 +1336,7 @@ public class TestHBaseFsck { setupTable(table); assertEquals(ROWKEYS.length, countRows()); - // make sure data in regions, if in hlog only there is no data loss + // make sure data in regions, if in wal only there is no data loss TEST_UTIL.getHBaseAdmin().flush(table); // Mess it up by deleting hdfs dirs @@ -1470,14 +1470,14 @@ public class TestHBaseFsck { TableName.valueOf("testFixByTable2"); try { setupTable(table1); - // make sure data in regions, if in hlog only there is no data loss + // make sure data in regions, if in wal only there is no data loss TEST_UTIL.getHBaseAdmin().flush(table1); // Mess them up by leaving a hole in the hdfs data deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), false, false, true); // don't rm meta setupTable(table2); - // make sure data in regions, if in hlog only there is no data loss + // make sure data in regions, if in wal only there is no data loss TEST_UTIL.getHBaseAdmin().flush(table2); // Mess them up by leaving a hole in the hdfs data deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), @@ -1517,7 +1517,7 @@ public class TestHBaseFsck { setupTable(table); assertEquals(ROWKEYS.length, countRows()); - // make sure data in regions, if in hlog only there is no data loss + // make sure data in regions, if in wal only there is no data loss TEST_UTIL.getHBaseAdmin().flush(table); HRegionLocation location = tbl.getRegionLocation("B"); @@ -1597,7 +1597,7 @@ public class TestHBaseFsck { setupTable(table); assertEquals(ROWKEYS.length, countRows()); - // make sure data in regions, if in hlog only there is no data loss + // make sure data in regions, if in wal only there is no data loss TEST_UTIL.getHBaseAdmin().flush(table); HRegionLocation location = tbl.getRegionLocation("B"); @@ -1647,7 +1647,7 @@ public class TestHBaseFsck { setupTable(table); assertEquals(ROWKEYS.length, countRows()); - // make sure data in regions, if in hlog only there is no data loss + // make sure data in regions, if in wal only there is no data loss TEST_UTIL.getHBaseAdmin().flush(table); HRegionLocation location = tbl.getRegionLocation("B"); @@ -2502,7 +2502,7 @@ public class TestHBaseFsck { setupTable(table); assertEquals(ROWKEYS.length, countRows()); - // make sure data in regions, if in hlog only there is no data loss + // make sure data in regions, if in wal only there is no data loss TEST_UTIL.getHBaseAdmin().flush(table); HRegionInfo region1 = tbl.getRegionLocation("A").getRegionInfo(); HRegionInfo region2 = tbl.getRegionLocation("B").getRegionInfo(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java index 2da631f5bbb..e5cd2123622 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java @@ -40,8 +40,8 @@ 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.HLogFactory; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -65,6 +65,7 @@ public class TestMergeTool extends HBaseTestCase { private HTableDescriptor desc; private byte [][][] rows; private MiniDFSCluster dfsCluster = null; + private WALFactory wals; @Before public void setUp() throws Exception { @@ -147,6 +148,7 @@ public class TestMergeTool extends HBaseTestCase { // we will end up with a local file system super.setUp(); + wals = new WALFactory(conf, null, "TestMergeTool"); try { // Create meta region createMetaRegion(); @@ -187,6 +189,7 @@ public class TestMergeTool extends HBaseTestCase { HRegion.closeHRegion(r); } } + wals.close(); TEST_UTIL.shutdownMiniCluster(); } @@ -200,7 +203,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 WAL log, final int upperbound) throws Exception { Merge merger = new Merge(this.conf); LOG.info(msg); @@ -274,39 +277,26 @@ public class TestMergeTool extends HBaseTestCase { // Close the region and delete the log HRegion.closeHRegion(regions[i]); } + WAL log = wals.getWAL(new byte[]{}); + // Merge Region 0 and Region 1 + HRegion merged = mergeAndVerify("merging regions 0 and 1 ", + this.sourceRegions[0].getRegionNameAsString(), + this.sourceRegions[1].getRegionNameAsString(), log, 2); - // Create a log that we can reuse when we need to open regions - Path logPath = new Path("/tmp"); - String logName = HConstants.HREGION_LOGDIR_NAME + "_" - + System.currentTimeMillis(); - LOG.info("Creating log " + logPath.toString() + "/" + logName); + // Merge the result of merging regions 0 and 1 with region 2 + merged = mergeAndVerify("merging regions 0+1 and 2", + merged.getRegionInfo().getRegionNameAsString(), + this.sourceRegions[2].getRegionNameAsString(), log, 3); - HLog log = HLogFactory.createHLog(this.fs, logPath, - logName, this.conf); + // Merge the result of merging regions 0, 1 and 2 with region 3 + merged = mergeAndVerify("merging regions 0+1+2 and 3", + merged.getRegionInfo().getRegionNameAsString(), + this.sourceRegions[3].getRegionNameAsString(), log, 4); - try { - // Merge Region 0 and Region 1 - HRegion merged = mergeAndVerify("merging regions 0 and 1 ", - this.sourceRegions[0].getRegionNameAsString(), - this.sourceRegions[1].getRegionNameAsString(), log, 2); - - // Merge the result of merging regions 0 and 1 with region 2 - merged = mergeAndVerify("merging regions 0+1 and 2", - merged.getRegionInfo().getRegionNameAsString(), - this.sourceRegions[2].getRegionNameAsString(), log, 3); - - // Merge the result of merging regions 0, 1 and 2 with region 3 - merged = mergeAndVerify("merging regions 0+1+2 and 3", - merged.getRegionInfo().getRegionNameAsString(), - this.sourceRegions[3].getRegionNameAsString(), log, 4); - - // Merge the result of merging regions 0, 1, 2 and 3 with region 4 - merged = mergeAndVerify("merging regions 0+1+2+3 and 4", - merged.getRegionInfo().getRegionNameAsString(), - this.sourceRegions[4].getRegionNameAsString(), log, rows.length); - } finally { - log.closeAndDelete(); - } + // Merge the result of merging regions 0, 1, 2 and 3 with region 4 + merged = mergeAndVerify("merging regions 0+1+2+3 and 4", + merged.getRegionInfo().getRegionNameAsString(), + this.sourceRegions[4].getRegionNameAsString(), log, rows.length); } } diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb index 971df06b10a..1fe496df33d 100644 --- a/hbase-shell/src/main/ruby/hbase/admin.rb +++ b/hbase-shell/src/main/ruby/hbase/admin.rb @@ -82,10 +82,12 @@ module Hbase end #---------------------------------------------------------------------------------------------- - # Requests a regionserver's HLog roll - def hlog_roll(server_name) - @admin.rollHLogWriter(server_name) + # Requests a regionserver's WAL roll + def wal_roll(server_name) + @admin.rollWALWriter(server_name) end + # TODO remove older hlog_roll version + alias :hlog_roll :wal_roll #---------------------------------------------------------------------------------------------- # Requests a table or region split diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb index a06371a70ae..620033ce93c 100644 --- a/hbase-shell/src/main/ruby/shell.rb +++ b/hbase-shell/src/main/ruby/shell.rb @@ -321,13 +321,17 @@ Shell.load_command_group( merge_region unassign zk_dump - hlog_roll + wal_roll catalogjanitor_run catalogjanitor_switch catalogjanitor_enabled compact_rs trace - ] + ], + # TODO remove older hlog_roll command + :aliases => { + 'wal_roll' => ['hlog_roll'] + } ) Shell.load_command_group( diff --git a/hbase-shell/src/main/ruby/shell/commands/hlog_roll.rb b/hbase-shell/src/main/ruby/shell/commands/hlog_roll.rb deleted file mode 100644 index b292791541f..00000000000 --- a/hbase-shell/src/main/ruby/shell/commands/hlog_roll.rb +++ /dev/null @@ -1,39 +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. -# -module Shell - module Commands - class HlogRoll < Command - def help - return <<-EOF -Roll the log writer. That is, start writing log messages to a new file. -The name of the regionserver should be given as the parameter. A -'server_name' is the host, port plus startcode of a regionserver. For -example: host187.example.com,60020,1289493121758 (find servername in -master ui or when you do detailed status in shell) -EOF - end - - def command(server_name) - format_simple_command do - admin.hlog_roll(server_name) - end - end - end - end -end diff --git a/src/main/docbkx/book.xml b/src/main/docbkx/book.xml index 06578caceb5..706936c8ca1 100644 --- a/src/main/docbkx/book.xml +++ b/src/main/docbkx/book.xml @@ -2136,7 +2136,7 @@ rs.close();
    LogRoller - Periodically checks the RegionServer's HLog. + Periodically checks the RegionServer's WAL.
    @@ -2531,13 +2531,22 @@ rs.close(); becomes unavailable before the MemStore is flushed, the WAL ensures that the changes to the data can be replayed. If writing to the WAL fails, the entire operation to modify the data fails. - HBase uses an implementation of the HLog - interface for the WAL. - Usually, there is only one instance of a WAL per RegionServer. The RegionServer records Puts and Deletes to - it, before recording them to the + HBase uses an implementation of the WAL interface. Usually, there is only one instance of a WAL per RegionServer. + The RegionServer records Puts and Deletes to it, before recording them to the for the affected . + linkend="store" />. + + + The HLog + + Prior to 2.0, the interface for WALs in HBase was named HLog. + In 0.94, HLog was the name of the implementation of the WAL. You will likely find + references to the HLog in documentation tailored to these older versions. + + The WAL resides in HDFS in the /hbase/WALs/ directory (prior to HBase 0.94, they were stored in /hbase/.logs/), with subdirectories per region. @@ -3439,10 +3448,10 @@ myHtd.setValue(HTableDescriptor.SPLIT_POLICY, MyCustomSplitPolicy.class.getName( hbase.regionserver.global.memstore.lowerLimit. - When the number of HLog per region server reaches the value specified in + When the number of WAL per region server reaches the value specified in hbase.regionserver.max.logs, MemStores from various regions - will be flushed out to disk to reduce HLog count. The flush order is based on time. - Regions with the oldest MemStores are flushed first until HLog count drops below + will be flushed out to disk to reduce WAL count. The flush order is based on time. + Regions with the oldest MemStores are flushed first until WAL count drops below hbase.regionserver.max.logs. diff --git a/src/main/docbkx/ops_mgt.xml b/src/main/docbkx/ops_mgt.xml index b52457abe4c..0af8f027fe7 100644 --- a/src/main/docbkx/ops_mgt.xml +++ b/src/main/docbkx/ops_mgt.xml @@ -318,9 +318,20 @@ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.mapreduce.UtilityNa
    - <classname>HLogPrettyPrinter</classname> - HLogPrettyPrinter is a tool with configurable options to - print the contents of an HLog. + WAL Pretty Printer + The WAL Pretty Printer is a tool with configurable options to + print the contents of a WAL. You can invoke it via the hbase cli with the 'wal' command. + + $ ./bin/hbase wal hdfs://example.org:8020/hbase/.logs/example.org,60020,1283516293161/10.10.21.10%3A60020.1283973724012 + + WAL Printing in older versions of HBase + Prior to version 2.0, the WAL Pretty Printer was called the + HLogPrettyPrinter, after an internal name for HBase's write + ahead log. In those versions, you can pring the contents of a WAL using the same + configuration as above, but with the 'hlog' command. + + $ ./bin/hbase hlog hdfs://example.org:8020/hbase/.logs/example.org,60020,1283516293161/10.10.21.10%3A60020.1283973724012 +
    @@ -1968,7 +1979,7 @@ $ for i in `cat conf/regionservers|sort`; do ./bin/graceful_stop.sh --restart -- source.logEditsRead - number of mutations read from HLogs at the replication source + number of mutations read from WALs at the replication source diff --git a/src/main/docbkx/performance.xml b/src/main/docbkx/performance.xml index 78a828feb2c..1757d3f43c5 100644 --- a/src/main/docbkx/performance.xml +++ b/src/main/docbkx/performance.xml @@ -690,9 +690,9 @@ admin.createTable(table, splits); xml:id="def.log.flush"> Table Creation: Deferred Log Flush The default behavior for Puts using the Write Ahead Log (WAL) is that - HLog edits will be written immediately. If deferred log flush is + WAL edits will be written immediately. If deferred log flush is used, WAL edits are kept in memory until the flush period. The benefit is aggregated and - asynchronous HLog- writes, but the potential downside is that if the + asynchronous WAL- writes, but the potential downside is that if the RegionServer goes down the yet-to-be-flushed edits are lost. This is safer, however, than not using WAL at all with Puts. Deferred log flush can be configured on tables via /hbase /.logs /<RegionServer> (RegionServers) - /<HLog> (WAL HLog files for the RegionServer) + /<WAL> (WAL files for the RegionServer) See the for other non-shell diagnostic utilities like fsck.
    - Zero size HLogs with data in them + Zero size WALs with data in them Problem: when getting a listing of all the files in a region server's .logs directory, one file has a size of 0 but it contains data. - Answer: It's an HDFS quirk. A file that's currently being to will appear to have a - size of 0 but once it's closed it will show its true size + Answer: It's an HDFS quirk. A file that's currently being written to will appear to + have a size of 0 but once it's closed it will show its true size
    @@ -1583,7 +1583,7 @@ WARN HDFS.DFSClient: DataStreamer Exception: java.io.IOException: Unable to crea If the operation was an attempt to split the log, the following type of message may also appear: -FATAL wal.HLogSplitter: WriterThread-xxx Got while writing log entry to log +FATAL wal.WALSplitter: WriterThread-xxx Got while writing log entry to log