You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2020/01/21 22:41:08 UTC

[kudu] branch master updated (45af2ad -> 0f22f12)

This is an automated email from the ASF dual-hosted git repository.

adar pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git.


    from 45af2ad  [spark] Replace bad Guava import
     new 07e3622  tablet copy client: delete WAL data using existing function
     new 37c3a96  compression: fix handling of NO_COMPRESSION
     new ceeabc5  rpc-test-base.h: squelch a warning
     new 0f22f12  file cache: evict open fd when descriptor goes out of scope

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/kudu/cfile/cfile_writer.cc                         |  8 ++------
 src/kudu/consensus/log-test.cc                         |  4 ++--
 src/kudu/consensus/log.cc                              | 11 +++--------
 src/kudu/integration-tests/disk_reservation-itest.cc   |  2 +-
 .../integration-tests/raft_consensus-itest-base.cc     |  3 +--
 src/kudu/integration-tests/raft_consensus-itest.cc     |  4 ++--
 src/kudu/integration-tests/ts_recovery-itest.cc        |  2 +-
 src/kudu/rpc/rpc-test-base.h                           |  4 +++-
 src/kudu/tserver/tablet_copy_client.cc                 | 11 +++++------
 src/kudu/tserver/tablet_copy_client.h                  |  6 ++++--
 src/kudu/util/compression/compression_codec.cc         |  2 +-
 src/kudu/util/file_cache-test.cc                       | 12 ++++++------
 src/kudu/util/file_cache.cc                            | 18 +++++++++++++-----
 src/kudu/util/file_cache.h                             | 10 +++++-----
 14 files changed, 49 insertions(+), 48 deletions(-)


[kudu] 02/04: compression: fix handling of NO_COMPRESSION

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

adar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 37c3a960e9b0f86069876f049e9a073d4b5fd907
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Mon Jan 20 00:43:07 2020 -0800

    compression: fix handling of NO_COMPRESSION
    
    The string values of CompressionType and GetCompressionCodecType() did not
    agree: the former used NO_COMPRESSION and the latter NONE to indicate the
    lack of compression. This led to some unnecessary warnings when a
    stringified CompressionType was fed into GetCompressionCodecType(), as is
    done in log-test.
    
    This patch changes GetCompressionCodecType() to expect NO_COMPRESSION rather
    than NONE. It shouldn't affect backwards compatibility: if someone really
    does use NONE (i.e. in a gflag), they'll just get no compression anyway,
    albeit with the ugly warning. That's not ideal, but the alternative (use
    NONE in CompressionType) may break backwards compatibility in JSON encoding,
    and NO_COMPRESSION is the value we use in our public APIs.
    
    Change-Id: I900458b7c7ed4be02906479becaaf60bad379029
    Reviewed-on: http://gerrit.cloudera.org:8080/15078
    Tested-by: Kudu Jenkins
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
---
 src/kudu/cfile/cfile_writer.cc                          |  8 ++------
 src/kudu/consensus/log-test.cc                          |  4 ++--
 src/kudu/consensus/log.cc                               | 11 +++--------
 src/kudu/integration-tests/disk_reservation-itest.cc    |  2 +-
 src/kudu/integration-tests/raft_consensus-itest-base.cc |  3 +--
 src/kudu/integration-tests/raft_consensus-itest.cc      |  4 ++--
 src/kudu/integration-tests/ts_recovery-itest.cc         |  2 +-
 src/kudu/util/compression/compression_codec.cc          |  2 +-
 8 files changed, 13 insertions(+), 23 deletions(-)

diff --git a/src/kudu/cfile/cfile_writer.cc b/src/kudu/cfile/cfile_writer.cc
index 5769e10..aa6f85d 100644
--- a/src/kudu/cfile/cfile_writer.cc
+++ b/src/kudu/cfile/cfile_writer.cc
@@ -52,7 +52,7 @@
 DEFINE_int32(cfile_default_block_size, 256*1024, "The default block size to use in cfiles");
 TAG_FLAG(cfile_default_block_size, advanced);
 
-DEFINE_string(cfile_default_compression_codec, "none",
+DEFINE_string(cfile_default_compression_codec, "no_compression",
               "Default cfile block compression codec.");
 TAG_FLAG(cfile_default_compression_codec, advanced);
 
@@ -80,10 +80,6 @@ const size_t kChecksumSize = sizeof(uint32_t);
 
 static const size_t kMinBlockSize = 512;
 
-static CompressionType GetDefaultCompressionCodec() {
-  return GetCompressionCodecType(FLAGS_cfile_default_compression_codec);
-}
-
 ////////////////////////////////////////////////////////////
 // CFileWriter
 ////////////////////////////////////////////////////////////
@@ -114,7 +110,7 @@ CFileWriter::CFileWriter(WriterOptions options,
 
   compression_ = options_.storage_attributes.compression;
   if (compression_ == DEFAULT_COMPRESSION) {
-    compression_ = GetDefaultCompressionCodec();
+    compression_ = GetCompressionCodecType(FLAGS_cfile_default_compression_codec);
   }
 
   if (options_.storage_attributes.cfile_block_size <= 0) {
diff --git a/src/kudu/consensus/log-test.cc b/src/kudu/consensus/log-test.cc
index 4a68189..653ed00 100644
--- a/src/kudu/consensus/log-test.cc
+++ b/src/kudu/consensus/log-test.cc
@@ -430,7 +430,7 @@ TEST_P(LogTestOptionalCompression, TestSegmentRollover) {
 }
 
 TEST_F(LogTest, TestWriteAndReadToAndFromInProgressSegment) {
-  FLAGS_log_compression_codec = "none";
+  FLAGS_log_compression_codec = "no_compression";
 
   const int kNumEntries = 4;
   ASSERT_OK(BuildLog());
@@ -1054,7 +1054,7 @@ TEST_P(LogTestOptionalCompression, TestReadReplicatesHighIndex) {
 // Test various situations where we expect different segments depending on what the
 // min log index is.
 TEST_F(LogTest, TestGetGCableDataSize) {
-  FLAGS_log_compression_codec = "none";
+  FLAGS_log_compression_codec = "no_compression";
   FLAGS_log_min_segments_to_retain = 2;
   ASSERT_OK(BuildLog());
 
diff --git a/src/kudu/consensus/log.cc b/src/kudu/consensus/log.cc
index 54400d2..429036f 100644
--- a/src/kudu/consensus/log.cc
+++ b/src/kudu/consensus/log.cc
@@ -42,7 +42,6 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/walltime.h"
 #include "kudu/util/async_util.h"
-#include "kudu/util/compression/compression.pb.h"
 #include "kudu/util/compression/compression_codec.h"
 #include "kudu/util/debug/trace_event.h"
 #include "kudu/util/env.h"
@@ -448,13 +447,9 @@ Status SegmentAllocator::Init(
     uint64_t sequence_number,
     scoped_refptr<ReadableLogSegment>* new_readable_segment) {
   // Init the compression codec.
-  if (!FLAGS_log_compression_codec.empty()) {
-    auto codec_type = GetCompressionCodecType(FLAGS_log_compression_codec);
-    if (codec_type != NO_COMPRESSION) {
-      RETURN_NOT_OK_PREPEND(GetCompressionCodec(codec_type, &codec_),
-                            "could not instantiate compression codec");
-    }
-  }
+  RETURN_NOT_OK_PREPEND(GetCompressionCodec(
+      GetCompressionCodecType(FLAGS_log_compression_codec), &codec_),
+                        "could not instantiate compression codec");
   active_segment_sequence_number_ = sequence_number;
   RETURN_NOT_OK(ThreadPoolBuilder("log-alloc")
       .set_max_threads(1)
diff --git a/src/kudu/integration-tests/disk_reservation-itest.cc b/src/kudu/integration-tests/disk_reservation-itest.cc
index 5f42295..90a2adc 100644
--- a/src/kudu/integration-tests/disk_reservation-itest.cc
+++ b/src/kudu/integration-tests/disk_reservation-itest.cc
@@ -131,7 +131,7 @@ TEST_F(DiskReservationITest, TestWalWriteToFullDiskAborts) {
     "--disable_core_dumps",
     // Disable compression so that our data being written doesn't end up
     // compressed away.
-    "--log_compression_codec=none"
+    "--log_compression_codec=no_compression"
   };
   NO_FATALS(StartCluster(ts_flags, {}, 1));
 
diff --git a/src/kudu/integration-tests/raft_consensus-itest-base.cc b/src/kudu/integration-tests/raft_consensus-itest-base.cc
index b357dcc..ef28531 100644
--- a/src/kudu/integration-tests/raft_consensus-itest-base.cc
+++ b/src/kudu/integration-tests/raft_consensus-itest-base.cc
@@ -26,7 +26,6 @@
 #include <vector>
 
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
@@ -181,7 +180,7 @@ void RaftConsensusITestBase::AddFlagsForLogRolls(vector<string>* extra_tserver_f
   //
   // Additionally, we disable log compression, since these tests write a lot of
   // repetitive data to cause the rolls, and compression would make it all tiny.
-  extra_tserver_flags->push_back("--log_compression_codec=none");
+  extra_tserver_flags->push_back("--log_compression_codec=no_compression");
   extra_tserver_flags->push_back("--log_cache_size_limit_mb=1");
   extra_tserver_flags->push_back("--log_segment_size_mb=1");
   extra_tserver_flags->push_back("--log_async_preallocate_segments=false");
diff --git a/src/kudu/integration-tests/raft_consensus-itest.cc b/src/kudu/integration-tests/raft_consensus-itest.cc
index 9072b2d..dfe2940 100644
--- a/src/kudu/integration-tests/raft_consensus-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus-itest.cc
@@ -815,7 +815,7 @@ TEST_F(RaftConsensusITest, TestCatchupAfterOpsEvicted) {
     // We write 128KB cells in this test, so bump the limit.
     "--max_cell_size_bytes=1000000",
     // And disable WAL compression so the 128KB cells don't get compressed away.
-    "--log_compression_codec=none"
+    "--log_compression_codec=no_compression"
   };
 
   NO_FATALS(BuildAndStart(kTsFlags));
@@ -2140,7 +2140,7 @@ TEST_F(RaftConsensusITest, TestLargeBatches) {
     // We write 128KB cells in this test, so bump the limit, and disable compression.
     "--max_cell_size_bytes=1000000",
     "--log_segment_size_mb=1",
-    "--log_compression_codec=none",
+    "--log_compression_codec=no_compression",
     "--log_min_segments_to_retain=100", // disable GC of logs.
   };
 
diff --git a/src/kudu/integration-tests/ts_recovery-itest.cc b/src/kudu/integration-tests/ts_recovery-itest.cc
index 6be498e..36b9ea8 100644
--- a/src/kudu/integration-tests/ts_recovery-itest.cc
+++ b/src/kudu/integration-tests/ts_recovery-itest.cc
@@ -153,7 +153,7 @@ TEST_F(TsRecoveryITest, TestTabletRecoveryAfterSegmentDelete) {
   vector<string> flags;
   flags.emplace_back("--log_segment_size_mb=1");
   flags.emplace_back("--log_min_segments_to_retain=3");
-  flags.emplace_back("--log_compression_codec=''");
+  flags.emplace_back("--log_compression_codec=no_compression");
   NO_FATALS(StartCluster(flags));
 
   const int kNumTablets = 1;
diff --git a/src/kudu/util/compression/compression_codec.cc b/src/kudu/util/compression/compression_codec.cc
index b927d48..dc69311 100644
--- a/src/kudu/util/compression/compression_codec.cc
+++ b/src/kudu/util/compression/compression_codec.cc
@@ -276,7 +276,7 @@ CompressionType GetCompressionCodecType(const std::string& name) {
     return LZ4;
   if (uname == "ZLIB")
     return ZLIB;
-  if (uname == "NONE")
+  if (uname == "NO_COMPRESSION")
     return NO_COMPRESSION;
 
   LOG(WARNING) << "Unable to recognize the compression codec '" << name


[kudu] 01/04: tablet copy client: delete WAL data using existing function

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

adar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 07e36221b8d8b3ce48690e40181a51a6953323b8
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Sun Jan 19 16:09:59 2020 -0800

    tablet copy client: delete WAL data using existing function
    
    I thought this might be needed for correctness (if Log::DeleteOnDiskData did
    something beyond a recursive deletion), but it's not. Nevertheless, it seems
    more robust, if DeleteOnDiskData _did_ were changed to do something
    interesting in the future.
    
    Change-Id: Ic18d11a3d857ab9a29a82d9f7ebe72f33737e25d
    Reviewed-on: http://gerrit.cloudera.org:8080/15077
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Tested-by: Kudu Jenkins
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
---
 src/kudu/tserver/tablet_copy_client.cc | 11 +++++------
 src/kudu/tserver/tablet_copy_client.h  |  6 ++++--
 2 files changed, 9 insertions(+), 8 deletions(-)

diff --git a/src/kudu/tserver/tablet_copy_client.cc b/src/kudu/tserver/tablet_copy_client.cc
index a80993b..d5e4a2c 100644
--- a/src/kudu/tserver/tablet_copy_client.cc
+++ b/src/kudu/tserver/tablet_copy_client.cc
@@ -24,8 +24,8 @@
 
 #include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
+#include <google/protobuf/stubs/port.h>
 
 #include "kudu/common/common.pb.h"
 #include "kudu/common/partition.h"
@@ -33,6 +33,7 @@
 #include "kudu/common/wire_protocol.h"
 #include "kudu/consensus/consensus_meta.h"
 #include "kudu/consensus/consensus_meta_manager.h"
+#include "kudu/consensus/log.h"
 #include "kudu/consensus/metadata.pb.h"
 #include "kudu/consensus/opid.pb.h"
 #include "kudu/consensus/opid_util.h"
@@ -148,7 +149,8 @@ TabletCopyClientMetrics::TabletCopyClientMetrics(const scoped_refptr<MetricEntit
       open_client_sessions(METRIC_tablet_copy_open_client_sessions.Instantiate(metric_entity, 0)) {
 }
 
-TabletCopyClient::TabletCopyClient(std::string tablet_id,
+TabletCopyClient::TabletCopyClient(
+    std::string tablet_id,
     FsManager* fs_manager,
     scoped_refptr<ConsensusMetadataManager> cmeta_manager,
     shared_ptr<Messenger> messenger,
@@ -159,7 +161,6 @@ TabletCopyClient::TabletCopyClient(std::string tablet_id,
       messenger_(std::move(messenger)),
       state_(kInitialized),
       replace_tombstoned_tablet_(false),
-      tablet_replica_(nullptr),
       session_idle_timeout_millis_(FLAGS_tablet_copy_begin_session_timeout_ms),
       start_time_micros_(0),
       rng_(GetRandomSeed32()),
@@ -532,10 +533,8 @@ Status TabletCopyClient::DownloadWALs() {
 
   // Delete and recreate WAL dir if it already exists, to ensure stray files are
   // not kept from previous copies and runs.
+  RETURN_NOT_OK(log::Log::DeleteOnDiskData(fs_manager_, tablet_id_));
   string path = fs_manager_->GetTabletWalDir(tablet_id_);
-  if (fs_manager_->env()->FileExists(path)) {
-    RETURN_NOT_OK(fs_manager_->env()->DeleteRecursively(path));
-  }
   RETURN_NOT_OK(fs_manager_->env()->CreateDir(path));
   RETURN_NOT_OK(fs_manager_->env()->SyncDir(DirName(path))); // fsync() parent dir.
 
diff --git a/src/kudu/tserver/tablet_copy_client.h b/src/kudu/tserver/tablet_copy_client.h
index c2adf6a..27ed996 100644
--- a/src/kudu/tserver/tablet_copy_client.h
+++ b/src/kudu/tserver/tablet_copy_client.h
@@ -80,8 +80,10 @@ class TabletCopyClient {
  public:
 
   // Construct the tablet copy client.
-  // 'fs_manager' and 'messenger' must remain valid until this object is destroyed.
-  TabletCopyClient(std::string tablet_id, FsManager* fs_manager,
+  //
+  // Objects behind raw pointers must remain valid until this object is destroyed.
+  TabletCopyClient(std::string tablet_id,
+                   FsManager* fs_manager,
                    scoped_refptr<consensus::ConsensusMetadataManager> cmeta_manager,
                    std::shared_ptr<rpc::Messenger> messenger,
                    TabletCopyClientMetrics* tablet_copy_metrics);


[kudu] 03/04: rpc-test-base.h: squelch a warning

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

adar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit ceeabc5460bf4d791441bc1b2d37347475c1b399
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Mon Jan 20 01:14:54 2020 -0800

    rpc-test-base.h: squelch a warning
    
    Change-Id: I4ae8ebb0cb49e32f08f5c3f0c0f5061b029e0638
    Reviewed-on: http://gerrit.cloudera.org:8080/15079
    Tested-by: Kudu Jenkins
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
---
 src/kudu/rpc/rpc-test-base.h | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/src/kudu/rpc/rpc-test-base.h b/src/kudu/rpc/rpc-test-base.h
index 0b6f047..fcc7452 100644
--- a/src/kudu/rpc/rpc-test-base.h
+++ b/src/kudu/rpc/rpc-test-base.h
@@ -559,7 +559,9 @@ class RpcTestBase : public KuduTest {
 
     // We shouldn't timeout significantly faster than our configured timeout, unless the
     // rpc is cancelled.
-    if (!will_be_cancelled) EXPECT_GE(elapsed_millis, expected_millis - 10);
+    if (!will_be_cancelled) {
+      EXPECT_GE(elapsed_millis, expected_millis - 10);
+    }
     // And we also shouldn't take the full time that we asked for
     EXPECT_LT(elapsed_millis * 1000, sleep_micros);
     if (will_be_cancelled) {


[kudu] 04/04: file cache: evict open fd when descriptor goes out of scope

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

adar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 0f22f1268c4fda855456d6245c28024daf881d00
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Sun Jan 19 15:54:25 2020 -0800

    file cache: evict open fd when descriptor goes out of scope
    
    Previously, an open fd remained in cache when its descriptor was destroyed
    unless the file was marked for deletion. This avoided an extra cache lookup
    at destruction time and provided faster access to a file if it was closed
    and then reopened (though that second benefit was irrelevant to the block
    managers where files were kept open permanently unless they were deleted).
    
    With this change, an open fd is forcefully evicted when its descriptor is
    destroyed. This provides better semantics if the goal is to close a file
    without deleting it and test that the fd is indeed closed. It also prevents
    "false positive" cache hits when the user closes the file, renames it, and
    creates a new file with the old file name. This is an access pattern used by
    the WAL when a replica fails and it is brought back to life via tablet copy
    from a healthy server.
    
    Change-Id: Iea5317add630753716ef538cc8a198c9b3547822
    Reviewed-on: http://gerrit.cloudera.org:8080/15080
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Tested-by: Adar Dembo <ad...@cloudera.com>
---
 src/kudu/util/file_cache-test.cc | 12 ++++++------
 src/kudu/util/file_cache.cc      | 18 +++++++++++++-----
 src/kudu/util/file_cache.h       | 10 +++++-----
 3 files changed, 24 insertions(+), 16 deletions(-)

diff --git a/src/kudu/util/file_cache-test.cc b/src/kudu/util/file_cache-test.cc
index 0c2dea8..1fbfc38 100644
--- a/src/kudu/util/file_cache-test.cc
+++ b/src/kudu/util/file_cache-test.cc
@@ -175,10 +175,10 @@ TYPED_TEST(FileCacheTest, TestBasicOperations) {
     }
   }
 
-  // The descriptors are all out of scope, but the open fds remain in the cache.
-  NO_FATALS(this->AssertFdsAndDescriptors(1, 0));
+  // The descriptors are all out of scope, and so are the cached fds.
+  NO_FATALS(this->AssertFdsAndDescriptors(0, 0));
 
-  // With the cache gone, so are the cached fds.
+  // With the cache gone, nothing changes.
   this->cache_.reset();
   ASSERT_EQ(this->initial_open_fds_, this->CountOpenFds());
 }
@@ -222,8 +222,8 @@ TYPED_TEST(FileCacheTest, TestDeletion) {
   ASSERT_EQ(this->initial_open_fds_, this->CountOpenFds());
 
   // Create a test file, open it, and let it go out of scope before
-  // deleting it. The deletion should evict the fd and close it, despite
-  // happening after the descriptor is gone.
+  // deleting it. The fd is already evicted and closed; this is effectively a
+  // non-cached deletion.
   const string kFile3 = this->GetTestPath("baz");
   const string kData3 = "test data 3";
   ASSERT_OK(this->WriteTestFile(kFile3, kData3));
@@ -232,7 +232,7 @@ TYPED_TEST(FileCacheTest, TestDeletion) {
     ASSERT_OK(this->cache_->OpenExistingFile(kFile3, &f3));
   }
   ASSERT_TRUE(this->env_->FileExists(kFile3));
-  ASSERT_EQ(this->initial_open_fds_ + 1, this->CountOpenFds());
+  ASSERT_EQ(this->initial_open_fds_, this->CountOpenFds());
   ASSERT_OK(this->cache_->DeleteFile(kFile3));
   ASSERT_FALSE(this->env_->FileExists(kFile3));
   ASSERT_EQ(this->initial_open_fds_, this->CountOpenFds());
diff --git a/src/kudu/util/file_cache.cc b/src/kudu/util/file_cache.cc
index 06e8965..8fb0adf 100644
--- a/src/kudu/util/file_cache.cc
+++ b/src/kudu/util/file_cache.cc
@@ -100,16 +100,24 @@ class BaseDescriptor {
   ~BaseDescriptor() {
     VLOG(2) << "Out of scope descriptor with file name: " << filename();
 
-    // The (now expired) weak_ptr remains in 'descriptors_', to be removed by
-    // the next call to RunDescriptorExpiry(). Removing it here would risk a
-    // deadlock on recursive acquisition of 'lock_'.
+    // The destruction of the descriptor indicates that there's no active user
+    // of the file at the moment. However, if the fd is still open in the LRU
+    // cache, should we leave it there, or should we evict it?
+    //
+    // We opt for eviction: your typical filesystem user is more likely to want
+    // the underlying resource released when they're done working with a file
+    // than they are to want the resource to be quickly accessible should the
+    // file be reopened.
+    cache()->Erase(filename());
 
     if (deleted()) {
-      cache()->Erase(filename());
-
       VLOG(1) << "Deleting file: " << filename();
       WARN_NOT_OK(env()->DeleteFile(filename()), "");
     }
+
+    // The (now expired) weak_ptr remains in 'descriptors_', to be removed by
+    // the next call to RunDescriptorExpiry(). Removing it here would risk a
+    // deadlock on recursive acquisition of 'lock_'.
   }
 
   // Insert a pointer to an open file object into the file cache with the
diff --git a/src/kudu/util/file_cache.h b/src/kudu/util/file_cache.h
index e8c3250..ed3663c 100644
--- a/src/kudu/util/file_cache.h
+++ b/src/kudu/util/file_cache.h
@@ -63,7 +63,7 @@ class Thread;
 // The core of the client-facing API is the cache descriptor. A descriptor
 // uniquely identifies an opened file. To a client, a descriptor is just an
 // open file interface of the variety defined in util/env.h. Clients open
-// descriptors via the OpenExisting*() cache methods.
+// descriptors via the OpenExistingFile() cache methods.
 //
 // Descriptors are shared objects; an existing descriptor is handed back to a
 // client if a file with the same name is already opened. To facilitate
@@ -118,9 +118,9 @@ class FileCache {
   // to a file-like interface but interfaces with the cache under the hood to
   // reopen a file as needed during file operations.
   //
-  // The descriptor is opened immediately to verify that the on-disk file can
-  // be opened, but may be closed later if the cache reaches its upper bound on
-  // the number of open files.
+  // The underlying file is opened immediately to verify that it indeed exists,
+  // but may be closed later if the cache reaches its upper bound on the number
+  // of open files. It is also closed when the descriptor's last reference is dropped.
   template <class FileType>
   Status OpenExistingFile(const std::string& file_name,
                           std::shared_ptr<FileType>* file);
@@ -128,7 +128,7 @@ class FileCache {
   // Deletes a file by name through the cache.
   //
   // If there is an outstanding descriptor for the file, the deletion will be
-  // deferred until the last referent is dropped. Otherwise, the file is
+  // deferred until the last reference is dropped. Otherwise, the file is
   // deleted immediately.
   Status DeleteFile(const std::string& file_name);