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

[1/5] kudu git commit: Fix false positive clang-tidy warning when using gtest

Repository: kudu
Updated Branches:
  refs/heads/master 2d30f3a96 -> 9ae11e6d8


Fix false positive clang-tidy warning when using gtest

This applies a patch from https://reviews.llvm.org/D27048 to fix a false
positive that has surfaced since upgrading to LLVM 4.0.

I verified locally that this solved the spurious warnings on a test
file.

Change-Id: Iff972e198ba297a001b965f0012e1140112e553f
Reviewed-on: http://gerrit.cloudera.org:8080/7119
Reviewed-by: Dan Burkert <da...@apache.org>
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: a9ac82ff42738e759df21145dda0382b2996bf43
Parents: 2d30f3a
Author: Todd Lipcon <to...@cloudera.com>
Authored: Thu Jun 8 14:10:52 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Thu Jun 8 21:45:28 2017 +0000

----------------------------------------------------------------------
 thirdparty/download-thirdparty.sh               |   4 +-
 ...y-redundant-declaration-false-positive.patch | 101 +++++++++++++++++++
 2 files changed, 104 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/a9ac82ff/thirdparty/download-thirdparty.sh
----------------------------------------------------------------------
diff --git a/thirdparty/download-thirdparty.sh b/thirdparty/download-thirdparty.sh
index 4238c08..68950f5 100755
--- a/thirdparty/download-thirdparty.sh
+++ b/thirdparty/download-thirdparty.sh
@@ -238,13 +238,15 @@ if [ ! -d $PYTHON_SOURCE ]; then
   fetch_and_expand python-${PYTHON_VERSION}.tar.gz
 fi
 
-LLVM_PATCHLEVEL=1
+LLVM_PATCHLEVEL=2
 delete_if_wrong_patchlevel $LLVM_SOURCE $LLVM_PATCHLEVEL
 if [ ! -d $LLVM_SOURCE ]; then
   fetch_and_expand llvm-${LLVM_VERSION}.src.tar.gz
 
   pushd $LLVM_SOURCE
   patch -p1 < $TP_DIR/patches/llvm-fix-amazon-linux.patch
+  patch -p1 -d $LLVM_SOURCE/tools/clang/tools/extra \
+    < $TP_DIR/patches/llvm-fix-readability-redundant-declaration-false-positive.patch
   touch patchlevel-$LLVM_PATCHLEVEL
   popd
   echo

http://git-wip-us.apache.org/repos/asf/kudu/blob/a9ac82ff/thirdparty/patches/llvm-fix-readability-redundant-declaration-false-positive.patch
----------------------------------------------------------------------
diff --git a/thirdparty/patches/llvm-fix-readability-redundant-declaration-false-positive.patch b/thirdparty/patches/llvm-fix-readability-redundant-declaration-false-positive.patch
new file mode 100644
index 0000000..87d682e
--- /dev/null
+++ b/thirdparty/patches/llvm-fix-readability-redundant-declaration-false-positive.patch
@@ -0,0 +1,101 @@
+From 5978f7dc14b9d4e36f12ec3bb850b5d80f5a9045 Mon Sep 17 00:00:00 2001
+From: Daniel Marjamaki <da...@evidente.se>
+Date: Fri, 24 Feb 2017 09:02:44 +0000
+Subject: [PATCH] [clang-tidy] Fix readability-redundant-declaration false
+ positive
+
+Differential Revision: https://reviews.llvm.org/D27048
+
+
+git-svn-id: https://llvm.org/svn/llvm-project/clang-tools-extra/trunk@296100 91177308-0d34-0410-b5e6-96231b3b80d8
+---
+ .../readability/RedundantDeclarationCheck.cpp      | 12 ++++-------
+ .../readability-redundant-declaration.cpp          | 24 ++++++++++++++--------
+ 2 files changed, 19 insertions(+), 17 deletions(-)
+
+diff --git a/clang-tidy/readability/RedundantDeclarationCheck.cpp b/clang-tidy/readability/RedundantDeclarationCheck.cpp
+index 672cda655..1ee65d8d7 100644
+--- a/clang-tidy/readability/RedundantDeclarationCheck.cpp
++++ b/clang-tidy/readability/RedundantDeclarationCheck.cpp
+@@ -19,7 +19,10 @@ namespace tidy {
+ namespace readability {
+ 
+ void RedundantDeclarationCheck::registerMatchers(MatchFinder *Finder) {
+-  Finder->addMatcher(namedDecl(anyOf(varDecl(), functionDecl())).bind("Decl"),
++  auto UnlessDefinition = unless(isDefinition());
++  Finder->addMatcher(namedDecl(anyOf(varDecl(UnlessDefinition),
++                                     functionDecl(UnlessDefinition)))
++                         .bind("Decl"),
+                      this);
+ }
+ 
+@@ -41,9 +44,6 @@ void RedundantDeclarationCheck::check(const MatchFinder::MatchResult &Result) {
+ 
+   bool MultiVar = false;
+   if (const auto *VD = dyn_cast<VarDecl>(D)) {
+-    if (VD->getPreviousDecl()->getStorageClass() == SC_Extern &&
+-        VD->getStorageClass() != SC_Extern)
+-      return;
+     // Is this a multivariable declaration?
+     for (const auto Other : VD->getDeclContext()->decls()) {
+       if (Other != D && Other->getLocStart() == VD->getLocStart()) {
+@@ -51,10 +51,6 @@ void RedundantDeclarationCheck::check(const MatchFinder::MatchResult &Result) {
+         break;
+       }
+     }
+-  } else {
+-    const auto *FD = cast<FunctionDecl>(D);
+-    if (FD->isThisDeclarationADefinition())
+-      return;
+   }
+ 
+   SourceLocation EndLoc = Lexer::getLocForEndOfToken(
+diff --git a/test/clang-tidy/readability-redundant-declaration.cpp b/test/clang-tidy/readability-redundant-declaration.cpp
+index e68b7f657..bedc68c4d 100644
+--- a/test/clang-tidy/readability-redundant-declaration.cpp
++++ b/test/clang-tidy/readability-redundant-declaration.cpp
+@@ -1,9 +1,9 @@
+ // RUN: %check_clang_tidy %s readability-redundant-declaration %t
+ 
+ extern int Xyz;
+-extern int Xyz;
++extern int Xyz; // Xyz
+ // CHECK-MESSAGES: :[[@LINE-1]]:12: warning: redundant 'Xyz' declaration [readability-redundant-declaration]
+-// CHECK-FIXES: {{^}}{{$}}
++// CHECK-FIXES: {{^}}// Xyz{{$}}
+ int Xyz = 123;
+ 
+ extern int A;
+@@ -12,19 +12,25 @@ extern int A, B;
+ // CHECK-FIXES: {{^}}extern int A, B;{{$}}
+ 
+ extern int Buf[10];
+-extern int Buf[10];
++extern int Buf[10]; // Buf[10]
+ // CHECK-MESSAGES: :[[@LINE-1]]:12: warning: redundant 'Buf' declaration
+-// CHECK-FIXES: {{^}}{{$}}
++// CHECK-FIXES: {{^}}// Buf[10]{{$}}
+ 
+ static int f();
+-static int f();
++static int f(); // f
+ // CHECK-MESSAGES: :[[@LINE-1]]:12: warning: redundant 'f' declaration
+-// CHECK-FIXES: {{^}}{{$}}
++// CHECK-FIXES: {{^}}// f{{$}}
+ static int f() {}
+ 
+ // Original check crashed for the code below.
+ namespace std {
+-  typedef decltype(sizeof(0)) size_t;
++typedef decltype(sizeof(0)) size_t;
+ }
+-void* operator new(std::size_t) __attribute__((__externally_visible__));
+-void* operator new[](std::size_t) __attribute__((__externally_visible__));
++void *operator new(std::size_t) __attribute__((__externally_visible__));
++void *operator new[](std::size_t) __attribute__((__externally_visible__));
++
++// Don't warn about static member definition.
++struct C {
++  static int I;
++};
++int C::I;


[3/5] kudu git commit: Temporary workaround for KUDU-1959 (race when selecting rowsets)

Posted by to...@apache.org.
Temporary workaround for KUDU-1959 (race when selecting rowsets)

As described in the JIRA, there is a race by which multiple MM threads
can race to pick the same rowsets for compaction. Rather than crash when
hitting this bug, it is safe to simply abort that compaction attempt.
The MM will warn about the compaction failure and try again.

This is a temporary workround for the 1.4 release since the issue was
recently reported in the wild on the user list.

Change-Id: I9db313849176e1bf05636d969fafb1682e6d78de
Reviewed-on: http://gerrit.cloudera.org:8080/7120
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: 8be2a59103da46472062f47f89efa6e1bddd0a5c
Parents: 693f675
Author: Todd Lipcon <to...@apache.org>
Authored: Thu Jun 8 14:07:52 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Thu Jun 8 22:04:19 2017 +0000

----------------------------------------------------------------------
 src/kudu/tablet/tablet.cc | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/8be2a591/src/kudu/tablet/tablet.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet.cc b/src/kudu/tablet/tablet.cc
index aaaa72b..fb6043b 100644
--- a/src/kudu/tablet/tablet.cc
+++ b/src/kudu/tablet/tablet.cc
@@ -1219,7 +1219,13 @@ Status Tablet::PickRowSetsToCompact(RowSetsInCompaction *picked,
       LOG_WITH_PREFIX(ERROR) << "Rowset selected for compaction but not available anymore: "
                              << not_found->ToString();
     }
-    LOG_WITH_PREFIX(FATAL) << "Was unable to find all rowsets selected for compaction";
+    // TODO(todd): this should never happen, but KUDU-1959 is a bug which causes us to
+    // sometimes concurrently decide to compact the same rowsets. It should be harmless
+    // to simply abort the compaction when we hit this bug, though long term we should
+    // fix the underlying race.
+    const char* msg = "Was unable to find all rowsets selected for compaction";
+    LOG_WITH_PREFIX(DFATAL) << msg;
+    return Status::RuntimeError(msg);
   }
   return Status::OK();
 }


[2/5] kudu git commit: build: make the failure summary nicer formatting

Posted by to...@apache.org.
build: make the failure summary nicer formatting

Change-Id: Id8ac20195026be6069cc3ec1e18139d09f102011
Reviewed-on: http://gerrit.cloudera.org:8080/7114
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin <as...@cloudera.com>
Reviewed-by: Adar Dembo <ad...@cloudera.com>


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

Branch: refs/heads/master
Commit: 693f675c580364f0a94f58c161074590a09913da
Parents: a9ac82f
Author: Todd Lipcon <to...@apache.org>
Authored: Wed Jun 7 20:37:04 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Thu Jun 8 22:00:17 2017 +0000

----------------------------------------------------------------------
 build-support/jenkins/build-and-test.sh | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/693f675c/build-support/jenkins/build-and-test.sh
----------------------------------------------------------------------
diff --git a/build-support/jenkins/build-and-test.sh b/build-support/jenkins/build-and-test.sh
index 8346472..5ebe1dd 100755
--- a/build-support/jenkins/build-and-test.sh
+++ b/build-support/jenkins/build-and-test.sh
@@ -511,9 +511,14 @@ fi
 set -e
 
 if [ -n "$FAILURES" ]; then
+  echo
+  echo
+  echo ======================================================================
   echo Failure summary
-  echo ------------------------------------------------------------
+  echo ======================================================================
   echo $FAILURES
+  echo
+  echo
 fi
 
 exit $EXIT_STATUS


[5/5] kudu git commit: tool: add a 'local-replica cmeta set-term' tool

Posted by to...@apache.org.
tool: add a 'local-replica cmeta set-term' tool

Through abuse of some other tools (and restoring from a backup of a
cmeta file) I ended up with a situation where a tablet's WAL contained
operations from a term higher than the term listed in its cmeta file.
This caused the tablet to fail to start due to seeing this inconsistency
(the highest term in the WAL should always be <= the term in the cmeta).

This patch adds a tool that I wrote in order to recover from the
situation. The tool allows the operator to override the term stored in
the cmeta file. It's restricted to only bumping the term upwards, since
doing so is always "safe" whereas reducing it backwards could have
really bad consequences.

I also took the opportunity to add some new tests for the 'cmeta' tool
functionality.

Change-Id: I7525ffbe772f214e0972a6b450f3f1609109ca05
Reviewed-on: http://gerrit.cloudera.org:8080/7049
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Reviewed-by: Mike Percy <mp...@apache.org>
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin <as...@cloudera.com>


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

Branch: refs/heads/master
Commit: 9ae11e6d86eb33321fbb219927b27424e58f6e2c
Parents: e77538b
Author: Todd Lipcon <to...@cloudera.com>
Authored: Thu Jun 1 14:15:49 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Thu Jun 8 22:54:59 2017 +0000

----------------------------------------------------------------------
 src/kudu/tools/kudu-tool-test.cc            | 61 +++++++++++++++++++-
 src/kudu/tools/tool_action_local_replica.cc | 72 ++++++++++++++++++++----
 2 files changed, 122 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/9ae11e6d/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 615ef1e..7bc23a7 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -405,7 +405,8 @@ TEST_F(ToolTest, TestModeHelp) {
   {
     const vector<string> kLocalReplicaCMetaRegexes = {
         "print_replica_uuids.*Print all tablet replica peer UUIDs",
-        "rewrite_raft_config.*Rewrite a tablet replica"
+        "rewrite_raft_config.*Rewrite a tablet replica",
+        "set_term.*Bump the current term"
     };
     NO_FATALS(RunTestHelp("local_replica cmeta", kLocalReplicaCMetaRegexes));
     // Try with a hyphen instead of an underscore.
@@ -1480,6 +1481,64 @@ TEST_F(ToolTest, TestLocalReplicaTombstoneDelete) {
   }
 }
 
+// Test for 'local_replica cmeta' functionality.
+TEST_F(ToolTest, TestLocalReplicaCMetaOps) {
+  NO_FATALS(StartMiniCluster());
+
+  // TestWorkLoad.Setup() internally generates a table.
+  TestWorkload workload(mini_cluster_.get());
+  workload.set_num_replicas(1);
+  workload.Setup();
+  MiniTabletServer* ts = mini_cluster_->mini_tablet_server(0);
+  const string ts_uuid = ts->uuid();
+  const string& flags = Substitute("-fs-wal-dir $0", ts->options()->fs_opts.wal_path);
+  string tablet_id;
+  {
+    vector<string> tablets;
+    NO_FATALS(RunActionStdoutLines(Substitute("local_replica list $0", flags), &tablets));
+    ASSERT_EQ(1, tablets.size());
+    tablet_id = tablets[0];
+  }
+  const auto& cmeta_path = ts->server()->fs_manager()->GetConsensusMetadataPath(tablet_id);
+
+  ts->Shutdown();
+
+  // Test print_replica_uuids.
+  // We only have a single replica, so we expect one line, with our server's UUID.
+  {
+    vector<string> uuids;
+    NO_FATALS(RunActionStdoutLines(Substitute("local_replica cmeta print_replica_uuids $0 $1",
+                                               flags, tablet_id), &uuids));
+    ASSERT_EQ(1, uuids.size());
+    EXPECT_EQ(ts_uuid, uuids[0]);
+  }
+
+  // Test using set-term to bump the term to 123.
+  {
+    NO_FATALS(RunActionStdoutNone(Substitute("local_replica cmeta set-term $0 $1 123",
+                                             flags, tablet_id)));
+
+    string stdout;
+    NO_FATALS(RunActionStdoutString(Substitute("pbc dump $0", cmeta_path),
+                                    &stdout));
+    ASSERT_STR_CONTAINS(stdout, "current_term: 123");
+  }
+
+  // Test that set-term refuses to decrease the term.
+  {
+    string stdout, stderr;
+    Status s = RunTool(Substitute("local_replica cmeta set-term $0 $1 10",
+                                  flags, tablet_id),
+                       &stdout, &stderr,
+                       /* stdout_lines = */ nullptr,
+                       /* stderr_lines = */ nullptr);
+    EXPECT_FALSE(s.ok());
+    EXPECT_EQ("", stdout);
+    EXPECT_THAT(stderr, testing::HasSubstr(
+        "specified term 10 must be higher than current term 123"));
+  }
+}
+
 TEST_F(ToolTest, TestTserverList) {
   NO_FATALS(StartExternalMiniCluster({}, {}, 1));
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/9ae11e6d/src/kudu/tools/tool_action_local_replica.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_local_replica.cc b/src/kudu/tools/tool_action_local_replica.cc
index 0736a66..0cb438c 100644
--- a/src/kudu/tools/tool_action_local_replica.cc
+++ b/src/kudu/tools/tool_action_local_replica.cc
@@ -129,6 +129,8 @@ namespace {
 const char* const kSeparatorLine =
     "----------------------------------------------------------------------\n";
 
+const char* const kTermArg = "term";
+
 string Indent(int indent) {
   return string(indent, ' ');
 }
@@ -224,6 +226,19 @@ Status PrintReplicaUuids(const RunnerContext& context) {
   return Status::OK();
 }
 
+Status BackupConsensusMetadata(FsManager* fs_manager,
+                               const string& tablet_id) {
+  Env* env = fs_manager->env();
+  string cmeta_filename = fs_manager->GetConsensusMetadataPath(tablet_id);
+  string backup_filename = Substitute("$0.pre_rewrite.$1", cmeta_filename, env->NowMicros());
+  WritableFileOptions opts;
+  opts.mode = Env::CREATE_NON_EXISTING;
+  opts.sync_on_close = true;
+  RETURN_NOT_OK(env_util::CopyFile(env, cmeta_filename, backup_filename, opts));
+  LOG(INFO) << "Backed up old consensus metadata to " << backup_filename;
+  return Status::OK();
+}
+
 Status RewriteRaftConfig(const RunnerContext& context) {
   // Parse tablet ID argument.
   const string& tablet_id = FindOrDie(context.required_args, kTabletIdArg);
@@ -246,15 +261,7 @@ Status RewriteRaftConfig(const RunnerContext& context) {
   Env* env = Env::Default();
   FsManager fs_manager(env, FsManagerOpts());
   RETURN_NOT_OK(fs_manager.Open());
-  string cmeta_filename = fs_manager.GetConsensusMetadataPath(tablet_id);
-  string backup_filename = Substitute("$0.pre_rewrite.$1",
-                                      cmeta_filename, env->NowMicros());
-  WritableFileOptions opts;
-  opts.mode = Env::CREATE_NON_EXISTING;
-  opts.sync_on_close = true;
-  RETURN_NOT_OK(env_util::CopyFile(env, cmeta_filename,
-                                   backup_filename, opts));
-  LOG(INFO) << "Backed up current config to " << backup_filename;
+  RETURN_NOT_OK(BackupConsensusMetadata(&fs_manager, tablet_id));
 
   // Load the cmeta file and rewrite the raft config.
   unique_ptr<ConsensusMetadata> cmeta;
@@ -276,6 +283,41 @@ Status RewriteRaftConfig(const RunnerContext& context) {
   return cmeta->Flush();
 }
 
+Status SetRaftTerm(const RunnerContext& context) {
+  // Parse tablet ID argument.
+  const string& tablet_id = FindOrDie(context.required_args, kTabletIdArg);
+  const string& new_term_str = FindOrDie(context.required_args, kTermArg);
+  int64_t new_term;
+  if (!safe_strto64(new_term_str, &new_term) || new_term <= 0) {
+    return Status::InvalidArgument("invalid term");
+  }
+
+  // Load the current metadata from disk and verify that the intended operation is safe.
+  Env* env = Env::Default();
+  FsManager fs_manager(env, FsManagerOpts());
+  RETURN_NOT_OK(fs_manager.Open());
+  // Load the cmeta file and rewrite the raft config.
+  unique_ptr<ConsensusMetadata> cmeta;
+  RETURN_NOT_OK(ConsensusMetadata::Load(&fs_manager, tablet_id,
+                                        fs_manager.uuid(), &cmeta));
+  if (new_term <= cmeta->current_term()) {
+    return Status::InvalidArgument(Substitute(
+        "specified term $0 must be higher than current term $1",
+        new_term, cmeta->current_term()));
+  }
+
+  // Make a copy of the old file before rewriting it.
+  RETURN_NOT_OK(BackupConsensusMetadata(&fs_manager, tablet_id));
+
+  // Update and flush.
+  cmeta->set_current_term(new_term);
+  // The 'voted_for' field is relative to the term stored in 'current_term'. So, if we
+  // have changed to a new term, we need to also clear any previous vote record that was
+  // associated with the old term.
+  cmeta->clear_voted_for();
+  return cmeta->Flush();
+}
+
 Status CopyFromRemote(const RunnerContext& context) {
   // Parse the tablet ID and source arguments.
   const string& tablet_id = FindOrDie(context.required_args, kTabletIdArg);
@@ -758,12 +800,23 @@ unique_ptr<Mode> BuildLocalReplicaMode() {
       .AddOptionalParameter("fs_data_dirs")
       .Build();
 
+  unique_ptr<Action> set_term =
+      ActionBuilder("set_term", &SetRaftTerm)
+      .Description("Bump the current term stored in consensus metadata")
+      .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
+      .AddRequiredParameter({ kTermArg, "the new raft term (must be greater "
+        "than the current term)" })
+      .AddOptionalParameter("fs_wal_dir")
+      .AddOptionalParameter("fs_data_dirs")
+      .Build();
+
   unique_ptr<Mode> cmeta =
       ModeBuilder("cmeta")
       .Description("Operate on a local tablet replica's consensus "
         "metadata file")
       .AddAction(std::move(print_replica_uuids))
       .AddAction(std::move(rewrite_raft_config))
+      .AddAction(std::move(set_term))
       .Build();
 
   unique_ptr<Action> copy_from_remote =
@@ -806,4 +859,3 @@ unique_ptr<Mode> BuildLocalReplicaMode() {
 
 } // namespace tools
 } // namespace kudu
-


[4/5] kudu git commit: log_block_manager: fix corruption after re-opening compacted metadata

Posted by to...@apache.org.
log_block_manager: fix corruption after re-opening compacted metadata

This fixes an issue discovered on a cluster due to the following
sequence of events:

- a block manager compacts a metadata file while starting up
- when it reopens the metadata file after replacing it with the
  compacted one, it gets a file_cache hit. Thus, the WritablePBContainer
  continues to write to the _deleted_ file instead of the compacted one.
  Metadata entries at this point are lost (which could cause block loss
  in the case of lost CREATE records, or dangling blocks in the case of
  lost DELETEs)
- if the server continues to run for a while, the FD will be evicted
  from the cache and eventually re-opened. At that point, a further
  DELETE record could end up writing to an offset past the end of the
  file, since the write offset was incremented by the "lost" records
  above.
- on the next restart, the metadata file would have a "gap" of zero
  bytes, which would surface as a checksum failure and failure to start
  up.

The fix is relatively simple: when we replace the metadata file we need
to invalidate and evict the cache entry so that when we "reopen", it
actually starts appending to the _new_ file and not the old deleted one.

The bulk of the changes here are to tests:
- the stress test now enforces a minimum number of live blocks before it
  starts deleting them. It also more aggressively compacts, and has a
  smaller cache. With these changes, I was sometimes able to reproduce
  the issue.
- A more targeted test issues a canned sequence of block creations and
  deletions that can reliably reproduce the above issue.

Change-Id: I14b2c64685e24d27591258911db4aeb9e8020a4d
Reviewed-on: http://gerrit.cloudera.org:8080/7113
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: e77538b541b112555ee62cff1506bf7fd1f0e461
Parents: 8be2a59
Author: Todd Lipcon <to...@apache.org>
Authored: Wed Jun 7 20:01:17 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Thu Jun 8 22:47:26 2017 +0000

----------------------------------------------------------------------
 src/kudu/fs/block_manager-stress-test.cc |  45 ++++++----
 src/kudu/fs/log_block_manager-test.cc    |  56 +++++++++++++
 src/kudu/fs/log_block_manager.cc         |  12 ++-
 src/kudu/util/file_cache-test.cc         |  34 ++++++++
 src/kudu/util/file_cache.cc              | 114 +++++++++++++++-----------
 src/kudu/util/file_cache.h               |  23 ++++++
 6 files changed, 218 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/e77538b5/src/kudu/fs/block_manager-stress-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/block_manager-stress-test.cc b/src/kudu/fs/block_manager-stress-test.cc
index 458f3fe..30cc6a9 100644
--- a/src/kudu/fs/block_manager-stress-test.cc
+++ b/src/kudu/fs/block_manager-stress-test.cc
@@ -43,10 +43,13 @@ DECLARE_int64(block_manager_max_open_files);
 DECLARE_uint64(log_container_max_size);
 DECLARE_uint64(log_container_preallocate_bytes);
 
-DEFINE_int32(test_duration_secs, 2, "Number of seconds to run the test");
+DEFINE_double(test_duration_secs, 2, "Number of seconds to run the test");
 DEFINE_int32(num_writer_threads, 4, "Number of writer threads to run");
 DEFINE_int32(num_reader_threads, 8, "Number of reader threads to run");
 DEFINE_int32(num_deleter_threads, 1, "Number of deleter threads to run");
+DEFINE_int32(minimum_live_blocks_for_delete, 1000,
+             "If there are fewer than this number of live blocks, the deleter "
+             "threads will not delete any");
 DEFINE_int32(block_group_size, 8, "Number of blocks to write per block "
              "group. Must be power of 2");
 DEFINE_int32(block_group_bytes, 32 * 1024,
@@ -104,13 +107,13 @@ class BlockManagerStressTest : public KuduTest {
     FLAGS_log_container_preallocate_bytes = 1 * 1024 * 1024;
 
     // Ensure the file cache is under stress too.
-    FLAGS_block_manager_max_open_files = 512;
+    FLAGS_block_manager_max_open_files = 32;
 
     // Maximize the amount of cleanup triggered by the extra space heuristic.
     FLAGS_log_container_excess_space_before_cleanup_fraction = 0.0;
 
     // Compact block manager metadata aggressively.
-    FLAGS_log_container_live_metadata_before_compact_ratio = 0.80;
+    FLAGS_log_container_live_metadata_before_compact_ratio = 0.99;
 
     if (FLAGS_block_manager_paths.empty()) {
       data_dirs_.push_back(test_dir_);
@@ -147,7 +150,7 @@ class BlockManagerStressTest : public KuduTest {
     return new T(env_, opts);
   }
 
-  void RunTest(int secs) {
+  void RunTest(double secs) {
     LOG(INFO) << "Starting all threads";
     this->StartThreads();
     SleepFor(MonoDelta::FromSeconds(secs));
@@ -389,8 +392,15 @@ void BlockManagerStressTest<T>::DeleterThread() {
     // Grab a block at random.
     BlockId to_delete;
     {
-      std::lock_guard<simple_spinlock> l(lock_);
-      if (written_blocks_.empty()) {
+      std::unique_lock<simple_spinlock> l(lock_);
+      // If we only have a small number of live blocks, don't delete any.
+      // This ensures that, when we restart, we always have a reasonable
+      // amount of data -- otherwise the deletion threads are likely to
+      // "keep up" with the writer threads and every restart will consist
+      // of a very small number of non-dead containers.
+      if (written_blocks_.size() < FLAGS_minimum_live_blocks_for_delete) {
+        l.unlock();
+        SleepFor(MonoDelta::FromMilliseconds(10));
         continue;
       }
 
@@ -460,6 +470,8 @@ TYPED_TEST(BlockManagerStressTest, StressTest) {
   OverrideFlagForSlowTests("block_group_size", "16");
   OverrideFlagForSlowTests("num_inconsistencies", "128");
 
+  const int kNumStarts = 3;
+
   if ((FLAGS_block_group_size & (FLAGS_block_group_size - 1)) != 0) {
     LOG(FATAL) << "block_group_size " << FLAGS_block_group_size
                << " is not a power of 2";
@@ -469,16 +481,19 @@ TYPED_TEST(BlockManagerStressTest, StressTest) {
 
   LOG(INFO) << "Running on fresh block manager";
   checker.Start();
-  this->RunTest(FLAGS_test_duration_secs / 2);
+  this->RunTest(FLAGS_test_duration_secs / kNumStarts);
   NO_FATALS(this->InjectNonFatalInconsistencies());
-  LOG(INFO) << "Running on populated block manager";
-  this->bm_.reset(this->CreateBlockManager());
-  FsReport report;
-  ASSERT_OK(this->bm_->Open(&report));
-  ASSERT_OK(this->bm_->dd_manager()->LoadDataDirGroupFromPB(this->test_tablet_name_,
-                                                            this->test_group_pb_));
-  ASSERT_OK(report.LogAndCheckForFatalErrors());
-  this->RunTest(FLAGS_test_duration_secs / 2);
+
+  for (int i = 1; i < kNumStarts; i++) {
+    LOG(INFO) << "Running on populated block manager (restart #" << i << ")";
+    this->bm_.reset(this->CreateBlockManager());
+    FsReport report;
+    ASSERT_OK(this->bm_->Open(&report));
+    ASSERT_OK(this->bm_->dd_manager()->LoadDataDirGroupFromPB(this->test_tablet_name_,
+                                                              this->test_group_pb_));
+    ASSERT_OK(report.LogAndCheckForFatalErrors());
+    this->RunTest(FLAGS_test_duration_secs / kNumStarts);
+  }
   checker.Stop();
 
   LOG(INFO) << "Printing test totals";

http://git-wip-us.apache.org/repos/asf/kudu/blob/e77538b5/src/kudu/fs/log_block_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/log_block_manager-test.cc b/src/kudu/fs/log_block_manager-test.cc
index f8f155a..f3f1cfe 100644
--- a/src/kudu/fs/log_block_manager-test.cc
+++ b/src/kudu/fs/log_block_manager-test.cc
@@ -44,6 +44,8 @@ using std::unordered_set;
 using std::vector;
 using strings::Substitute;
 
+DECLARE_int64(block_manager_max_open_files);
+DECLARE_bool(cache_force_single_shard);
 DECLARE_double(log_container_excess_space_before_cleanup_fraction);
 DECLARE_double(log_container_live_metadata_before_compact_ratio);
 DECLARE_int64(log_container_max_blocks);
@@ -1198,5 +1200,59 @@ TEST_F(LogBlockManagerTest, TestCompactFullContainerMetadataAtStartup) {
   ASSERT_EQ(last_live_aligned_bytes, report.stats.live_block_bytes_aligned);
 }
 
+// Regression test for a bug in which, after a metadata file was compacted,
+// we would not properly handle appending to the new (post-compaction) metadata.
+//
+// The bug was related to a stale file descriptor left in the file_cache, so
+// this test explicitly targets that scenario.
+TEST_F(LogBlockManagerTest, TestDeleteFromContainerAfterMetadataCompaction) {
+  // Compact aggressively.
+  FLAGS_log_container_live_metadata_before_compact_ratio = 0.99;
+  // Use a small file cache (smaller than the number of containers).
+  FLAGS_block_manager_max_open_files = 50;
+  // Use a single shard so that we have an accurate max cache capacity
+  // regardless of the number of cores on the machine.
+  FLAGS_cache_force_single_shard = true;
+  // Use very small containers, so that we generate a lot of them (and thus
+  // consume a lot of file descriptors).
+  FLAGS_log_container_max_blocks = 4;
+  // Reopen so the flags take effect.
+  ASSERT_OK(ReopenBlockManager(nullptr));
+
+  // Create many container with a bunch of blocks, half of which are deleted.
+  vector<BlockId> block_ids;
+  for (int i = 0; i < 1000; i++) {
+    unique_ptr<WritableBlock> block;
+    ASSERT_OK(bm_->CreateBlock(test_block_opts_, &block));
+    ASSERT_OK(block->Close());
+    if (i % 2 == 1) {
+      ASSERT_OK(bm_->DeleteBlock(block->id()));
+    } else {
+      block_ids.emplace_back(block->id());
+    }
+  }
+
+  // Reopen the block manager. This will cause it to compact all of the metadata
+  // files, since we've deleted half the blocks in every container and the
+  // threshold is set high above.
+  FsReport report;
+  ASSERT_OK(ReopenBlockManager(&report));
+
+  // Delete the remaining blocks in a random order. This will append to metadata
+  // files which have just been compacted. Since we have more metadata files than
+  // we have file_cache capacity, this will also generate a mix of cache hits,
+  // misses, and re-insertions.
+  std::random_shuffle(block_ids.begin(), block_ids.end());
+  for (const BlockId& b : block_ids) {
+    ASSERT_OK(bm_->DeleteBlock(b));
+  }
+
+  // Reopen to make sure that the metadata can be properly loaded and
+  // that the resulting block manager is empty.
+  ASSERT_OK(ReopenBlockManager(&report));
+  ASSERT_EQ(0, report.stats.live_block_count);
+  ASSERT_EQ(0, report.stats.live_block_bytes_aligned);
+}
+
 } // namespace fs
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/e77538b5/src/kudu/fs/log_block_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/log_block_manager.cc b/src/kudu/fs/log_block_manager.cc
index 4579b7f..c73bd49 100644
--- a/src/kudu/fs/log_block_manager.cc
+++ b/src/kudu/fs/log_block_manager.cc
@@ -2255,9 +2255,8 @@ Status LogBlockManager::Repair(
 
     // Rewrite this metadata file. Failures are non-fatal.
     int64_t file_bytes_delta;
-    Status s = RewriteMetadataFile(StrCat(e.first, kContainerMetadataFileSuffix),
-                                   e.second,
-                                   &file_bytes_delta);
+    const auto& meta_path = StrCat(e.first, kContainerMetadataFileSuffix);
+    Status s = RewriteMetadataFile(meta_path, e.second, &file_bytes_delta);
     if (!s.ok()) {
       WARN_NOT_OK(s, "could not rewrite metadata file");
       continue;
@@ -2269,6 +2268,9 @@ Status LogBlockManager::Repair(
 
     metadata_files_compacted++;
     metadata_bytes_delta += file_bytes_delta;
+    VLOG(1) << "Compacted metadata file " << meta_path
+            << " (saved " << file_bytes_delta << " bytes)";
+
   }
 
   // The data directory can be synchronized once for all of the new metadata files.
@@ -2322,6 +2324,10 @@ Status LogBlockManager::RewriteMetadataFile(const string& metadata_file_name,
                         "could not get file size of temporary metadata file");
   RETURN_NOT_OK_PREPEND(env_->RenameFile(tmp_file_name, metadata_file_name),
                         "could not rename temporary metadata file");
+  // Evict the old path from the file cache, so that when we re-open the new
+  // metadata file for write, we don't accidentally get a cache hit on the
+  // old file descriptor pointing to the now-deleted old version.
+  file_cache_.Invalidate(metadata_file_name);
 
   tmp_deleter.Cancel();
   *file_bytes_delta = (static_cast<int64_t>(old_metadata_size) - new_metadata_size);

http://git-wip-us.apache.org/repos/asf/kudu/blob/e77538b5/src/kudu/util/file_cache-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/file_cache-test.cc b/src/kudu/util/file_cache-test.cc
index 5ac568e..1752bd2 100644
--- a/src/kudu/util/file_cache-test.cc
+++ b/src/kudu/util/file_cache-test.cc
@@ -223,6 +223,40 @@ TYPED_TEST(FileCacheTest, TestDeletion) {
   ASSERT_EQ(this->initial_open_fds_, CountOpenFds(this->env_));
 }
 
+TYPED_TEST(FileCacheTest, TestInvalidation) {
+  const string kFile1 = this->GetTestPath("foo");
+  const string kData1 = "test data 1";
+  ASSERT_OK(this->WriteTestFile(kFile1, kData1));
+
+  // Open the file.
+  shared_ptr<TypeParam> f;
+  ASSERT_OK(this->cache_->OpenExistingFile(kFile1, &f));
+
+  // Write a new file and rename it in place on top of file1.
+  const string kFile2 = this->GetTestPath("foo2");
+  const string kData2 = "test data 2 (longer than original)";
+  ASSERT_OK(this->WriteTestFile(kFile2, kData2));
+  ASSERT_OK(this->env_->RenameFile(kFile2, kFile1));
+
+  // We should still be able to access the file, since it has a cached fd.
+  uint64_t size;
+  ASSERT_OK(f->Size(&size));
+  ASSERT_EQ(kData1.size(), size);
+
+  // If we invalidate it from the cache and try again, it should crash because
+  // the existing descriptor was invalidated.
+  this->cache_->Invalidate(kFile1);
+  ASSERT_DEATH({ f->Size(&size); }, "invalidated");
+
+  // But if we re-open the path again, the new descriptor should read the
+  // new data.
+  shared_ptr<TypeParam> f2;
+  ASSERT_OK(this->cache_->OpenExistingFile(kFile1, &f2));
+  ASSERT_OK(f2->Size(&size));
+  ASSERT_EQ(kData2.size(), size);
+}
+
+
 TYPED_TEST(FileCacheTest, TestHeavyReads) {
   const int kNumFiles = 20;
   const int kNumIterations = 100;

http://git-wip-us.apache.org/repos/asf/kudu/blob/e77538b5/src/kudu/util/file_cache.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/file_cache.cc b/src/kudu/util/file_cache.cc
index b8d17cf..07babad 100644
--- a/src/kudu/util/file_cache.cc
+++ b/src/kudu/util/file_cache.cc
@@ -17,6 +17,7 @@
 
 #include "kudu/util/file_cache.h"
 
+#include <atomic>
 #include <memory>
 #include <mutex>
 #include <string>
@@ -96,7 +97,7 @@ class BaseDescriptor {
     // the next call to RunDescriptorExpiry(). Removing it here would risk a
     // deadlock on recursive acquisition of 'lock_'.
 
-    if (deleted_) {
+    if (deleted()) {
       cache()->Erase(filename());
 
       VLOG(1) << "Deleting file: " << filename();
@@ -136,8 +137,21 @@ class BaseDescriptor {
 
   // Mark this descriptor as to-be-deleted later.
   void MarkDeleted() {
-    DCHECK(!deleted_);
-    deleted_ = true;
+    DCHECK(!deleted());
+    while (true) {
+      auto v = flags_.load();
+      if (flags_.compare_exchange_weak(v, v | FILE_DELETED)) return;
+    }
+  }
+
+  // Mark this descriptor as invalidated. No further access is allowed
+  // to this file.
+  void MarkInvalidated() {
+    DCHECK(!invalidated());
+    while (true) {
+      auto v = flags_.load();
+      if (flags_.compare_exchange_weak(v, v | INVALIDATED)) return;
+    }
   }
 
   Cache* cache() const { return file_cache_->cache_.get(); }
@@ -146,13 +160,17 @@ class BaseDescriptor {
 
   const string& filename() const { return file_name_; }
 
-  bool deleted() const { return deleted_; }
+  bool deleted() const { return flags_.load() & FILE_DELETED; }
+  bool invalidated() const { return flags_.load() & INVALIDATED; }
 
  private:
   FileCache<FileType>* file_cache_;
   const string file_name_;
-
-  bool deleted_ = false;
+  enum Flags {
+    FILE_DELETED = 1 << 0,
+    INVALIDATED = 1 << 1
+  };
+  std::atomic<uint8_t> flags_ {0};
 
   DISALLOW_COPY_AND_ASSIGN(BaseDescriptor);
 };
@@ -296,6 +314,7 @@ class Descriptor<RWFile> : public RWFile {
 
   Status ReopenFileIfNecessary(ScopedOpenedDescriptor<RWFile>* out) const {
     ScopedOpenedDescriptor<RWFile> found(base_.LookupFromCache());
+    CHECK(!base_.invalidated());
     if (found.opened()) {
       // The file is already open in the cache, return it.
       if (out) {
@@ -398,6 +417,7 @@ class Descriptor<RandomAccessFile> : public RandomAccessFile {
   Status ReopenFileIfNecessary(
       ScopedOpenedDescriptor<RandomAccessFile>* out) const {
     ScopedOpenedDescriptor<RandomAccessFile> found(base_.LookupFromCache());
+    CHECK(!base_.invalidated());
     if (found.opened()) {
       // The file is already open in the cache, return it.
       if (out) {
@@ -479,7 +499,7 @@ Status FileCache<FileType>::OpenExistingFile(const string& file_name,
   // Check that the underlying file can be opened (no-op for found
   // descriptors). Done outside the lock.
   RETURN_NOT_OK(desc->Init());
-  *file = desc;
+  *file = std::move(desc);
   return Status::OK();
 }
 
@@ -506,6 +526,42 @@ Status FileCache<FileType>::DeleteFile(const string& file_name) {
 }
 
 template <class FileType>
+void FileCache<FileType>::Invalidate(const string& file_name) {
+  // Ensure that there is an invalidated descriptor in the map for this filename.
+  //
+  // This ensures that any concurrent OpenExistingFile() during this method wil
+  // see the invalidation and issue a CHECK failure.
+  shared_ptr<internal::Descriptor<FileType>> desc;
+  {
+    // Find an existing descriptor, or create one if none exists.
+    std::lock_guard<simple_spinlock> l(lock_);
+    auto it = descriptors_.find(file_name);
+    if (it != descriptors_.end()) {
+      desc = it->second.lock();
+    }
+    if (!desc) {
+      desc = std::make_shared<internal::Descriptor<FileType>>(this, file_name);
+      descriptors_.emplace(file_name, desc);
+    }
+
+    desc->base_.MarkInvalidated();
+  }
+  // Remove it from the cache so that if the same path is opened again, we
+  // will re-open a new FD rather than retrieving one that might have been
+  // cached prior to invalidation.
+  cache_->Erase(file_name);
+
+  // Remove the invalidated descriptor from the map. We are guaranteed it
+  // is still there because we've held a strong reference to it for
+  // the duration of this method, and no other methods erase strong
+  // references from the map.
+  {
+    std::lock_guard<simple_spinlock> l(lock_);
+    CHECK_EQ(1, descriptors_.erase(file_name));
+  }
+}
+
+template <class FileType>
 int FileCache<FileType>::NumDescriptorsForTests() const {
   std::lock_guard<simple_spinlock> l(lock_);
   return descriptors_.size();
@@ -552,6 +608,7 @@ Status FileCache<FileType>::FindDescriptorUnlocked(
     // Found the descriptor. Has it expired?
     shared_ptr<internal::Descriptor<FileType>> desc = it->second.lock();
     if (desc) {
+      CHECK(!desc->base_.invalidated());
       if (desc->base_.deleted()) {
         return Status::NotFound("File already marked for deletion", file_name);
       }
@@ -584,46 +641,7 @@ void FileCache<FileType>::RunDescriptorExpiry() {
 }
 
 // Explicit specialization for callers outside this compilation unit.
-template
-FileCache<RWFile>::FileCache(
-    const string& cache_name,
-    Env* env,
-    int max_open_files,
-    const scoped_refptr<MetricEntity>& entity);
-template
-FileCache<RWFile>::~FileCache();
-template
-Status FileCache<RWFile>::Init();
-template
-Status FileCache<RWFile>::OpenExistingFile(
-    const string& file_name,
-    shared_ptr<RWFile>* file);
-template
-Status FileCache<RWFile>::DeleteFile(const string& file_name);
-template
-int FileCache<RWFile>::NumDescriptorsForTests() const;
-template
-string FileCache<RWFile>::ToDebugString() const;
-
-template
-FileCache<RandomAccessFile>::FileCache(
-    const string& cache_name,
-    Env* env,
-    int max_open_files,
-    const scoped_refptr<MetricEntity>& entity);
-template
-FileCache<RandomAccessFile>::~FileCache();
-template
-Status FileCache<RandomAccessFile>::Init();
-template
-Status FileCache<RandomAccessFile>::OpenExistingFile(
-    const string& file_name,
-    shared_ptr<RandomAccessFile>* file);
-template
-Status FileCache<RandomAccessFile>::DeleteFile(const string& file_name);
-template
-int FileCache<RandomAccessFile>::NumDescriptorsForTests() const;
-template
-string FileCache<RandomAccessFile>::ToDebugString() const;
+template class FileCache<RWFile>;
+template class FileCache<RandomAccessFile>;
 
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/e77538b5/src/kudu/util/file_cache.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/file_cache.h b/src/kudu/util/file_cache.h
index 3ef87ad..44330d3 100644
--- a/src/kudu/util/file_cache.h
+++ b/src/kudu/util/file_cache.h
@@ -130,6 +130,29 @@ class FileCache {
   // deleted immediately.
   Status DeleteFile(const std::string& file_name);
 
+  // Invalidate the given path in the cache if present. This removes the
+  // path from the cache, and invalidates any previously-opened descriptors
+  // associated with this file.
+  //
+  // If a file with the same path is opened again, the actual path will be opened from
+  // disk.
+  //
+  // This operation should be used during 'rename-to-replace' patterns, eg:
+  //
+  //    WriteNewDataTo(tmp_path);
+  //    env->RenameFile(tmp_path, p);
+  //    file_cache->Invalidate(p);
+  //
+  // NOTE: if any reader of 'p' holds an open descriptor from the cache
+  // prior to this operation, that descriptor is invalidated and any
+  // further operations on that descriptor will result in a CHECK failure.
+  // Hence this is not safe to use without some external synchronization
+  // which prevents concurrent access to the same file.
+  //
+  // NOTE: this function must not be called concurrently on the same file name
+  // from multiple threads.
+  void Invalidate(const std::string& file_name);
+
   // Returns the number of entries in the descriptor map.
   //
   // Only intended for unit tests.