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 2016/08/21 04:22:14 UTC

[1/2] incubator-impala git commit: IMPALA-3981: Fix crash when accessing statestored / catalogd /memz page

Repository: incubator-impala
Updated Branches:
  refs/heads/master 1522da351 -> 6b5f9cf0f


IMPALA-3981: Fix crash when accessing statestored / catalogd /memz page

The /memz page tried to add JVM metrics even when they didn't exist for
all daemons, not just Impala. This led to a crash when they tried to
access ExecEnv::GetInstance() without an initialised ExecEnv at
statestored and catalogd

To fix, changed the memz handler method to take an optional metric
group, provided by the caller.  memz handler will check the existence of
the metric group.

Used C++11 lambdas rather than boost::bind to help simplify the code.

Testing: Ran locally and looked at impalad/memz, statestored/memz
and catalogd/memz

Add a test file test_web_pages.py to test sending request to /memz on
impalad / statestored / catalogd

Change-Id: If59b10f20044d1a468f27810a3029fe18fb19f29
Reviewed-on: http://gerrit.cloudera.org:8080/3998
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Internal 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/1fc487c7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/1fc487c7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/1fc487c7

Branch: refs/heads/master
Commit: 1fc487c7dfab862ca88a8f6ae7790e3b004756a8
Parents: 1522da3
Author: KathySun-c <ka...@cloudera.com>
Authored: Mon Aug 15 19:01:15 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Sat Aug 20 04:23:50 2016 +0000

----------------------------------------------------------------------
 be/src/catalog/catalogd-main.cc            |  5 ++--
 be/src/runtime/data-stream-mgr.cc          |  2 +-
 be/src/runtime/exec-env.cc                 |  2 +-
 be/src/scheduling/simple-scheduler.cc      |  2 +-
 be/src/service/impala-server.cc            |  3 ++-
 be/src/statestore/statestore-subscriber.cc |  2 +-
 be/src/statestore/statestored-main.cc      |  4 +--
 be/src/util/default-path-handlers.cc       | 35 ++++++++++++++++---------
 be/src/util/default-path-handlers.h        |  4 ++-
 be/src/util/memory-metrics.cc              |  2 +-
 be/src/util/metrics-test.cc                | 19 +++++++++++---
 be/src/util/metrics.cc                     |  9 ++++++-
 be/src/util/metrics.h                      |  5 +++-
 tests/webserver/test_web_pages.py          | 30 +++++++++++++++++++++
 www/memz.tmpl                              |  3 ++-
 15 files changed, 96 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1fc487c7/be/src/catalog/catalogd-main.cc
----------------------------------------------------------------------
diff --git a/be/src/catalog/catalogd-main.cc b/be/src/catalog/catalogd-main.cc
index 3a6463f..52bf283 100644
--- a/be/src/catalog/catalogd-main.cc
+++ b/be/src/catalog/catalogd-main.cc
@@ -60,14 +60,15 @@ int CatalogdMain(int argc, char** argv) {
 
   MemTracker process_mem_tracker;
   scoped_ptr<Webserver> webserver(new Webserver());
+  scoped_ptr<MetricGroup> metrics(new MetricGroup("catalog"));
+
   if (FLAGS_enable_webserver) {
-    AddDefaultUrlCallbacks(webserver.get(), &process_mem_tracker);
+    AddDefaultUrlCallbacks(webserver.get(), &process_mem_tracker, metrics.get());
     ABORT_IF_ERROR(webserver->Start());
   } else {
     LOG(INFO) << "Not starting webserver";
   }
 
-  scoped_ptr<MetricGroup> metrics(new MetricGroup("catalog"));
   metrics->Init(FLAGS_enable_webserver ? webserver.get() : NULL);
   ABORT_IF_ERROR(RegisterMemoryMetrics(metrics.get(), true));
   StartThreadInstrumentation(metrics.get(), webserver.get());

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1fc487c7/be/src/runtime/data-stream-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/data-stream-mgr.cc b/be/src/runtime/data-stream-mgr.cc
index cb1adf3..ef9669e 100644
--- a/be/src/runtime/data-stream-mgr.cc
+++ b/be/src/runtime/data-stream-mgr.cc
@@ -55,7 +55,7 @@ const int32_t STREAM_EXPIRATION_TIME_MS = 300 * 1000;
 namespace impala {
 
 DataStreamMgr::DataStreamMgr(MetricGroup* metrics) {
-  metrics_ = metrics->GetChildGroup("datastream-manager");
+  metrics_ = metrics->GetOrCreateChildGroup("datastream-manager");
   num_senders_waiting_ =
       metrics_->AddGauge<int64_t>("senders-blocked-on-recvr-creation", 0L);
   total_senders_waited_ =

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1fc487c7/be/src/runtime/exec-env.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/exec-env.cc b/be/src/runtime/exec-env.cc
index cb74223..38e9f96 100644
--- a/be/src/runtime/exec-env.cc
+++ b/be/src/runtime/exec-env.cc
@@ -425,7 +425,7 @@ Status ExecEnv::StartServices() {
 
   // Start services in order to ensure that dependencies between them are met
   if (enable_webserver_) {
-    AddDefaultUrlCallbacks(webserver_.get(), mem_tracker_.get());
+    AddDefaultUrlCallbacks(webserver_.get(), mem_tracker_.get(), metrics_.get());
     RETURN_IF_ERROR(webserver_->Start());
   } else {
     LOG(INFO) << "Not starting webserver";

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1fc487c7/be/src/scheduling/simple-scheduler.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/simple-scheduler.cc b/be/src/scheduling/simple-scheduler.cc
index 2339007..a4d866f 100644
--- a/be/src/scheduling/simple-scheduler.cc
+++ b/be/src/scheduling/simple-scheduler.cc
@@ -82,7 +82,7 @@ SimpleScheduler::SimpleScheduler(StatestoreSubscriber* subscriber,
     MetricGroup* metrics, Webserver* webserver, ResourceBroker* resource_broker,
     RequestPoolService* request_pool_service)
   : backend_config_(std::make_shared<const BackendConfig>()),
-    metrics_(metrics->GetChildGroup("scheduler")),
+    metrics_(metrics->GetOrCreateChildGroup("scheduler")),
     webserver_(webserver),
     statestore_subscriber_(subscriber),
     local_backend_id_(backend_id),

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1fc487c7/be/src/service/impala-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 307910a..9bb8c77 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -323,7 +323,8 @@ ImpalaServer::ImpalaServer(ExecEnv* exec_env)
   http_handler_->RegisterHandlers(exec_env->webserver());
 
   // Initialize impalad metrics
-  ImpaladMetrics::CreateMetrics(exec_env->metrics()->GetChildGroup("impala-server"));
+  ImpaladMetrics::CreateMetrics(
+      exec_env->metrics()->GetOrCreateChildGroup("impala-server"));
   ImpaladMetrics::IMPALA_SERVER_START_TIME->set_value(
       TimestampValue::LocalTime().DebugString());
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1fc487c7/be/src/statestore/statestore-subscriber.cc
----------------------------------------------------------------------
diff --git a/be/src/statestore/statestore-subscriber.cc b/be/src/statestore/statestore-subscriber.cc
index 8fa1a7a..28ac553 100644
--- a/be/src/statestore/statestore-subscriber.cc
+++ b/be/src/statestore/statestore-subscriber.cc
@@ -102,7 +102,7 @@ StatestoreSubscriber::StatestoreSubscriber(const std::string& subscriber_id,
       client_cache_(new StatestoreClientCache(FLAGS_statestore_subscriber_cnxn_attempts,
           FLAGS_statestore_subscriber_cnxn_retry_interval_ms, 0, 0, "",
           !FLAGS_ssl_client_ca_certificate.empty())),
-      metrics_(metrics->GetChildGroup("statestore-subscriber")) {
+      metrics_(metrics->GetOrCreateChildGroup("statestore-subscriber")) {
   connected_to_statestore_metric_ =
       metrics_->AddProperty("statestore-subscriber.connected", false);
   last_recovery_duration_metric_ = metrics_->AddGauge(

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1fc487c7/be/src/statestore/statestored-main.cc
----------------------------------------------------------------------
diff --git a/be/src/statestore/statestored-main.cc b/be/src/statestore/statestored-main.cc
index 730277e..5f1ac25 100644
--- a/be/src/statestore/statestored-main.cc
+++ b/be/src/statestore/statestored-main.cc
@@ -54,15 +54,15 @@ int StatestoredMain(int argc, char** argv) {
 
   MemTracker mem_tracker;
   scoped_ptr<Webserver> webserver(new Webserver());
+  scoped_ptr<MetricGroup> metrics(new MetricGroup("statestore"));
 
   if (FLAGS_enable_webserver) {
-    AddDefaultUrlCallbacks(webserver.get(), &mem_tracker);
+    AddDefaultUrlCallbacks(webserver.get(), &mem_tracker, metrics.get());
     ABORT_IF_ERROR(webserver->Start());
   } else {
     LOG(INFO) << "Not starting webserver";
   }
 
-  scoped_ptr<MetricGroup> metrics(new MetricGroup("statestore"));
   metrics->Init(FLAGS_enable_webserver ? webserver.get() : NULL);
   ABORT_IF_ERROR(RegisterMemoryMetrics(metrics.get(), false));
   StartThreadInstrumentation(metrics.get(), webserver.get());

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1fc487c7/be/src/util/default-path-handlers.cc
----------------------------------------------------------------------
diff --git a/be/src/util/default-path-handlers.cc b/be/src/util/default-path-handlers.cc
index e1e1f44..91c51d3 100644
--- a/be/src/util/default-path-handlers.cc
+++ b/be/src/util/default-path-handlers.cc
@@ -26,7 +26,6 @@
 #include <gutil/strings/substitute.h>
 
 #include "common/logging.h"
-#include "runtime/exec-env.h"
 #include "runtime/mem-tracker.h"
 #include "util/debug-util.h"
 #include "util/pprof-path-handlers.h"
@@ -117,8 +116,8 @@ void FlagsHandler(const Webserver::ArgumentMap& args, Document* document) {
 }
 
 // Registered to handle "/memz"
-void MemUsageHandler(MemTracker* mem_tracker, const Webserver::ArgumentMap& args,
-    Document* document) {
+void MemUsageHandler(MemTracker* mem_tracker, MetricGroup* metric_group,
+    const Webserver::ArgumentMap& args, Document* document) {
   DCHECK(mem_tracker != NULL);
   Value mem_limit(PrettyPrinter::Print(mem_tracker->limit(), TUnit::BYTES).c_str(),
       document->GetAllocator());
@@ -144,25 +143,35 @@ void MemUsageHandler(MemTracker* mem_tracker, const Webserver::ArgumentMap& args
   Value detailed(mem_tracker->LogUsage().c_str(), document->GetAllocator());
   document->AddMember("detailed", detailed, document->GetAllocator());
 
-  Value jvm(kObjectType);
-  ExecEnv::GetInstance()->metrics()->GetChildGroup("jvm")->ToJson(false, document, &jvm);
+  if (metric_group != NULL) {
+    MetricGroup* jvm_group = metric_group->FindChildGroup("jvm");
+    if (jvm_group != NULL) {
+      Value jvm(kObjectType);
+      jvm_group->ToJson(false, document, &jvm);
+      Value total(kArrayType);
+      for (SizeType i = 0; i < jvm["metrics"].Size(); ++i) {
+        if (strstr(jvm["metrics"][i]["name"].GetString(), "total") != nullptr) {
+          total.PushBack(jvm["metrics"][i], document->GetAllocator());
+        }
+      }
+      document->AddMember("jvm", total, document->GetAllocator());
 
-  Value total(kArrayType);
-  for(SizeType i = 0; i < jvm["metrics"].Size(); ++i){
-    if (strstr(jvm["metrics"][i]["name"].GetString(), "total") != nullptr){
-      total.PushBack(jvm["metrics"][i], document->GetAllocator());
     }
   }
-  document->AddMember("jvm", total, document->GetAllocator());
+
 }
 
+
 void impala::AddDefaultUrlCallbacks(
-    Webserver* webserver, MemTracker* process_mem_tracker) {
+    Webserver* webserver, MemTracker* process_mem_tracker, MetricGroup* metric_group) {
   webserver->RegisterUrlCallback("/logs", "logs.tmpl", LogsHandler);
   webserver->RegisterUrlCallback("/varz", "flags.tmpl", FlagsHandler);
   if (process_mem_tracker != NULL) {
-    webserver->RegisterUrlCallback("/memz","memz.tmpl",
-        bind<void>(&MemUsageHandler, process_mem_tracker, _1, _2));
+    auto callback = [process_mem_tracker, metric_group]
+        (const Webserver::ArgumentMap& args, Document* doc) {
+      MemUsageHandler(process_mem_tracker, metric_group, args, doc);
+    };
+    webserver->RegisterUrlCallback("/memz", "memz.tmpl", callback);
   }
 
 #ifndef ADDRESS_SANITIZER

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1fc487c7/be/src/util/default-path-handlers.h
----------------------------------------------------------------------
diff --git a/be/src/util/default-path-handlers.h b/be/src/util/default-path-handlers.h
index 2926c79..aca8b83 100644
--- a/be/src/util/default-path-handlers.h
+++ b/be/src/util/default-path-handlers.h
@@ -24,10 +24,12 @@ namespace impala {
 
 class MemTracker;
 class Webserver;
+class MetricGroup;
 
 /// Adds a set of default path handlers to the webserver to display
 /// logs and configuration flags
-void AddDefaultUrlCallbacks(Webserver* webserver, MemTracker* process_mem_tracker = NULL);
+void AddDefaultUrlCallbacks(Webserver* webserver, MemTracker* process_mem_tracker = NULL,
+    MetricGroup* metric_group = NULL);
 }
 
 #endif // IMPALA_UTIL_DEFAULT_PATH_HANDLERS_H

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1fc487c7/be/src/util/memory-metrics.cc
----------------------------------------------------------------------
diff --git a/be/src/util/memory-metrics.cc b/be/src/util/memory-metrics.cc
index 31afe07..ae00e5a 100644
--- a/be/src/util/memory-metrics.cc
+++ b/be/src/util/memory-metrics.cc
@@ -59,7 +59,7 @@ Status impala::RegisterMemoryMetrics(MetricGroup* metrics, bool register_jvm_met
 #endif
 
   if (register_jvm_metrics) {
-    RETURN_IF_ERROR(JvmMetric::InitMetrics(metrics->GetChildGroup("jvm")));
+    RETURN_IF_ERROR(JvmMetric::InitMetrics(metrics->GetOrCreateChildGroup("jvm")));
   }
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1fc487c7/be/src/util/metrics-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/metrics-test.cc b/be/src/util/metrics-test.cc
index 9b8cbbd..8a0f983 100644
--- a/be/src/util/metrics-test.cc
+++ b/be/src/util/metrics-test.cc
@@ -230,12 +230,12 @@ TEST_F(MetricsTest, JvmMetrics) {
   MetricGroup metrics("JvmMetrics");
   RegisterMemoryMetrics(&metrics, true);
   UIntGauge* jvm_total_used =
-      metrics.GetChildGroup("jvm")->FindMetricForTesting<UIntGauge>(
+      metrics.GetOrCreateChildGroup("jvm")->FindMetricForTesting<UIntGauge>(
           "jvm.total.current-usage-bytes");
   ASSERT_TRUE(jvm_total_used != NULL);
   EXPECT_GT(jvm_total_used->value(), 0);
   UIntGauge* jvm_peak_total_used =
-      metrics.GetChildGroup("jvm")->FindMetricForTesting<UIntGauge>(
+      metrics.GetOrCreateChildGroup("jvm")->FindMetricForTesting<UIntGauge>(
           "jvm.total.peak-current-usage-bytes");
   ASSERT_TRUE(jvm_peak_total_used != NULL);
   EXPECT_GT(jvm_peak_total_used->value(), 0);
@@ -346,9 +346,12 @@ TEST_F(MetricsTest, MetricGroupJson) {
   metrics.AddCounter("counter1", 2048);
   metrics.AddCounter("counter2", 2048);
 
-  metrics.GetChildGroup("child1");
+  MetricGroup* find_result = metrics.FindChildGroup("child1");
+  EXPECT_EQ(find_result, reinterpret_cast<MetricGroup*>(NULL));
+
+  metrics.GetOrCreateChildGroup("child1");
   AddMetricDef("child_counter", TMetricKind::COUNTER, TUnit::BYTES, "description");
-  metrics.GetChildGroup("child2")->AddCounter("child_counter", 0);
+  metrics.GetOrCreateChildGroup("child2")->AddCounter("child_counter", 0);
 
   IntCounter* counter = metrics.FindMetricForTesting<IntCounter>(string("child_counter"));
   ASSERT_NE(counter, reinterpret_cast<IntCounter*>(NULL));
@@ -364,6 +367,14 @@ TEST_F(MetricsTest, MetricGroupJson) {
   EXPECT_EQ(val["child_groups"][0u]["metrics"].Size(), 0);
   EXPECT_EQ(val["child_groups"][1]["name"].GetString(), string("child2"));
   EXPECT_EQ(val["child_groups"][1]["metrics"].Size(), 1);
+
+  find_result = metrics.FindChildGroup("child1");
+  ASSERT_NE(find_result, reinterpret_cast<MetricGroup*>(NULL));
+  Value val2;
+  find_result->ToJson(true, &document, &val2);
+
+  EXPECT_EQ(val2["metrics"].Size(), 0);
+  EXPECT_EQ(val2["name"].GetString(), string("child1"));
 }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1fc487c7/be/src/util/metrics.cc
----------------------------------------------------------------------
diff --git a/be/src/util/metrics.cc b/be/src/util/metrics.cc
index a63ac5f..dc85093 100644
--- a/be/src/util/metrics.cc
+++ b/be/src/util/metrics.cc
@@ -194,7 +194,7 @@ void MetricGroup::ToJson(bool include_children, Document* document, Value* out_v
   *out_val = container;
 }
 
-MetricGroup* MetricGroup::GetChildGroup(const string& name) {
+MetricGroup* MetricGroup::GetOrCreateChildGroup(const string& name) {
   lock_guard<SpinLock> l(lock_);
   ChildGroupMap::iterator it = children_.find(name);
   if (it != children_.end()) return it->second;
@@ -203,6 +203,13 @@ MetricGroup* MetricGroup::GetChildGroup(const string& name) {
   return group;
 }
 
+MetricGroup* MetricGroup::FindChildGroup(const string& name) {
+  lock_guard<SpinLock> l(lock_);
+  ChildGroupMap::iterator it = children_.find(name);
+  if (it != children_.end()) return it->second;
+  return NULL;
+}
+
 string MetricGroup::DebugString() {
   Webserver::ArgumentMap empty_map;
   Document document;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1fc487c7/be/src/util/metrics.h
----------------------------------------------------------------------
diff --git a/be/src/util/metrics.h b/be/src/util/metrics.h
index 181222f..8b3942d 100644
--- a/be/src/util/metrics.h
+++ b/be/src/util/metrics.h
@@ -299,7 +299,10 @@ class MetricGroup {
       rapidjson::Value* out_val);
 
   /// Creates or returns an already existing child metric group.
-  MetricGroup* GetChildGroup(const std::string& name);
+  MetricGroup* GetOrCreateChildGroup(const std::string& name);
+
+  /// Returns a child metric group with name 'name', or NULL if that group doesn't exist
+  MetricGroup* FindChildGroup(const std::string& name);
 
   /// Useful for debuggers, returns the output of CMCompatibleCallback().
   std::string DebugString();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1fc487c7/tests/webserver/test_web_pages.py
----------------------------------------------------------------------
diff --git a/tests/webserver/test_web_pages.py b/tests/webserver/test_web_pages.py
new file mode 100644
index 0000000..c54a211
--- /dev/null
+++ b/tests/webserver/test_web_pages.py
@@ -0,0 +1,30 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from tests.common.impala_test_suite import ImpalaTestSuite
+import requests
+
+class TestWebPage(ImpalaTestSuite):
+  def test_memz(self):
+    """test /memz at impalad / statestored / catalogd"""
+
+    page = requests.get("http://localhost:25000/memz")
+    assert page.status_code == requests.codes.ok
+    page = requests.get("http://localhost:25010/memz")
+    assert page.status_code == requests.codes.ok
+    page = requests.get("http://localhost:25020/memz")
+    assert page.status_code == requests.codes.ok

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1fc487c7/www/memz.tmpl
----------------------------------------------------------------------
diff --git a/www/memz.tmpl b/www/memz.tmpl
index 6a2918d..f54cdaa 100644
--- a/www/memz.tmpl
+++ b/www/memz.tmpl
@@ -28,8 +28,8 @@ Memory consumption / limit: <strong>{{consumption}}</strong> / <strong>{{mem_lim
 <h3>Breakdown</h3>
 <pre>{{detailed}}</pre>
 
+{{?jvm}}
 <h3>JVM memory total usage</h3>
-
 <table class='table table-bordered table-hover'>
   <tr>
     <th>Name</th>
@@ -56,5 +56,6 @@ Memory consumption / limit: <strong>{{consumption}}</strong> / <strong>{{mem_lim
   </tr>
   {{/jvm}}
 </table>
+{{/jvm}}
 
 {{> www/common-footer.tmpl }}


[2/2] incubator-impala git commit: IMPALA-3996: Migrate to updated Kudu insert string API

Posted by he...@apache.org.
IMPALA-3996: Migrate to updated Kudu insert string API

In Kudu 0.10 the Insert API changed for Strings and Binary
values to avoid ambiguity around memory copying. They
changed SetString() to copy the value given to it to avoid
misuse, and added a SetStringNoCopy() with the old behavior.

We upgraded to 0.10 which has some perf impact but is still
correct. This changes the code to use the NoCopy call which
was the behavior we had previously.

See Kudu commit:
https://github.com/apache/kudu/commit/48766a4ce17d422ced9a6ec78c9a9969ac44d8c9

Change-Id: I910c24724d0bc887b2d4a3e62ecdf72420a76f6f
Reviewed-on: http://gerrit.cloudera.org:8080/4055
Reviewed-by: Matthew Jacobs <mj...@cloudera.com>
Tested-by: Internal 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/6b5f9cf0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/6b5f9cf0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/6b5f9cf0

Branch: refs/heads/master
Commit: 6b5f9cf0f53d8a9b35182af81ffecb1351e8d477
Parents: 1fc487c
Author: Matthew Jacobs <mj...@cloudera.com>
Authored: Thu Aug 18 18:03:30 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Sun Aug 21 02:48:52 2016 +0000

----------------------------------------------------------------------
 be/src/exec/kudu-table-sink.cc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/6b5f9cf0/be/src/exec/kudu-table-sink.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-table-sink.cc b/be/src/exec/kudu-table-sink.cc
index 79a6a83..661489f 100644
--- a/be/src/exec/kudu-table-sink.cc
+++ b/be/src/exec/kudu-table-sink.cc
@@ -173,7 +173,7 @@ Status KuduTableSink::Send(RuntimeState* state, RowBatch* batch) {
         case TYPE_STRING: {
           StringValue* sv = reinterpret_cast<StringValue*>(value);
           kudu::Slice slice(reinterpret_cast<uint8_t*>(sv->ptr), sv->len);
-          KUDU_RETURN_IF_ERROR(write->mutable_row()->SetString(col, slice),
+          KUDU_RETURN_IF_ERROR(write->mutable_row()->SetStringNoCopy(col, slice),
               "Could not add Kudu WriteOp.");
           break;
         }