You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by he...@apache.org on 2017/03/29 02:53:54 UTC

[13/14] incubator-impala git commit: IMPALA-4758: (2/2) Impala-side changes to build with latest gutil

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/gutil/strings/strip.cc
----------------------------------------------------------------------
diff --git a/be/src/gutil/strings/strip.cc b/be/src/gutil/strings/strip.cc
index 1a6a547..ff7449b 100644
--- a/be/src/gutil/strings/strip.cc
+++ b/be/src/gutil/strings/strip.cc
@@ -4,7 +4,7 @@
 // This file contains functions that remove a defined part from the string,
 // i.e., strip the string.
 
-#include "kudu/gutil/strings/strip.h"
+#include "gutil/strings/strip.h"
 
 #include <assert.h>
 #include <string.h>
@@ -18,8 +18,8 @@ using std::swap;
 #include <string>
 using std::string;
 
-#include "kudu/gutil/strings/ascii_ctype.h"
-#include "kudu/gutil/strings/stringpiece.h"
+#include "gutil/strings/ascii_ctype.h"
+#include "gutil/strings/stringpiece.h"
 
 string StripPrefixString(StringPiece str, const StringPiece& prefix) {
   if (str.starts_with(prefix))

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/gutil/strings/strip.h
----------------------------------------------------------------------
diff --git a/be/src/gutil/strings/strip.h b/be/src/gutil/strings/strip.h
index 8104b76..cdde845 100644
--- a/be/src/gutil/strings/strip.h
+++ b/be/src/gutil/strings/strip.h
@@ -11,8 +11,8 @@
 #include <string>
 using std::string;
 
-#include "kudu/gutil/strings/ascii_ctype.h"
-#include "kudu/gutil/strings/stringpiece.h"
+#include "gutil/strings/ascii_ctype.h"
+#include "gutil/strings/stringpiece.h"
 
 // Given a string and a putative prefix, returns the string minus the
 // prefix string if the prefix matches, otherwise the original

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/gutil/strings/substitute.cc
----------------------------------------------------------------------
diff --git a/be/src/gutil/strings/substitute.cc b/be/src/gutil/strings/substitute.cc
index 245894b..76ca151 100644
--- a/be/src/gutil/strings/substitute.cc
+++ b/be/src/gutil/strings/substitute.cc
@@ -1,13 +1,13 @@
 // Copyright 2008 Google Inc.  All rights reserved.
 
-#include "kudu/gutil/strings/substitute.h"
+#include "gutil/strings/substitute.h"
 
-#include <glog/logging.h>
-#include "kudu/gutil/logging-inl.h"
-#include "kudu/gutil/macros.h"
-#include "kudu/gutil/strings/ascii_ctype.h"
-#include "kudu/gutil/strings/escaping.h"
-#include "kudu/gutil/stl_util.h"
+#include <common/logging.h>
+#include "gutil/logging-inl.h"
+#include "gutil/macros.h"
+#include "gutil/strings/ascii_ctype.h"
+#include "gutil/strings/escaping.h"
+#include "gutil/stl_util.h"
 
 namespace strings {
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/gutil/strings/substitute.h
----------------------------------------------------------------------
diff --git a/be/src/gutil/strings/substitute.h b/be/src/gutil/strings/substitute.h
index 0812c3f..84d362a 100644
--- a/be/src/gutil/strings/substitute.h
+++ b/be/src/gutil/strings/substitute.h
@@ -4,9 +4,9 @@
 #include <string>
 using std::string;
 
-#include "kudu/gutil/basictypes.h"
-#include "kudu/gutil/strings/numbers.h"
-#include "kudu/gutil/strings/stringpiece.h"
+#include "gutil/basictypes.h"
+#include "gutil/strings/numbers.h"
+#include "gutil/strings/stringpiece.h"
 
 
 #ifndef STRINGS_SUBSTITUTE_H_

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/gutil/strings/util.cc
----------------------------------------------------------------------
diff --git a/be/src/gutil/strings/util.cc b/be/src/gutil/strings/util.cc
index c16d9b2..a5e29c4 100644
--- a/be/src/gutil/strings/util.cc
+++ b/be/src/gutil/strings/util.cc
@@ -5,7 +5,7 @@
 // TODO(user): visit each const_cast.  Some of them are no longer necessary
 // because last Single Unix Spec and grte v2 are more const-y.
 
-#include "kudu/gutil/strings/util.h"
+#include "gutil/strings/util.h"
 
 #include <assert.h>
 #include <stdarg.h>
@@ -24,13 +24,13 @@ using std::string;
 #include <vector>
 using std::vector;
 
-#include <glog/logging.h>
-#include "kudu/gutil/logging-inl.h"
-#include "kudu/gutil/strings/ascii_ctype.h"
-#include "kudu/gutil/strings/numbers.h"
-#include "kudu/gutil/strings/stringpiece.h"
-#include "kudu/gutil/stl_util.h"  // for string_as_array, STLAppendToString
-#include "kudu/gutil/utf/utf.h"
+#include <common/logging.h>
+#include "gutil/logging-inl.h"
+#include "gutil/strings/ascii_ctype.h"
+#include "gutil/strings/numbers.h"
+#include "gutil/strings/stringpiece.h"
+#include "gutil/stl_util.h"  // for string_as_array, STLAppendToString
+#include "gutil/utf/utf.h"
 
 #ifdef OS_WINDOWS
 #ifdef min  // windows.h defines this to something silly

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/gutil/strings/util.h
----------------------------------------------------------------------
diff --git a/be/src/gutil/strings/util.h b/be/src/gutil/strings/util.h
index 59db97d..2090e0f 100644
--- a/be/src/gutil/strings/util.h
+++ b/be/src/gutil/strings/util.h
@@ -42,9 +42,9 @@ using std::string;
 #include <vector>
 using std::vector;
 
-#include "kudu/gutil/integral_types.h"
-#include "kudu/gutil/port.h"
-#include "kudu/gutil/strings/stringpiece.h"
+#include "gutil/integral_types.h"
+#include "gutil/port.h"
+#include "gutil/strings/stringpiece.h"
 
 // Newer functions.
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/gutil/strtoint.cc
----------------------------------------------------------------------
diff --git a/be/src/gutil/strtoint.cc b/be/src/gutil/strtoint.cc
index bb96a57..9df29a3 100644
--- a/be/src/gutil/strtoint.cc
+++ b/be/src/gutil/strtoint.cc
@@ -5,8 +5,8 @@
 //
 
 #include <errno.h>
-#include "kudu/gutil/port.h"
-#include "kudu/gutil/strtoint.h"
+#include "gutil/port.h"
+#include "gutil/strtoint.h"
 
 // Replacement strto[u]l functions that have identical overflow and underflow
 // characteristics for both ILP-32 and LP-64 platforms, including errno

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/gutil/strtoint.h
----------------------------------------------------------------------
diff --git a/be/src/gutil/strtoint.h b/be/src/gutil/strtoint.h
index b581385..581ebf9 100644
--- a/be/src/gutil/strtoint.h
+++ b/be/src/gutil/strtoint.h
@@ -33,9 +33,9 @@
 #include <stdlib.h> // For strtol* functions.
 #include <string>
 using std::string;
-#include "kudu/gutil/integral_types.h"
-#include "kudu/gutil/macros.h"
-#include "kudu/gutil/port.h"
+#include "gutil/integral_types.h"
+#include "gutil/macros.h"
+#include "gutil/port.h"
 
 // Adapter functions for handling overflow and errno.
 int32 strto32_adapter(const char *nptr, char **endptr, int base);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/gutil/synchronization_profiling.h
----------------------------------------------------------------------
diff --git a/be/src/gutil/synchronization_profiling.h b/be/src/gutil/synchronization_profiling.h
index f008871..3292787 100644
--- a/be/src/gutil/synchronization_profiling.h
+++ b/be/src/gutil/synchronization_profiling.h
@@ -35,7 +35,7 @@
 #ifndef BASE_AUXILIARY_SYNCHRONIZATION_PROFILING_H_
 #define BASE_AUXILIARY_SYNCHRONIZATION_PROFILING_H_
 
-#include "kudu/gutil/basictypes.h"
+#include "gutil/basictypes.h"
 
 namespace gutil {
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/gutil/sysinfo.cc
----------------------------------------------------------------------
diff --git a/be/src/gutil/sysinfo.cc b/be/src/gutil/sysinfo.cc
index 6006b6d..a2b6077 100644
--- a/be/src/gutil/sysinfo.cc
+++ b/be/src/gutil/sysinfo.cc
@@ -53,11 +53,11 @@
 #include <shlwapi.h>          // for SHGetValueA()
 #include <tlhelp32.h>         // for Module32First()
 #endif
-#include "kudu/gutil/dynamic_annotations.h"   // for RunningOnValgrind
-#include "kudu/gutil/macros.h"
-#include "kudu/gutil/sysinfo.h"
-#include "kudu/gutil/walltime.h"
-#include <glog/logging.h>
+#include "gutil/dynamic_annotations.h"   // for RunningOnValgrind
+#include "gutil/macros.h"
+#include "gutil/sysinfo.h"
+#include "gutil/walltime.h"
+#include <common/logging.h>
 
 // This isn't in the 'base' namespace in tcmallc. But, tcmalloc
 // exports these functions, so we need to namespace them to avoid

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/gutil/threading/thread_collision_warner.cc
----------------------------------------------------------------------
diff --git a/be/src/gutil/threading/thread_collision_warner.cc b/be/src/gutil/threading/thread_collision_warner.cc
index 89270d8..2fa4415 100644
--- a/be/src/gutil/threading/thread_collision_warner.cc
+++ b/be/src/gutil/threading/thread_collision_warner.cc
@@ -2,11 +2,11 @@
 // Use of this source code is governed by a BSD-style license that can be
 // found in the LICENSE file.
 
-#include "kudu/gutil/threading/thread_collision_warner.h"
+#include "gutil/threading/thread_collision_warner.h"
 
-#include <glog/logging.h>
+#include <common/logging.h>
 
-#include "kudu/gutil/linux_syscall_support.h"
+#include "gutil/linux_syscall_support.h"
 
 namespace base {
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/gutil/threading/thread_collision_warner.h
----------------------------------------------------------------------
diff --git a/be/src/gutil/threading/thread_collision_warner.h b/be/src/gutil/threading/thread_collision_warner.h
index d59ea67..92d3e5c 100644
--- a/be/src/gutil/threading/thread_collision_warner.h
+++ b/be/src/gutil/threading/thread_collision_warner.h
@@ -7,9 +7,9 @@
 
 #include <memory>
 
-#include "kudu/gutil/atomicops.h"
-#include "kudu/gutil/port.h"
-#include "kudu/gutil/macros.h"
+#include "gutil/atomicops.h"
+#include "gutil/port.h"
+#include "gutil/macros.h"
  
 #ifndef BASE_EXPORT
 #define BASE_EXPORT

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/gutil/type_traits.h
----------------------------------------------------------------------
diff --git a/be/src/gutil/type_traits.h b/be/src/gutil/type_traits.h
index a4e874f..db03e54 100644
--- a/be/src/gutil/type_traits.h
+++ b/be/src/gutil/type_traits.h
@@ -63,7 +63,7 @@
 using std::make_pair;
 using std::pair;                  // For pair
 
-#include "kudu/gutil/template_util.h"     // For true_type and false_type
+#include "gutil/template_util.h"     // For true_type and false_type
 
 namespace base {
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/gutil/walltime.cc
----------------------------------------------------------------------
diff --git a/be/src/gutil/walltime.cc b/be/src/gutil/walltime.cc
index 89a805e..8f19f97 100644
--- a/be/src/gutil/walltime.cc
+++ b/be/src/gutil/walltime.cc
@@ -24,7 +24,7 @@
 #define _GNU_SOURCE   // Linux wants that for strptime in time.h
 #endif
 
-#include "kudu/gutil/walltime.h"
+#include "gutil/walltime.h"
 
 #include <stdio.h>
 #include <string.h>
@@ -183,6 +183,7 @@ WallTime WallTime_Now() {
 #endif  // defined(__APPLE__)
 }
 
+
 void StringAppendStrftime(string* dst,
                           const char* format,
                           time_t when,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/gutil/walltime.h
----------------------------------------------------------------------
diff --git a/be/src/gutil/walltime.h b/be/src/gutil/walltime.h
index e6a1294..e16e5ca 100644
--- a/be/src/gutil/walltime.h
+++ b/be/src/gutil/walltime.h
@@ -21,7 +21,7 @@
 
 #include <sys/time.h>
 
-#include <glog/logging.h>
+#include <common/logging.h>
 #include <string>
 using std::string;
 
@@ -30,10 +30,16 @@ using std::string;
 #include <mach/mach.h>
 #include <mach/mach_time.h>
 
-#include "kudu/gutil/once.h"
+#include "gutil/once.h"
 #endif  // defined(__APPLE__)
 
-#include "kudu/gutil/integral_types.h"
+#include "gutil/integral_types.h"
+
+#define NANOS_PER_SEC  1000000000ll
+#define NANOS_PER_MICRO      1000ll
+#define MICROS_PER_SEC    1000000ll
+#define MICROS_PER_MILLI     1000ll
+#define MILLIS_PER_SEC       1000ll
 
 typedef double WallTime;
 
@@ -80,7 +86,7 @@ inline void GetCurrentTime(mach_timespec_t* ts) {
 inline MicrosecondsInt64 GetCurrentTimeMicros() {
   mach_timespec_t ts;
   GetCurrentTime(&ts);
-  return ts.tv_sec * 1e6 + ts.tv_nsec / 1e3;
+  return ts.tv_sec * MICROS_PER_SEC + ts.tv_nsec / NANOS_PER_MICRO;
 }
 
 inline int64_t GetMonoTimeNanos() {
@@ -95,7 +101,7 @@ inline int64_t GetMonoTimeNanos() {
 }
 
 inline MicrosecondsInt64 GetMonoTimeMicros() {
-  return GetMonoTimeNanos() / 1e3;
+  return GetMonoTimeNanos() / NANOS_PER_MICRO;
 }
 
 inline MicrosecondsInt64 GetThreadCpuTimeMicros() {
@@ -121,7 +127,8 @@ inline MicrosecondsInt64 GetThreadCpuTimeMicros() {
     return 0;
   }
 
-  return thread_info_data.user_time.seconds * 1e6 + thread_info_data.user_time.microseconds;
+  return thread_info_data.user_time.seconds * MICROS_PER_SEC +
+      thread_info_data.user_time.microseconds;
 }
 
 #else
@@ -129,7 +136,7 @@ inline MicrosecondsInt64 GetThreadCpuTimeMicros() {
 inline MicrosecondsInt64 GetClockTimeMicros(clockid_t clock) {
   timespec ts;
   clock_gettime(clock, &ts);
-  return ts.tv_sec * 1e6 + ts.tv_nsec / 1e3;
+  return ts.tv_sec * MICROS_PER_SEC + ts.tv_nsec / NANOS_PER_MICRO;
 }
 
 #endif // defined(__APPLE__)
@@ -175,5 +182,5 @@ class CycleClock {
   CycleClock();
 };
 
-#include "kudu/gutil/cycleclock-inl.h"  // inline method bodies
+#include "gutil/cycleclock-inl.h"  // inline method bodies
 #endif  // GUTIL_WALLTIME_H_

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/runtime/buffered-tuple-stream-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/buffered-tuple-stream-test.cc b/be/src/runtime/buffered-tuple-stream-test.cc
index 7793767..3c50ec5 100644
--- a/be/src/runtime/buffered-tuple-stream-test.cc
+++ b/be/src/runtime/buffered-tuple-stream-test.cc
@@ -43,7 +43,7 @@
 
 #include "common/names.h"
 
-using base::FreeDeleter;
+using kudu::FreeDeleter;
 
 static const int BATCH_SIZE = 250;
 static const int IO_BLOCK_SIZE = 8 * 1024 * 1024;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/runtime/buffered-tuple-stream-v2-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/buffered-tuple-stream-v2-test.cc b/be/src/runtime/buffered-tuple-stream-v2-test.cc
index 5ff6471..da75212 100644
--- a/be/src/runtime/buffered-tuple-stream-v2-test.cc
+++ b/be/src/runtime/buffered-tuple-stream-v2-test.cc
@@ -44,7 +44,7 @@
 
 #include "common/names.h"
 
-using base::FreeDeleter;
+using kudu::FreeDeleter;
 using std::numeric_limits;
 
 static const int BATCH_SIZE = 250;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/runtime/bufferpool/suballocator.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/bufferpool/suballocator.cc b/be/src/runtime/bufferpool/suballocator.cc
index a6ab1ff..1d74d86 100644
--- a/be/src/runtime/bufferpool/suballocator.cc
+++ b/be/src/runtime/bufferpool/suballocator.cc
@@ -19,7 +19,6 @@
 
 #include <new>
 
-#include "gutil/bits.h"
 #include "runtime/bufferpool/reservation-tracker.h"
 #include "util/bit-util.h"
 
@@ -83,7 +82,7 @@ Status Suballocator::Allocate(int64_t bytes, unique_ptr<Suballocation>* result)
 }
 
 int Suballocator::ComputeListIndex(int64_t bytes) const {
-  return Bits::Log2CeilingNonZero64(bytes) - LOG_MIN_ALLOCATION_BYTES;
+  return BitUtil::Log2CeilingNonZero64(bytes) - LOG_MIN_ALLOCATION_BYTES;
 }
 
 Status Suballocator::AllocateBuffer(int64_t bytes, unique_ptr<Suballocation>* result) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/runtime/collection-value-builder.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/collection-value-builder.h b/be/src/runtime/collection-value-builder.h
index 701b22b..1811a9a 100644
--- a/be/src/runtime/collection-value-builder.h
+++ b/be/src/runtime/collection-value-builder.h
@@ -60,7 +60,8 @@ class CollectionValueBuilder {
       DCHECK_GE(buffer_size_, bytes_written);
       if (buffer_size_ == bytes_written) {
         // Double tuple buffer
-        int64_t new_buffer_size = max<int64_t>(buffer_size_ * 2, tuple_desc_.byte_size());
+        int64_t new_buffer_size =
+            std::max<int64_t>(buffer_size_ * 2, tuple_desc_.byte_size());
         uint8_t* new_buf = pool_->TryAllocate(new_buffer_size);
         if (UNLIKELY(new_buf == NULL)) {
           *tuple_mem = NULL;
@@ -113,4 +114,3 @@ class CollectionValueBuilder {
 }
 
 #endif
-

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/util/bit-util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/bit-util-test.cc b/be/src/util/bit-util-test.cc
index 2559d18..948e5cf 100644
--- a/be/src/util/bit-util-test.cc
+++ b/be/src/util/bit-util-test.cc
@@ -247,16 +247,14 @@ TEST(BitUtil, ByteSwap) {
 }
 
 TEST(BitUtil, Log2) {
-  // We use gutil's implementation in place of an older custom implementation in BitUtil.
-  // We leave this test here to ensure no test coverage is lost.
-  EXPECT_EQ(Bits::Log2CeilingNonZero64(1), 0);
-  EXPECT_EQ(Bits::Log2CeilingNonZero64(2), 1);
-  EXPECT_EQ(Bits::Log2CeilingNonZero64(3), 2);
-  EXPECT_EQ(Bits::Log2CeilingNonZero64(4), 2);
-  EXPECT_EQ(Bits::Log2CeilingNonZero64(5), 3);
-  EXPECT_EQ(Bits::Log2CeilingNonZero64(INT_MAX), 31);
-  EXPECT_EQ(Bits::Log2CeilingNonZero64(UINT_MAX), 32);
-  EXPECT_EQ(Bits::Log2CeilingNonZero64(ULLONG_MAX), 64);
+  EXPECT_EQ(BitUtil::Log2CeilingNonZero64(1), 0);
+  EXPECT_EQ(BitUtil::Log2CeilingNonZero64(2), 1);
+  EXPECT_EQ(BitUtil::Log2CeilingNonZero64(3), 2);
+  EXPECT_EQ(BitUtil::Log2CeilingNonZero64(4), 2);
+  EXPECT_EQ(BitUtil::Log2CeilingNonZero64(5), 3);
+  EXPECT_EQ(BitUtil::Log2CeilingNonZero64(INT_MAX), 31);
+  EXPECT_EQ(BitUtil::Log2CeilingNonZero64(UINT_MAX), 32);
+  EXPECT_EQ(BitUtil::Log2CeilingNonZero64(ULLONG_MAX), 64);
 }
 
 TEST(BitUtil, RoundUpToPowerOf2) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/util/bit-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/bit-util.h b/be/src/util/bit-util.h
index 571f1ae..25a8c96 100644
--- a/be/src/util/bit-util.h
+++ b/be/src/util/bit-util.h
@@ -32,6 +32,7 @@
 #include <boost/type_traits/make_unsigned.hpp>
 
 #include "common/compiler-util.h"
+#include "gutil/bits.h"
 #include "util/cpu-info.h"
 #include "util/sse-util.h"
 
@@ -285,6 +286,40 @@ class BitUtil {
     if (UNLIKELY(v == 0)) return otherwise;
     return __builtin_ctzll(v);
   }
+
+  /// More efficient version of similar functions found in gutil/
+  static inline int Log2Ceiling(uint32 n) {
+    int floor = Bits::Log2Floor(n);
+    // Check if zero or a power of two. This pattern is recognised by gcc and optimised
+    // into branch-free code.
+    if (0 == (n & (n - 1))) {
+      return floor;
+    } else {
+      return floor + 1;
+    }
+  }
+
+  static inline int Log2Ceiling64(uint64 n) {
+    int floor = Bits::Log2Floor64(n);
+    // Check if zero or a power of two. This pattern is recognised by gcc and optimised
+    // into branch-free code.
+    if (0 == (n & (n - 1))) {
+      return floor;
+    } else {
+      return floor + 1;
+    }
+  }
+
+  static inline int Log2CeilingNonZero64(uint64 n) {
+    int floor = Bits::Log2FloorNonZero64(n);
+    // Check if zero or a power of two. This pattern is recognised by gcc and optimised
+    // into branch-free code.
+    if (0 == (n & (n - 1))) {
+      return floor;
+    } else {
+      return floor + 1;
+    }
+  }
 };
 
 /// An encapsulation class of SIMD byteswap functions

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/util/error-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/error-util.h b/be/src/util/error-util.h
index 25fa0ff..06441cb 100644
--- a/be/src/util/error-util.h
+++ b/be/src/util/error-util.h
@@ -22,9 +22,10 @@
 #include <string>
 #include <vector>
 
+#include "common/logging.h"
 #include "gen-cpp/CatalogObjects_types.h"
-#include "gen-cpp/ErrorCodes_types.h"
 #include "gen-cpp/ErrorCodes_constants.h"
+#include "gen-cpp/ErrorCodes_types.h"
 #include "gen-cpp/ImpalaInternalService_types.h"
 #include "gutil/strings/substitute.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/be/src/util/time.h
----------------------------------------------------------------------
diff --git a/be/src/util/time.h b/be/src/util/time.h
index efe6a3b..3ec009a 100644
--- a/be/src/util/time.h
+++ b/be/src/util/time.h
@@ -31,7 +31,9 @@ namespace impala {
 /// time. Typically the value corresponds to elapsed time since the system booted. See
 /// UnixMillis() below if you need to send a time to a different host.
 inline int64_t MonotonicNanos() {
-  return GetMonoTimeNanos();
+  timespec ts;
+  clock_gettime(CLOCK_MONOTONIC, &ts);
+  return ts.tv_sec * NANOS_PER_SEC + ts.tv_nsec;
 }
 
 inline int64_t MonotonicMicros() {  // 63 bits ~= 5K years uptime

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/23100102/cmake_modules/kudu_cmake_fns.txt
----------------------------------------------------------------------
diff --git a/cmake_modules/kudu_cmake_fns.txt b/cmake_modules/kudu_cmake_fns.txt
index 8cd2cbc..ebbf114 100644
--- a/cmake_modules/kudu_cmake_fns.txt
+++ b/cmake_modules/kudu_cmake_fns.txt
@@ -55,8 +55,12 @@ function(ADD_EXPORTABLE_LIBRARY LIB_NAME)
   set(EXECUTABLE_OUTPUT_PATH "${BUILD_OUTPUT_ROOT_DIRECTORY}/${LIB_NAME}")
 
   add_library(${LIB_NAME} ${ARG_SRCS})
+  if(ARG_COMPILE_FLAGS)
+    set_target_properties(${LIB_NAME}
+      PROPERTIES COMPILE_FLAGS ${ARG_COMPILE_FLAGS})
+  endif()
   if (ARG_DEPS)
-    add_dependencies(${LIB_NAME} ${ARG_DEPS})
+    target_link_libraries(${LIB_NAME} ${ARG_DEPS})
   endif()
 endfunction()