You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2018/02/01 04:33:24 UTC

[1/3] impala git commit: IMPALA-6450: fix EventSequence::Start()

Repository: impala
Updated Branches:
  refs/heads/master f6576adc2 -> 999a3f60c


IMPALA-6450: fix EventSequence::Start()

It looks like this newly-added DCHECK is being hit because of the same
underlying issue as IMPALA-4631. This patch loosens the DCHECK to accept
time going backward 1 tick, the same as the original workaround for
IMPALA-4631.

'offset_' also isn't being used, so fix that too.

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


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

Branch: refs/heads/master
Commit: 4b16ecd032ac19c040d5cfa2a26ab8a11e4d554d
Parents: f6576ad
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Mon Jan 29 15:52:59 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 1 01:43:12 2018 +0000

----------------------------------------------------------------------
 be/src/util/runtime-profile-counters.h | 10 +++++++---
 1 file changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/4b16ecd0/be/src/util/runtime-profile-counters.h
----------------------------------------------------------------------
diff --git a/be/src/util/runtime-profile-counters.h b/be/src/util/runtime-profile-counters.h
index 62281f1..c8d8bcd 100644
--- a/be/src/util/runtime-profile-counters.h
+++ b/be/src/util/runtime-profile-counters.h
@@ -302,14 +302,17 @@ class RuntimeProfile::EventSequence {
   /// 'start_time_ns', which must have been obtained by calling MonotonicStopWatch::Now().
   void Start(int64_t start_time_ns) {
     offset_ = MonotonicStopWatch::Now() - start_time_ns;
-    DCHECK_GE(offset_, 0);
+    // TODO: IMPALA-4631: Occasionally we see MonotonicStopWatch::Now() return
+    // (start_time_ns - 1), even though 'start_time_ns' was obtained using
+    // MonotonicStopWatch::Now().
+    DCHECK_GE(offset_, -1);
     sw_.Start();
   }
 
   /// Stores an event in sequence with the given label and the current time
   /// (relative to the first time Start() was called) as the timestamp.
   void MarkEvent(std::string label) {
-    Event event = make_pair(move(label), sw_.ElapsedTime());
+    Event event = make_pair(move(label), sw_.ElapsedTime() + offset_);
     boost::lock_guard<SpinLock> event_lock(lock_);
     events_.emplace_back(move(event));
   }
@@ -334,7 +337,8 @@ class RuntimeProfile::EventSequence {
   }
 
   /// Adds all events from the input parameters that are newer than the last member of
-  /// 'events_'. The caller must make sure that 'timestamps' is sorted.
+  /// 'events_'. The caller must make sure that 'timestamps' is sorted. Does not adjust
+  /// added timestamps by 'offset_'.
   void AddNewerEvents(
       const std::vector<int64_t>& timestamps, const std::vector<std::string>& labels) {
     DCHECK_EQ(timestamps.size(), labels.size());


[2/3] impala git commit: IMPALA-6430: Log relevant debug pages if wait_for_metric_value times out

Posted by ta...@apache.org.
IMPALA-6430: Log relevant debug pages if wait_for_metric_value times out

Log the memz, metrics and query page if the method wait_for_metric_value
times out. This would help us understand the state of the defaulting
impalad when the time out happens.

Change-Id: I069dad48ede709c4114f4d7175861f98321be6cf
Reviewed-on: http://gerrit.cloudera.org:8080/9098
Reviewed-by: Bikramjeet Vig <bi...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/master
Commit: cb2fa2475ee504c9d849a7a139b89c55cc59ce30
Parents: 4b16ecd
Author: Bikramjeet Vig <bi...@cloudera.com>
Authored: Wed Jan 17 14:23:27 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 1 02:26:50 2018 +0000

----------------------------------------------------------------------
 tests/common/impala_service.py | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/cb2fa247/tests/common/impala_service.py
----------------------------------------------------------------------
diff --git a/tests/common/impala_service.py b/tests/common/impala_service.py
index dfd4a6e..3ad0e84 100644
--- a/tests/common/impala_service.py
+++ b/tests/common/impala_service.py
@@ -115,8 +115,14 @@ class BaseImpalaService(object):
             (metric_name, expected_value, value))
       LOG.info("Sleeping %ds before next retry." % interval)
       sleep(interval)
-    assert 0, 'Metric value %s did not reach value %s in %ss' %\
-        (metric_name, expected_value, timeout)
+    assert 0, 'Metric value %s did not reach value %s in %ss\nDumping impalad debug ' \
+              'pages:\nmemz: %s\nmetrics: %s\nqueries: %s\nthreadz: %s\nrpcz: %s' % \
+              (metric_name, expected_value, timeout,
+               json.dumps(self.read_debug_webpage('memz?json')),
+               json.dumps(self.read_debug_webpage('metrics?json')),
+               json.dumps(self.read_debug_webpage('queries?json')),
+               json.dumps(self.read_debug_webpage('threadz?json')),
+               json.dumps(self.read_debug_webpage('rpcz?json')))
 
 # Allows for interacting with an Impalad instance to perform operations such as creating
 # new connections or accessing the debug webpage.


[3/3] impala git commit: IMPALA-6242: Change runtime-profile-test into using the same clock

Posted by ta...@apache.org.
IMPALA-6242: Change runtime-profile-test into using the same clock

In runtime-profile-test, both MonotonicStopWatch::Now() and
MonotonicNanos() are used. The former may use CLOCK_MONOTONIC_COARSE or
CLOCK_MONOTONIC while the latter always uses CLOCK_MONOTONIC. This may
contribute to the flakiness of this test. This patch changes this test
into using MonotonicStopWatch::Now() uniformly.

Change-Id: I20648d88e666b08d6f2a67a4158244865a50b629
Reviewed-on: http://gerrit.cloudera.org:8080/9171
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/master
Commit: 999a3f60c11d4bcaea77da887abf26b743fc6794
Parents: cb2fa24
Author: Tianyi Wang <tw...@cloudera.com>
Authored: Wed Jan 31 14:28:54 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 1 02:45:51 2018 +0000

----------------------------------------------------------------------
 be/src/util/runtime-profile-test.cc | 21 +++++++++------------
 1 file changed, 9 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/999a3f60/be/src/util/runtime-profile-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/runtime-profile-test.cc b/be/src/util/runtime-profile-test.cc
index f85b473..d3394b2 100644
--- a/be/src/util/runtime-profile-test.cc
+++ b/be/src/util/runtime-profile-test.cc
@@ -16,7 +16,6 @@
 // under the License.
 
 #include <stdlib.h>
-#include <stdio.h>
 #include <iostream>
 #include <boost/bind.hpp>
 
@@ -24,9 +23,7 @@
 #include "testutil/gtest-util.h"
 #include "util/periodic-counter-updater.h"
 #include "util/runtime-profile-counters.h"
-#include "util/streaming-sampler.h"
 #include "util/thread.h"
-#include "util/time.h"
 
 #include "common/names.h"
 
@@ -668,7 +665,7 @@ class TimerCounterTest {
 };
 
 void ValidateTimerValue(const TimerCounterTest& timer, int64_t start) {
-  int64_t expected_value = MonotonicNanos() - start;
+  int64_t expected_value = MonotonicStopWatch::Now() - start;
   int64_t stopwatch_value = timer.csw_.TotalRunningTime();
   EXPECT_GE(stopwatch_value, expected_value - TimerCounterTest::MAX_TIMER_ERROR_NS);
   EXPECT_LE(stopwatch_value, expected_value + TimerCounterTest::MAX_TIMER_ERROR_NS);
@@ -690,7 +687,7 @@ void ValidateLapTime(TimerCounterTest* timer, int64_t expected_value) {
 
 TEST(TimerCounterTest, CountersTestOneThread) {
   TimerCounterTest tester;
-  uint64_t start = MonotonicNanos();
+  int64_t start = MonotonicStopWatch::Now();
   tester.StartWorkers(1, 0);
   SleepForMs(500);
   ValidateTimerValue(tester, start);
@@ -700,7 +697,7 @@ TEST(TimerCounterTest, CountersTestOneThread) {
 
 TEST(TimerCounterTest, CountersTestTwoThreads) {
   TimerCounterTest tester;
-  uint64_t start = MonotonicNanos();
+  int64_t start = MonotonicStopWatch::Now();
   tester.StartWorkers(2, 10);
   SleepForMs(500);
   ValidateTimerValue(tester, start);
@@ -710,7 +707,7 @@ TEST(TimerCounterTest, CountersTestTwoThreads) {
 
 TEST(TimerCounterTest, CountersTestRandom) {
   TimerCounterTest tester;
-  uint64_t start = MonotonicNanos();
+  int64_t start = MonotonicStopWatch::Now();
   ValidateTimerValue(tester, start);
   // First working period
   tester.StartWorkers(5, 10);
@@ -726,23 +723,23 @@ TEST(TimerCounterTest, CountersTestRandom) {
   ValidateTimerValue(tester, start);
   tester.Reset();
 
-  ValidateLapTime(&tester, MonotonicNanos() - start);
-  uint64_t first_run_end = MonotonicNanos();
+  ValidateLapTime(&tester, MonotonicStopWatch::Now() - start);
+  int64_t first_run_end = MonotonicStopWatch::Now();
   // Adding some idle time. concurrent stopwatch and timer should not count the idle time.
   SleepForMs(200);
-  start += MonotonicNanos() - first_run_end;
+  start += MonotonicStopWatch::Now() - first_run_end;
 
   // Second working period
   tester.StartWorkers(2, 0);
   // We just get lap time after first run finish. so at start of second run, expect lap time == 0
   ValidateLapTime(&tester, 0);
-  uint64_t lap_time_start = MonotonicNanos();
+  int64_t lap_time_start = MonotonicStopWatch::Now();
   SleepForMs(200);
   ValidateTimerValue(tester, start);
   SleepForMs(200);
   tester.StopWorkers(-1);
   ValidateTimerValue(tester, start);
-  ValidateLapTime(&tester, MonotonicNanos() - lap_time_start);
+  ValidateLapTime(&tester, MonotonicStopWatch::Now() - lap_time_start);
 }
 
 }