You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by jr...@apache.org on 2017/01/13 23:47:58 UTC

[2/9] incubator-impala git commit: IMPALA-2615: warn if Status is ignored

IMPALA-2615: warn if Status is ignored

This introduces a WARN_UNUSED_RESULT macro. It can be used at the end
of function declarations to issue warnings if the return value of the
function is implicitly ignored by the caller. E.g. if a Status or
bool return value indicates the success/failure of a method.

It can also be prepended to function definitions (gcc doesn't allow
it to be put before the { in function definitions).

This can help find bugs. E.g. I found IMPALA-4391 by applying this
to some other places in HdfsScanner.

This commit uses the macro in a few key APIs. We can use it in more
places if we agree on the pattern.

Note:
Gcc has a bug https://gcc.gnu.org/bugzilla/show_bug.cgi?id=38172
that prevents this from being effective for many functions that
return C++ classes or structs. Clang does not have this issue.

Change-Id: I8c26e7532b5f2c7fe167accc73179e8b72b192bc
Reviewed-on: http://gerrit.cloudera.org:8080/4878
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/master
Commit: b989efbb5607f45f0873d25764b5adc21cb87535
Parents: db7facd
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Thu Oct 27 17:03:45 2016 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Fri Jan 13 03:02:03 2017 +0000

----------------------------------------------------------------------
 be/src/common/status.h                          |  5 +++
 be/src/exec/exec-node.h                         | 35 ++++++++++--------
 be/src/exec/hdfs-scan-node-base.cc              |  3 +-
 be/src/runtime/bufferpool/buffer-allocator.h    |  2 +-
 be/src/runtime/bufferpool/buffer-pool.h         | 14 ++++---
 be/src/runtime/bufferpool/reservation-tracker.h |  4 +-
 be/src/runtime/disk-io-mgr-stress.cc            |  5 +--
 be/src/runtime/disk-io-mgr-test.cc              | 32 ++++++++--------
 be/src/runtime/disk-io-mgr.cc                   |  3 +-
 be/src/runtime/disk-io-mgr.h                    | 39 +++++++++++---------
 be/src/runtime/mem-tracker.h                    |  3 +-
 be/src/runtime/test-env.cc                      |  9 +++--
 be/src/runtime/tmp-file-mgr-test.cc             | 10 ++---
 be/src/runtime/tmp-file-mgr.h                   | 34 +++++++++--------
 be/src/util/openssl-util.h                      | 10 ++---
 15 files changed, 112 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b989efbb/be/src/common/status.h
----------------------------------------------------------------------
diff --git a/be/src/common/status.h b/be/src/common/status.h
index a7f31bc..6cd9035 100644
--- a/be/src/common/status.h
+++ b/be/src/common/status.h
@@ -289,6 +289,11 @@ class Status {
     exit(1); \
   } while (false)
 
+/// This macro can be appended to a function definition to generate a compiler warning
+/// if the result is ignored.
+/// TODO: when we upgrade gcc from 4.9.2, we may be able to apply this to the Status
+/// type to get this automatically for all Status-returning functions.
+#define WARN_UNUSED_RESULT __attribute__((warn_unused_result))
 }
 
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b989efbb/be/src/exec/exec-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/exec-node.h b/be/src/exec/exec-node.h
index 6a7737b..aa0c379 100644
--- a/be/src/exec/exec-node.h
+++ b/be/src/exec/exec-node.h
@@ -58,13 +58,13 @@ class ExecNode {
   /// Initializes this object from the thrift tnode desc. The subclass should
   /// do any initialization that can fail in Init() rather than the ctor.
   /// If overridden in subclass, must first call superclass's Init().
-  virtual Status Init(const TPlanNode& tnode, RuntimeState* state);
+  virtual Status Init(const TPlanNode& tnode, RuntimeState* state) WARN_UNUSED_RESULT;
 
   /// Sets up internal structures, etc., without doing any actual work.
   /// Must be called prior to Open(). Will only be called once in this
   /// node's lifetime.
   /// If overridden in subclass, must first call superclass's Prepare().
-  virtual Status Prepare(RuntimeState* state);
+  virtual Status Prepare(RuntimeState* state) WARN_UNUSED_RESULT;
 
   /// Recursively calls Codegen() on all children.
   /// Expected to be overriden in subclass to generate LLVM IR functions and register
@@ -79,7 +79,7 @@ class ExecNode {
   /// If overridden in subclass, must first call superclass's Open().
   /// Open() is called after Prepare() or Reset(), i.e., possibly multiple times
   /// throughout the lifetime of this node.
-  virtual Status Open(RuntimeState* state);
+  virtual Status Open(RuntimeState* state) WARN_UNUSED_RESULT;
 
   /// Retrieves rows and returns them via row_batch. Sets eos to true
   /// if subsequent calls will not retrieve any more rows.
@@ -94,7 +94,8 @@ class ExecNode {
   /// row_batch's tuple_data_pool.
   /// Caller must not be holding any io buffers. This will cause deadlock.
   /// TODO: AggregationNode and HashJoinNode cannot be "re-opened" yet.
-  virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) = 0;
+  virtual Status GetNext(
+      RuntimeState* state, RowBatch* row_batch, bool* eos) WARN_UNUSED_RESULT = 0;
 
   /// Resets the stream of row batches to be retrieved by subsequent GetNext() calls.
   /// Clears all internal state, returning this node to the state it was in after calling
@@ -109,7 +110,7 @@ class ExecNode {
   /// implementation calls Reset() on children.
   /// Note that this function may be called many times (proportional to the input data),
   /// so should be fast.
-  virtual Status Reset(RuntimeState* state);
+  virtual Status Reset(RuntimeState* state) WARN_UNUSED_RESULT;
 
   /// Close() will get called for every exec node, regardless of what else is called and
   /// the status of these calls (i.e. Prepare() may never have been called, or
@@ -130,11 +131,11 @@ class ExecNode {
   /// traversal. All nodes are placed in state->obj_pool() and have Init() called on them.
   /// Returns error if 'plan' is corrupted, otherwise success.
   static Status CreateTree(RuntimeState* state, const TPlan& plan,
-      const DescriptorTbl& descs, ExecNode** root);
+      const DescriptorTbl& descs, ExecNode** root) WARN_UNUSED_RESULT;
 
   /// Set debug action for node with given id in 'tree'
-  static void SetDebugOptions(int node_id, TExecNodePhase::type phase,
-                              TDebugAction::type action, ExecNode* tree);
+  static void SetDebugOptions(
+      int node_id, TExecNodePhase::type phase, TDebugAction::type action, ExecNode* tree);
 
   /// Collect all nodes of given 'node_type' that are part of this subtree, and return in
   /// 'nodes'.
@@ -150,9 +151,9 @@ class ExecNode {
 
   /// Codegen EvalConjuncts(). Returns a non-OK status if the function couldn't be
   /// codegen'd. The codegen'd version uses inlined, codegen'd GetBooleanVal() functions.
-  static Status CodegenEvalConjuncts(
-      LlvmCodeGen* codegen, const std::vector<ExprContext*>& conjunct_ctxs,
-      llvm::Function** fn, const char* name = "EvalConjuncts");
+  static Status CodegenEvalConjuncts(LlvmCodeGen* codegen,
+      const std::vector<ExprContext*>& conjunct_ctxs, llvm::Function** fn,
+      const char* name = "EvalConjuncts") WARN_UNUSED_RESULT;
 
   /// Returns a string representation in DFS order of the plan rooted at this.
   std::string DebugString() const;
@@ -222,7 +223,7 @@ class ExecNode {
     /// Returns true if the element was added to the queue, false if it wasn't. If this
     /// method returns false, the queue didn't take ownership of the batch and it must be
     /// managed externally.
-    bool AddBatchWithTimeout(RowBatch* batch, int64_t timeout_micros);
+    bool AddBatchWithTimeout(RowBatch* batch, int64_t timeout_micros) WARN_UNUSED_RESULT;
 
     /// Gets a row batch from the queue. Returns NULL if there are no more.
     /// This function blocks.
@@ -285,11 +286,12 @@ class ExecNode {
 
   /// Create a single exec node derived from thrift node; place exec node in 'pool'.
   static Status CreateNode(ObjectPool* pool, const TPlanNode& tnode,
-      const DescriptorTbl& descs, ExecNode** node, RuntimeState* state);
+      const DescriptorTbl& descs, ExecNode** node,
+      RuntimeState* state) WARN_UNUSED_RESULT;
 
   static Status CreateTreeHelper(RuntimeState* state,
       const std::vector<TPlanNode>& tnodes, const DescriptorTbl& descs, ExecNode* parent,
-      int* node_idx, ExecNode** root);
+      int* node_idx, ExecNode** root) WARN_UNUSED_RESULT;
 
   virtual bool IsScanNode() const { return false; }
 
@@ -297,7 +299,8 @@ class ExecNode {
 
   /// Executes debug_action_ if phase matches debug_phase_.
   /// 'phase' must not be INVALID.
-  Status ExecDebugAction(TExecNodePhase::type phase, RuntimeState* state);
+  Status ExecDebugAction(
+      TExecNodePhase::type phase, RuntimeState* state) WARN_UNUSED_RESULT;
 
   /// Frees any local allocations made by expr_ctxs_to_free_ and returns the result of
   /// state->CheckQueryState(). Nodes should call this periodically, e.g. once per input
@@ -306,7 +309,7 @@ class ExecNode {
   /// Nodes may override this to add extra periodic cleanup, e.g. freeing other local
   /// allocations. ExecNodes overriding this function should return
   /// ExecNode::QueryMaintenance().
-  virtual Status QueryMaintenance(RuntimeState* state);
+  virtual Status QueryMaintenance(RuntimeState* state) WARN_UNUSED_RESULT;
 
   /// Add an ExprContext to have its local allocations freed by QueryMaintenance().
   /// Exprs that are evaluated in the main execution thread should be added. Exprs

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b989efbb/be/src/exec/hdfs-scan-node-base.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node-base.cc b/be/src/exec/hdfs-scan-node-base.cc
index 738ef36..7b00fcc 100644
--- a/be/src/exec/hdfs-scan-node-base.cc
+++ b/be/src/exec/hdfs-scan-node-base.cc
@@ -372,8 +372,7 @@ Status HdfsScanNodeBase::Open(RuntimeState* state) {
         partition_desc->partition_key_value_ctxs(), scan_node_pool_.get(), state);
   }
 
-  RETURN_IF_ERROR(runtime_state_->io_mgr()->RegisterContext(
-      &reader_context_, mem_tracker()));
+  runtime_state_->io_mgr()->RegisterContext(&reader_context_, mem_tracker());
 
   // Initialize HdfsScanNode specific counters
   // TODO: Revisit counters and move the counters specific to multi-threaded scans

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b989efbb/be/src/runtime/bufferpool/buffer-allocator.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/bufferpool/buffer-allocator.h b/be/src/runtime/bufferpool/buffer-allocator.h
index 54b667a..0c6e36e 100644
--- a/be/src/runtime/bufferpool/buffer-allocator.h
+++ b/be/src/runtime/bufferpool/buffer-allocator.h
@@ -35,7 +35,7 @@ class BufferAllocator {
 
   /// Allocate memory for a buffer of 'len' bytes. 'len' must be a power-of-two multiple
   /// of the minimum buffer length.
-  Status Allocate(int64_t len, uint8_t** buffer);
+  Status Allocate(int64_t len, uint8_t** buffer) WARN_UNUSED_RESULT;
 
   /// Free the memory for a previously-allocated buffer.
   void Free(uint8_t* buffer, int64_t len);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b989efbb/be/src/runtime/bufferpool/buffer-pool.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/bufferpool/buffer-pool.h b/be/src/runtime/bufferpool/buffer-pool.h
index 7839612..0327425 100644
--- a/be/src/runtime/bufferpool/buffer-pool.h
+++ b/be/src/runtime/bufferpool/buffer-pool.h
@@ -172,7 +172,7 @@ class BufferPool {
   /// 'profile'. 'client' is the client to register. 'client' should not already be
   /// registered.
   Status RegisterClient(const std::string& name, ReservationTracker* reservation,
-      RuntimeProfile* profile, Client* client);
+      RuntimeProfile* profile, Client* client) WARN_UNUSED_RESULT;
 
   /// Deregister 'client' if it is registered. Idempotent.
   void DeregisterClient(Client* client);
@@ -183,7 +183,7 @@ class BufferPool {
   /// CreatePage() only fails when a system error prevents the buffer pool from fulfilling
   /// the reservation.
   /// On success, the handle is mapped to the new page.
-  Status CreatePage(Client* client, int64_t len, PageHandle* handle);
+  Status CreatePage(Client* client, int64_t len, PageHandle* handle) WARN_UNUSED_RESULT;
 
   /// Increment the pin count of 'handle'. After Pin() the underlying page will
   /// be mapped to a buffer, which will be accessible through 'handle'. Uses
@@ -191,7 +191,7 @@ class BufferPool {
   /// unused reservation before calling Pin() (otherwise it will DCHECK). Pin() only
   /// fails when a system error prevents the buffer pool from fulfilling the reservation.
   /// 'handle' must be open.
-  Status Pin(Client* client, PageHandle* handle);
+  Status Pin(Client* client, PageHandle* handle) WARN_UNUSED_RESULT;
 
   /// Decrement the pin count of 'handle'. Decrease client's reservation usage. If the
   /// handle's pin count becomes zero, it is no longer valid for the underlying page's
@@ -219,7 +219,8 @@ class BufferPool {
   /// is responsible for ensuring it has enough unused reservation before calling
   /// AllocateBuffer() (otherwise it will DCHECK). AllocateBuffer() only fails when
   /// a system error prevents the buffer pool from fulfilling the reservation.
-  Status AllocateBuffer(Client* client, int64_t len, BufferHandle* handle);
+  Status AllocateBuffer(
+      Client* client, int64_t len, BufferHandle* handle) WARN_UNUSED_RESULT;
 
   /// If 'handle' is open, close 'handle', free the buffer and and decrease the
   /// reservation usage from 'client'. Idempotent.
@@ -231,7 +232,7 @@ class BufferPool {
   /// closed before calling. 'src'/'dst' and 'src_client'/'dst_client' must be different.
   /// After a successful call, 'src' is closed and 'dst' is open.
   Status TransferBuffer(Client* src_client, BufferHandle* src, Client* dst_client,
-      BufferHandle* dst);
+      BufferHandle* dst) WARN_UNUSED_RESULT;
 
   /// Print a debug string with the state of the buffer pool.
   std::string DebugString();
@@ -255,7 +256,8 @@ class BufferPool {
   /// Allocate a buffer of length 'len'. Assumes that the client's reservation has already
   /// been consumed for the buffer. Returns an error if the pool is unable to fulfill the
   /// reservation.
-  Status AllocateBufferInternal(Client* client, int64_t len, BufferHandle* buffer);
+  Status AllocateBufferInternal(
+      Client* client, int64_t len, BufferHandle* buffer) WARN_UNUSED_RESULT;
 
   /// Frees 'buffer', which must be open before calling. Closes 'buffer' and updates
   /// internal state but does not release to any reservation.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b989efbb/be/src/runtime/bufferpool/reservation-tracker.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/bufferpool/reservation-tracker.h b/be/src/runtime/bufferpool/reservation-tracker.h
index c786f80..9edd37f 100644
--- a/be/src/runtime/bufferpool/reservation-tracker.h
+++ b/be/src/runtime/bufferpool/reservation-tracker.h
@@ -116,14 +116,14 @@ class ReservationTracker {
   /// Returns true if the reservation increase is granted, or false if not granted.
   /// If the reservation is not granted, no modifications are made to the state of
   /// any ReservationTrackers.
-  bool IncreaseReservation(int64_t bytes);
+  bool IncreaseReservation(int64_t bytes) WARN_UNUSED_RESULT;
 
   /// Tries to ensure that 'bytes' of unused reservation is available. If not already
   /// available, tries to increase the reservation such that the unused reservation is
   /// exactly equal to 'bytes'. Uses any unused reservation on ancestors and increase
   /// ancestors' reservations if needed to fit the increased reservation.
   /// Returns true if the reservation increase was successful or not necessary.
-  bool IncreaseReservationToFit(int64_t bytes);
+  bool IncreaseReservationToFit(int64_t bytes) WARN_UNUSED_RESULT;
 
   /// Decrease tracker's reservation by 'bytes'. This tracker's reservation must be at
   /// least 'bytes' before calling this method.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b989efbb/be/src/runtime/disk-io-mgr-stress.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/disk-io-mgr-stress.cc b/be/src/runtime/disk-io-mgr-stress.cc
index 73e055d..c25d6ef 100644
--- a/be/src/runtime/disk-io-mgr-stress.cc
+++ b/be/src/runtime/disk-io-mgr-stress.cc
@@ -238,8 +238,7 @@ void DiskIoMgrStress::NewClient(int i) {
   }
 
   client_mem_trackers_[i].reset(new MemTracker(-1, "", &mem_tracker_));
-  Status status = io_mgr_->RegisterContext(&client.reader, client_mem_trackers_[i].get());
-  CHECK(status.ok());
-  status = io_mgr_->AddScanRanges(client.reader, client.scan_ranges);
+  io_mgr_->RegisterContext(&client.reader, client_mem_trackers_[i].get());
+  Status status = io_mgr_->AddScanRanges(client.reader, client.scan_ranges);
   CHECK(status.ok());
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b989efbb/be/src/runtime/disk-io-mgr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/disk-io-mgr-test.cc b/be/src/runtime/disk-io-mgr-test.cc
index 016b14f..9167a6c 100644
--- a/be/src/runtime/disk-io-mgr-test.cc
+++ b/be/src/runtime/disk-io-mgr-test.cc
@@ -204,7 +204,7 @@ TEST_F(DiskIoMgrTest, SingleWriter) {
   MemTracker reader_mem_tracker(LARGE_MEM_LIMIT);
   ASSERT_OK(read_io_mgr->Init(&reader_mem_tracker));
   DiskIoRequestContext* reader;
-  ASSERT_OK(read_io_mgr->RegisterContext(&reader, &reader_mem_tracker));
+  read_io_mgr->RegisterContext(&reader, &reader_mem_tracker);
   for (int num_threads_per_disk = 1; num_threads_per_disk <= 5; ++num_threads_per_disk) {
     for (int num_disks = 1; num_disks <= 5; num_disks += 2) {
       pool_.reset(new ObjectPool);
@@ -248,7 +248,7 @@ TEST_F(DiskIoMgrTest, InvalidWrite) {
   DiskIoMgr io_mgr(1, 1, 1, 10);
   ASSERT_OK(io_mgr.Init(&mem_tracker));
   DiskIoRequestContext* writer;
-  ASSERT_OK(io_mgr.RegisterContext(&writer, NULL));
+  io_mgr.RegisterContext(&writer, NULL);
   int32_t* data = pool_->Add(new int32_t);
   *data = rand();
 
@@ -310,7 +310,7 @@ TEST_F(DiskIoMgrTest, SingleWriterCancel) {
   MemTracker reader_mem_tracker(LARGE_MEM_LIMIT);
   ASSERT_OK(read_io_mgr->Init(&reader_mem_tracker));
   DiskIoRequestContext* reader;
-  ASSERT_OK(read_io_mgr->RegisterContext(&reader, &reader_mem_tracker));
+  read_io_mgr->RegisterContext(&reader, &reader_mem_tracker);
   for (int num_threads_per_disk = 1; num_threads_per_disk <= 5; ++num_threads_per_disk) {
     for (int num_disks = 1; num_disks <= 5; num_disks += 2) {
       pool_.reset(new ObjectPool);
@@ -381,7 +381,7 @@ TEST_F(DiskIoMgrTest, SingleReader) {
           ASSERT_OK(io_mgr.Init(&mem_tracker));
           MemTracker reader_mem_tracker;
           DiskIoRequestContext* reader;
-          ASSERT_OK(io_mgr.RegisterContext(&reader, &reader_mem_tracker));
+          io_mgr.RegisterContext(&reader, &reader_mem_tracker);
 
           vector<DiskIoMgr::ScanRange*> ranges;
           for (int i = 0; i < len; ++i) {
@@ -435,7 +435,7 @@ TEST_F(DiskIoMgrTest, AddScanRangeTest) {
         ASSERT_OK(io_mgr.Init(&mem_tracker));
         MemTracker reader_mem_tracker;
         DiskIoRequestContext* reader;
-        ASSERT_OK(io_mgr.RegisterContext(&reader, &reader_mem_tracker));
+        io_mgr.RegisterContext(&reader, &reader_mem_tracker);
 
         vector<DiskIoMgr::ScanRange*> ranges_first_half;
         vector<DiskIoMgr::ScanRange*> ranges_second_half;
@@ -508,7 +508,7 @@ TEST_F(DiskIoMgrTest, SyncReadTest) {
         ASSERT_OK(io_mgr.Init(&mem_tracker));
         MemTracker reader_mem_tracker;
         DiskIoRequestContext* reader;
-        ASSERT_OK(io_mgr.RegisterContext(&reader, &reader_mem_tracker));
+        io_mgr.RegisterContext(&reader, &reader_mem_tracker);
 
         DiskIoMgr::ScanRange* complete_range = InitRange(1, tmp_file, 0, strlen(data), 0,
             stat_val.st_mtime);
@@ -578,7 +578,7 @@ TEST_F(DiskIoMgrTest, SingleReaderCancel) {
         ASSERT_OK(io_mgr.Init(&mem_tracker));
         MemTracker reader_mem_tracker;
         DiskIoRequestContext* reader;
-        ASSERT_OK(io_mgr.RegisterContext(&reader, &reader_mem_tracker));
+        io_mgr.RegisterContext(&reader, &reader_mem_tracker);
 
         vector<DiskIoMgr::ScanRange*> ranges;
         for (int i = 0; i < len; ++i) {
@@ -643,7 +643,7 @@ TEST_F(DiskIoMgrTest, MemLimits) {
     ASSERT_OK(io_mgr.Init(&root_mem_tracker));
     MemTracker reader_mem_tracker(-1, "Reader", &root_mem_tracker);
     DiskIoRequestContext* reader;
-    ASSERT_OK(io_mgr.RegisterContext(&reader, &reader_mem_tracker));
+    io_mgr.RegisterContext(&reader, &reader_mem_tracker);
 
     vector<DiskIoMgr::ScanRange*> ranges;
     for (int i = 0; i < num_buffers; ++i) {
@@ -718,7 +718,7 @@ TEST_F(DiskIoMgrTest, CachedReads) {
     ASSERT_OK(io_mgr.Init(&mem_tracker));
     MemTracker reader_mem_tracker;
     DiskIoRequestContext* reader;
-    ASSERT_OK(io_mgr.RegisterContext(&reader, &reader_mem_tracker));
+    io_mgr.RegisterContext(&reader, &reader_mem_tracker);
 
     DiskIoMgr::ScanRange* complete_range =
         InitRange(1, tmp_file, 0, strlen(data), 0, stat_val.st_mtime, NULL, true);
@@ -788,9 +788,9 @@ TEST_F(DiskIoMgrTest, MultipleReaderWriter) {
     for (int threads_per_disk = 1; threads_per_disk <= 5; ++threads_per_disk) {
       for (int num_disks = 1; num_disks <= 5; num_disks += 2) {
         DiskIoMgr io_mgr(num_disks, threads_per_disk, MIN_BUFFER_SIZE, MAX_BUFFER_SIZE);
-        io_mgr.Init(&mem_tracker);
+        ASSERT_OK(io_mgr.Init(&mem_tracker));
         for (int file_index = 0; file_index < num_contexts; ++file_index) {
-          ASSERT_OK(io_mgr.RegisterContext(&contexts[file_index], &mem_tracker));
+          io_mgr.RegisterContext(&contexts[file_index], &mem_tracker);
         }
         pool_.reset(new ObjectPool);
         int read_offset = 0;
@@ -904,7 +904,7 @@ TEST_F(DiskIoMgrTest, MultipleReader) {
           EXPECT_OK(io_mgr.Init(&mem_tracker));
 
           for (int i = 0; i < NUM_READERS; ++i) {
-            ASSERT_OK(io_mgr.RegisterContext(&readers[i], &mem_tracker));
+            io_mgr.RegisterContext(&readers[i], &mem_tracker);
 
             vector<DiskIoMgr::ScanRange*> ranges;
             for (int j = 0; j < DATA_LEN; ++j) {
@@ -958,7 +958,7 @@ TEST_F(DiskIoMgrTest, Buffers) {
 
   MemTracker reader_mem_tracker(-1, "Reader", &root_mem_tracker);
   DiskIoRequestContext* reader;
-  ASSERT_OK(io_mgr.RegisterContext(&reader, &reader_mem_tracker));
+  io_mgr.RegisterContext(&reader, &reader_mem_tracker);
 
   DiskIoMgr::ScanRange* dummy_range = InitRange(1, "dummy", 0, 0, 0, 0);
 
@@ -1030,7 +1030,7 @@ TEST_F(DiskIoMgrTest, PartialRead) {
   ASSERT_OK(io_mgr->Init(&mem_tracker));
   MemTracker reader_mem_tracker;
   DiskIoRequestContext* reader;
-  ASSERT_OK(io_mgr->RegisterContext(&reader, &reader_mem_tracker));
+  io_mgr->RegisterContext(&reader, &reader_mem_tracker);
 
   // We should not read past the end of file.
   DiskIoMgr::ScanRange* range = InitRange(1, tmp_file, 0, read_len, 0, stat_val.st_mtime);
@@ -1063,7 +1063,7 @@ TEST_F(DiskIoMgrTest, ReadIntoClientBuffer) {
   // Reader doesn't need to provide mem tracker if it's providing buffers.
   MemTracker* reader_mem_tracker = NULL;
   DiskIoRequestContext* reader;
-  ASSERT_OK(io_mgr->RegisterContext(&reader, reader_mem_tracker));
+  io_mgr->RegisterContext(&reader, reader_mem_tracker);
 
   for (int buffer_len : vector<int>({len - 1, len, len + 1})) {
     vector<uint8_t> client_buffer(buffer_len);
@@ -1105,7 +1105,7 @@ TEST_F(DiskIoMgrTest, ReadIntoClientBufferError) {
   DiskIoRequestContext* reader;
   vector<uint8_t> client_buffer(SCAN_LEN);
   for (int i = 0; i < 1000; ++i) {
-    ASSERT_OK(io_mgr->RegisterContext(&reader, reader_mem_tracker));
+    io_mgr->RegisterContext(&reader, reader_mem_tracker);
     DiskIoMgr::ScanRange* range = AllocateRange(1);
     range->Reset(NULL, tmp_file, SCAN_LEN, 0, 0, true,
         DiskIoMgr::BufferOpts::ReadInto(&client_buffer[0], SCAN_LEN));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b989efbb/be/src/runtime/disk-io-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/disk-io-mgr.cc b/be/src/runtime/disk-io-mgr.cc
index 20cb9b5..16fd211 100644
--- a/be/src/runtime/disk-io-mgr.cc
+++ b/be/src/runtime/disk-io-mgr.cc
@@ -415,12 +415,11 @@ Status DiskIoMgr::Init(MemTracker* process_mem_tracker) {
   return Status::OK();
 }
 
-Status DiskIoMgr::RegisterContext(DiskIoRequestContext** request_context,
+void DiskIoMgr::RegisterContext(DiskIoRequestContext** request_context,
     MemTracker* mem_tracker) {
   DCHECK(request_context_cache_.get() != NULL) << "Must call Init() first.";
   *request_context = request_context_cache_->GetNewContext();
   (*request_context)->Reset(mem_tracker);
-  return Status::OK();
 }
 
 void DiskIoMgr::UnregisterContext(DiskIoRequestContext* reader) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b989efbb/be/src/runtime/disk-io-mgr.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/disk-io-mgr.h b/be/src/runtime/disk-io-mgr.h
index c67f69d..b7222cb 100644
--- a/be/src/runtime/disk-io-mgr.h
+++ b/be/src/runtime/disk-io-mgr.h
@@ -436,7 +436,7 @@ class DiskIoMgr {
     /// called when all buffers have been returned, *buffer is set to NULL and Status::OK
     /// is returned.
     /// Only one thread can be in GetNext() at any time.
-    Status GetNext(BufferDescriptor** buffer);
+    Status GetNext(BufferDescriptor** buffer) WARN_UNUSED_RESULT;
 
     /// Cancel this scan range. This cleans up all queued buffers and
     /// wakes up any threads blocked on GetNext().
@@ -481,13 +481,14 @@ class DiskIoMgr {
 
     /// Reads from this range into 'buffer', which has length 'buffer_len' bytes. Returns
     /// the number of bytes read. The read position in this scan range is updated.
-    Status Read(uint8_t* buffer, int64_t buffer_len, int64_t* bytes_read, bool* eosr);
+    Status Read(uint8_t* buffer, int64_t buffer_len, int64_t* bytes_read,
+        bool* eosr) WARN_UNUSED_RESULT;
 
     /// Reads from the DN cache. On success, sets cached_buffer_ to the DN buffer
     /// and *read_succeeded to true.
     /// If the data is not cached, returns ok() and *read_succeeded is set to false.
     /// Returns a non-ok status if it ran into a non-continuable error.
-    Status ReadFromCache(bool* read_succeeded);
+    Status ReadFromCache(bool* read_succeeded) WARN_UNUSED_RESULT;
 
     /// Pointer to caller specified metadata. This is untouched by the io manager
     /// and the caller can put whatever auxiliary data in here.
@@ -645,7 +646,7 @@ class DiskIoMgr {
   ~DiskIoMgr();
 
   /// Initialize the IoMgr. Must be called once before any of the other APIs.
-  Status Init(MemTracker* process_mem_tracker);
+  Status Init(MemTracker* process_mem_tracker) WARN_UNUSED_RESULT;
 
   /// Allocates tracking structure for a request context.
   /// Register a new request context which is returned in *request_context.
@@ -655,7 +656,7 @@ class DiskIoMgr {
   ///    used for this reader will be tracked by this. If the limit is exceeded
   ///    the reader will be cancelled and MEM_LIMIT_EXCEEDED will be returned via
   ///    GetNext().
-  Status RegisterContext(DiskIoRequestContext** request_context,
+  void RegisterContext(DiskIoRequestContext** request_context,
       MemTracker* reader_mem_tracker);
 
   /// Unregisters context from the disk IoMgr. This must be called for every
@@ -683,20 +684,22 @@ class DiskIoMgr {
   /// This can be used to do synchronous reads as well as schedule dependent ranges,
   /// as in the case for columnar formats.
   Status AddScanRanges(DiskIoRequestContext* reader,
-      const std::vector<ScanRange*>& ranges, bool schedule_immediately = false);
-  Status AddScanRange(
-      DiskIoRequestContext* reader, ScanRange* range, bool schedule_immediately = false);
+      const std::vector<ScanRange*>& ranges,
+      bool schedule_immediately = false) WARN_UNUSED_RESULT;
+  Status AddScanRange(DiskIoRequestContext* reader, ScanRange* range,
+      bool schedule_immediately = false) WARN_UNUSED_RESULT;
 
   /// Add a WriteRange for the writer. This is non-blocking and schedules the context
   /// on the IoMgr disk queue. Does not create any files.
-  Status AddWriteRange(DiskIoRequestContext* writer, WriteRange* write_range);
+  Status AddWriteRange(
+      DiskIoRequestContext* writer, WriteRange* write_range) WARN_UNUSED_RESULT;
 
   /// Returns the next unstarted scan range for this reader. When the range is returned,
   /// the disk threads in the IoMgr will already have started reading from it. The
   /// caller is expected to call ScanRange::GetNext on the returned range.
   /// If there are no more unstarted ranges, NULL is returned.
   /// This call is blocking.
-  Status GetNextRange(DiskIoRequestContext* reader, ScanRange** range);
+  Status GetNextRange(DiskIoRequestContext* reader, ScanRange** range) WARN_UNUSED_RESULT;
 
   /// Reads the range and returns the result in buffer.
   /// This behaves like the typical synchronous read() api, blocking until the data
@@ -705,7 +708,8 @@ class DiskIoMgr {
   /// range *cannot* have already been added via AddScanRanges.
   /// This can only be used if the scan range fits in a single IO buffer (i.e. is smaller
   /// than max_read_buffer_size()) or if reading into a client-provided buffer.
-  Status Read(DiskIoRequestContext* reader, ScanRange* range, BufferDescriptor** buffer);
+  Status Read(DiskIoRequestContext* reader, ScanRange* range,
+      BufferDescriptor** buffer) WARN_UNUSED_RESULT;
 
   /// Determine which disk queue this file should be assigned to.  Returns an index into
   /// disk_queues_.  The disk_id is the volume ID for the local disk that holds the
@@ -715,7 +719,7 @@ class DiskIoMgr {
 
   /// TODO: The functions below can be moved to DiskIoRequestContext.
   /// Returns the current status of the context.
-  Status context_status(DiskIoRequestContext* context) const;
+  Status context_status(DiskIoRequestContext* context) const WARN_UNUSED_RESULT;
 
   void set_bytes_read_counter(DiskIoRequestContext*, RuntimeProfile::Counter*);
   void set_read_timer(DiskIoRequestContext*, RuntimeProfile::Counter*);
@@ -932,11 +936,11 @@ class DiskIoMgr {
   /// The write_status does not affect the writer->status_. That is, an write error does
   /// not cancel the writer context - that decision is left to the callback handler.
   /// TODO: On the read path, consider not canceling the reader context on error.
-  void HandleWriteFinished(DiskIoRequestContext* writer, WriteRange* write_range,
-      const Status& write_status);
+  void HandleWriteFinished(
+      DiskIoRequestContext* writer, WriteRange* write_range, const Status& write_status);
 
   /// Validates that range is correctly initialized
-  Status ValidateScanRange(ScanRange* range);
+  Status ValidateScanRange(ScanRange* range) WARN_UNUSED_RESULT;
 
   /// Write the specified range to disk and calls HandleWriteFinished when done.
   /// Responsible for opening and closing the file that is written.
@@ -945,11 +949,10 @@ class DiskIoMgr {
   /// Helper method to write a range using the specified FILE handle. Returns Status:OK
   /// if the write succeeded, or a RUNTIME_ERROR with an appropriate message otherwise.
   /// Does not open or close the file that is written.
-  Status WriteRangeHelper(FILE* file_handle, WriteRange* write_range);
+  Status WriteRangeHelper(FILE* file_handle, WriteRange* write_range) WARN_UNUSED_RESULT;
 
   /// Reads the specified scan range and calls HandleReadFinished when done.
-  void ReadRange(DiskQueue* disk_queue, DiskIoRequestContext* reader,
-      ScanRange* range);
+  void ReadRange(DiskQueue* disk_queue, DiskIoRequestContext* reader, ScanRange* range);
 
   /// Try to allocate the next buffer for the scan range, returning the new buffer
   /// if successful. If 'reader' is cancelled, cancels the range and returns NULL.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b989efbb/be/src/runtime/mem-tracker.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/mem-tracker.h b/be/src/runtime/mem-tracker.h
index 17b8ba3..2d35c43 100644
--- a/be/src/runtime/mem-tracker.h
+++ b/be/src/runtime/mem-tracker.h
@@ -151,6 +151,7 @@ class MemTracker {
   /// they can all consume 'bytes'. If this brings any of them over, none of them
   /// are updated.
   /// Returns true if the try succeeded.
+  WARN_UNUSED_RESULT
   bool TryConsume(int64_t bytes) {
     if (consumption_metric_ != NULL) RefreshConsumptionFromMetric();
     if (UNLIKELY(bytes <= 0)) return true;
@@ -318,7 +319,7 @@ class MemTracker {
   /// If 'failed_allocation_size' is greater than zero, logs the allocation size. If
   /// 'failed_allocation_size' is zero, nothing about the allocation size is logged.
   Status MemLimitExceeded(RuntimeState* state, const std::string& details,
-      int64_t failed_allocation = 0);
+      int64_t failed_allocation = 0) WARN_UNUSED_RESULT;
 
   static const std::string COUNTER_NAME;
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b989efbb/be/src/runtime/test-env.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/test-env.cc b/be/src/runtime/test-env.cc
index 8675ae9..a1ac3c4 100644
--- a/be/src/runtime/test-env.cc
+++ b/be/src/runtime/test-env.cc
@@ -39,10 +39,12 @@ TestEnv::TestEnv() {
   exec_env_.reset(new ExecEnv);
   exec_env_->InitForFeTests();
   io_mgr_tracker_.reset(new MemTracker(-1));
-  exec_env_->disk_io_mgr()->Init(io_mgr_tracker_.get());
+  Status status = exec_env_->disk_io_mgr()->Init(io_mgr_tracker_.get());
+  CHECK(status.ok()) << status.msg().msg();
   InitMetrics();
   tmp_file_mgr_.reset(new TmpFileMgr);
-  tmp_file_mgr_->Init(metrics_.get());
+  status = tmp_file_mgr_->Init(metrics_.get());
+  CHECK(status.ok()) << status.msg().msg();
 }
 
 void TestEnv::InitMetrics() {
@@ -53,7 +55,8 @@ void TestEnv::InitTmpFileMgr(const vector<string>& tmp_dirs, bool one_dir_per_de
   // Need to recreate metrics to avoid error when registering metric twice.
   InitMetrics();
   tmp_file_mgr_.reset(new TmpFileMgr);
-  tmp_file_mgr_->InitCustom(tmp_dirs, one_dir_per_device, metrics_.get());
+  Status status = tmp_file_mgr_->InitCustom(tmp_dirs, one_dir_per_device, metrics_.get());
+  CHECK(status.ok()) << status.msg().msg();
 }
 
 TestEnv::~TestEnv() {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b989efbb/be/src/runtime/tmp-file-mgr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/tmp-file-mgr-test.cc b/be/src/runtime/tmp-file-mgr-test.cc
index 5e629ef..791c99c 100644
--- a/be/src/runtime/tmp-file-mgr-test.cc
+++ b/be/src/runtime/tmp-file-mgr-test.cc
@@ -201,7 +201,7 @@ TEST_F(TmpFileMgrTest, TestOneDirPerDevice) {
   vector<string> tmp_dirs({"/tmp/tmp-file-mgr-test.1", "/tmp/tmp-file-mgr-test.2"});
   RemoveAndCreateDirs(tmp_dirs);
   TmpFileMgr tmp_file_mgr;
-  tmp_file_mgr.InitCustom(tmp_dirs, true, metrics_.get());
+  ASSERT_OK(tmp_file_mgr.InitCustom(tmp_dirs, true, metrics_.get()));
   TUniqueId id;
   TmpFileMgr::FileGroup file_group(&tmp_file_mgr, io_mgr(), profile_, id);
 
@@ -225,7 +225,7 @@ TEST_F(TmpFileMgrTest, TestMultiDirsPerDevice) {
   vector<string> tmp_dirs({"/tmp/tmp-file-mgr-test.1", "/tmp/tmp-file-mgr-test.2"});
   RemoveAndCreateDirs(tmp_dirs);
   TmpFileMgr tmp_file_mgr;
-  tmp_file_mgr.InitCustom(tmp_dirs, false, metrics_.get());
+  ASSERT_OK(tmp_file_mgr.InitCustom(tmp_dirs, false, metrics_.get()));
   TUniqueId id;
   TmpFileMgr::FileGroup file_group(&tmp_file_mgr, io_mgr(), profile_, id);
 
@@ -253,7 +253,7 @@ TEST_F(TmpFileMgrTest, TestReportError) {
   vector<string> tmp_dirs({"/tmp/tmp-file-mgr-test.1", "/tmp/tmp-file-mgr-test.2"});
   RemoveAndCreateDirs(tmp_dirs);
   TmpFileMgr tmp_file_mgr;
-  tmp_file_mgr.InitCustom(tmp_dirs, false, metrics_.get());
+  ASSERT_OK(tmp_file_mgr.InitCustom(tmp_dirs, false, metrics_.get()));
   TUniqueId id;
   TmpFileMgr::FileGroup file_group(&tmp_file_mgr, io_mgr(), profile_, id);
 
@@ -302,7 +302,7 @@ TEST_F(TmpFileMgrTest, TestAllocateNonWritable) {
   }
   RemoveAndCreateDirs(tmp_dirs);
   TmpFileMgr tmp_file_mgr;
-  tmp_file_mgr.InitCustom(tmp_dirs, false, metrics_.get());
+  ASSERT_OK(tmp_file_mgr.InitCustom(tmp_dirs, false, metrics_.get()));
   TUniqueId id;
   TmpFileMgr::FileGroup file_group(&tmp_file_mgr, io_mgr(), profile_, id);
 
@@ -328,7 +328,7 @@ TEST_F(TmpFileMgrTest, TestScratchLimit) {
   vector<string> tmp_dirs({"/tmp/tmp-file-mgr-test.1", "/tmp/tmp-file-mgr-test.2"});
   RemoveAndCreateDirs(tmp_dirs);
   TmpFileMgr tmp_file_mgr;
-  tmp_file_mgr.InitCustom(tmp_dirs, false, metrics_.get());
+  ASSERT_OK(tmp_file_mgr.InitCustom(tmp_dirs, false, metrics_.get()));
 
   const int64_t LIMIT = 128;
   // A power-of-two so that FileGroup allocates exactly this amount of scratch space.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b989efbb/be/src/runtime/tmp-file-mgr.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/tmp-file-mgr.h b/be/src/runtime/tmp-file-mgr.h
index a8d63b2..65476cb 100644
--- a/be/src/runtime/tmp-file-mgr.h
+++ b/be/src/runtime/tmp-file-mgr.h
@@ -119,20 +119,20 @@ class TmpFileMgr {
     ///
     /// 'handle' must be destroyed by passing the DestroyWriteHandle() or
     /// CancelWriteAndRestoreData().
-    Status Write(
-        MemRange buffer, WriteDoneCallback cb, std::unique_ptr<WriteHandle>* handle);
+    Status Write(MemRange buffer, WriteDoneCallback cb,
+        std::unique_ptr<WriteHandle>* handle) WARN_UNUSED_RESULT;
 
     /// Synchronously read the data referenced by 'handle' from the temporary file into
     /// 'buffer'. buffer.len() must be the same as handle->len(). Can only be called
     /// after a write successfully completes.
-    Status Read(WriteHandle* handle, MemRange buffer);
+    Status Read(WriteHandle* handle, MemRange buffer) WARN_UNUSED_RESULT;
 
     /// Cancels the write referenced by 'handle' and destroy associate resources. Also
     /// restore the original data in the 'buffer' passed to Write(), decrypting or
     /// decompressing as necessary. The cancellation always succeeds, but an error
     /// is returned if restoring the data fails.
     Status CancelWriteAndRestoreData(
-        std::unique_ptr<WriteHandle> handle, MemRange buffer);
+        std::unique_ptr<WriteHandle> handle, MemRange buffer) WARN_UNUSED_RESULT;
 
     /// Wait for the in-flight I/Os to complete and destroy resources associated with
     /// 'handle'.
@@ -153,12 +153,13 @@ class TmpFileMgr {
     /// directory. Returns OK if at least one temporary file could be created.
     /// Returns an error if no temporary files were successfully created. Must only be
     /// called once. Must be called with 'lock_' held.
-    Status CreateFiles();
+    Status CreateFiles() WARN_UNUSED_RESULT;
 
     /// Allocate 'num_bytes' bytes in a temporary file. Try multiple disks if error
     /// occurs. Returns an error only if no temporary files are usable or the scratch
     /// limit is exceeded. Must be called without 'lock_' held.
-    Status AllocateSpace(int64_t num_bytes, File** tmp_file, int64_t* file_offset);
+    Status AllocateSpace(
+        int64_t num_bytes, File** tmp_file, int64_t* file_offset) WARN_UNUSED_RESULT;
 
     /// Add the scratch range from 'handle' to 'free_ranges_' and destroy handle. Must be
     /// called without 'lock_' held.
@@ -175,7 +176,8 @@ class TmpFileMgr {
     /// successfully reissued the write. Returns an error status if the original error
     /// was unrecoverable or an unrecoverable error is encountered when reissuing the
     /// write. The error status will include all previous I/O errors in its details.
-    Status RecoverWriteError(WriteHandle* handle, const Status& write_status);
+    Status RecoverWriteError(
+        WriteHandle* handle, const Status& write_status) WARN_UNUSED_RESULT;
 
     /// The TmpFileMgr it is associated with.
     TmpFileMgr* const tmp_file_mgr_;
@@ -281,12 +283,12 @@ class TmpFileMgr {
     /// Starts a write of 'buffer' to 'offset' of 'file'.
     Status Write(DiskIoMgr* io_mgr, DiskIoRequestContext* io_ctx, File* file,
         int64_t offset, MemRange buffer,
-        DiskIoMgr::WriteRange::WriteDoneCallback callback);
+        DiskIoMgr::WriteRange::WriteDoneCallback callback) WARN_UNUSED_RESULT;
 
     /// Retry the write after the initial write failed with an error, instead writing to
     /// 'offset' of 'file'.
-    Status RetryWrite(
-        DiskIoMgr* io_mgr, DiskIoRequestContext* io_ctx, File* file, int64_t offset);
+    Status RetryWrite(DiskIoMgr* io_mgr, DiskIoRequestContext* io_ctx, File* file,
+        int64_t offset) WARN_UNUSED_RESULT;
 
     /// Cancels the write asynchronously. After Cancel() is called, writes are not
     /// retried.
@@ -300,10 +302,10 @@ class TmpFileMgr {
     void WriteComplete(const Status& write_status);
 
     /// Encrypts the data in 'buffer' in-place and computes 'hash_'.
-    Status EncryptAndHash(MemRange buffer);
+    Status EncryptAndHash(MemRange buffer) WARN_UNUSED_RESULT;
 
     /// Verifies the integrity hash and decrypts the contents of 'buffer' in place.
-    Status CheckHashAndDecrypt(MemRange buffer);
+    Status CheckHashAndDecrypt(MemRange buffer) WARN_UNUSED_RESULT;
 
     /// Callback to be called when the write completes.
     WriteDoneCallback cb_;
@@ -346,13 +348,13 @@ class TmpFileMgr {
 
   /// Creates the configured tmp directories. If multiple directories are specified per
   /// disk, only one is created and used. Must be called after DiskInfo::Init().
-  Status Init(MetricGroup* metrics);
+  Status Init(MetricGroup* metrics) WARN_UNUSED_RESULT;
 
   /// Custom initialization - initializes with the provided list of directories.
   /// If one_dir_per_device is true, only use one temporary directory per device.
   /// This interface is intended for testing purposes.
   Status InitCustom(const std::vector<std::string>& tmp_dirs, bool one_dir_per_device,
-      MetricGroup* metrics);
+      MetricGroup* metrics) WARN_UNUSED_RESULT;
 
   /// Return the scratch directory path for the device.
   std::string GetTmpDirPath(DeviceId device_id) const;
@@ -373,8 +375,8 @@ class TmpFileMgr {
   /// directory on the specified device id. The caller owns the returned handle and is
   /// responsible for deleting it. The file is not created - creation is deferred until
   /// the file is written.
-  Status NewFile(
-      FileGroup* file_group, DeviceId device_id, std::unique_ptr<File>* new_file);
+  Status NewFile(FileGroup* file_group, DeviceId device_id,
+      std::unique_ptr<File>* new_file) WARN_UNUSED_RESULT;
 
   bool initialized_;
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b989efbb/be/src/util/openssl-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/openssl-util.h b/be/src/util/openssl-util.h
index b214ee9..4b32db6 100644
--- a/be/src/util/openssl-util.h
+++ b/be/src/util/openssl-util.h
@@ -38,7 +38,7 @@ class IntegrityHash {
 
   /// Verify that the data in a buffer matches this hash. Returns true on match, false
   /// otherwise.
-  bool Verify(const uint8_t* data, int64_t len) const;
+  bool Verify(const uint8_t* data, int64_t len) const WARN_UNUSED_RESULT;
 
  private:
   uint8_t hash_[SHA256_DIGEST_LENGTH];
@@ -67,13 +67,13 @@ class EncryptionKey {
   /// Encrypts a buffer of input data 'data' of length 'len' into an output buffer 'out'.
   /// Exactly 'len' bytes will be written to 'out'. This key must be initialized before
   /// calling. Operates in-place if 'in' == 'out', otherwise the buffers must not overlap.
-  Status Encrypt(const uint8_t* data, int64_t len, uint8_t* out) const;
+  Status Encrypt(const uint8_t* data, int64_t len, uint8_t* out) const WARN_UNUSED_RESULT;
 
   /// Decrypts a buffer of input data 'data' of length 'len' that was encrypted with this
   /// key into an output buffer 'out'. Exactly 'len' bytes will be written to 'out'.
   /// This key must be initialized before calling. Operates in-place if 'in' == 'out',
   /// otherwise the buffers must not overlap.
-  Status Decrypt(const uint8_t* data, int64_t len, uint8_t* out) const;
+  Status Decrypt(const uint8_t* data, int64_t len, uint8_t* out) const WARN_UNUSED_RESULT;
 
  private:
   /// Helper method that encrypts/decrypts if 'encrypt' is true/false respectively.
@@ -81,8 +81,8 @@ class EncryptionKey {
   /// key into an output buffer 'out'. Exactly 'len' bytes will be written to 'out'.
   /// This key must be initialized before calling. Operates in-place if 'in' == 'out',
   /// otherwise the buffers must not overlap.
-  Status EncryptInternal(
-      bool encrypt, const uint8_t* data, int64_t len, uint8_t* out) const;
+  Status EncryptInternal(bool encrypt, const uint8_t* data, int64_t len,
+      uint8_t* out) const WARN_UNUSED_RESULT;
 
   /// Track whether this key has been initialized, to avoid accidentally using
   /// uninitialized keys.