You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by mp...@apache.org on 2016/08/31 16:21:16 UTC

[2/2] kudu git commit: tool: port log-dump

tool: port log-dump

This one was more complicated, because log-dump can run against a single
file or an entire tablet. So I put all the common functionality in one place
and referenced it from both modes.

I consolidated similar gflags where it made sense to do so, and I tweaked
the endline handling for help generation so that each parameter is separated
from the next with an empty line.

Semantic changes from log-dump:
- If called with print_entries=no, will also print the footer.
- The print_headers flag is now print_meta, to be more consistent with 'kudu
  cfile dump'.

Change-Id: I14f048169c0b211e3c72fcd255c76dd59cbb05c9
Reviewed-on: http://gerrit.cloudera.org:8080/4167
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/1f660ee9
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/1f660ee9
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/1f660ee9

Branch: refs/heads/master
Commit: 1f660ee9fd23520cdded155c1d7a18f5331cec5b
Parents: 362a307
Author: Adar Dembo <ad...@cloudera.com>
Authored: Mon Aug 29 17:50:10 2016 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Aug 31 01:23:05 2016 +0000

----------------------------------------------------------------------
 docs/release_notes.adoc              |   6 +
 src/kudu/consensus/CMakeLists.txt    |   6 -
 src/kudu/consensus/log-dump.cc       | 254 ------------------------------
 src/kudu/tools/CMakeLists.txt        |   6 +-
 src/kudu/tools/kudu-tool-test.cc     | 233 ++++++++++++++++++++++-----
 src/kudu/tools/tool_action.cc        |   4 +
 src/kudu/tools/tool_action.h         |   1 +
 src/kudu/tools/tool_action_common.cc | 210 ++++++++++++++++++++++++
 src/kudu/tools/tool_action_common.h  |  42 +++++
 src/kudu/tools/tool_action_fs.cc     |   4 +-
 src/kudu/tools/tool_action_tablet.cc |  65 ++++++--
 src/kudu/tools/tool_action_wal.cc    |  69 ++++++++
 src/kudu/tools/tool_main.cc          |   7 +-
 src/kudu/util/test_macros.h          |   3 +
 14 files changed, 595 insertions(+), 315 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/1f660ee9/docs/release_notes.adoc
----------------------------------------------------------------------
diff --git a/docs/release_notes.adoc b/docs/release_notes.adoc
index 00bf69b..281eb81 100644
--- a/docs/release_notes.adoc
+++ b/docs/release_notes.adoc
@@ -46,6 +46,12 @@ detailed below.
 - The `kudu-ksck` tool has been removed. The same functionality is now
   implemented as `kudu cluster ksck`.
 
+- The `cfile-dump` tool has been removed. The same functionality is now
+  implemented as `kudu fs cfile_dump`.
+
+- The `log-dump` tool has been removed. The same functionality is now
+  implemented as `kudu wal dump` and `kudu tablet dump_wals`.
+
 - KuduSession methods in the C++ library are no longer advertised as thread-safe
   to have one set of semantics for both C++ and Java Kudu client libraries.
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f660ee9/src/kudu/consensus/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/CMakeLists.txt b/src/kudu/consensus/CMakeLists.txt
index a097869..88f0c61 100644
--- a/src/kudu/consensus/CMakeLists.txt
+++ b/src/kudu/consensus/CMakeLists.txt
@@ -145,9 +145,3 @@ ADD_KUDU_TEST(raft_consensus-test)
 # googletest). Remove this when we upgrade.
 set_source_files_properties(raft_consensus-test.cc
     PROPERTIES COMPILE_FLAGS -Wno-inconsistent-missing-override)
-
-#Tools
-add_executable(log-dump log-dump.cc)
-target_link_libraries(log-dump
-  log
-  ${KUDU_BASE_LIBS})

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f660ee9/src/kudu/consensus/log-dump.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log-dump.cc b/src/kudu/consensus/log-dump.cc
deleted file mode 100644
index 8bab8fc..0000000
--- a/src/kudu/consensus/log-dump.cc
+++ /dev/null
@@ -1,254 +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.
-
-#include <gflags/gflags.h>
-#include <glog/logging.h>
-#include <iostream>
-#include <memory>
-#include <vector>
-
-#include "kudu/common/row_operations.h"
-#include "kudu/common/wire_protocol.h"
-#include "kudu/consensus/consensus.pb.h"
-#include "kudu/consensus/log_index.h"
-#include "kudu/consensus/log_reader.h"
-#include "kudu/gutil/stl_util.h"
-#include "kudu/gutil/strings/numbers.h"
-#include "kudu/rpc/rpc_header.pb.h"
-#include "kudu/util/env.h"
-#include "kudu/util/flags.h"
-#include "kudu/util/logging.h"
-#include "kudu/util/metrics.h"
-#include "kudu/util/pb_util.h"
-
-DEFINE_bool(print_headers, true, "print the log segment headers/footers");
-DEFINE_string(print_entries, "decoded",
-              "How to print entries:\n"
-              "  false|0|no = don't print\n"
-              "  true|1|yes|decoded = print them decoded\n"
-              "  pb = print the raw protobuf\n"
-              "  id = print only their ids");
-DEFINE_int32(truncate_data, 100,
-             "Truncate the data fields to the given number of bytes "
-             "before printing. Set to 0 to disable");
-namespace kudu {
-namespace log {
-
-using consensus::CommitMsg;
-using consensus::OperationType;
-using consensus::ReplicateMsg;
-using tserver::WriteRequestPB;
-using std::shared_ptr;
-using std::string;
-using std::vector;
-using std::cout;
-using std::endl;
-
-enum PrintEntryType {
-  DONT_PRINT,
-  PRINT_PB,
-  PRINT_DECODED,
-  PRINT_ID
-};
-
-static PrintEntryType ParsePrintType() {
-  if (ParseLeadingBoolValue(FLAGS_print_entries.c_str(), true) == false) {
-    return DONT_PRINT;
-  } else if (ParseLeadingBoolValue(FLAGS_print_entries.c_str(), false) == true ||
-             FLAGS_print_entries == "decoded") {
-    return PRINT_DECODED;
-  } else if (FLAGS_print_entries == "pb") {
-    return PRINT_PB;
-  } else if (FLAGS_print_entries == "id") {
-    return PRINT_ID;
-  } else {
-    LOG(FATAL) << "Unknown value for --print_entries: " << FLAGS_print_entries;
-  }
-}
-
-void PrintIdOnly(const LogEntryPB& entry) {
-  switch (entry.type()) {
-    case log::REPLICATE:
-    {
-      cout << entry.replicate().id().term() << "." << entry.replicate().id().index()
-           << "@" << entry.replicate().timestamp() << "\t";
-      cout << "REPLICATE "
-           << OperationType_Name(entry.replicate().op_type());
-      break;
-    }
-    case log::COMMIT:
-    {
-      cout << "COMMIT " << entry.commit().commited_op_id().term()
-           << "." << entry.commit().commited_op_id().index();
-      break;
-    }
-    default:
-      cout << "UNKNOWN: " << entry.ShortDebugString();
-  }
-
-  cout << endl;
-}
-
-Status PrintDecodedWriteRequestPB(const string& indent,
-                                  const Schema& tablet_schema,
-                                  const WriteRequestPB& write,
-                                  const rpc::RequestIdPB* request_id) {
-  Schema request_schema;
-  RETURN_NOT_OK(SchemaFromPB(write.schema(), &request_schema));
-
-  Arena arena(32 * 1024, 1024 * 1024);
-  RowOperationsPBDecoder dec(&write.row_operations(), &request_schema, &tablet_schema, &arena);
-  vector<DecodedRowOperation> ops;
-  RETURN_NOT_OK(dec.DecodeOperations(&ops));
-
-  cout << indent << "Tablet: " << write.tablet_id() << endl;
-  cout << indent << "RequestId: "
-      << (request_id ? request_id->ShortDebugString() : "None") << endl;
-  cout << indent << "Consistency: "
-       << ExternalConsistencyMode_Name(write.external_consistency_mode()) << endl;
-  if (write.has_propagated_timestamp()) {
-    cout << indent << "Propagated TS: " << write.propagated_timestamp() << endl;
-  }
-
-  int i = 0;
-  for (const DecodedRowOperation& op : ops) {
-    // TODO (KUDU-515): Handle the case when a tablet's schema changes
-    // mid-segment.
-    cout << indent << "op " << (i++) << ": " << op.ToString(tablet_schema) << endl;
-  }
-
-  return Status::OK();
-}
-
-Status PrintDecoded(const LogEntryPB& entry, const Schema& tablet_schema) {
-  PrintIdOnly(entry);
-
-  const string indent = "\t";
-  if (entry.has_replicate()) {
-    // We can actually decode REPLICATE messages.
-
-    const ReplicateMsg& replicate = entry.replicate();
-    if (replicate.op_type() == consensus::WRITE_OP) {
-      RETURN_NOT_OK(PrintDecodedWriteRequestPB(
-          indent,
-          tablet_schema,
-          replicate.write_request(),
-          replicate.has_request_id() ? &replicate.request_id() : nullptr));
-    } else {
-      cout << indent << replicate.ShortDebugString() << endl;
-    }
-  } else if (entry.has_commit()) {
-    // For COMMIT we'll just dump the PB
-    cout << indent << entry.commit().ShortDebugString() << endl;
-  }
-
-  return Status::OK();
-}
-
-Status PrintSegment(const scoped_refptr<ReadableLogSegment>& segment) {
-  PrintEntryType print_type = ParsePrintType();
-  if (FLAGS_print_headers) {
-    cout << "Header:\n" << segment->header().DebugString();
-  }
-  if (print_type == DONT_PRINT) return Status::OK();
-
-  Schema tablet_schema;
-  RETURN_NOT_OK(SchemaFromPB(segment->header().schema(), &tablet_schema));
-
-  LogEntryReader reader(segment.get());
-  LogEntryPB entry;
-  while (true) {
-    Status s = reader.ReadNextEntry(&entry);
-    if (s.IsEndOfFile()) break;
-    RETURN_NOT_OK(s);
-
-    if (print_type == PRINT_PB) {
-      if (FLAGS_truncate_data > 0) {
-        pb_util::TruncateFields(&entry, FLAGS_truncate_data);
-      }
-
-      cout << "Entry:\n" << entry.DebugString();
-    } else if (print_type == PRINT_DECODED) {
-      RETURN_NOT_OK(PrintDecoded(entry, tablet_schema));
-    } else if (print_type == PRINT_ID) {
-      PrintIdOnly(entry);
-    }
-  }
-  if (FLAGS_print_headers && segment->HasFooter()) {
-    cout << "Footer:\n" << segment->footer().DebugString();
-  }
-
-  return Status::OK();
-}
-
-Status DumpLog(const string& tablet_id) {
-  Env *env = Env::Default();
-  shared_ptr<LogReader> reader;
-  FsManagerOpts fs_opts;
-  fs_opts.read_only = true;
-  FsManager fs_manager(env, fs_opts);
-  RETURN_NOT_OK(fs_manager.Open());
-  RETURN_NOT_OK(LogReader::Open(&fs_manager,
-                                scoped_refptr<LogIndex>(),
-                                tablet_id,
-                                scoped_refptr<MetricEntity>(),
-                                &reader));
-
-  SegmentSequence segments;
-  RETURN_NOT_OK(reader->GetSegmentsSnapshot(&segments));
-
-  for (const scoped_refptr<ReadableLogSegment>& segment : segments) {
-    RETURN_NOT_OK(PrintSegment(segment));
-  }
-
-  return Status::OK();
-}
-
-Status DumpSegment(const string &segment_path) {
-  Env *env = Env::Default();
-  scoped_refptr<ReadableLogSegment> segment;
-  RETURN_NOT_OK(ReadableLogSegment::Open(env, segment_path, &segment));
-  RETURN_NOT_OK(PrintSegment(segment));
-
-  return Status::OK();
-}
-
-} // namespace log
-} // namespace kudu
-
-int main(int argc, char **argv) {
-  kudu::ParseCommandLineFlags(&argc, &argv, true);
-  if (argc != 2) {
-    std::cerr << "usage: " << argv[0]
-              << " -fs_wal_dir <dir> -fs_data_dirs <dirs>"
-              << " <tablet_name> | <log segment path>"
-              << std::endl;
-    return 1;
-  }
-  kudu::InitGoogleLoggingSafe(argv[0]);
-  kudu::Status s = kudu::log::DumpSegment(argv[1]);
-  if (s.ok()) {
-    return 0;
-  } else if (s.IsNotFound()) {
-    s = kudu::log::DumpLog(argv[1]);
-  }
-  if (!s.ok()) {
-    std::cerr << "Error: " << s.ToString() << std::endl;
-    return 1;
-  }
-  return 0;
-}

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f660ee9/src/kudu/tools/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/tools/CMakeLists.txt b/src/kudu/tools/CMakeLists.txt
index 05a72e3..c7c5a5f 100644
--- a/src/kudu/tools/CMakeLists.txt
+++ b/src/kudu/tools/CMakeLists.txt
@@ -87,18 +87,22 @@ target_link_libraries(ksck
 add_executable(kudu
   tool_action.cc
   tool_action_cluster.cc
+  tool_action_common.cc
   tool_action_fs.cc
   tool_action_pbc.cc
   tool_action_tablet.cc
+  tool_action_wal.cc
   tool_main.cc
 )
 target_link_libraries(kudu
   consensus
   gutil
+  krpc
+  ksck
   kudu_common
   kudu_fs
   kudu_util
-  ksck
+  log
   master
   tserver
   ${KUDU_BASE_LIBS}

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f660ee9/src/kudu/tools/kudu-tool-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 637cdf9..b018f3b 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -24,12 +24,25 @@
 #include "kudu/cfile/cfile-test-base.h"
 #include "kudu/cfile/cfile_util.h"
 #include "kudu/cfile/cfile_writer.h"
+#include "kudu/common/schema.h"
+#include "kudu/common/wire_protocol.h"
+#include "kudu/common/wire_protocol-test-util.h"
+#include "kudu/consensus/consensus.pb.h"
+#include "kudu/consensus/log.h"
+#include "kudu/consensus/log_util.h"
+#include "kudu/consensus/opid.pb.h"
+#include "kudu/consensus/opid_util.h"
+#include "kudu/consensus/ref_counted_replicate.h"
 #include "kudu/fs/block_manager.h"
 #include "kudu/fs/fs_manager.h"
 #include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/tserver/tserver.pb.h"
+#include "kudu/util/async_util.h"
 #include "kudu/util/env.h"
+#include "kudu/util/metrics.h"
 #include "kudu/util/oid_generator.h"
 #include "kudu/util/path_util.h"
 #include "kudu/util/subprocess.h"
@@ -42,10 +55,16 @@ namespace tools {
 using cfile::CFileWriter;
 using cfile::StringDataGenerator;
 using cfile::WriterOptions;
+using consensus::OpId;
+using consensus::ReplicateRefPtr;
+using consensus::ReplicateMsg;
 using fs::WritableBlock;
+using log::Log;
+using log::LogOptions;
 using std::string;
 using std::vector;
 using strings::Substitute;
+using tserver::WriteRequestPB;
 
 class ToolTest : public KuduTest {
  public:
@@ -59,6 +78,8 @@ class ToolTest : public KuduTest {
   }
 
   Status RunTool(const string& arg_str,
+                 string* stdout,
+                 string* stderr,
                  vector<string>* stdout_lines,
                  vector<string>* stderr_lines) const {
     vector<string> args = { tool_path_ };
@@ -66,28 +87,43 @@ class ToolTest : public KuduTest {
                                               strings::SkipEmpty());
     args.insert(args.end(), more_args.begin(), more_args.end());
 
-    string stdout;
-    string stderr;
-    Status s = Subprocess::Call(args, &stdout, &stderr);
-    StripWhiteSpace(&stdout);
-    StripWhiteSpace(&stderr);
-    *stdout_lines = strings::Split(stdout, "\n", strings::SkipEmpty());
-    *stderr_lines = strings::Split(stderr, "\n", strings::SkipEmpty());
+    string out;
+    string err;
+    Status s = Subprocess::Call(args, &out, &err);
+    if (stdout) {
+      *stdout = out;
+      StripWhiteSpace(stdout);
+    }
+    if (stderr) {
+      *stderr = err;
+      StripWhiteSpace(stderr);
+    }
+    if (stdout_lines) {
+      *stdout_lines = strings::Split(out, "\n", strings::SkipEmpty());
+    }
+    if (stderr_lines) {
+      *stderr_lines = strings::Split(err, "\n", strings::SkipEmpty());
+    }
     return s;
-
   }
 
-  void RunTestActionNoOut(const string& arg_str) const {
-    vector<string> stdout;
-    RunTestAction(arg_str, &stdout);
+  void RunActionStdoutNone(const string& arg_str) const {
+    string stdout;
+    Status s = RunTool(arg_str, &stdout, nullptr, nullptr, nullptr);
+    SCOPED_TRACE(stdout);
+    ASSERT_OK(s);
     ASSERT_TRUE(stdout.empty());
   }
 
-  void RunTestAction(const string& arg_str, vector<string>* stdout) const {
-    vector<string> stderr;
-    Status s = RunTool(arg_str, stdout, &stderr);
+  void RunActionStdoutString(const string& arg_str, string* stdout) const {
+    Status s = RunTool(arg_str, stdout, nullptr, nullptr, nullptr);
     SCOPED_TRACE(*stdout);
-    SCOPED_TRACE(stderr);
+    ASSERT_OK(s);
+  }
+
+  void RunActionStdoutLines(const string& arg_str, vector<string>* stdout_lines) const {
+    Status s = RunTool(arg_str, nullptr, nullptr, stdout_lines, nullptr);
+    SCOPED_TRACE(*stdout_lines);
     ASSERT_OK(s);
   }
 
@@ -96,7 +132,7 @@ class ToolTest : public KuduTest {
                    const Status& expected_status = Status::OK()) const {
     vector<string> stdout;
     vector<string> stderr;
-    Status s = RunTool(arg_str, &stdout, &stderr);
+    Status s = RunTool(arg_str, nullptr, nullptr, &stdout, &stderr);
     SCOPED_TRACE(stdout);
     SCOPED_TRACE(stderr);
 
@@ -132,7 +168,8 @@ TEST_F(ToolTest, TestTopLevelHelp) {
       "cluster.*Kudu cluster",
       "fs.*Kudu filesystem",
       "pbc.*protobuf container",
-      "tablet.*Kudu replica"
+      "tablet.*Kudu replica",
+      "wal.*write-ahead log"
   };
   NO_FATALS(RunTestHelp("", kTopLevelRegexes));
   NO_FATALS(RunTestHelp("--help", kTopLevelRegexes));
@@ -153,7 +190,8 @@ TEST_F(ToolTest, TestModeHelp) {
   {
     const vector<string> kTabletModeRegexes = {
         "cmeta.*consensus metadata file",
-        "copy.*Copy a replica"
+        "copy.*Copy a replica",
+        "dump_wals.*Dump all WAL"
     };
     NO_FATALS(RunTestHelp("tablet", kTabletModeRegexes));
   }
@@ -176,6 +214,12 @@ TEST_F(ToolTest, TestModeHelp) {
     };
     NO_FATALS(RunTestHelp("pbc", kPbcModeRegexes));
   }
+  {
+    const vector<string> kWalModeRegexes = {
+        "dump.*Dump a WAL",
+    };
+    NO_FATALS(RunTestHelp("wal", kWalModeRegexes));
+  }
 }
 
 TEST_F(ToolTest, TestActionHelp) {
@@ -191,7 +235,7 @@ TEST_F(ToolTest, TestActionHelp) {
 
 TEST_F(ToolTest, TestFsFormat) {
   const string kTestDir = GetTestPath("test");
-  NO_FATALS(RunTestActionNoOut(Substitute("fs format --fs_wal_dir=$0", kTestDir)));
+  NO_FATALS(RunActionStdoutNone(Substitute("fs format --fs_wal_dir=$0", kTestDir)));
   FsManager fs(env_.get(), kTestDir);
   ASSERT_OK(fs.Open());
 
@@ -205,7 +249,7 @@ TEST_F(ToolTest, TestFsFormatWithUuid) {
   const string kTestDir = GetTestPath("test");
   ObjectIdGenerator generator;
   string original_uuid = generator.Next();
-  NO_FATALS(RunTestActionNoOut(Substitute(
+  NO_FATALS(RunActionStdoutNone(Substitute(
       "fs format --fs_wal_dir=$0 --uuid=$1", kTestDir, original_uuid)));
   FsManager fs(env_.get(), kTestDir);
   ASSERT_OK(fs.Open());
@@ -225,12 +269,11 @@ TEST_F(ToolTest, TestFsPrintUuid) {
     ASSERT_OK(fs.Open());
     uuid = fs.uuid();
   }
-  vector<string> stdout;
-  NO_FATALS(RunTestAction(Substitute(
+  string stdout;
+  NO_FATALS(RunActionStdoutString(Substitute(
       "fs print_uuid --fs_wal_dir=$0", kTestDir), &stdout));
   SCOPED_TRACE(stdout);
-  ASSERT_EQ(1, stdout.size());
-  ASSERT_EQ(uuid, stdout[0]);
+  ASSERT_EQ(uuid, stdout);
 }
 
 TEST_F(ToolTest, TestPbcDump) {
@@ -245,9 +288,9 @@ TEST_F(ToolTest, TestPbcDump) {
     uuid = fs.uuid();
     instance_path = fs.GetInstanceMetadataPath(kTestDir);
   }
-  vector<string> stdout;
   {
-    NO_FATALS(RunTestAction(Substitute(
+    vector<string> stdout;
+    NO_FATALS(RunActionStdoutLines(Substitute(
         "pbc dump $0", instance_path), &stdout));
     SCOPED_TRACE(stdout);
     ASSERT_EQ(4, stdout.size());
@@ -257,13 +300,12 @@ TEST_F(ToolTest, TestPbcDump) {
     ASSERT_STR_MATCHES(stdout[3], "^format_stamp: \"Formatted at .*\"$");
   }
   {
-    NO_FATALS(RunTestAction(Substitute(
+    string stdout;
+    NO_FATALS(RunActionStdoutString(Substitute(
         "pbc dump $0/instance --oneline", kTestDir), &stdout));
     SCOPED_TRACE(stdout);
-    ASSERT_EQ(1, stdout.size());
-    ASSERT_STR_MATCHES(
-        stdout[0], Substitute(
-            "^0\tuuid: \"$0\" format_stamp: \"Formatted at .*\"$$", uuid));
+    ASSERT_STR_MATCHES(stdout, Substitute(
+        "^0\tuuid: \"$0\" format_stamp: \"Formatted at .*\"$$", uuid));
   }
 }
 
@@ -287,16 +329,14 @@ TEST_F(ToolTest, TestFsDumpCFile) {
   ASSERT_OK_FAST(writer.AppendEntries(generator.values(), kNumEntries));
   ASSERT_OK(writer.Finish());
 
-  vector<string> stdout;
   {
-    NO_FATALS(RunTestAction(Substitute(
+    NO_FATALS(RunActionStdoutNone(Substitute(
         "fs dump_cfile --fs_wal_dir=$0 $1 --noprint_meta --noprint_rows",
-        kTestDir, block_id.ToString()), &stdout));
-    SCOPED_TRACE(stdout);
-    ASSERT_TRUE(stdout.empty());
+        kTestDir, block_id.ToString())));
   }
+  vector<string> stdout;
   {
-    NO_FATALS(RunTestAction(Substitute(
+    NO_FATALS(RunActionStdoutLines(Substitute(
         "fs dump_cfile --fs_wal_dir=$0 $1 --noprint_rows",
         kTestDir, block_id.ToString()), &stdout));
     SCOPED_TRACE(stdout);
@@ -305,14 +345,14 @@ TEST_F(ToolTest, TestFsDumpCFile) {
     ASSERT_EQ(stdout[3], "Footer:");
   }
   {
-    NO_FATALS(RunTestAction(Substitute(
+    NO_FATALS(RunActionStdoutLines(Substitute(
         "fs dump_cfile --fs_wal_dir=$0 $1 --noprint_meta",
         kTestDir, block_id.ToString()), &stdout));
     SCOPED_TRACE(stdout);
     ASSERT_EQ(kNumEntries, stdout.size());
   }
   {
-    NO_FATALS(RunTestAction(Substitute(
+    NO_FATALS(RunActionStdoutLines(Substitute(
         "fs dump_cfile --fs_wal_dir=$0 $1",
         kTestDir, block_id.ToString()), &stdout));
     SCOPED_TRACE(stdout);
@@ -322,5 +362,120 @@ TEST_F(ToolTest, TestFsDumpCFile) {
   }
 }
 
+TEST_F(ToolTest, TestWalDump) {
+  const string kTestDir = GetTestPath("test");
+  const string kTestTablet = "test-tablet";
+  const Schema kSchema(GetSimpleTestSchema());
+  const Schema kSchemaWithIds(SchemaBuilder(kSchema).Build());
+
+  FsManager fs(env_.get(), kTestDir);
+  ASSERT_OK(fs.CreateInitialFileSystemLayout());
+  ASSERT_OK(fs.Open());
+
+  {
+    scoped_refptr<Log> log;
+    ASSERT_OK(Log::Open(LogOptions(),
+                        &fs,
+                        kTestTablet,
+                        kSchemaWithIds,
+                        0, // schema_version
+                        scoped_refptr<MetricEntity>(),
+                        &log));
+
+    OpId opid = consensus::MakeOpId(1, 1);
+    ReplicateRefPtr replicate =
+        consensus::make_scoped_refptr_replicate(new ReplicateMsg());
+    replicate->get()->set_op_type(consensus::WRITE_OP);
+    replicate->get()->mutable_id()->CopyFrom(opid);
+    replicate->get()->set_timestamp(1);
+    WriteRequestPB* write = replicate->get()->mutable_write_request();
+    ASSERT_OK(SchemaToPB(kSchema, write->mutable_schema()));
+    AddTestRowToPB(RowOperationsPB::INSERT, kSchema,
+                   opid.index(),
+                   0,
+                   "this is a test insert",
+                   write->mutable_row_operations());
+    write->set_tablet_id(kTestTablet);
+    Synchronizer s;
+    ASSERT_OK(log->AsyncAppendReplicates({ replicate }, s.AsStatusCallback()));
+    ASSERT_OK(s.Wait());
+  }
+
+  string wal_path = fs.GetWalSegmentFileName(kTestTablet, 1);
+  string stdout;
+  for (const auto& args : { Substitute("wal dump $0", wal_path),
+                            Substitute("tablet dump_wals --fs_wal_dir=$0 $1", kTestDir, kTestTablet)
+                           }) {
+    SCOPED_TRACE(args);
+    for (const auto& print_entries : { "true", "1", "yes", "decoded" }) {
+      SCOPED_TRACE(print_entries);
+      NO_FATALS(RunActionStdoutString(Substitute("$0 --print_entries=$1",
+                                                 args, print_entries), &stdout));
+      SCOPED_TRACE(stdout);
+      ASSERT_STR_MATCHES(stdout, "Header:");
+      ASSERT_STR_MATCHES(stdout, "1\\.1@1");
+      ASSERT_STR_MATCHES(stdout, "this is a test insert");
+      ASSERT_STR_NOT_MATCHES(stdout, "t<truncated>");
+      ASSERT_STR_NOT_MATCHES(stdout, "row_operations \\{");
+      ASSERT_STR_MATCHES(stdout, "Footer:");
+    }
+    for (const auto& print_entries : { "false", "0", "no" }) {
+      SCOPED_TRACE(print_entries);
+      NO_FATALS(RunActionStdoutString(Substitute("$0 --print_entries=$1",
+                                                 args, print_entries), &stdout));
+      SCOPED_TRACE(stdout);
+      ASSERT_STR_MATCHES(stdout, "Header:");
+      ASSERT_STR_NOT_MATCHES(stdout, "1\\.1@1");
+      ASSERT_STR_NOT_MATCHES(stdout, "this is a test insert");
+      ASSERT_STR_NOT_MATCHES(stdout, "t<truncated>");
+      ASSERT_STR_NOT_MATCHES(stdout, "row_operations \\{");
+      ASSERT_STR_MATCHES(stdout, "Footer:");
+    }
+    {
+      NO_FATALS(RunActionStdoutString(Substitute("$0 --print_entries=pb",
+                                                 args), &stdout));
+      SCOPED_TRACE(stdout);
+      ASSERT_STR_MATCHES(stdout, "Header:");
+      ASSERT_STR_NOT_MATCHES(stdout, "1\\.1@1");
+      ASSERT_STR_MATCHES(stdout, "this is a test insert");
+      ASSERT_STR_NOT_MATCHES(stdout, "t<truncated>");
+      ASSERT_STR_MATCHES(stdout, "row_operations \\{");
+      ASSERT_STR_MATCHES(stdout, "Footer:");
+    }
+    {
+      NO_FATALS(RunActionStdoutString(Substitute(
+          "$0 --print_entries=pb --truncate_data=1", args), &stdout));
+      SCOPED_TRACE(stdout);
+      ASSERT_STR_MATCHES(stdout, "Header:");
+      ASSERT_STR_NOT_MATCHES(stdout, "1\\.1@1");
+      ASSERT_STR_NOT_MATCHES(stdout, "this is a test insert");
+      ASSERT_STR_MATCHES(stdout, "t<truncated>");
+      ASSERT_STR_MATCHES(stdout, "row_operations \\{");
+      ASSERT_STR_MATCHES(stdout, "Footer:");
+    }
+    {
+      NO_FATALS(RunActionStdoutString(Substitute(
+          "$0 --print_entries=id", args), &stdout));
+      SCOPED_TRACE(stdout);
+      ASSERT_STR_MATCHES(stdout, "Header:");
+      ASSERT_STR_MATCHES(stdout, "1\\.1@1");
+      ASSERT_STR_NOT_MATCHES(stdout, "this is a test insert");
+      ASSERT_STR_NOT_MATCHES(stdout, "t<truncated>");
+      ASSERT_STR_NOT_MATCHES(stdout, "row_operations \\{");
+      ASSERT_STR_MATCHES(stdout, "Footer:");
+    }
+    {
+      NO_FATALS(RunActionStdoutString(Substitute(
+          "$0 --print_meta=false", args), &stdout));
+      SCOPED_TRACE(stdout);
+      ASSERT_STR_NOT_MATCHES(stdout, "Header:");
+      ASSERT_STR_MATCHES(stdout, "1\\.1@1");
+      ASSERT_STR_MATCHES(stdout, "this is a test insert");
+      ASSERT_STR_NOT_MATCHES(stdout, "row_operations \\{");
+      ASSERT_STR_NOT_MATCHES(stdout, "Footer:");
+    }
+  }
+}
+
 } // namespace tools
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f660ee9/src/kudu/tools/tool_action.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action.cc b/src/kudu/tools/tool_action.cc
index d142465..aa5cdb6 100644
--- a/src/kudu/tools/tool_action.cc
+++ b/src/kudu/tools/tool_action.cc
@@ -166,6 +166,7 @@ string Mode::BuildHelp(const vector<Mode*>& chain) const {
     msg += "\n";
   }
 
+  msg += "\n";
   return msg;
 }
 
@@ -238,11 +239,13 @@ string Action::BuildHelp(const vector<Mode*>& chain) const {
   for (const auto& param : args_.required) {
     usage_msg += Substitute(" <$0>", param.name);
     desc_msg += FakeDescribeOneFlag(param);
+    desc_msg += "\n";
   }
   if (args_.variadic) {
     const ActionArgsDescriptor::Arg& param = args_.variadic.get();
     usage_msg += Substitute(" <$0>...", param.name);
     desc_msg += FakeDescribeOneFlag(param);
+    desc_msg += "\n";
   }
   for (const auto& param : args_.optional) {
     google::CommandLineFlagInfo gflag_info =
@@ -266,6 +269,7 @@ string Action::BuildHelp(const vector<Mode*>& chain) const {
       usage_msg += Substitute(" [-$0=<$1>]", param, noun);
     }
     desc_msg += google::DescribeOneFlag(gflag_info);
+    desc_msg += "\n";
   }
   string msg;
   AppendHardWrapped(usage_msg, 8, &msg);

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f660ee9/src/kudu/tools/tool_action.h
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action.h b/src/kudu/tools/tool_action.h
index bac09dd..e8bdc90 100644
--- a/src/kudu/tools/tool_action.h
+++ b/src/kudu/tools/tool_action.h
@@ -276,6 +276,7 @@ std::unique_ptr<Mode> BuildClusterMode();
 std::unique_ptr<Mode> BuildFsMode();
 std::unique_ptr<Mode> BuildPbcMode();
 std::unique_ptr<Mode> BuildTabletMode();
+std::unique_ptr<Mode> BuildWalMode();
 
 } // namespace tools
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f660ee9/src/kudu/tools/tool_action_common.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_common.cc b/src/kudu/tools/tool_action_common.cc
new file mode 100644
index 0000000..1daa12d
--- /dev/null
+++ b/src/kudu/tools/tool_action_common.cc
@@ -0,0 +1,210 @@
+// 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.
+
+#include "kudu/tools/tool_action_common.h"
+
+#include <iostream>
+#include <string>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/common/common.pb.h"
+#include "kudu/common/schema.h"
+#include "kudu/common/row_operations.h"
+#include "kudu/common/wire_protocol.h"
+#include "kudu/consensus/consensus.pb.h"
+#include "kudu/consensus/log.pb.h"
+#include "kudu/consensus/log_util.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/tserver/tserver.pb.h"
+#include "kudu/util/memory/arena.h"
+#include "kudu/util/pb_util.h"
+#include "kudu/util/status.h"
+
+DEFINE_bool(print_meta, true, "Include metadata in output");
+DEFINE_string(print_entries, "decoded",
+              "How to print entries:\n"
+              "  false|0|no = don't print\n"
+              "  true|1|yes|decoded = print them decoded\n"
+              "  pb = print the raw protobuf\n"
+              "  id = print only their ids");
+DEFINE_int32(truncate_data, 100,
+             "Truncate the data fields to the given number of bytes "
+             "before printing. Set to 0 to disable");
+
+namespace kudu {
+namespace tools {
+
+using consensus::ReplicateMsg;
+using log::LogEntryPB;
+using log::LogEntryReader;
+using log::ReadableLogSegment;
+using rpc::RequestIdPB;
+using std::cout;
+using std::cerr;
+using std::endl;
+using std::string;
+using std::vector;
+using tserver::WriteRequestPB;
+
+namespace {
+
+enum PrintEntryType {
+  DONT_PRINT,
+  PRINT_PB,
+  PRINT_DECODED,
+  PRINT_ID
+};
+
+PrintEntryType ParsePrintType() {
+  if (ParseLeadingBoolValue(FLAGS_print_entries.c_str(), true) == false) {
+    return DONT_PRINT;
+  } else if (ParseLeadingBoolValue(FLAGS_print_entries.c_str(), false) == true ||
+             FLAGS_print_entries == "decoded") {
+    return PRINT_DECODED;
+  } else if (FLAGS_print_entries == "pb") {
+    return PRINT_PB;
+  } else if (FLAGS_print_entries == "id") {
+    return PRINT_ID;
+  } else {
+    LOG(FATAL) << "Unknown value for --print_entries: " << FLAGS_print_entries;
+  }
+}
+
+void PrintIdOnly(const LogEntryPB& entry) {
+  switch (entry.type()) {
+    case log::REPLICATE:
+    {
+      cout << entry.replicate().id().term() << "." << entry.replicate().id().index()
+           << "@" << entry.replicate().timestamp() << "\t";
+      cout << "REPLICATE "
+           << OperationType_Name(entry.replicate().op_type());
+      break;
+    }
+    case log::COMMIT:
+    {
+      cout << "COMMIT " << entry.commit().commited_op_id().term()
+           << "." << entry.commit().commited_op_id().index();
+      break;
+    }
+    default:
+      cout << "UNKNOWN: " << entry.ShortDebugString();
+  }
+
+  cout << endl;
+}
+
+Status PrintDecodedWriteRequestPB(const string& indent,
+                                  const Schema& tablet_schema,
+                                  const WriteRequestPB& write,
+                                  const RequestIdPB* request_id) {
+  Schema request_schema;
+  RETURN_NOT_OK(SchemaFromPB(write.schema(), &request_schema));
+
+  Arena arena(32 * 1024, 1024 * 1024);
+  RowOperationsPBDecoder dec(&write.row_operations(), &request_schema, &tablet_schema, &arena);
+  vector<DecodedRowOperation> ops;
+  RETURN_NOT_OK(dec.DecodeOperations(&ops));
+
+  cout << indent << "Tablet: " << write.tablet_id() << endl;
+  cout << indent << "RequestId: "
+      << (request_id ? request_id->ShortDebugString() : "None") << endl;
+  cout << indent << "Consistency: "
+       << ExternalConsistencyMode_Name(write.external_consistency_mode()) << endl;
+  if (write.has_propagated_timestamp()) {
+    cout << indent << "Propagated TS: " << write.propagated_timestamp() << endl;
+  }
+
+  int i = 0;
+  for (const DecodedRowOperation& op : ops) {
+    // TODO (KUDU-515): Handle the case when a tablet's schema changes
+    // mid-segment.
+    cout << indent << "op " << (i++) << ": " << op.ToString(tablet_schema) << endl;
+  }
+
+  return Status::OK();
+}
+
+Status PrintDecoded(const LogEntryPB& entry, const Schema& tablet_schema) {
+  PrintIdOnly(entry);
+
+  const string indent = "\t";
+  if (entry.has_replicate()) {
+    // We can actually decode REPLICATE messages.
+
+    const ReplicateMsg& replicate = entry.replicate();
+    if (replicate.op_type() == consensus::WRITE_OP) {
+      RETURN_NOT_OK(PrintDecodedWriteRequestPB(
+          indent,
+          tablet_schema,
+          replicate.write_request(),
+          replicate.has_request_id() ? &replicate.request_id() : nullptr));
+    } else {
+      cout << indent << replicate.ShortDebugString() << endl;
+    }
+  } else if (entry.has_commit()) {
+    // For COMMIT we'll just dump the PB
+    cout << indent << entry.commit().ShortDebugString() << endl;
+  }
+
+  return Status::OK();
+}
+
+} // anonymous namespace
+
+Status PrintSegment(const scoped_refptr<ReadableLogSegment>& segment) {
+  PrintEntryType print_type = ParsePrintType();
+  if (FLAGS_print_meta) {
+    cout << "Header:\n" << segment->header().DebugString();
+  }
+  if (print_type != DONT_PRINT) {
+    Schema tablet_schema;
+    RETURN_NOT_OK(SchemaFromPB(segment->header().schema(), &tablet_schema));
+
+    LogEntryReader reader(segment.get());
+    LogEntryPB entry;
+    while (true) {
+      Status s = reader.ReadNextEntry(&entry);
+      if (s.IsEndOfFile()) break;
+      RETURN_NOT_OK(s);
+
+      if (print_type == PRINT_PB) {
+        if (FLAGS_truncate_data > 0) {
+          pb_util::TruncateFields(&entry, FLAGS_truncate_data);
+        }
+
+        cout << "Entry:\n" << entry.DebugString();
+      } else if (print_type == PRINT_DECODED) {
+        RETURN_NOT_OK(PrintDecoded(entry, tablet_schema));
+      } else if (print_type == PRINT_ID) {
+        PrintIdOnly(entry);
+      }
+    }
+  }
+  if (FLAGS_print_meta && segment->HasFooter()) {
+    cout << "Footer:\n" << segment->footer().DebugString();
+  }
+
+  return Status::OK();
+}
+
+} // namespace tools
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f660ee9/src/kudu/tools/tool_action_common.h
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_common.h b/src/kudu/tools/tool_action_common.h
new file mode 100644
index 0000000..0391bb3
--- /dev/null
+++ b/src/kudu/tools/tool_action_common.h
@@ -0,0 +1,42 @@
+// 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.
+
+#pragma once
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+namespace log {
+class ReadableLogSegment;
+} // namespace log
+
+namespace tools {
+
+// Utility methods used by multiple actions across different modes.
+
+// Prints the contents of a WAL segment to stdout.
+//
+// The following gflags affect the output:
+// - print_entries: in what style entries should be printed.
+// - print_meta: whether or not headers/footers are printed.
+// - truncate_data: how many bytes to print for each data field.
+Status PrintSegment(const scoped_refptr<log::ReadableLogSegment>& segment);
+
+} // namespace tools
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f660ee9/src/kudu/tools/tool_action_fs.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_fs.cc b/src/kudu/tools/tool_action_fs.cc
index 63fd702..a09a085 100644
--- a/src/kudu/tools/tool_action_fs.cc
+++ b/src/kudu/tools/tool_action_fs.cc
@@ -31,8 +31,7 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/status.h"
 
-DEFINE_bool(print_meta, true,
-            "Print the header and footer from the CFile");
+DECLARE_bool(print_meta);
 DEFINE_bool(print_rows, true,
             "Print each row in the CFile");
 DEFINE_string(uuid, "",
@@ -47,7 +46,6 @@ using std::cout;
 using std::endl;
 using std::string;
 using std::unique_ptr;
-using std::vector;
 using strings::Substitute;
 
 namespace {

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f660ee9/src/kudu/tools/tool_action_tablet.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_tablet.cc b/src/kudu/tools/tool_action_tablet.cc
index 437c26f..736c2e5 100644
--- a/src/kudu/tools/tool_action_tablet.cc
+++ b/src/kudu/tools/tool_action_tablet.cc
@@ -25,26 +25,38 @@
 
 #include "kudu/common/wire_protocol.h"
 #include "kudu/consensus/consensus_meta.h"
+#include "kudu/consensus/log_index.h"
+#include "kudu/consensus/log_reader.h"
+#include "kudu/consensus/log_util.h"
 #include "kudu/fs/fs_manager.h"
 #include "kudu/gutil/map-util.h"
+#include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/stringpiece.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/sys_catalog.h"
 #include "kudu/rpc/messenger.h"
+#include "kudu/tools/tool_action_common.h"
 #include "kudu/tserver/tablet_copy_client.h"
 #include "kudu/util/env.h"
 #include "kudu/util/env_util.h"
+#include "kudu/util/metrics.h"
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/status.h"
 
-using kudu::consensus::ConsensusMetadata;
-using kudu::consensus::RaftConfigPB;
-using kudu::consensus::RaftPeerPB;
-using kudu::rpc::Messenger;
-using kudu::rpc::MessengerBuilder;
-using kudu::tserver::TabletCopyClient;
+namespace kudu {
+namespace tools {
+
+using consensus::ConsensusMetadata;
+using consensus::RaftConfigPB;
+using consensus::RaftPeerPB;
+using log::LogIndex;
+using log::LogReader;
+using log::ReadableLogSegment;
+using log::SegmentSequence;
+using rpc::Messenger;
+using rpc::MessengerBuilder;
 using std::cout;
 using std::endl;
 using std::list;
@@ -54,9 +66,7 @@ using std::unique_ptr;
 using std::vector;
 using strings::Split;
 using strings::Substitute;
-
-namespace kudu {
-namespace tools {
+using tserver::TabletCopyClient;
 
 namespace {
 
@@ -186,6 +196,31 @@ Status Copy(const RunnerContext& context) {
   return client.Finish();
 }
 
+Status DumpWals(const RunnerContext& context) {
+  string tablet_id = FindOrDie(context.required_args, "tablet_id");
+
+  FsManagerOpts fs_opts;
+  fs_opts.read_only = true;
+  FsManager fs_manager(Env::Default(), fs_opts);
+  RETURN_NOT_OK(fs_manager.Open());
+
+  shared_ptr<LogReader> reader;
+  RETURN_NOT_OK(LogReader::Open(&fs_manager,
+                                scoped_refptr<LogIndex>(),
+                                tablet_id,
+                                scoped_refptr<MetricEntity>(),
+                                &reader));
+
+  SegmentSequence segments;
+  RETURN_NOT_OK(reader->GetSegmentsSnapshot(&segments));
+
+  for (const scoped_refptr<ReadableLogSegment>& segment : segments) {
+    RETURN_NOT_OK(PrintSegment(segment));
+  }
+
+  return Status::OK();
+}
+
 } // anonymous namespace
 
 unique_ptr<Mode> BuildTabletMode() {
@@ -223,10 +258,22 @@ unique_ptr<Mode> BuildTabletMode() {
       .AddOptionalParameter("fs_data_dirs")
       .Build();
 
+  unique_ptr<Action> dump_wals =
+      ActionBuilder("dump_wals", &DumpWals)
+      .Description("Dump all WAL (write-ahead log) segments of a tablet")
+      .AddRequiredParameter({ "tablet_id", "Tablet identifier" })
+      .AddOptionalParameter("fs_wal_dir")
+      .AddOptionalParameter("fs_data_dirs")
+      .AddOptionalParameter("print_entries")
+      .AddOptionalParameter("print_meta")
+      .AddOptionalParameter("truncate_data")
+      .Build();
+
   return ModeBuilder("tablet")
       .Description("Operate on a local Kudu replica")
       .AddMode(std::move(cmeta))
       .AddAction(std::move(copy))
+      .AddAction(std::move(dump_wals))
       .Build();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f660ee9/src/kudu/tools/tool_action_wal.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_wal.cc b/src/kudu/tools/tool_action_wal.cc
new file mode 100644
index 0000000..9b8cdb8
--- /dev/null
+++ b/src/kudu/tools/tool_action_wal.cc
@@ -0,0 +1,69 @@
+// 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.
+
+#include "kudu/tools/tool_action.h"
+
+#include <memory>
+#include <string>
+
+#include "kudu/consensus/log_util.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/tools/tool_action_common.h"
+#include "kudu/util/env.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace tools {
+
+using log::ReadableLogSegment;
+using std::string;
+using std::unique_ptr;
+
+namespace {
+
+const char* const kPathArg = "path";
+
+Status Dump(const RunnerContext& context) {
+  string segment_path = FindOrDie(context.required_args, kPathArg);
+
+  scoped_refptr<ReadableLogSegment> segment;
+  RETURN_NOT_OK(ReadableLogSegment::Open(Env::Default(), segment_path, &segment));
+  RETURN_NOT_OK(PrintSegment(segment));
+  return Status::OK();
+}
+
+} // anonymous namespace
+
+unique_ptr<Mode> BuildWalMode() {
+  unique_ptr<Action> dump =
+      ActionBuilder("dump", &Dump)
+      .Description("Dump a WAL (write-ahead log) file")
+      .AddRequiredParameter({ kPathArg, "path to WAL file" })
+      .AddOptionalParameter("print_entries")
+      .AddOptionalParameter("print_meta")
+      .AddOptionalParameter("truncate_data")
+      .Build();
+
+  return ModeBuilder("wal")
+      .Description("Operate on WAL (write-ahead log) files")
+      .AddAction(std::move(dump))
+      .Build();
+}
+
+} // namespace tools
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f660ee9/src/kudu/tools/tool_main.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_main.cc b/src/kudu/tools/tool_main.cc
index 3a6f4c8..f1fbe93 100644
--- a/src/kudu/tools/tool_main.cc
+++ b/src/kudu/tools/tool_main.cc
@@ -115,6 +115,7 @@ int RunTool(int argc, char** argv, bool show_help) {
     .AddMode(BuildFsMode())
     .AddMode(BuildPbcMode())
     .AddMode(BuildTabletMode())
+    .AddMode(BuildWalMode())
     .Build();
 
   // Initialize arg parsing state.
@@ -150,7 +151,7 @@ int RunTool(int argc, char** argv, bool show_help) {
       chain.push_back(next_mode);
     } else if (next_action) {
       if (show_help) {
-        cerr << next_action->BuildHelp(chain) << endl;
+        cerr << next_action->BuildHelp(chain);
         return 1;
       } else {
         // Invoke the action with whatever arguments remain, skipping this one.
@@ -164,7 +165,7 @@ int RunTool(int argc, char** argv, bool show_help) {
       // Couldn't match the argument at all. Print the help.
       Status s = Status::InvalidArgument(
           Substitute("unknown command '$0'\n", argv[i]));
-      cerr << s.ToString() << cur->BuildHelp(chain) << endl;
+      cerr << s.ToString() << cur->BuildHelp(chain);
       return 1;
     }
   }
@@ -172,7 +173,7 @@ int RunTool(int argc, char** argv, bool show_help) {
   // Ran out of arguments before reaching an action. Print the last mode's help.
   DCHECK(!chain.empty());
   const Mode* last = chain.back();
-  cerr << last->BuildHelp(chain) << endl;
+  cerr << last->BuildHelp(chain);
   return 1;
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f660ee9/src/kudu/util/test_macros.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/test_macros.h b/src/kudu/util/test_macros.h
index a5f151f..c4fdca0 100644
--- a/src/kudu/util/test_macros.h
+++ b/src/kudu/util/test_macros.h
@@ -62,6 +62,9 @@
 #define ASSERT_STR_MATCHES(str, pattern) \
   ASSERT_THAT(str, testing::ContainsRegex(pattern))
 
+#define ASSERT_STR_NOT_MATCHES(str, pattern) \
+  ASSERT_THAT(str, testing::Not(testing::ContainsRegex(pattern)))
+
 // Batched substring regular expressions in extended regex (POSIX) syntax.
 //
 // All strings must match the pattern.