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/09/21 04:00:38 UTC

[1/7] incubator-impala git commit: IMPALA-1616: Improve the Memory Limit Exceeded error report

Repository: incubator-impala
Updated Branches:
  refs/heads/master 57210ecd5 -> cd6d86b83


IMPALA-1616: Improve the Memory Limit Exceeded error report

The error report has been changed to include the id of the fragment
instance that exceeded the memory limit.

Change-Id: Ibb4e0c359d889938b4c351771ba539850bdb95ea
Reviewed-on: http://gerrit.cloudera.org:8080/4335
Reviewed-by: Michael Ho <kw...@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/3be61f90
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/3be61f90
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/3be61f90

Branch: refs/heads/master
Commit: 3be61f902425a026a5f29f6943f9d236f77f58b4
Parents: 57210ec
Author: Attila Jeges <at...@cloudera.com>
Authored: Thu Sep 1 12:04:38 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Mon Sep 19 23:39:33 2016 +0000

----------------------------------------------------------------------
 be/src/runtime/runtime-state.cc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3be61f90/be/src/runtime/runtime-state.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-state.cc b/be/src/runtime/runtime-state.cc
index 36e9bcf..57aae58 100644
--- a/be/src/runtime/runtime-state.cc
+++ b/be/src/runtime/runtime-state.cc
@@ -247,7 +247,7 @@ void RuntimeState::LogMemLimitExceeded(const MemTracker* tracker,
   DCHECK_GE(failed_allocation_size, 0);
   DCHECK(query_mem_tracker_.get() != NULL);
   stringstream ss;
-  ss << "Memory Limit Exceeded\n";
+  ss << "Memory Limit Exceeded by fragment: " << fragment_instance_id() << endl;
   if (failed_allocation_size != 0) {
     DCHECK(tracker != NULL);
     ss << "  " << tracker->label() << " could not allocate "


[3/7] incubator-impala git commit: IMPALA-4160: Remove Llama support.

Posted by he...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/scheduling/query-schedule.h
----------------------------------------------------------------------
diff --git a/be/src/scheduling/query-schedule.h b/be/src/scheduling/query-schedule.h
index 5ee60d6..c8ebd5d 100644
--- a/be/src/scheduling/query-schedule.h
+++ b/be/src/scheduling/query-schedule.h
@@ -26,12 +26,10 @@
 
 #include "common/global-types.h"
 #include "common/status.h"
-#include "scheduling/query-resource-mgr.h"
 #include "util/promise.h"
 #include "util/runtime-profile.h"
 #include "gen-cpp/Types_types.h"  // for TNetworkAddress
 #include "gen-cpp/Frontend_types.h"
-#include "gen-cpp/ResourceBrokerService_types.h"
 
 namespace impala {
 
@@ -74,30 +72,19 @@ class QuerySchedule {
       const TQueryOptions& query_options, RuntimeProfile* summary_profile,
       RuntimeProfile::EventSequence* query_events);
 
-  /// Returns OK if reservation_ contains a matching resource for each
-  /// of the hosts in fragment_exec_params_. Returns an error otherwise.
-  Status ValidateReservation();
-
   const TUniqueId& query_id() const { return query_id_; }
   const TQueryExecRequest& request() const { return request_; }
   const TQueryOptions& query_options() const { return query_options_; }
   const std::string& request_pool() const { return request_pool_; }
   void set_request_pool(const std::string& pool_name) { request_pool_ = pool_name; }
-  bool HasReservation() const { return !reservation_.allocated_resources.empty(); }
-
-  /// Granted or timed out reservations need to be released. In both such cases,
-  /// the reservation_'s reservation_id is set.
-  bool NeedsRelease() const { return reservation_.__isset.reservation_id; }
 
-  /// Gets the estimated memory (bytes) and vcores per-node. Returns the user specified
-  /// estimate (MEM_LIMIT query parameter) if provided or the estimate from planning if
-  /// available, but is capped at the amount of physical memory to avoid problems if
-  /// either estimate is unreasonably large.
+  /// Gets the estimated memory (bytes) per-node. Returns the user specified estimate
+  /// (MEM_LIMIT query parameter) if provided or the estimate from planning if available,
+  /// but is capped at the amount of physical memory to avoid problems if either estimate
+  /// is unreasonably large.
   int64_t GetPerHostMemoryEstimate() const;
-  int16_t GetPerHostVCores() const;
   /// Total estimated memory for all nodes. set_num_hosts() must be set before calling.
   int64_t GetClusterMemoryEstimate() const;
-  void GetResourceHostport(const TNetworkAddress& src, TNetworkAddress* dst);
 
   /// Helper methods used by scheduler to populate this QuerySchedule.
   void AddScanRanges(int64_t delta) { num_scan_ranges_ += delta; }
@@ -116,10 +103,6 @@ class QuerySchedule {
   const boost::unordered_set<TNetworkAddress>& unique_hosts() const {
     return unique_hosts_;
   }
-  TResourceBrokerReservationResponse* reservation() { return &reservation_; }
-  const TResourceBrokerReservationRequest& reservation_request() const {
-    return reservation_request_;
-  }
   bool is_admitted() const { return is_admitted_; }
   void set_is_admitted(bool is_admitted) { is_admitted_ = is_admitted; }
   RuntimeProfile* summary_profile() { return summary_profile_; }
@@ -127,10 +110,6 @@ class QuerySchedule {
 
   void SetUniqueHosts(const boost::unordered_set<TNetworkAddress>& unique_hosts);
 
-  /// Populates reservation_request_ ready to submit a query to Llama for all initial
-  /// resources required for this query.
-  void PrepareReservationRequest(const std::string& pool, const std::string& user);
-
  private:
 
   /// These references are valid for the lifetime of this query schedule because they
@@ -165,18 +144,9 @@ class QuerySchedule {
   /// Request pool to which the request was submitted for admission.
   std::string request_pool_;
 
-  /// Reservation request to be submitted to Llama. Set in PrepareReservationRequest().
-  TResourceBrokerReservationRequest reservation_request_;
-
-  /// Fulfilled reservation request. Populated by scheduler.
-  TResourceBrokerReservationResponse reservation_;
-
   /// Indicates if the query has been admitted for execution.
   bool is_admitted_;
 
-  /// Resolves unique_hosts_ to node mgr addresses. Valid only after SetUniqueHosts() has
-  /// been called.
-  boost::scoped_ptr<ResourceResolver> resource_resolver_;
 };
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/scheduling/request-pool-service.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/request-pool-service.cc b/be/src/scheduling/request-pool-service.cc
index 9f3363d..ea2553e 100644
--- a/be/src/scheduling/request-pool-service.cc
+++ b/be/src/scheduling/request-pool-service.cc
@@ -44,6 +44,7 @@ static const string DEFAULT_USER = "default";
 
 DEFINE_string(fair_scheduler_allocation_path, "", "Path to the fair scheduler "
     "allocation file (fair-scheduler.xml).");
+// TODO: Rename / cleanup now that Llama is removed (see IMPALA-4159).
 DEFINE_string(llama_site_path, "", "Path to the Llama configuration file "
     "(llama-site.xml). If set, fair_scheduler_allocation_path must also be set.");
 
@@ -74,7 +75,6 @@ DEFINE_bool(disable_pool_mem_limits, false, "Disables all per-pool mem limits.")
 DEFINE_bool(disable_pool_max_requests, false, "Disables all per-pool limits on the "
     "maximum number of running requests.");
 
-DECLARE_bool(enable_rm);
 
 // Pool name used when the configuration files are not specified.
 static const string DEFAULT_POOL_NAME = "default-pool";
@@ -94,12 +94,7 @@ RequestPoolService::RequestPoolService(MetricGroup* metrics) :
   resolve_pool_ms_metric_ =
       StatsMetric<double>::CreateAndRegister(metrics, RESOLVE_POOL_METRIC_NAME);
 
-  if (FLAGS_fair_scheduler_allocation_path.empty() &&
-      FLAGS_llama_site_path.empty()) {
-    if (FLAGS_enable_rm) {
-      CLEAN_EXIT_WITH_ERROR("If resource management is enabled, "
-          "-fair_scheduler_allocation_path is required.");
-    }
+  if (FLAGS_fair_scheduler_allocation_path.empty()) {
     default_pool_only_ = true;
     bool is_percent; // not used
     int64_t bytes_limit = ParseUtil::ParseMemSpec(FLAGS_default_pool_mem_limit,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/scheduling/scheduler.h
----------------------------------------------------------------------
diff --git a/be/src/scheduling/scheduler.h b/be/src/scheduling/scheduler.h
index 8b074d2..4c3a967 100644
--- a/be/src/scheduling/scheduler.h
+++ b/be/src/scheduling/scheduler.h
@@ -31,7 +31,6 @@
 #include "gen-cpp/PlanNodes_types.h"
 #include "gen-cpp/Frontend_types.h"
 #include "gen-cpp/ImpalaInternalService_types.h"
-#include "gen-cpp/ResourceBrokerService_types.h"
 
 namespace impala {
 
@@ -57,21 +56,6 @@ class Scheduler {
   /// Releases the reserved resources (if any) from the given schedule.
   virtual Status Release(QuerySchedule* schedule) = 0;
 
-  /// Notifies this scheduler that a resource reservation has been preempted by the
-  /// central scheduler (Yarn via Llama). All affected queries are cancelled
-  /// via their coordinator.
-  virtual void HandlePreemptedReservation(const TUniqueId& reservation_id) = 0;
-
-  /// Notifies this scheduler that a single resource with the given client resource id
-  /// has been preempted by the central scheduler (Yarn via Llama). All affected queries
-  /// are cancelled via their coordinator.
-  virtual void HandlePreemptedResource(const TUniqueId& client_resource_id) = 0;
-
-  /// Notifies this scheduler that a single resource with the given client resource id
-  /// has been lost by the central scheduler (Yarn via Llama). All affected queries
-  /// are cancelled via their coordinator.
-  virtual void HandleLostResource(const TUniqueId& client_resource_id) = 0;
-
   /// Initialises the scheduler, acquiring all resources needed to make
   /// scheduling decisions once this method returns.
   virtual Status Init() = 0;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/scheduling/simple-scheduler-test.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/simple-scheduler-test.cc b/be/src/scheduling/simple-scheduler-test.cc
index 76da6f9..7116e21 100644
--- a/be/src/scheduling/simple-scheduler-test.cc
+++ b/be/src/scheduling/simple-scheduler-test.cc
@@ -830,8 +830,8 @@ class SchedulerWrapper {
     scheduler_backend_address.hostname = scheduler_host.ip;
     scheduler_backend_address.port = scheduler_host.be_port;
 
-    scheduler_.reset(new SimpleScheduler(NULL, scheduler_backend_id,
-        scheduler_backend_address, &metrics_, NULL, NULL, NULL));
+    scheduler_.reset(new SimpleScheduler(
+        NULL, scheduler_backend_id, scheduler_backend_address, &metrics_, NULL, NULL));
     scheduler_->Init();
     // Initialize the scheduler backend maps.
     SendFullMembershipMap();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/scheduling/simple-scheduler.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/simple-scheduler.cc b/be/src/scheduling/simple-scheduler.cc
index f4b90cc..f3ba9a5 100644
--- a/be/src/scheduling/simple-scheduler.cc
+++ b/be/src/scheduling/simple-scheduler.cc
@@ -29,7 +29,6 @@
 
 #include "common/logging.h"
 #include "util/metrics.h"
-#include "resourcebroker/resource-broker.h"
 #include "runtime/exec-env.h"
 #include "runtime/coordinator.h"
 #include "service/impala-server.h"
@@ -43,11 +42,9 @@
 #include "util/container-util.h"
 #include "util/debug-util.h"
 #include "util/error-util.h"
-#include "util/llama-util.h"
 #include "util/mem-info.h"
 #include "util/parse-util.h"
 #include "util/runtime-profile-counters.h"
-#include "gen-cpp/ResourceBrokerService_types.h"
 
 #include "common/names.h"
 
@@ -58,9 +55,6 @@ using namespace strings;
 
 DECLARE_int32(be_port);
 DECLARE_string(hostname);
-DECLARE_bool(enable_rm);
-DECLARE_int32(rm_default_cpu_vcores);
-DECLARE_string(rm_default_memory);
 
 DEFINE_bool(disable_admission_control, false, "Disables admission control.");
 
@@ -79,8 +73,7 @@ const string SimpleScheduler::IMPALA_MEMBERSHIP_TOPIC("impala-membership");
 
 SimpleScheduler::SimpleScheduler(StatestoreSubscriber* subscriber,
     const string& backend_id, const TNetworkAddress& backend_address,
-    MetricGroup* metrics, Webserver* webserver, ResourceBroker* resource_broker,
-    RequestPoolService* request_pool_service)
+    MetricGroup* metrics, Webserver* webserver, RequestPoolService* request_pool_service)
   : backend_config_(std::make_shared<const BackendConfig>()),
     metrics_(metrics->GetOrCreateChildGroup("scheduler")),
     webserver_(webserver),
@@ -90,7 +83,6 @@ SimpleScheduler::SimpleScheduler(StatestoreSubscriber* subscriber,
     total_assignments_(NULL),
     total_local_assignments_(NULL),
     initialized_(NULL),
-    resource_broker_(resource_broker),
     request_pool_service_(request_pool_service) {
   local_backend_descriptor_.address = backend_address;
 
@@ -99,32 +91,10 @@ SimpleScheduler::SimpleScheduler(StatestoreSubscriber* subscriber,
     admission_controller_.reset(
         new AdmissionController(request_pool_service_, metrics, backend_address));
   }
-
-  if (FLAGS_enable_rm) {
-    if (FLAGS_rm_default_cpu_vcores <= 0) {
-      LOG(ERROR) << "Bad value for --rm_default_cpu_vcores (must be postive): "
-                 << FLAGS_rm_default_cpu_vcores;
-      exit(1);
-    }
-    bool is_percent;
-    int64_t mem_bytes =
-        ParseUtil::ParseMemSpec(
-            FLAGS_rm_default_memory, &is_percent, MemInfo::physical_mem());
-    if (mem_bytes <= 1024 * 1024) {
-      LOG(ERROR) << "Bad value for --rm_default_memory (must be larger than 1M):"
-                 << FLAGS_rm_default_memory;
-      exit(1);
-    } else if (is_percent) {
-      LOG(ERROR) << "Must use absolute value for --rm_default_memory: "
-                 << FLAGS_rm_default_memory;
-      exit(1);
-    }
-  }
 }
 
 SimpleScheduler::SimpleScheduler(const vector<TNetworkAddress>& backends,
-    MetricGroup* metrics, Webserver* webserver, ResourceBroker* resource_broker,
-    RequestPoolService* request_pool_service)
+    MetricGroup* metrics, Webserver* webserver, RequestPoolService* request_pool_service)
   : backend_config_(std::make_shared<const BackendConfig>(backends)),
     metrics_(metrics),
     webserver_(webserver),
@@ -133,7 +103,6 @@ SimpleScheduler::SimpleScheduler(const vector<TNetworkAddress>& backends,
     total_assignments_(NULL),
     total_local_assignments_(NULL),
     initialized_(NULL),
-    resource_broker_(resource_broker),
     request_pool_service_(request_pool_service) {
   DCHECK(backends.size() > 0);
   local_backend_descriptor_.address = MakeNetworkAddress(FLAGS_hostname, FLAGS_be_port);
@@ -289,10 +258,7 @@ void SimpleScheduler::UpdateMembership(
 
     // If this impalad is not in our view of the membership list, we should add it and
     // tell the statestore.
-    bool is_offline = ExecEnv::GetInstance() &&
-        ExecEnv::GetInstance()->impala_server()->IsOffline();
-    if (!is_offline &&
-        current_membership_.find(local_backend_id_) == current_membership_.end()) {
+    if (current_membership_.find(local_backend_id_) == current_membership_.end()) {
       VLOG(1) << "Registering local backend with statestore";
       subscriber_topic_updates->push_back(TTopicDelta());
       TTopicDelta& update = subscriber_topic_updates->back();
@@ -308,13 +274,6 @@ void SimpleScheduler::UpdateMembership(
                      << " " << status.GetDetail();
         subscriber_topic_updates->pop_back();
       }
-    } else if (is_offline &&
-        current_membership_.find(local_backend_id_) != current_membership_.end()) {
-      LOG(WARNING) << "Removing offline ImpalaServer from statestore";
-      subscriber_topic_updates->push_back(TTopicDelta());
-      TTopicDelta& update = subscriber_topic_updates->back();
-      update.topic_name = IMPALA_MEMBERSHIP_TOPIC;
-      update.topic_deletions.push_back(local_backend_id_);
     }
     if (metrics_ != NULL) {
       num_fragment_instances_metric_->set_value(current_membership_.size());
@@ -626,7 +585,6 @@ void SimpleScheduler::ComputeFragmentHosts(const TQueryExecRequest& exec_request
   for (const FragmentExecParams& exec_params: *fragment_exec_params) {
     unique_hosts.insert(exec_params.hosts.begin(), exec_params.hosts.end());
   }
-
   schedule->SetUniqueHosts(unique_hosts);
 }
 
@@ -723,38 +681,12 @@ Status SimpleScheduler::Schedule(Coordinator* coord, QuerySchedule* schedule) {
   schedule->set_request_pool(resolved_pool);
   schedule->summary_profile()->AddInfoString("Request Pool", resolved_pool);
 
-  if (ExecEnv::GetInstance()->impala_server()->IsOffline()) {
-    return Status("This Impala server is offline. Please retry your query later.");
-  }
-
   RETURN_IF_ERROR(ComputeScanRangeAssignment(schedule->request(), schedule));
   ComputeFragmentHosts(schedule->request(), schedule);
   ComputeFragmentExecParams(schedule->request(), schedule);
   if (!FLAGS_disable_admission_control) {
     RETURN_IF_ERROR(admission_controller_->AdmitQuery(schedule));
   }
-  if (!FLAGS_enable_rm) return Status::OK();
-  string user = GetEffectiveUser(schedule->request().query_ctx.session);
-  if (user.empty()) user = "default";
-  schedule->PrepareReservationRequest(resolved_pool, user);
-  const TResourceBrokerReservationRequest& reservation_request =
-      schedule->reservation_request();
-  if (!reservation_request.resources.empty()) {
-    Status status = resource_broker_->Reserve(
-        reservation_request, schedule->reservation());
-    if (!status.ok()) {
-      // Warn about missing table and/or column stats if necessary.
-      const TQueryCtx& query_ctx = schedule->request().query_ctx;
-      if (!query_ctx.__isset.parent_query_id &&
-          query_ctx.__isset.tables_missing_stats &&
-          !query_ctx.tables_missing_stats.empty()) {
-        status.AddDetail(GetTablesMissingStatsWarning(query_ctx.tables_missing_stats));
-      }
-      return status;
-    }
-    RETURN_IF_ERROR(schedule->ValidateReservation());
-    AddToActiveResourceMaps(*schedule->reservation(), coord);
-  }
   return Status::OK();
 }
 
@@ -762,106 +694,9 @@ Status SimpleScheduler::Release(QuerySchedule* schedule) {
   if (!FLAGS_disable_admission_control) {
     RETURN_IF_ERROR(admission_controller_->ReleaseQuery(schedule));
   }
-  if (FLAGS_enable_rm && schedule->NeedsRelease()) {
-    DCHECK(resource_broker_ != NULL);
-    Status status = resource_broker_->ReleaseReservation(
-        schedule->reservation()->reservation_id);
-    // Remove the reservation from the active-resource maps even if there was an error
-    // releasing the reservation because the query running in the reservation is done.
-    RemoveFromActiveResourceMaps(*schedule->reservation());
-    RETURN_IF_ERROR(status);
-  }
   return Status::OK();
 }
 
-void SimpleScheduler::AddToActiveResourceMaps(
-    const TResourceBrokerReservationResponse& reservation, Coordinator* coord) {
-  lock_guard<mutex> l(active_resources_lock_);
-  active_reservations_[reservation.reservation_id] = coord;
-  map<TNetworkAddress, llama::TAllocatedResource>::const_iterator iter;
-  for (iter = reservation.allocated_resources.begin();
-      iter != reservation.allocated_resources.end();
-      ++iter) {
-    TUniqueId client_resource_id;
-    client_resource_id << iter->second.client_resource_id;
-    active_client_resources_[client_resource_id] = coord;
-  }
-}
-
-void SimpleScheduler::RemoveFromActiveResourceMaps(
-    const TResourceBrokerReservationResponse& reservation) {
-  lock_guard<mutex> l(active_resources_lock_);
-  active_reservations_.erase(reservation.reservation_id);
-  map<TNetworkAddress, llama::TAllocatedResource>::const_iterator iter;
-  for (iter = reservation.allocated_resources.begin();
-      iter != reservation.allocated_resources.end();
-      ++iter) {
-    TUniqueId client_resource_id;
-    client_resource_id << iter->second.client_resource_id;
-    active_client_resources_.erase(client_resource_id);
-  }
-}
-
-// TODO: Refactor the Handle*{Reservation,Resource} functions to avoid code duplication.
-void SimpleScheduler::HandlePreemptedReservation(const TUniqueId& reservation_id) {
-  VLOG_QUERY << "HandlePreemptedReservation client_id=" << reservation_id;
-  Coordinator* coord = NULL;
-  {
-    lock_guard<mutex> l(active_resources_lock_);
-    ActiveReservationsMap::iterator it = active_reservations_.find(reservation_id);
-    if (it != active_reservations_.end()) coord = it->second;
-  }
-  if (coord == NULL) {
-    LOG(WARNING) << "Ignoring preempted reservation id " << reservation_id
-                 << " because no active query using it was found.";
-  } else {
-    stringstream err_msg;
-    err_msg << "Reservation " << reservation_id << " was preempted";
-    Status status(err_msg.str());
-    coord->Cancel(&status);
-  }
-}
-
-void SimpleScheduler::HandlePreemptedResource(const TUniqueId& client_resource_id) {
-  VLOG_QUERY << "HandlePreemptedResource client_id=" << client_resource_id;
-  Coordinator* coord = NULL;
-  {
-    lock_guard<mutex> l(active_resources_lock_);
-    ActiveClientResourcesMap::iterator it =
-        active_client_resources_.find(client_resource_id);
-    if (it != active_client_resources_.end()) coord = it->second;
-  }
-  if (coord == NULL) {
-    LOG(WARNING) << "Ignoring preempted client resource id " << client_resource_id
-                 << " because no active query using it was found.";
-  } else {
-    stringstream err_msg;
-    err_msg << "Resource " << client_resource_id << " was preempted";
-    Status status(err_msg.str());
-    coord->Cancel(&status);
-  }
-}
-
-void SimpleScheduler::HandleLostResource(const TUniqueId& client_resource_id) {
-  VLOG_QUERY << "HandleLostResource preempting client_id=" << client_resource_id;
-  Coordinator* coord = NULL;
-  {
-    lock_guard<mutex> l(active_resources_lock_);
-    ActiveClientResourcesMap::iterator it =
-        active_client_resources_.find(client_resource_id);
-    if (it != active_client_resources_.end()) coord = it->second;
-  }
-  if (coord == NULL) {
-    LOG(WARNING) << "Ignoring lost client resource id " << client_resource_id
-                 << " because no active query using it was found.";
-  } else {
-    stringstream err_msg;
-    err_msg << "Resource " << client_resource_id << " was lost";
-    Status status(err_msg.str());
-    coord->Cancel(&status);
-  }
-}
-
 SimpleScheduler::AssignmentCtx::AssignmentCtx(
     const BackendConfig& backend_config,
     IntCounter* total_assignments, IntCounter* total_local_assignments)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/scheduling/simple-scheduler.h
----------------------------------------------------------------------
diff --git a/be/src/scheduling/simple-scheduler.h b/be/src/scheduling/simple-scheduler.h
index dd119c2..8c96dde 100644
--- a/be/src/scheduling/simple-scheduler.h
+++ b/be/src/scheduling/simple-scheduler.h
@@ -36,12 +36,10 @@
 #include "scheduling/admission-controller.h"
 #include "scheduling/backend-config.h"
 #include "gen-cpp/Types_types.h"  // for TNetworkAddress
-#include "gen-cpp/ResourceBrokerService_types.h"
 #include "rapidjson/rapidjson.h"
 
 namespace impala {
 
-class ResourceBroker;
 class Coordinator;
 
 class SchedulerWrapper;
@@ -78,22 +76,18 @@ class SimpleScheduler : public Scheduler {
   ///  - backend_address - the address that this backend listens on
   SimpleScheduler(StatestoreSubscriber* subscriber, const std::string& backend_id,
       const TNetworkAddress& backend_address, MetricGroup* metrics, Webserver* webserver,
-      ResourceBroker* resource_broker, RequestPoolService* request_pool_service);
+      RequestPoolService* request_pool_service);
 
   /// Initialize with a list of <host:port> pairs in 'static' mode - i.e. the set of
   /// backends is fixed and will not be updated.
   SimpleScheduler(const std::vector<TNetworkAddress>& backends, MetricGroup* metrics,
-      Webserver* webserver, ResourceBroker* resource_broker,
-      RequestPoolService* request_pool_service);
+      Webserver* webserver, RequestPoolService* request_pool_service);
 
   /// Register with the subscription manager if required
   virtual impala::Status Init();
 
   virtual Status Schedule(Coordinator* coord, QuerySchedule* schedule);
   virtual Status Release(QuerySchedule* schedule);
-  virtual void HandlePreemptedReservation(const TUniqueId& reservation_id);
-  virtual void HandlePreemptedResource(const TUniqueId& client_resource_id);
-  virtual void HandleLostResource(const TUniqueId& client_resource_id);
 
  private:
   /// Map from a host's IP address to the next backend to be round-robin scheduled for
@@ -306,27 +300,6 @@ class SimpleScheduler : public Scheduler {
   /// Current number of backends
   IntGauge* num_fragment_instances_metric_;
 
-  /// Protect active_reservations_ and active_client_resources_.
-  boost::mutex active_resources_lock_;
-
-  /// Map from a Llama reservation id to the coordinator of the query using that
-  /// reservation. The map is used to cancel queries whose reservation has been preempted.
-  /// Entries are added in Schedule() calls that result in granted resource allocations.
-  /// Entries are removed in Release().
-  typedef boost::unordered_map<TUniqueId, Coordinator*> ActiveReservationsMap;
-  ActiveReservationsMap active_reservations_;
-
-  /// Map from client resource id to the coordinator of the query using that resource.
-  /// The map is used to cancel queries whose resource(s) have been preempted.
-  /// Entries are added in Schedule() calls that result in granted resource allocations.
-  /// Entries are removed in Release().
-  typedef boost::unordered_map<TUniqueId, Coordinator*> ActiveClientResourcesMap;
-  ActiveClientResourcesMap active_client_resources_;
-
-  /// Resource broker that mediates resource requests between Impala and the Llama.
-  /// Set to NULL if resource management is disabled.
-  ResourceBroker* resource_broker_;
-
   /// Used for user-to-pool resolution and looking up pool configurations. Not owned by
   /// us.
   RequestPoolService* request_pool_service_;
@@ -339,16 +312,6 @@ class SimpleScheduler : public Scheduler {
   BackendConfigPtr GetBackendConfig() const;
   void SetBackendConfig(const BackendConfigPtr& backend_config);
 
-  /// Add the granted reservation and resources to the active_reservations_ and
-  /// active_client_resources_ maps, respectively.
-  void AddToActiveResourceMaps(
-      const TResourceBrokerReservationResponse& reservation, Coordinator* coord);
-
-  /// Remove the given reservation and resources from the active_reservations_ and
-  /// active_client_resources_ maps, respectively.
-  void RemoveFromActiveResourceMaps(
-      const TResourceBrokerReservationResponse& reservation);
-
   /// Called asynchronously when an update is received from the subscription manager
   void UpdateMembership(const StatestoreSubscriber::TopicDeltaMap& incoming_topic_deltas,
       std::vector<TTopicDelta>* subscriber_topic_updates);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/service/impala-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index c749a6a..1b10aec 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -60,7 +60,6 @@
 #include "service/query-exec-state.h"
 #include "scheduling/simple-scheduler.h"
 #include "util/bit-util.h"
-#include "util/cgroups-mgr.h"
 #include "util/container-util.h"
 #include "util/debug-util.h"
 #include "util/error-util.h"
@@ -177,9 +176,9 @@ DEFINE_int32(idle_query_timeout, 0, "The time, in seconds, that a query may be i
     "QUERY_TIMEOUT_S overrides this setting, but, if set, --idle_query_timeout represents"
     " the maximum allowable timeout.");
 
-DEFINE_string(local_nodemanager_url, "", "The URL of the local Yarn Node Manager's HTTP "
-    "interface, used to detect if the Node Manager fails");
-DECLARE_bool(enable_rm);
+// TODO: Remove for Impala 3.0.
+DEFINE_string(local_nodemanager_url, "", "Deprecated");
+
 DECLARE_bool(compact_catalog_topic);
 
 namespace impala {
@@ -361,12 +360,6 @@ ImpalaServer::ImpalaServer(ExecEnv* exec_env)
   query_expiration_thread_.reset(new Thread("impala-server", "query-expirer",
       bind<void>(&ImpalaServer::ExpireQueries, this)));
 
-  is_offline_ = false;
-  if (FLAGS_enable_rm) {
-    nm_failure_detection_thread_.reset(new Thread("impala-server", "nm-failure-detector",
-            bind<void>(&ImpalaServer::DetectNmFailures, this)));
-  }
-
   exec_env_->SetImpalaServer(this);
 }
 
@@ -783,9 +776,7 @@ Status ImpalaServer::ExecuteInternal(
     shared_ptr<QueryExecState>* exec_state) {
   DCHECK(session_state != NULL);
   *registered_exec_state = false;
-  if (IsOffline()) {
-    return Status("This Impala server is offline. Please retry your query later.");
-  }
+
   exec_state->reset(new QueryExecState(query_ctx, exec_env_, exec_env_->frontend(),
       this, session_state));
 
@@ -1938,81 +1929,6 @@ shared_ptr<ImpalaServer::QueryExecState> ImpalaServer::GetQueryExecState(
   }
 }
 
-void ImpalaServer::SetOffline(bool is_offline) {
-  lock_guard<mutex> l(is_offline_lock_);
-  is_offline_ = is_offline;
-  ImpaladMetrics::IMPALA_SERVER_READY->set_value(is_offline);
-}
-
-void ImpalaServer::DetectNmFailures() {
-  DCHECK(FLAGS_enable_rm);
-  if (FLAGS_local_nodemanager_url.empty()) {
-    LOG(WARNING) << "No NM address set (--nm_addr is empty), no NM failure detection "
-                 << "thread started";
-    return;
-  }
-  // We only want a network address to open a socket to, for now. Get rid of http(s)://
-  // prefix, and split the string into hostname:port.
-  if (istarts_with(FLAGS_local_nodemanager_url, "http://")) {
-    FLAGS_local_nodemanager_url =
-        FLAGS_local_nodemanager_url.substr(string("http://").size());
-  } else if (istarts_with(FLAGS_local_nodemanager_url, "https://")) {
-    FLAGS_local_nodemanager_url =
-        FLAGS_local_nodemanager_url.substr(string("https://").size());
-  }
-  vector<string> components;
-  split(components, FLAGS_local_nodemanager_url, is_any_of(":"));
-  if (components.size() < 2) {
-    LOG(ERROR) << "Could not parse network address from --local_nodemanager_url, no NM"
-               << " failure detection thread started";
-    return;
-  }
-  DCHECK_GE(components.size(), 2);
-  TNetworkAddress nm_addr =
-      MakeNetworkAddress(components[0], atoi(components[1].c_str()));
-
-  MissedHeartbeatFailureDetector failure_detector(MAX_NM_MISSED_HEARTBEATS,
-      MAX_NM_MISSED_HEARTBEATS / 2);
-  struct addrinfo* addr;
-  if (getaddrinfo(nm_addr.hostname.c_str(), components[1].c_str(), NULL, &addr)) {
-    LOG(WARNING) << "Could not resolve NM address: " << nm_addr << ". Error was: "
-                 << GetStrErrMsg();
-    return;
-  }
-  LOG(INFO) << "Starting NM failure-detection thread, NM at: " << nm_addr;
-  // True if the last time through the loop Impala had failed, otherwise false. Used to
-  // only change the offline status when there's a change in state.
-  bool last_failure_state = false;
-  while (true) {
-    int sockfd = socket(AF_INET, SOCK_STREAM, 0);
-    if (sockfd >= 0) {
-      if (connect(sockfd, addr->ai_addr, sizeof(sockaddr)) < 0) {
-        failure_detector.UpdateHeartbeat(FLAGS_local_nodemanager_url, false);
-      } else {
-        failure_detector.UpdateHeartbeat(FLAGS_local_nodemanager_url, true);
-      }
-      ::close(sockfd);
-    } else {
-      LOG(ERROR) << "Could not create socket! Error was: " << GetStrErrMsg();
-    }
-    bool is_failed = (failure_detector.GetPeerState(FLAGS_local_nodemanager_url) ==
-        FailureDetector::FAILED);
-    if (is_failed != last_failure_state) {
-      if (is_failed) {
-        LOG(WARNING) <<
-            "ImpalaServer is going offline while local node-manager connectivity is bad";
-      } else {
-        LOG(WARNING) <<
-            "Node-manager connectivity has been restored. ImpalaServer is now online";
-      }
-      SetOffline(is_failed);
-    }
-    last_failure_state = is_failed;
-    SleepForMs(2000);
-  }
-  freeaddrinfo(addr);
-}
-
 void ImpalaServer::UpdateFilter(TUpdateFilterResult& result,
     const TUpdateFilterParams& params) {
   shared_ptr<QueryExecState> query_exec_state = GetQueryExecState(params.query_id, false);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/service/impala-server.h
----------------------------------------------------------------------
diff --git a/be/src/service/impala-server.h b/be/src/service/impala-server.h
index f756932..2104c5e 100644
--- a/be/src/service/impala-server.h
+++ b/be/src/service/impala-server.h
@@ -240,12 +240,6 @@ class ImpalaServer : public ImpalaServiceIf, public ImpalaHiveServer2ServiceIf,
   void CatalogUpdateCallback(const StatestoreSubscriber::TopicDeltaMap& topic_deltas,
       std::vector<TTopicDelta>* topic_updates);
 
-  /// Returns true if Impala is offline (and not accepting queries), false otherwise.
-  bool IsOffline() {
-    boost::lock_guard<boost::mutex> l(is_offline_lock_);
-    return is_offline_;
-  }
-
   /// Returns true if lineage logging is enabled, false otherwise.
   bool IsLineageLoggingEnabled();
 
@@ -633,15 +627,6 @@ class ImpalaServer : public ImpalaServiceIf, public ImpalaHiveServer2ServiceIf,
   /// FLAGS_idle_query_timeout seconds.
   void ExpireQueries();
 
-  /// Periodically opens a socket to FLAGS_local_nodemanager_url to check if the Yarn Node
-  /// Manager is running. If not, this method calls SetOffline(true), and when the NM
-  /// recovers, calls SetOffline(false). Only called (in nm_failure_detection_thread_) if
-  /// FLAGS_enable_rm is true.
-  void DetectNmFailures();
-
-  /// Set is_offline_ to the argument's value.
-  void SetOffline(bool offline);
-
   /// Guards query_log_ and query_log_index_
   boost::mutex query_log_lock_;
 
@@ -963,15 +948,6 @@ class ImpalaServer : public ImpalaServiceIf, public ImpalaHiveServer2ServiceIf,
 
   /// Container for a thread that runs ExpireQueries() if FLAGS_idle_query_timeout is set.
   boost::scoped_ptr<Thread> query_expiration_thread_;
-
-  /// Container thread for DetectNmFailures().
-  boost::scoped_ptr<Thread> nm_failure_detection_thread_;
-
-  /// Protects is_offline_
-  boost::mutex is_offline_lock_;
-
-  /// True if Impala server is offline, false otherwise.
-  bool is_offline_;
 };
 
 /// Create an ImpalaServer and Thrift servers.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/service/impalad-main.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impalad-main.cc b/be/src/service/impalad-main.cc
index de1a56b..55d9ad6 100644
--- a/be/src/service/impalad-main.cc
+++ b/be/src/service/impalad-main.cc
@@ -54,6 +54,7 @@ DECLARE_int32(beeswax_port);
 DECLARE_int32(hs2_port);
 DECLARE_int32(be_port);
 DECLARE_string(principal);
+DECLARE_bool(enable_rm);
 
 int ImpaladMain(int argc, char** argv) {
   InitCommonRuntime(argc, argv, true);
@@ -66,6 +67,13 @@ int ImpaladMain(int argc, char** argv) {
   ABORT_IF_ERROR(HiveUdfCall::Init());
   InitFeSupport();
 
+  if (FLAGS_enable_rm) {
+    // TODO: Remove in Impala 3.0.
+    LOG(WARNING) << "*****************************************************************";
+    LOG(WARNING) << "Llama support has been deprecated. FLAGS_enable_rm has no effect.";
+    LOG(WARNING) << "*****************************************************************";
+  }
+
   // start backend service for the coordinator on be_port
   ExecEnv exec_env;
   StartThreadInstrumentation(exec_env.metrics(), exec_env.webserver());

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/service/query-exec-state.cc
----------------------------------------------------------------------
diff --git a/be/src/service/query-exec-state.cc b/be/src/service/query-exec-state.cc
index cea24bb..eb710fb 100644
--- a/be/src/service/query-exec-state.cc
+++ b/be/src/service/query-exec-state.cc
@@ -21,7 +21,6 @@
 
 #include "exprs/expr.h"
 #include "exprs/expr-context.h"
-#include "resourcebroker/resource-broker.h"
 #include "runtime/mem-tracker.h"
 #include "runtime/row-batch.h"
 #include "runtime/runtime-state.h"
@@ -48,7 +47,6 @@ using namespace strings;
 
 DECLARE_int32(catalog_service_port);
 DECLARE_string(catalog_service_host);
-DECLARE_bool(enable_rm);
 DECLARE_int64(max_result_cache_size);
 
 namespace impala {
@@ -428,34 +426,16 @@ Status ImpalaServer::QueryExecState::ExecQueryOrDmlRequest(
       query_exec_request.fragments[0].partition.type == TPartitionType::UNPARTITIONED;
   DCHECK(has_coordinator_fragment || query_exec_request.__isset.desc_tbl);
 
-  if (FLAGS_enable_rm) {
-    DCHECK(exec_env_->resource_broker() != NULL);
-  }
   schedule_.reset(new QuerySchedule(query_id(), query_exec_request,
       exec_request_.query_options, &summary_profile_, query_events_));
   coord_.reset(new Coordinator(exec_request_.query_options, exec_env_, query_events_));
   Status status = exec_env_->scheduler()->Schedule(coord_.get(), schedule_.get());
-  if (FLAGS_enable_rm) {
-    if (status.ok()) {
-      stringstream reservation_request_ss;
-      reservation_request_ss << schedule_->reservation_request();
-      summary_profile_.AddInfoString("Resource reservation request",
-          reservation_request_ss.str());
-    }
-  }
 
   {
     lock_guard<mutex> l(lock_);
     RETURN_IF_ERROR(UpdateQueryStatus(status));
   }
 
-  if (FLAGS_enable_rm && schedule_->HasReservation()) {
-    // Add the granted reservation to the query profile.
-    stringstream reservation_ss;
-    reservation_ss << *schedule_->reservation();
-    summary_profile_.AddInfoString("Granted resource reservation", reservation_ss.str());
-    query_events_->MarkEvent("Resources reserved");
-  }
   status = coord_->Exec(*schedule_, &output_expr_ctxs_);
   {
     lock_guard<mutex> l(lock_);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/util/CMakeLists.txt b/be/src/util/CMakeLists.txt
index da682f4..62fff80 100644
--- a/be/src/util/CMakeLists.txt
+++ b/be/src/util/CMakeLists.txt
@@ -36,7 +36,6 @@ add_library(Util
   bitmap.cc
   bit-util.cc
   bloom-filter.cc
-  cgroups-mgr.cc
   coding-util.cc
   codec.cc
   compress.cc
@@ -54,7 +53,6 @@ add_library(Util
   hdr-histogram.cc
   impalad-metrics.cc
   jni-util.cc
-  llama-util.cc
   logging-support.cc
   mem-info.cc
   memory-metrics.cc

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/util/cgroups-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/util/cgroups-mgr.cc b/be/src/util/cgroups-mgr.cc
deleted file mode 100644
index e49d57c..0000000
--- a/be/src/util/cgroups-mgr.cc
+++ /dev/null
@@ -1,238 +0,0 @@
-// 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.
-
-#include "util/cgroups-mgr.h"
-
-#include <fstream>
-#include <sstream>
-#include <boost/filesystem.hpp>
-#include "util/debug-util.h"
-#include <gutil/strings/substitute.h>
-
-#include "common/names.h"
-
-using boost::filesystem::create_directory;
-using boost::filesystem::exists;
-using boost::filesystem::remove;
-using namespace impala;
-using namespace strings;
-
-namespace impala {
-
-// Suffix appended to Yarn resource ids to form an Impala-internal cgroups.
-const std::string IMPALA_CGROUP_SUFFIX = "_impala";
-
-// Yarn's default multiplier for translating virtual CPU cores into cgroup CPU shares.
-// See Yarn's CgroupsLCEResourcesHandler.java for more details.
-const int32_t CPU_DEFAULT_WEIGHT = 1024;
-
-CgroupsMgr::CgroupsMgr(MetricGroup* metrics) {
-  active_cgroups_metric_ = metrics->AddGauge<int64_t>("cgroups-mgr.active-cgroups", 0);
-}
-
-Status CgroupsMgr::Init(const string& cgroups_hierarchy_path,
-      const string& staging_cgroup) {
-  cgroups_hierarchy_path_ = cgroups_hierarchy_path;
-  staging_cgroup_ = staging_cgroup;
-  // Set up the staging cgroup for Impala to retire execution threads into.
-  RETURN_IF_ERROR(CreateCgroup(staging_cgroup, true));
-  return Status::OK();
-}
-
-string CgroupsMgr::UniqueIdToCgroup(const string& unique_id) const {
-  if (unique_id.empty()) return "";
-  return unique_id + IMPALA_CGROUP_SUFFIX;
-}
-
-int32_t CgroupsMgr::VirtualCoresToCpuShares(int16_t v_cpu_cores) {
-  if (v_cpu_cores <= 0) return -1;
-  return CPU_DEFAULT_WEIGHT * v_cpu_cores;
-}
-
-Status CgroupsMgr::CreateCgroup(const string& cgroup, bool if_not_exists) const {
-  const string& cgroup_path = Substitute("$0/$1", cgroups_hierarchy_path_, cgroup);
-  try {
-    // Returns false if the dir already exists, otherwise throws an exception.
-    if (!create_directory(cgroup_path) && !if_not_exists) {
-      stringstream err_msg;
-      err_msg << "Failed to create CGroup at path " << cgroup_path
-              << ". Path already exists.";
-      return Status(err_msg.str());
-    }
-    LOG(INFO) << "Created CGroup " << cgroup_path;
-  } catch (std::exception& e) {
-    stringstream err_msg;
-    err_msg << "Failed to create CGroup at path " << cgroup_path << ". " << e.what();
-    return Status(err_msg.str());
-  }
-  return Status::OK();
-}
-
-Status CgroupsMgr::DropCgroup(const string& cgroup, bool if_exists) const {
-  const string& cgroup_path = Substitute("$0/$1", cgroups_hierarchy_path_, cgroup);
-  LOG(INFO) << "Dropping CGroup " << cgroups_hierarchy_path_ << " " << cgroup;
-  try {
-    if(!remove(cgroup_path) && !if_exists) {
-      stringstream err_msg;
-      err_msg << "Failed to create CGroup at path " << cgroup_path
-              << ". Path does not exist.";
-      return Status(err_msg.str());
-    }
-  } catch (std::exception& e) {
-    stringstream err_msg;
-    err_msg << "Failed to drop CGroup at path " << cgroup_path << ". " << e.what();
-    return Status(err_msg.str());
-  }
-  return Status::OK();
-}
-
-Status CgroupsMgr::SetCpuShares(const string& cgroup, int32_t num_shares) {
-  string cgroup_path;
-  string tasks_path;
-  RETURN_IF_ERROR(GetCgroupPaths(cgroup, &cgroup_path, &tasks_path));
-
-  const string& cpu_shares_path = Substitute("$0/$1", cgroup_path, "cpu.shares");
-  ofstream cpu_shares(tasks_path.c_str(), ios::out | ios::trunc);
-  if (!cpu_shares.is_open()) {
-    stringstream err_msg;
-    err_msg << "CGroup CPU shares file: " << cpu_shares_path
-            << " is not writable by Impala";
-    return Status(err_msg.str());
-  }
-
-  LOG(INFO) << "Setting CPU shares of CGroup " << cgroup_path << " to " << num_shares;
-  cpu_shares << num_shares << endl;
-  return Status::OK();
-}
-
-Status CgroupsMgr::GetCgroupPaths(const std::string& cgroup,
-    std::string* cgroup_path, std::string* tasks_path) const {
-  stringstream cgroup_path_ss;
-  cgroup_path_ss << cgroups_hierarchy_path_ << "/" << cgroup;
-  *cgroup_path = cgroup_path_ss.str();
-  if (!exists(*cgroup_path)) {
-    stringstream err_msg;
-    err_msg << "CGroup " << *cgroup_path << " does not exist";
-    return Status(err_msg.str());
-  }
-
-  stringstream tasks_path_ss;
-  tasks_path_ss << *cgroup_path << "/tasks";
-  *tasks_path = tasks_path_ss.str();
-  if (!exists(*tasks_path)) {
-    stringstream err_msg;
-    err_msg << "CGroup " << *cgroup_path << " does not have a /tasks file";
-    return Status(err_msg.str());
-  }
-  return Status::OK();
-}
-
-Status CgroupsMgr::AssignThreadToCgroup(const Thread& thread,
-    const string& cgroup) const {
-  string cgroup_path;
-  string tasks_path;
-  RETURN_IF_ERROR(GetCgroupPaths(cgroup, &cgroup_path, &tasks_path));
-
-  ofstream tasks(tasks_path.c_str(), ios::out | ios::app);
-  if (!tasks.is_open()) {
-    stringstream err_msg;
-    err_msg << "CGroup tasks file: " << tasks_path << " is not writable by Impala";
-    return Status(err_msg.str());
-  }
-  tasks << thread.tid() << endl;
-
-  VLOG_ROW << "Thread " << thread.tid() << " moved to CGroup " << cgroup_path;
-  tasks.close();
-  return Status::OK();
-}
-
-Status CgroupsMgr::RelocateThreads(const string& src_cgroup,
-    const string& dst_cgroup) const {
-  string src_cgroup_path;
-  string src_tasks_path;
-  RETURN_IF_ERROR(GetCgroupPaths(src_cgroup, &src_cgroup_path, &src_tasks_path));
-
-  string dst_cgroup_path;
-  string dst_tasks_path;
-  RETURN_IF_ERROR(GetCgroupPaths(dst_cgroup, &dst_cgroup_path, &dst_tasks_path));
-
-  ifstream src_tasks(src_tasks_path.c_str());
-  if (!src_tasks) {
-    stringstream err_msg;
-    err_msg << "Failed to open source CGroup tasks file at: " << src_tasks_path;
-    return Status(err_msg.str());
-  }
-
-  ofstream dst_tasks(dst_tasks_path.c_str(), ios::out | ios::app);
-  if (!dst_tasks) {
-    stringstream err_msg;
-    err_msg << "Failed to open destination CGroup tasks file at: " << dst_tasks_path;
-    return Status(err_msg.str());
-  }
-
-  int32_t tid;
-  while (src_tasks >> tid) {
-    dst_tasks << tid << endl;
-    // Attempting to write a non-existent tid/pid will result in an error,
-    // so clear the error flags after every append.
-    dst_tasks.clear();
-    VLOG_ROW << "Relocating thread id " << tid << " from " << src_tasks_path
-             << " to " << dst_tasks_path;
-  }
-
-  return Status::OK();
-}
-
-Status CgroupsMgr::RegisterFragment(const TUniqueId& fragment_instance_id,
-    const string& cgroup, bool* is_first) {
-  if (cgroup.empty() || cgroups_hierarchy_path_.empty()) return Status::OK();
-
-  LOG(INFO) << "Registering fragment " << PrintId(fragment_instance_id)
-            << " with CGroup " << cgroups_hierarchy_path_ << "/" << cgroup;
-  lock_guard<mutex> l(active_cgroups_lock_);
-  if (++active_cgroups_[cgroup] == 1) {
-    *is_first = true;
-    RETURN_IF_ERROR(CreateCgroup(cgroup, false));
-    active_cgroups_metric_->Increment(1);
-  } else {
-    *is_first = false;
-  }
-  return Status::OK();
-}
-
-Status CgroupsMgr::UnregisterFragment(const TUniqueId& fragment_instance_id,
-    const string& cgroup) {
-  if (cgroup.empty() || cgroups_hierarchy_path_.empty()) return Status::OK();
-
-  LOG(INFO) << "Unregistering fragment " << PrintId(fragment_instance_id)
-            << " from CGroup " << cgroups_hierarchy_path_ << "/" << cgroup;
-  lock_guard<mutex> l(active_cgroups_lock_);
-  unordered_map<string, int32_t>::iterator entry = active_cgroups_.find(cgroup);
-  DCHECK(entry != active_cgroups_.end());
-
-  int32_t* ref_count = &entry->second;
-  --(*ref_count);
-  if (*ref_count == 0) {
-    RETURN_IF_ERROR(RelocateThreads(cgroup, staging_cgroup_));
-    RETURN_IF_ERROR(DropCgroup(cgroup, false));
-    active_cgroups_metric_->Increment(-1);
-    active_cgroups_.erase(entry);
-  }
-  return Status::OK();
-}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/util/cgroups-mgr.h
----------------------------------------------------------------------
diff --git a/be/src/util/cgroups-mgr.h b/be/src/util/cgroups-mgr.h
deleted file mode 100644
index 2e52f6b..0000000
--- a/be/src/util/cgroups-mgr.h
+++ /dev/null
@@ -1,175 +0,0 @@
-// 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.
-
-#ifndef IMPALA_UTIL_CGROUPS_MGR_H
-#define IMPALA_UTIL_CGROUPS_MGR_H
-
-#include <string>
-#include <boost/thread/mutex.hpp>
-#include <boost/unordered_map.hpp>
-#include "common/status.h"
-#include "util/metrics.h"
-#include "util/thread.h"
-
-namespace impala {
-
-/// Control Groups, or 'cgroups', are a Linux-specific mechanism for arbitrating resources
-/// amongst threads.
-//
-/// CGroups are organised in a forest of 'hierarchies', each of which are mounted at a path
-/// in the filesystem. Each hierarchy contains one or more cgroups, arranged
-/// hierarchically. Each hierarchy has one or more 'subsystems' attached. Each subsystem
-/// represents a resource to manage, so for example there is a CPU subsystem and a MEMORY
-/// subsystem. There are rules about when subsystems may be attached to more than one
-/// hierarchy, which are out of scope of this description.
-//
-/// Each thread running on a kernel with cgroups enabled belongs to exactly one cgroup in
-/// every hierarchy at once. Impala is only concerned with a single hierarchy that assigns
-/// CPU resources in the first instance. Threads are assigned to cgroups by writing their
-/// thread ID to a file in the special cgroup filesystem.
-//
-/// For more information:
-/// access.redhat.com/site/documentation/en-US/Red_Hat_Enterprise_Linux/6/html/Resource_Management_Guide/
-/// www.kernel.org/doc/Documentation/cgroups/cgroups.txt
-
-/// Manages the lifecycle of Impala-internal cgroups as well as the assignment of
-/// execution threads into cgroups.
-/// To execute queries Impala requests resources from Yarn via the Llama. Yarn returns
-/// granted resources via the Llama in the form or "RM resource ids" that conventionally
-/// correspond to a CGroups that the Yarn NM creates. Instead of directly using the
-/// NM-provided CGroups, Impala creates and manages its own CGroups for the
-/// following reasons:
-/// 1. In typical CM/Yarn setups, Impala would not have permissions to write to the tasks
-///    file of NM-provided CGroups. It is arguably not even desirable (e.g., for security
-///    reasons) for external process to be able to manipulate the permissions of
-///    NM-generated CGroups either directly or indirectly.
-/// 2. Yarn-granted CGroups are created asynchronously (the AM calls to create the
-///    CGroups are non-blocking). From Impala's perspective that means that once Impala
-///    receives notice from the Llama that resources have been granted, it cannot
-///    assume that the corresponding containers have been created (although the Yarn
-///    NMs eventually will). While each of Impala's plan fragments could wait for the
-///    CGroups to be created, it seems unnecessarily complicated and slow to do so.
-/// 3. Impala will probably want to manage its own CGroups eventually, e.g., for
-///    optimistic query scheduling.
-//
-/// In summary, the typical CGroups-related flow of an Impala query is as follows:
-/// 1. Impala receives granted resources from Llama and sends out plan fragments
-/// 2. On each node execution such a fragment, convert the Yarn resource id into
-///    a CGroup that Impala should create and assign the query's threads to
-/// 3. Register the fragment(s) and the CGroup for the query with the
-///    node-local CGroup manager. The registration creates the CGroup maintains a
-///    count of all fragments using that CGroup.
-/// 4. Execute the fragments, assigning threads into the Impala-managed CGroup.
-/// 5. Complete the fragments by unregistering them with the CGroup from the node-local
-///    CGroups manager. When the last fragment for a CGroup is unregistered, all threads
-///    from that CGroup are relocated into a special staging CGroup, so that the now
-///    unused CGroup can safely be deleted (otherwise, we'd have to wait for the OS to
-///    drain all entries from the CGroup's tasks file)
-class CgroupsMgr {
- public:
-  CgroupsMgr(MetricGroup* metrics);
-
-  /// Sets the cgroups mgr's corresponding members and creates the staging cgroup
-  /// under <cgroups_hierarchy_path>/<staging_cgroup>. Returns a non-OK status if
-  /// creation of the staging cgroup failed, e.g., because of insufficient privileges.
-  Status Init(const std::string& cgroups_hierarchy_path,
-      const std::string& staging_cgroup);
-
-  /// Returns the cgroup Impala should create and use for enforcing granted resources
-  /// identified by the given unique ID (which usually corresponds to a query ID). Returns
-  /// an empty string if unique_id is empty.
-  std::string UniqueIdToCgroup(const std::string& unique_id) const;
-
-  /// Returns the cgroup CPU shares corresponding to the given number of virtual cores.
-  /// Returns -1 if v_cpu_cores is <= 0 (which is invalid).
-  int32_t VirtualCoresToCpuShares(int16_t v_cpu_cores);
-
-  /// Informs the cgroups mgr that a plan fragment intends to use the given cgroup.
-  /// If this is the first fragment requesting use of cgroup, then the cgroup will
-  /// be created and *is_first will be set to true (otherwise to false). In any case the
-  /// reference count active_cgroups_[cgroup] is incremented. Returns a non-OK status
-  /// if there was an error creating the cgroup.
-  Status RegisterFragment(const TUniqueId& fragment_instance_id,
-      const std::string& cgroup, bool* is_first);
-
-  /// Informs the cgroups mgr that a plan fragment using the given cgroup is complete.
-  /// Decrements the corresponding reference count active_cgroups_[cgroup]. If the
-  /// reference count reaches zero this function relocates all thread ids from
-  /// the cgroup to the staging_cgroup_ and drops cgroup (a cgroup with active thread ids
-  /// cannot be dropped, so we relocate the thread ids first).
-  /// Returns a non-OK status there was an error creating the cgroup.
-  Status UnregisterFragment(const TUniqueId& fragment_instance_id,
-      const std::string& cgroup);
-
-  /// Creates a cgroup at <cgroups_hierarchy_path_>/<cgroup>. Returns a non-OK status
-  /// if the cgroup creation failed, e.g., because of insufficient privileges.
-  /// If is_not_exists is true then no error is returned if the cgroup already exists.
-  Status CreateCgroup(const std::string& cgroup, bool if_not_exists) const;
-
-  /// Drops the cgroup at <cgroups_hierarchy_path_>/<cgroup>. Returns a non-OK status
-  /// if the cgroup deletion failed, e.g., because of insufficient privileges.
-  /// If if_exists is true then no error is returned if the cgroup does not exist.
-  Status DropCgroup(const std::string& cgroup, bool if_exists) const;
-
-  /// Sets the number of CPU shares for the given cgroup by writing num_shares into the
-  /// cgroup's cpu.shares file. Returns a non-OK status if there was an error writing
-  /// to the file, e.g., because of insufficient privileges.
-  Status SetCpuShares(const std::string& cgroup, int32_t num_shares);
-
-  /// Assigns a given thread to a cgroup, by writing its thread id to
-  /// <cgroups_hierarchy_path_>/<cgroup>/tasks. If there is no file at that
-  /// location, returns an error. Otherwise no attempt is made to check that the
-  /// target belongs to a cgroup hierarchy due to the cost of reading and parsing
-  /// cgroup information from the filesystem.
-  Status AssignThreadToCgroup(const Thread& thread, const std::string& cgroup) const;
-
-  /// Reads the <cgroups_hierarchy_path_>/<src_cgroup>/tasks file and writing all the
-  /// contained thread ids to <cgroups_hierarchy_path_>/<dst_cgroup>/tasks.
-  /// Assumes that the destination cgroup has already been created. Returns a non-OK
-  /// status if there was an error reading src_cgroup and/or writing dst_cgroup.
-  Status RelocateThreads(const std::string& src_cgroup,
-      const std::string& dst_cgroup) const;
-
- private:
-  /// Checks that the cgroups_hierarchy_path_ and the given cgroup under it exists.
-  /// Returns an error if either of them do not exist.
-  /// Returns the absolute cgroup path and the absolute path to its tasks file.
-  Status GetCgroupPaths(const std::string& cgroup,
-      std::string* cgroup_path, std::string* tasks_path) const;
-
-  /// Number of currently active Impala-managed cgroups.
-  IntGauge* active_cgroups_metric_;
-
-  /// Root of the CPU cgroup hierarchy. Created cgroups are placed directly under it.
-  std::string cgroups_hierarchy_path_;
-
-  /// Cgroup that threads from completed queries are relocated into such that the
-  /// query's cgroup can be dropped.
-  std::string staging_cgroup_;
-
-  /// Protects active_cgroups_.
-  boost::mutex active_cgroups_lock_;
-
-  /// Process-wide map from cgroup to number of fragments using the cgroup.
-  /// A cgroup can be safely dropped once the number of fragments in the cgroup,
-  /// according to this map, reaches zero.
-  boost::unordered_map<std::string, int32_t> active_cgroups_;
-};
-
-}
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/util/debug-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/debug-util.h b/be/src/util/debug-util.h
index 4b17ed7..a02a288 100644
--- a/be/src/util/debug-util.h
+++ b/be/src/util/debug-util.h
@@ -32,7 +32,6 @@
 #include "gen-cpp/RuntimeProfile_types.h"
 #include "gen-cpp/ImpalaService_types.h"
 #include "gen-cpp/parquet_types.h"
-#include "gen-cpp/Llama_types.h"
 
 #include "runtime/descriptors.h" // for SchemaPath
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/util/llama-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/llama-util.cc b/be/src/util/llama-util.cc
deleted file mode 100644
index 82f2bd6..0000000
--- a/be/src/util/llama-util.cc
+++ /dev/null
@@ -1,152 +0,0 @@
-// 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.
-
-#include "util/llama-util.h"
-
-#include <sstream>
-#include <boost/algorithm/string/join.hpp>
-#include <boost/algorithm/string.hpp>
-
-#include "common/names.h"
-#include "util/debug-util.h"
-#include "util/uid-util.h"
-
-using boost::algorithm::is_any_of;
-using boost::algorithm::join;
-using boost::algorithm::split;
-using namespace llama;
-
-namespace llama {
-
-string PrintId(const TUniqueId& id, const string& separator) {
-  return PrintId(impala::CastTUniqueId<TUniqueId, impala::TUniqueId>(id), separator);
-}
-
-ostream& operator<<(ostream& os, const TUniqueId& id) {
-  os << hex << id.hi << ":" << id.lo;
-  return os;
-}
-
-ostream& operator<<(ostream& os, const TNetworkAddress& address) {
-  os << address.hostname << ":" << dec << address.port;
-  return os;
-}
-
-ostream& operator<<(ostream& os, const TResource& resource) {
-  os << "Resource("
-     << "client_resource_id=" << resource.client_resource_id << " "
-     << "v_cpu_cores=" << dec << resource.v_cpu_cores << " "
-     << "memory_mb=" << dec << resource.memory_mb << " "
-     << "asked_location=" << resource.askedLocation << " "
-     << "enforcement=" << resource.enforcement << ")";
-  return os;
-}
-
-ostream& operator<<(ostream& os, const TAllocatedResource& resource) {
-  os << "Allocated Resource("
-     << "reservation_id=" << resource.reservation_id << " "
-     << "client_resource_id=" << resource.client_resource_id << " "
-     << "rm_resource_id=" << resource.rm_resource_id << " "
-     << "v_cpu_cores=" << dec << resource.v_cpu_cores << " "
-     << "memory_mb=" << dec << resource.memory_mb << " "
-     << "location=" << resource.location << ")";
-  return os;
-}
-
-ostream& operator<<(ostream& os, const llama::TLlamaAMGetNodesRequest& request) {
-  os << "GetNodes Request(llama handle=" << request.am_handle << ")";
-  return os;
-}
-
-ostream& operator<<(ostream& os, const llama::TLlamaAMReservationRequest& request) {
-  os << "Reservation Request("
-     << "llama handle=" << request.am_handle << " "
-     << "queue=" << request.queue << " "
-     << "user=" << request.user << " "
-     << "gang=" << request.gang << " "
-     << "resources=[";
-  for (int i = 0; i < request.resources.size(); ++i) {
-    os << request.resources[i];
-    if (i + 1 != request.resources.size()) os << ",";
-  }
-  os << "])";
-  return os;
-}
-
-ostream& operator<<(ostream& os,
-    const llama::TLlamaAMReservationExpansionRequest& request) {
-  os << "Expansion Request("
-     << "llama handle=" << request.am_handle << " "
-     << "reservation id=" << request.expansion_of << " "
-     << "resource=" << request.resource << ")";
-  return os;
-}
-
-ostream& operator<<(ostream& os, const llama::TLlamaAMReleaseRequest& request) {
-  os << "Release Request("
-     << "llama handle=" << request.am_handle << " "
-     << "reservation id=" << request.reservation_id << ")";
-  return os;
-}
-
-llama::TUniqueId& operator<<(llama::TUniqueId& dest, const impala::TUniqueId& src) {
-  dest.lo = src.lo;
-  dest.hi = src.hi;
-  return dest;
-}
-
-impala::TUniqueId& operator<<(impala::TUniqueId& dest, const llama::TUniqueId& src) {
-  dest.lo = src.lo;
-  dest.hi = src.hi;
-  return dest;
-}
-
-bool operator==(const impala::TUniqueId& impala_id, const llama::TUniqueId& llama_id) {
-  return impala_id.lo == llama_id.lo && impala_id.hi == llama_id.hi;
-}
-
-llama::TNetworkAddress& operator<<(llama::TNetworkAddress& dest,
-    const impala::TNetworkAddress& src) {
-  dest.hostname = src.hostname;
-  dest.port = src.port;
-  return dest;
-}
-
-impala::TNetworkAddress& operator<<(impala::TNetworkAddress& dest,
-    const llama::TNetworkAddress& src) {
-  dest.hostname = src.hostname;
-  dest.port = src.port;
-  return dest;
-}
-
-impala::Status LlamaStatusToImpalaStatus(const TStatus& status,
-    const string& err_prefix) {
-  if (status.status_code == TStatusCode::OK) return impala::Status::OK();
-  stringstream ss;
-  ss << err_prefix << " " << join(status.error_msgs, ", ");
-  return impala::Status(ss.str());
-}
-
-string GetShortName(const string& user) {
-  if (user.empty() || user[0] == '/' || user[0] == '@') return user;
-
-  vector<string> components;
-  split(components, user, is_any_of("/@"));
-  return components[0];
-}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/util/llama-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/llama-util.h b/be/src/util/llama-util.h
deleted file mode 100644
index f6fc4ce..0000000
--- a/be/src/util/llama-util.h
+++ /dev/null
@@ -1,75 +0,0 @@
-// 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.
-
-#ifndef IMPALA_UTIL_LLAMA_UTIL_H
-#define IMPALA_UTIL_LLAMA_UTIL_H
-
-#include <ostream>
-#include <string>
-#include <boost/functional/hash.hpp>
-
-#include "gen-cpp/Types_types.h"  // for TUniqueId
-#include "gen-cpp/Llama_types.h"  // for TUniqueId
-#include "common/status.h"
-
-namespace llama {
-
-std::ostream& operator<<(std::ostream& os, const llama::TUniqueId& id);
-std::ostream& operator<<(std::ostream& os, const llama::TNetworkAddress& address);
-std::ostream& operator<<(std::ostream& os, const llama::TResource& resource);
-std::ostream& operator<<(std::ostream& os, const llama::TAllocatedResource& resource);
-
-std::ostream& operator<<(std::ostream& os,
-    const llama::TLlamaAMGetNodesRequest& request);
-std::ostream& operator<<(std::ostream& os,
-    const llama::TLlamaAMReservationRequest& request);
-std::ostream& operator<<(std::ostream& os,
-    const llama::TLlamaAMReservationExpansionRequest& request);
-std::ostream& operator<<(std::ostream& os,
-    const llama::TLlamaAMReleaseRequest& request);
-
-/// 'Assignment' operators to convert types between the llama and impala namespaces.
-llama::TUniqueId& operator<<(llama::TUniqueId& dest, const impala::TUniqueId& src);
-impala::TUniqueId& operator<<(impala::TUniqueId& dest, const llama::TUniqueId& src);
-
-std::string PrintId(const llama::TUniqueId& id, const std::string& separator = ":");
-
-bool operator==(const impala::TUniqueId& impala_id, const llama::TUniqueId& llama_id);
-
-llama::TNetworkAddress& operator<<(llama::TNetworkAddress& dest,
-    const impala::TNetworkAddress& src);
-impala::TNetworkAddress& operator<<(impala::TNetworkAddress& dest,
-    const llama::TNetworkAddress& src);
-
-impala::Status LlamaStatusToImpalaStatus(const llama::TStatus& status,
-    const std::string& err_prefix = "");
-
-/// This function must be called 'hash_value' to be picked up by boost.
-inline std::size_t hash_value(const llama::TUniqueId& id) {
-  std::size_t seed = 0;
-  boost::hash_combine(seed, id.lo);
-  boost::hash_combine(seed, id.hi);
-  return seed;
-}
-
-/// Get the short version of the user name (the user's name up to the first '/' or '@')
-/// If neither are found (or are found at the beginning of the user name) return username.
-std::string GetShortName(const std::string& user);
-
-}
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/util/thread-pool.h
----------------------------------------------------------------------
diff --git a/be/src/util/thread-pool.h b/be/src/util/thread-pool.h
index 8fc1bcc..ccc49c9 100644
--- a/be/src/util/thread-pool.h
+++ b/be/src/util/thread-pool.h
@@ -114,10 +114,6 @@ class ThreadPool {
     Join();
   }
 
-  Status AssignToCgroup(const std::string& cgroup) {
-    return threads_.SetCgroup(cgroup);
-  }
-
  private:
   /// Driver method for each thread in the pool. Continues to read work from the queue
   /// until the pool is shutdown.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/util/thread.cc
----------------------------------------------------------------------
diff --git a/be/src/util/thread.cc b/be/src/util/thread.cc
index 6a8b9b8..757ba59 100644
--- a/be/src/util/thread.cc
+++ b/be/src/util/thread.cc
@@ -26,7 +26,6 @@
 #include "util/coding-util.h"
 #include "util/debug-util.h"
 #include "util/error-util.h"
-#include "util/cgroups-mgr.h"
 #include "util/metrics.h"
 #include "util/webserver.h"
 #include "util/os-util.h"
@@ -321,10 +320,6 @@ void Thread::SuperviseThread(const string& name, const string& category,
 
 Status ThreadGroup::AddThread(Thread* thread) {
   threads_.push_back(thread);
-  if (!cgroup_path_.empty()) {
-    DCHECK(cgroups_mgr_ != NULL);
-    RETURN_IF_ERROR(cgroups_mgr_->AssignThreadToCgroup(*thread, cgroup_path_));
-  }
   return Status::OK();
 }
 
@@ -332,13 +327,4 @@ void ThreadGroup::JoinAll() {
   for (const Thread& thread: threads_) thread.Join();
 }
 
-Status ThreadGroup::SetCgroup(const string& cgroup) {
-  DCHECK(cgroups_mgr_ != NULL);
-  cgroup_path_ = cgroup;
-  for (const Thread& t: threads_) {
-    RETURN_IF_ERROR(cgroups_mgr_->AssignThreadToCgroup(t, cgroup));
-  }
-  return Status::OK();
-}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/util/thread.h
----------------------------------------------------------------------
diff --git a/be/src/util/thread.h b/be/src/util/thread.h
index 8c880d2..4e2b65d 100644
--- a/be/src/util/thread.h
+++ b/be/src/util/thread.h
@@ -31,7 +31,6 @@ namespace impala {
 
 class MetricGroup;
 class Webserver;
-class CgroupsMgr;
 
 /// Thin wrapper around boost::thread that can register itself with the singleton
 /// ThreadMgr (a private class implemented in thread.cc entirely, which tracks all live
@@ -165,39 +164,19 @@ class ThreadGroup {
  public:
   ThreadGroup() {}
 
-  ThreadGroup(CgroupsMgr* cgroups_mgr, const std::string& cgroup)
-        : cgroups_mgr_(cgroups_mgr), cgroup_path_(cgroup) { }
-
   /// Adds a new Thread to this group. The ThreadGroup takes ownership of the Thread, and
   /// will destroy it when the ThreadGroup is destroyed.  Threads will linger until that
   /// point (even if terminated), however, so callers should be mindful of the cost of
   /// placing very many threads in this set.
-  /// If cgroup_path_ / cgroup_prefix_ are set, the thread will be added to the specified
-  /// cgroup and an error will be returned if that operation fails.
   Status AddThread(Thread* thread);
 
   /// Waits for all threads to finish. DO NOT call this from a thread inside this set;
   /// deadlock will predictably ensue.
   void JoinAll();
 
-  /// Assigns all current and future threads to the given cgroup managed by cgroups_mgr_.
-  /// Must be called after SetCgroupsMgr() if groups_mgr_ has not been set already.
-  /// Returns an error if any assignment was not possible, but does not undo previously
-  /// successful assignments.
-  Status SetCgroup(const std::string& cgroup);
-
-  void SetCgroupsMgr(CgroupsMgr* cgroups_mgr) { cgroups_mgr_ = cgroups_mgr; }
-
  private:
   /// All the threads grouped by this set.
   boost::ptr_vector<Thread> threads_;
-
-  /// Cgroups manager for assigning threads in this group to cgroups. Not owned.
-  CgroupsMgr* cgroups_mgr_;
-
-  /// If not empty, every thread added to this group will also be placed in the
-  /// cgroup_path_ managed by the cgroups_mgr_.
-  std::string cgroup_path_;
 };
 
 /// Initialises the threading subsystem. Must be called before a Thread is created.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/util/uid-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/uid-util.h b/be/src/util/uid-util.h
index c78a9ea..f0f87ec 100644
--- a/be/src/util/uid-util.h
+++ b/be/src/util/uid-util.h
@@ -37,10 +37,7 @@ inline std::size_t hash_value(const impala::TUniqueId& id) {
   return seed;
 }
 
-/// Templated so that this method is not namespace-specific (since we also call this on
-/// llama::TUniqueId)
-template <typename T>
-inline void UUIDToTUniqueId(const boost::uuids::uuid& uuid, T* unique_id) {
+inline void UUIDToTUniqueId(const boost::uuids::uuid& uuid, TUniqueId* unique_id) {
   memcpy(&(unique_id->hi), &uuid.data[0], 8);
   memcpy(&(unique_id->lo), &uuid.data[8], 8);
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/bin/bootstrap_toolchain.py
----------------------------------------------------------------------
diff --git a/bin/bootstrap_toolchain.py b/bin/bootstrap_toolchain.py
index d8621c4..6524e82 100755
--- a/bin/bootstrap_toolchain.py
+++ b/bin/bootstrap_toolchain.py
@@ -22,7 +22,7 @@
 # that we can deduce the version settings of the dependencies from the environment.
 # IMPALA_TOOLCHAIN indicates the location where the prebuilt artifacts should be extracted
 # to. If DOWNLOAD_CDH_COMPONENTS is set to true, this script will also download and extract
-# the CDH components (i.e. Hadoop, Hive, HBase, Llama, Llama-minikdc and Sentry) into
+# the CDH components (i.e. Hadoop, Hive, HBase and Sentry) into
 # CDH_COMPONENTS_HOME.
 #
 # The script is called as follows without any additional parameters:

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/bin/create-test-configuration.sh
----------------------------------------------------------------------
diff --git a/bin/create-test-configuration.sh b/bin/create-test-configuration.sh
index bbe8f61..8d695a4 100755
--- a/bin/create-test-configuration.sh
+++ b/bin/create-test-configuration.sh
@@ -148,7 +148,7 @@ if ${CLUSTER_DIR}/admin is_kerberized; then
   # strange, but making these symlinks also results in data loading
   # failures in the non-kerberized case.  Without these, mapreduce
   # jobs die in a kerberized cluster because they can't find their
-  # kerberos principals.  Obviously this has to be sorted out before
+  # kerberos principals. Obviously this has to be sorted out before
   # a kerberized cluster can load data.
   echo "Linking yarn and mapred from local cluster"
   ln -s ${CLUSTER_HADOOP_CONF_DIR}/yarn-site.xml

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/bin/generate_minidump_collection_testdata.py
----------------------------------------------------------------------
diff --git a/bin/generate_minidump_collection_testdata.py b/bin/generate_minidump_collection_testdata.py
index 27f9a42..a408e05 100755
--- a/bin/generate_minidump_collection_testdata.py
+++ b/bin/generate_minidump_collection_testdata.py
@@ -71,7 +71,6 @@ CONFIG_FILE = '''-beeswax_port=21000
 -max_lineage_log_file_size=5000
 -hostname=vb0204.halxg.cloudera.com
 -state_store_host=vb0202.halxg.cloudera.com
--enable_rm=false
 -state_store_port=24000
 -catalog_service_host=vb0202.halxg.cloudera.com
 -catalog_service_port=26000

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/bin/start-impala-cluster.py
----------------------------------------------------------------------
diff --git a/bin/start-impala-cluster.py b/bin/start-impala-cluster.py
index bfde4c3..b92fcf1 100755
--- a/bin/start-impala-cluster.py
+++ b/bin/start-impala-cluster.py
@@ -41,8 +41,6 @@ parser.add_option("--build_type", dest="build_type", default= 'latest',
 parser.add_option("--impalad_args", dest="impalad_args", action="append", type="string",
                   default=[],
                   help="Additional arguments to pass to each Impalad during startup")
-parser.add_option("--enable_rm", dest="enable_rm", action="store_true", default=False,
-                  help="Enable resource management with Yarn and Llama.")
 parser.add_option("--state_store_args", dest="state_store_args", action="append",
                   type="string", default=[],
                   help="Additional arguments to pass to State Store during startup")
@@ -91,8 +89,6 @@ IMPALAD_PORTS = ("-beeswax_port=%d -hs2_port=%d  -be_port=%d "
                  "-llama_callback_port=%d")
 JVM_ARGS = "-jvm_debug_port=%s -jvm_args=%s"
 BE_LOGGING_ARGS = "-log_filename=%s -log_dir=%s -v=%s -logbufsecs=5 -max_log_files=%s"
-RM_ARGS = ("-enable_rm=true -llama_addresses=%s -cgroup_hierarchy_path=%s "
-           "-fair_scheduler_allocation_path=%s")
 CLUSTER_WAIT_TIMEOUT_IN_SECONDS = 240
 # Kills have a timeout to prevent automated scripts from hanging indefinitely.
 # It is set to a high value to avoid failing if processes are slow to shut down.
@@ -208,20 +204,6 @@ def build_jvm_args(instance_num):
   BASE_JVM_DEBUG_PORT = 30000
   return JVM_ARGS % (BASE_JVM_DEBUG_PORT + instance_num, options.jvm_args)
 
-def build_rm_args(instance_num):
-  if not options.enable_rm: return ""
-  try:
-    cgroup_path = cgroups.create_impala_cgroup_path(instance_num + 1)
-  except Exception, ex:
-    raise RuntimeError("Unable to initialize RM: %s" % str(ex))
-  llama_address = "localhost:15000"
-
-  # Don't bother checking if the path doesn't exist, the impalad won't start up
-  relative_fs_cfg_path = 'cdh%s/node-%d/etc/hadoop/conf/fair-scheduler.xml' %\
-      (os.environ.get('CDH_MAJOR_VERSION'), instance_num + 1)
-  fs_cfg_path = os.path.join(os.environ.get('CLUSTER_DIR'), relative_fs_cfg_path)
-  return RM_ARGS % (llama_address, cgroup_path, fs_cfg_path)
-
 def start_impalad_instances(cluster_size):
   if cluster_size == 0:
     # No impalad instances should be started.
@@ -250,11 +232,10 @@ def start_impalad_instances(cluster_size):
 
     # impalad args from the --impalad_args flag. Also replacing '#ID' with the instance.
     param_args = (" ".join(options.impalad_args)).replace("#ID", str(i))
-    args = "--mem_limit=%s %s %s %s %s %s" %\
+    args = "--mem_limit=%s %s %s %s %s" %\
           (mem_limit,  # Goes first so --impalad_args will override it.
            build_impalad_logging_args(i, service_name), build_jvm_args(i),
-           build_impalad_port_args(i), param_args,
-           build_rm_args(i))
+           build_impalad_port_args(i), param_args)
     stderr_log_file_path = os.path.join(options.log_dir, '%s-error.log' % service_name)
     exec_impala_process(IMPALAD_PATH, args, stderr_log_file_path)
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/common/thrift/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/common/thrift/CMakeLists.txt b/common/thrift/CMakeLists.txt
index e66bd89..3104ee2 100644
--- a/common/thrift/CMakeLists.txt
+++ b/common/thrift/CMakeLists.txt
@@ -162,7 +162,6 @@ set (SRC_FILES
   ImpalaService.thrift
   JniCatalog.thrift
   LineageGraph.thrift
-  Llama.thrift
   Logging.thrift
   NetworkTest.thrift
   MetricDefs.thrift
@@ -171,7 +170,6 @@ set (SRC_FILES
   Planner.thrift
   Partitions.thrift
   parquet.thrift
-  ResourceBrokerService.thrift
   Results.thrift
   RuntimeProfile.thrift
   StatestoreService.thrift

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/common/thrift/Frontend.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/Frontend.thrift b/common/thrift/Frontend.thrift
index 708cb46..732ea4a 100644
--- a/common/thrift/Frontend.thrift
+++ b/common/thrift/Frontend.thrift
@@ -387,6 +387,7 @@ struct TQueryExecRequest {
 
   // Estimated per-host CPU requirements in YARN virtual cores.
   // Used for resource management.
+  // TODO: Remove this and associated code in Planner.
   11: optional i16 per_host_vcores
 
   // List of replica hosts.  Used by the host_idx field of TScanRangeLocation.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/common/thrift/ImpalaInternalService.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/ImpalaInternalService.thrift b/common/thrift/ImpalaInternalService.thrift
index bf03d98..003a618 100644
--- a/common/thrift/ImpalaInternalService.thrift
+++ b/common/thrift/ImpalaInternalService.thrift
@@ -33,7 +33,6 @@ include "DataSinks.thrift"
 include "Results.thrift"
 include "RuntimeProfile.thrift"
 include "ImpalaService.thrift"
-include "Llama.thrift"
 
 // constants for TQueryOptions.num_nodes
 const i32 NUM_NODES_ALL = 0
@@ -366,12 +365,6 @@ struct TPlanFragmentInstanceCtx {
 
   // Id of this fragment in its role as a sender.
   11: optional i32 sender_id
-
-  // Resource reservation to run this plan fragment in.
-  12: optional Llama.TAllocatedResource reserved_resource
-
-  // Address of local node manager (used for expanding resource allocations)
-  13: optional Types.TNetworkAddress local_resource_address
 }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/common/thrift/Llama.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/Llama.thrift b/common/thrift/Llama.thrift
deleted file mode 100644
index a9b7f5f..0000000
--- a/common/thrift/Llama.thrift
+++ /dev/null
@@ -1,276 +0,0 @@
-// 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.
-
-namespace cpp llama
-namespace java com.cloudera.llama.thrift
-
-////////////////////////////////////////////////////////////////////////////////
-// DATA TYPES
-
-enum TLlamaServiceVersion {
-   V1
-}
-
-struct TUniqueId {
-  1: required i64 hi;
-  2: required i64 lo;
-}
-
-struct TNetworkAddress {
-  1: required string hostname;
-  2: required i32    port;
-}
-
-enum TStatusCode {
-  OK,
-  REQUEST_ERROR,
-  INTERNAL_ERROR
-}
-
-struct TStatus {
-  1: required TStatusCode status_code;
-  2: i16 error_code;
-  3: list<string> error_msgs;
-}
-
-enum TLocationEnforcement {
-  MUST,
-  PREFERRED,
-  DONT_CARE
-}
-
-struct TResource {
-  1: required TUniqueId            client_resource_id;
-  2: required i16                  v_cpu_cores;
-  3: required i32                  memory_mb;
-  4: required string               askedLocation;
-  5: required TLocationEnforcement enforcement;
-}
-
-struct TAllocatedResource {
-  1: required TUniqueId reservation_id;
-  2: required TUniqueId client_resource_id;
-  3: required string    rm_resource_id;
-  4: required i16       v_cpu_cores;
-  5: required i32       memory_mb;
-  6: required string    location;
-}
-
-struct TNodeCapacity {
-  1: required i16 total_v_cpu_cores;
-  2: required i32 total_memory_mb;
-  3: required i16 free_v_cpu_cores;
-  4: required i32 free_memory_mb;
-}
-
-////////////////////////////////////////////////////////////////////////////////
-// Llama AM Service
-
-struct TLlamaAMRegisterRequest {
-  1: required TLlamaServiceVersion version;
-  2: required TUniqueId            client_id;
-  3: required TNetworkAddress      notification_callback_service;
-}
-
-struct TLlamaAMRegisterResponse {
-  1: required TStatus   status;
-  2: optional TUniqueId am_handle;
-}
-
-struct TLlamaAMUnregisterRequest {
-  1: required TLlamaServiceVersion version;
-  2: required TUniqueId am_handle;
-}
-
-struct TLlamaAMUnregisterResponse {
-  1: required TStatus status;
-}
-
-struct TLlamaAMReservationRequest {
-  1: required TLlamaServiceVersion version;
-  2: required TUniqueId            am_handle;
-  3: required string               user;
-  4: optional string               queue;
-  5: required list<TResource>      resources;
-  6: required bool                 gang;
-  7: optional TUniqueId            reservation_id;
-}
-
-struct TLlamaAMReservationResponse {
-  1: required TStatus   status;
-  2: optional TUniqueId reservation_id;
-}
-
-struct TLlamaAMReservationExpansionRequest {
-  1: required TLlamaServiceVersion version;
-  2: required TUniqueId            am_handle;
-  3: required TUniqueId            expansion_of;
-  4: required TResource            resource;
-  5: optional TUniqueId            expansion_id;
-}
-
-struct TLlamaAMReservationExpansionResponse {
-  1: required TStatus   status;
-  2: optional TUniqueId expansion_id;
-}
-
-struct TLlamaAMReleaseRequest {
-  1: required TLlamaServiceVersion version;
-  2: required TUniqueId            am_handle;
-  3: required TUniqueId            reservation_id;
-}
-
-struct TLlamaAMReleaseResponse {
-  1: required TStatus status;
-}
-
-struct TLlamaAMGetNodesRequest {
-  1: required TLlamaServiceVersion version;
-  2: required TUniqueId            am_handle;
-}
-
-struct TLlamaAMGetNodesResponse {
-  1: required TStatus status;
-  2: optional list<string> nodes;
-}
-
-service LlamaAMService {
-
-  TLlamaAMRegisterResponse Register(1: TLlamaAMRegisterRequest request);
-
-  TLlamaAMUnregisterResponse Unregister(1: TLlamaAMUnregisterRequest request);
-
-  TLlamaAMReservationResponse Reserve(1: TLlamaAMReservationRequest request);
-
-    TLlamaAMReservationExpansionResponse Expand(
-    1: TLlamaAMReservationExpansionRequest request);
-
-  TLlamaAMReleaseResponse Release(1: TLlamaAMReleaseRequest request);
-
-  TLlamaAMGetNodesResponse GetNodes(1: TLlamaAMGetNodesRequest request);
-
-}
-
-////////////////////////////////////////////////////////////////////////////////
-// Llama AM Admin Service
-
-struct TLlamaAMAdminReleaseRequest {
-  1: required TLlamaServiceVersion version;
-  2: optional bool                 do_not_cache = false;
-  3: optional list<string>         queues;
-  4: optional list<TUniqueId>      handles;
-  5: optional list<TUniqueId>      reservations;
-}
-
-struct TLlamaAMAdminReleaseResponse {
-  1: required TStatus status;
-}
-
-struct TLlamaAMAdminEmptyCacheRequest {
-  1: required TLlamaServiceVersion version;
-  2: optional bool                 allQueues = false;
-  3: optional list<string>         queues;
-}
-
-struct TLlamaAMAdminEmptyCacheResponse {
-  1: required TStatus status;
-}
-
-service LlamaAMAdminService {
-
-  TLlamaAMAdminReleaseResponse Release
-  (1: TLlamaAMAdminReleaseRequest request);
-
-  TLlamaAMAdminEmptyCacheResponse EmptyCache
-  (1: TLlamaAMAdminEmptyCacheRequest request);
-
-}
-
-////////////////////////////////////////////////////////////////////////////////
-// Llama NM Service
-
-struct TLlamaNMRegisterRequest {
-  1: required TLlamaServiceVersion version;
-  2: required TUniqueId            client_id;
-  3: required TNetworkAddress      notification_callback_service;
-}
-
-struct TLlamaNMRegisterResponse {
-  1: required TStatus   status;
-  2: optional TUniqueId nm_handle;
-}
-
-struct TLlamaNMUnregisterRequest {
-  1: required TLlamaServiceVersion version;
-  2: required TUniqueId            nm_handle;
-}
-
-struct TLlamaNMUnregisterResponse {
-  1: required TStatus status;
-}
-
-service LlamaNMService {
-
-  TLlamaNMRegisterResponse Register(1: TLlamaNMRegisterRequest request);
-
-  TLlamaNMUnregisterResponse Unregister(1: TLlamaNMUnregisterRequest request);
-
-}
-
-////////////////////////////////////////////////////////////////////////////////
-// Llama Notification Callback Service
-
-struct TLlamaAMNotificationRequest {
-  1: required TLlamaServiceVersion     version;
-  2: required TUniqueId                am_handle;
-  3: required bool                     heartbeat;
-  4: optional list<TUniqueId>          allocated_reservation_ids;
-  5: optional list<TAllocatedResource> allocated_resources;
-  6: optional list<TUniqueId>          rejected_reservation_ids;
-  7: optional list<TUniqueId>          rejected_client_resource_ids;
-  8: optional list<TUniqueId>          lost_client_resource_ids;
-  9: optional list<TUniqueId>          preempted_reservation_ids;
-  10: optional list<TUniqueId>         preempted_client_resource_ids;
-  11: optional list<TUniqueId>         admin_released_reservation_ids;
-  12: optional list<TUniqueId>         lost_reservation_ids;
-}
-
-struct TLlamaAMNotificationResponse {
-  1: required TStatus status;
-}
-
-struct TLlamaNMNotificationRequest {
-  1: required TLlamaServiceVersion version;
-  2: required TUniqueId            nm_handle;
-  3: required TNodeCapacity        node_capacity;
-  4: list<string>                  preempted_rm_resource_ids;
-}
-
-struct TLlamaNMNotificationResponse {
-  1: required TStatus status;
-}
-
-service LlamaNotificationService {
-
-  TLlamaAMNotificationResponse AMNotification(
-    1: TLlamaAMNotificationRequest request);
-
-  TLlamaNMNotificationResponse NMNotification(
-    1: TLlamaNMNotificationRequest request);
-}
-
-////////////////////////////////////////////////////////////////////////////////


[4/7] incubator-impala git commit: IMPALA-4160: Remove Llama support.

Posted by he...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/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 089ada1..0f93875 100644
--- a/be/src/runtime/disk-io-mgr-test.cc
+++ b/be/src/runtime/disk-io-mgr-test.cc
@@ -635,7 +635,7 @@ TEST_F(DiskIoMgrTest, MemLimits) {
     DiskIoMgr io_mgr(1, 1, MIN_BUFFER_SIZE, MAX_BUFFER_SIZE);
 
     ASSERT_OK(io_mgr.Init(&root_mem_tracker));
-    MemTracker reader_mem_tracker(-1, -1, "Reader", &root_mem_tracker);
+    MemTracker reader_mem_tracker(-1, "Reader", &root_mem_tracker);
     DiskIoRequestContext* reader;
     ASSERT_OK(io_mgr.RegisterContext(&reader, &reader_mem_tracker));
 
@@ -950,7 +950,7 @@ TEST_F(DiskIoMgrTest, Buffers) {
   ASSERT_OK(io_mgr.Init(&root_mem_tracker));
   ASSERT_EQ(root_mem_tracker.consumption(), 0);
 
-  MemTracker reader_mem_tracker(-1, -1, "Reader", &root_mem_tracker);
+  MemTracker reader_mem_tracker(-1, "Reader", &root_mem_tracker);
   DiskIoRequestContext* reader;
   ASSERT_OK(io_mgr.RegisterContext(&reader, &reader_mem_tracker));
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/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 5df69ed..dd45932 100644
--- a/be/src/runtime/disk-io-mgr.cc
+++ b/be/src/runtime/disk-io-mgr.cc
@@ -370,9 +370,9 @@ DiskIoMgr::~DiskIoMgr() {
 Status DiskIoMgr::Init(MemTracker* process_mem_tracker) {
   DCHECK(process_mem_tracker != NULL);
   free_buffer_mem_tracker_.reset(
-      new MemTracker(-1, -1, "Free Disk IO Buffers", process_mem_tracker, false));
+      new MemTracker(-1, "Free Disk IO Buffers", process_mem_tracker, false));
   unowned_buffer_mem_tracker_.reset(
-      new MemTracker(-1, -1, "Untracked Disk IO Buffers", process_mem_tracker, false));
+      new MemTracker(-1, "Untracked Disk IO Buffers", process_mem_tracker, false));
   // If we hit the process limit, see if we can reclaim some memory by removing
   // previously allocated (but unused) io buffers.
   process_mem_tracker->AddGcFunction(bind(&DiskIoMgr::GcIoBuffers, this));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/exec-env.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/exec-env.cc b/be/src/runtime/exec-env.cc
index 38e9f96..1b3fa14 100644
--- a/be/src/runtime/exec-env.cc
+++ b/be/src/runtime/exec-env.cc
@@ -24,7 +24,6 @@
 #include <gutil/strings/substitute.h>
 
 #include "common/logging.h"
-#include "resourcebroker/resource-broker.h"
 #include "runtime/backend-client.h"
 #include "runtime/client-cache.h"
 #include "runtime/coordinator.h"
@@ -51,7 +50,6 @@
 #include "util/webserver.h"
 #include "util/mem-info.h"
 #include "util/debug-util.h"
-#include "util/cgroups-mgr.h"
 #include "util/memory-metrics.h"
 #include "util/pretty-printer.h"
 #include "util/thread-pool.h"
@@ -84,49 +82,21 @@ DECLARE_int32(num_cores);
 DECLARE_int32(be_port);
 DECLARE_string(mem_limit);
 
-DEFINE_bool(enable_rm, false, "Whether to enable resource management. If enabled, "
-                              "-fair_scheduler_allocation_path is required.");
-DEFINE_int32(llama_callback_port, 28000,
-             "Port where Llama notification callback should be started");
-// TODO: Deprecate llama_host and llama_port in favor of the new llama_hostports.
-// This needs to be coordinated with CM.
-DEFINE_string(llama_host, "",
-              "Host of Llama service that the resource broker should connect to");
-DEFINE_int32(llama_port, 15000,
-             "Port of Llama service that the resource broker should connect to");
-DEFINE_string(llama_addresses, "",
-             "Llama availability group given as a comma-separated list of hostports.");
-DEFINE_int64(llama_registration_timeout_secs, 30,
-             "Maximum number of seconds that Impala will attempt to (re-)register "
-             "with Llama before aborting the triggering action with an error "
-             "(e.g. Impalad startup or a Llama RPC request). "
-             "A setting of -1 means try indefinitely.");
-DEFINE_int64(llama_registration_wait_secs, 3,
-             "Number of seconds to wait between attempts during Llama registration.");
-DEFINE_int64(llama_max_request_attempts, 5,
-             "Maximum number of times a non-registration Llama RPC request "
-             "(reserve/expand/release, etc.) is retried until the request is aborted. "
-             "An attempt is counted once Impala is registered with Llama, i.e., a "
-             "request survives at most llama_max_request_attempts-1 re-registrations.");
-DEFINE_string(cgroup_hierarchy_path, "", "If Resource Management is enabled, this must "
-    "be set to the Impala-writeable root of the cgroups hierarchy into which execution "
-    "threads are assigned.");
-DEFINE_string(staging_cgroup, "impala_staging", "Name of the cgroup that a query's "
-    "execution threads are moved into once the query completes.");
-
-// Use a low default value because the reconnection logic is performed manually
-// for the purpose of faster Llama failover (otherwise we may try to reconnect to the
-// inactive Llama for a long time).
-DEFINE_int32(resource_broker_cnxn_attempts, 1, "The number of times to retry an "
-    "RPC connection to Llama. A setting of 0 means retry indefinitely");
-DEFINE_int32(resource_broker_cnxn_retry_interval_ms, 3000, "The interval, in ms, "
-    "to wait between attempts to make an RPC connection to the Llama.");
-DEFINE_int32(resource_broker_send_timeout, 0, "Time to wait, in ms, "
-    "for the underlying socket of an RPC to Llama to successfully send data. "
-    "A setting of 0 means the socket will wait indefinitely.");
-DEFINE_int32(resource_broker_recv_timeout, 0, "Time to wait, in ms, "
-    "for the underlying socket of an RPC to Llama to successfully receive data. "
-    "A setting of 0 means the socket will wait indefinitely.");
+// TODO: Remove the following RM-related flags in Impala 3.0.
+DEFINE_bool(enable_rm, false, "Deprecated");
+DEFINE_int32(llama_callback_port, 28000, "Deprecated");
+DEFINE_string(llama_host, "", "Deprecated");
+DEFINE_int32(llama_port, 15000, "Deprecated");
+DEFINE_string(llama_addresses, "", "Deprecated");
+DEFINE_int64(llama_registration_timeout_secs, 30, "Deprecated");
+DEFINE_int64(llama_registration_wait_secs, 3, "Deprecated");
+DEFINE_int64(llama_max_request_attempts, 5, "Deprecated");
+DEFINE_string(cgroup_hierarchy_path, "", "Deprecated");
+DEFINE_string(staging_cgroup, "impala_staging", "Deprecated");
+DEFINE_int32(resource_broker_cnxn_attempts, 1, "Deprecated");
+DEFINE_int32(resource_broker_cnxn_retry_interval_ms, 3000, "Deprecated");
+DEFINE_int32(resource_broker_send_timeout, 0, "Deprecated");
+DEFINE_int32(resource_broker_recv_timeout, 0, "Deprecated");
 
 DEFINE_int32(coordinator_rpc_threads, 12, "(Advanced) Number of threads available to "
     "start fragments on remote Impala daemons.");
@@ -145,11 +115,6 @@ DEFINE_int32(catalog_client_connection_num_retries, 3, "Retry catalog connection
 DEFINE_int32(catalog_client_rpc_timeout_ms, 0, "(Advanced) The underlying TSocket "
     "send/recv timeout in milliseconds for a catalog client RPC.");
 
-// The key for a variable set in Impala's test environment only, to allow the
-// resource-broker to correctly map node addresses into a form that Llama understand.
-const static string PSEUDO_DISTRIBUTED_CONFIG_KEY =
-    "yarn.scheduler.include-port-in-node-name";
-
 const static string DEFAULT_FS = "fs.defaultFS";
 
 namespace impala {
@@ -160,35 +125,29 @@ ExecEnv::ExecEnv()
   : metrics_(new MetricGroup("impala-metrics")),
     stream_mgr_(new DataStreamMgr(metrics_.get())),
     impalad_client_cache_(
-        new ImpalaBackendClientCache(FLAGS_backend_client_connection_num_retries,
-            0, FLAGS_backend_client_rpc_timeout_ms,
-            FLAGS_backend_client_rpc_timeout_ms,
-            "", !FLAGS_ssl_client_ca_certificate.empty())),
+        new ImpalaBackendClientCache(FLAGS_backend_client_connection_num_retries, 0,
+            FLAGS_backend_client_rpc_timeout_ms, FLAGS_backend_client_rpc_timeout_ms, "",
+            !FLAGS_ssl_client_ca_certificate.empty())),
     catalogd_client_cache_(
-        new CatalogServiceClientCache(FLAGS_catalog_client_connection_num_retries,
-            0, FLAGS_catalog_client_rpc_timeout_ms,
-            FLAGS_catalog_client_rpc_timeout_ms,
-            "", !FLAGS_ssl_client_ca_certificate.empty())),
+        new CatalogServiceClientCache(FLAGS_catalog_client_connection_num_retries, 0,
+            FLAGS_catalog_client_rpc_timeout_ms, FLAGS_catalog_client_rpc_timeout_ms, "",
+            !FLAGS_ssl_client_ca_certificate.empty())),
     htable_factory_(new HBaseTableFactory()),
     disk_io_mgr_(new DiskIoMgr()),
     webserver_(new Webserver()),
     mem_tracker_(NULL),
     thread_mgr_(new ThreadResourceMgr),
-    cgroups_mgr_(NULL),
     hdfs_op_thread_pool_(
         CreateHdfsOpThreadPool("hdfs-worker-pool", FLAGS_num_hdfs_worker_threads, 1024)),
     tmp_file_mgr_(new TmpFileMgr),
     request_pool_service_(new RequestPoolService(metrics_.get())),
     frontend_(new Frontend()),
-    fragment_exec_thread_pool_(
-        new CallableThreadPool("coordinator-fragment-rpc", "worker",
-            FLAGS_coordinator_rpc_threads, numeric_limits<int32_t>::max())),
+    fragment_exec_thread_pool_(new CallableThreadPool("coordinator-fragment-rpc",
+        "worker", FLAGS_coordinator_rpc_threads, numeric_limits<int32_t>::max())),
     async_rpc_pool_(new CallableThreadPool("rpc-pool", "async-rpc-sender", 8, 10000)),
     enable_webserver_(FLAGS_enable_webserver),
     is_fe_tests_(false),
-    backend_address_(MakeNetworkAddress(FLAGS_hostname, FLAGS_be_port)),
-    is_pseudo_distributed_llama_(false) {
-  if (FLAGS_enable_rm) InitRm();
+    backend_address_(MakeNetworkAddress(FLAGS_hostname, FLAGS_be_port)) {
   // Initialize the scheduler either dynamically (with a statestore) or statically (with
   // a standalone single backend)
   if (FLAGS_use_statestore) {
@@ -202,33 +161,30 @@ ExecEnv::ExecEnv()
         subscriber_address, statestore_address, metrics_.get()));
 
     scheduler_.reset(new SimpleScheduler(statestore_subscriber_.get(),
-        statestore_subscriber_->id(), backend_address_, metrics_.get(),
-        webserver_.get(), resource_broker_.get(), request_pool_service_.get()));
+        statestore_subscriber_->id(), backend_address_, metrics_.get(), webserver_.get(),
+        request_pool_service_.get()));
   } else {
     vector<TNetworkAddress> addresses;
     addresses.push_back(MakeNetworkAddress(FLAGS_hostname, FLAGS_be_port));
-    scheduler_.reset(new SimpleScheduler(addresses, metrics_.get(), webserver_.get(),
-        resource_broker_.get(), request_pool_service_.get()));
+    scheduler_.reset(new SimpleScheduler(
+        addresses, metrics_.get(), webserver_.get(), request_pool_service_.get()));
   }
   if (exec_env_ == NULL) exec_env_ = this;
-  if (FLAGS_enable_rm) resource_broker_->set_scheduler(scheduler_.get());
 }
 
 // TODO: Need refactor to get rid of duplicated code.
 ExecEnv::ExecEnv(const string& hostname, int backend_port, int subscriber_port,
-                 int webserver_port, const string& statestore_host, int statestore_port)
+    int webserver_port, const string& statestore_host, int statestore_port)
   : metrics_(new MetricGroup("impala-metrics")),
     stream_mgr_(new DataStreamMgr(metrics_.get())),
     impalad_client_cache_(
-        new ImpalaBackendClientCache(FLAGS_backend_client_connection_num_retries,
-            0, FLAGS_backend_client_rpc_timeout_ms,
-            FLAGS_backend_client_rpc_timeout_ms,
-            "", !FLAGS_ssl_client_ca_certificate.empty())),
+        new ImpalaBackendClientCache(FLAGS_backend_client_connection_num_retries, 0,
+            FLAGS_backend_client_rpc_timeout_ms, FLAGS_backend_client_rpc_timeout_ms, "",
+            !FLAGS_ssl_client_ca_certificate.empty())),
     catalogd_client_cache_(
-        new CatalogServiceClientCache(FLAGS_catalog_client_connection_num_retries,
-            0, FLAGS_catalog_client_rpc_timeout_ms,
-            FLAGS_catalog_client_rpc_timeout_ms,
-            "", !FLAGS_ssl_client_ca_certificate.empty())),
+        new CatalogServiceClientCache(FLAGS_catalog_client_connection_num_retries, 0,
+            FLAGS_catalog_client_rpc_timeout_ms, FLAGS_catalog_client_rpc_timeout_ms, "",
+            !FLAGS_ssl_client_ca_certificate.empty())),
     htable_factory_(new HBaseTableFactory()),
     disk_io_mgr_(new DiskIoMgr()),
     webserver_(new Webserver(webserver_port)),
@@ -238,16 +194,13 @@ ExecEnv::ExecEnv(const string& hostname, int backend_port, int subscriber_port,
         CreateHdfsOpThreadPool("hdfs-worker-pool", FLAGS_num_hdfs_worker_threads, 1024)),
     tmp_file_mgr_(new TmpFileMgr),
     frontend_(new Frontend()),
-    fragment_exec_thread_pool_(
-        new CallableThreadPool("coordinator-fragment-rpc", "worker",
-            FLAGS_coordinator_rpc_threads, numeric_limits<int32_t>::max())),
+    fragment_exec_thread_pool_(new CallableThreadPool("coordinator-fragment-rpc",
+        "worker", FLAGS_coordinator_rpc_threads, numeric_limits<int32_t>::max())),
     async_rpc_pool_(new CallableThreadPool("rpc-pool", "async-rpc-sender", 8, 10000)),
     enable_webserver_(FLAGS_enable_webserver && webserver_port > 0),
     is_fe_tests_(false),
-    backend_address_(MakeNetworkAddress(FLAGS_hostname, FLAGS_be_port)),
-    is_pseudo_distributed_llama_(false) {
+    backend_address_(MakeNetworkAddress(FLAGS_hostname, FLAGS_be_port)) {
   request_pool_service_.reset(new RequestPoolService(metrics_.get()));
-  if (FLAGS_enable_rm) InitRm();
 
   if (FLAGS_use_statestore && statestore_port > 0) {
     TNetworkAddress subscriber_address =
@@ -260,73 +213,23 @@ ExecEnv::ExecEnv(const string& hostname, int backend_port, int subscriber_port,
         subscriber_address, statestore_address, metrics_.get()));
 
     scheduler_.reset(new SimpleScheduler(statestore_subscriber_.get(),
-        statestore_subscriber_->id(), backend_address_, metrics_.get(),
-        webserver_.get(), resource_broker_.get(), request_pool_service_.get()));
+        statestore_subscriber_->id(), backend_address_, metrics_.get(), webserver_.get(),
+        request_pool_service_.get()));
   } else {
     vector<TNetworkAddress> addresses;
     addresses.push_back(MakeNetworkAddress(hostname, backend_port));
-    scheduler_.reset(new SimpleScheduler(addresses, metrics_.get(), webserver_.get(),
-        resource_broker_.get(), request_pool_service_.get()));
+    scheduler_.reset(new SimpleScheduler(
+        addresses, metrics_.get(), webserver_.get(), request_pool_service_.get()));
   }
   if (exec_env_ == NULL) exec_env_ = this;
-  if (FLAGS_enable_rm) resource_broker_->set_scheduler(scheduler_.get());
 }
 
-void ExecEnv::InitRm() {
-  // Unique addresses from FLAGS_llama_addresses and FLAGS_llama_host/FLAGS_llama_port.
-  vector<TNetworkAddress> llama_addresses;
-  if (!FLAGS_llama_addresses.empty()) {
-    vector<string> components;
-    split(components, FLAGS_llama_addresses, is_any_of(","), token_compress_on);
-    for (int i = 0; i < components.size(); ++i) {
-      to_lower(components[i]);
-      TNetworkAddress llama_address = MakeNetworkAddress(components[i]);
-      if (find(llama_addresses.begin(), llama_addresses.end(), llama_address)
-          == llama_addresses.end()) {
-        llama_addresses.push_back(llama_address);
-      }
-    }
-  }
-  // Add Llama hostport from deprecated flags (if it does not already exist).
-  if (!FLAGS_llama_host.empty()) {
-    to_lower(FLAGS_llama_host);
-    TNetworkAddress llama_address =
-        MakeNetworkAddress(FLAGS_llama_host, FLAGS_llama_port);
-    if (find(llama_addresses.begin(), llama_addresses.end(), llama_address)
-        == llama_addresses.end()) {
-      llama_addresses.push_back(llama_address);
-    }
-  }
-  for (int i = 0; i < llama_addresses.size(); ++i) {
-    LOG(INFO) << "Llama address " << i << ": " << llama_addresses[i];
-  }
-
-  TNetworkAddress llama_callback_address =
-      MakeNetworkAddress(FLAGS_hostname, FLAGS_llama_callback_port);
-  resource_broker_.reset(new ResourceBroker(llama_addresses, llama_callback_address,
-      metrics_.get()));
-  cgroups_mgr_.reset(new CgroupsMgr(metrics_.get()));
-
-  TGetHadoopConfigRequest config_request;
-  config_request.__set_name(PSEUDO_DISTRIBUTED_CONFIG_KEY);
-  TGetHadoopConfigResponse config_response;
-  frontend_->GetHadoopConfig(config_request, &config_response);
-  if (config_response.__isset.value) {
-    to_lower(config_response.value);
-    is_pseudo_distributed_llama_ = (config_response.value == "true");
-  } else {
-    is_pseudo_distributed_llama_ = false;
-  }
-  if (is_pseudo_distributed_llama_) {
-    LOG(INFO) << "Pseudo-distributed Llama cluster detected";
-  }
-}
 
 ExecEnv::~ExecEnv() {
 }
 
 Status ExecEnv::InitForFeTests() {
-  mem_tracker_.reset(new MemTracker(-1, -1, "Process"));
+  mem_tracker_.reset(new MemTracker(-1, "Process"));
   is_fe_tests_ = true;
   return Status::OK();
 }
@@ -334,15 +237,6 @@ Status ExecEnv::InitForFeTests() {
 Status ExecEnv::StartServices() {
   LOG(INFO) << "Starting global services";
 
-  if (FLAGS_enable_rm) {
-    // Initialize the resource broker to make sure the Llama is up and reachable.
-    DCHECK(resource_broker_.get() != NULL);
-    RETURN_IF_ERROR(resource_broker_->Init());
-    DCHECK(cgroups_mgr_.get() != NULL);
-    RETURN_IF_ERROR(
-        cgroups_mgr_->Init(FLAGS_cgroup_hierarchy_path, FLAGS_staging_cgroup));
-  }
-
   // Initialize global memory limit.
   // Depending on the system configuration, we will have to calculate the process
   // memory limit either based on the available physical memory, or if overcommitting
@@ -397,7 +291,7 @@ Status ExecEnv::StartServices() {
 #ifndef ADDRESS_SANITIZER
   // Limit of -1 means no memory limit.
   mem_tracker_.reset(new MemTracker(TcmallocMetric::PHYSICAL_BYTES_RESERVED,
-      bytes_limit > 0 ? bytes_limit : -1, -1, "Process"));
+      bytes_limit > 0 ? bytes_limit : -1, "Process"));
 
   // Since tcmalloc does not free unused memory, we may exceed the process mem limit even
   // if Impala is not actually using that much memory. Add a callback to free any unused
@@ -407,7 +301,7 @@ Status ExecEnv::StartServices() {
 #else
   // tcmalloc metrics aren't defined in ASAN builds, just use the default behavior to
   // track process memory usage (sum of all children trackers).
-  mem_tracker_.reset(new MemTracker(bytes_limit > 0 ? bytes_limit : -1, -1, "Process"));
+  mem_tracker_.reset(new MemTracker(bytes_limit > 0 ? bytes_limit : -1, "Process"));
 #endif
 
   mem_tracker_->RegisterMetrics(metrics_.get(), "mem-tracker.process");

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/exec-env.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/exec-env.h b/be/src/runtime/exec-env.h
index 1f37572..303876f 100644
--- a/be/src/runtime/exec-env.h
+++ b/be/src/runtime/exec-env.h
@@ -29,7 +29,6 @@
 namespace impala {
 
 class CallableThreadPool;
-class CgroupsMgr;
 class DataStreamMgr;
 class DiskIoMgr;
 class FragmentMgr;
@@ -42,7 +41,6 @@ class MemTracker;
 class MetricGroup;
 class QueryResourceMgr;
 class RequestPoolService;
-class ResourceBroker;
 class Scheduler;
 class StatestoreSubscriber;
 class TestExecEnv;
@@ -89,7 +87,6 @@ class ExecEnv {
   MetricGroup* metrics() { return metrics_.get(); }
   MemTracker* process_mem_tracker() { return mem_tracker_.get(); }
   ThreadResourceMgr* thread_mgr() { return thread_mgr_.get(); }
-  CgroupsMgr* cgroups_mgr() { return cgroups_mgr_.get(); }
   HdfsOpThreadPool* hdfs_op_thread_pool() { return hdfs_op_thread_pool_.get(); }
   TmpFileMgr* tmp_file_mgr() { return tmp_file_mgr_.get(); }
   CallableThreadPool* fragment_exec_thread_pool() {
@@ -102,7 +99,6 @@ class ExecEnv {
 
   void set_enable_webserver(bool enable) { enable_webserver_ = enable; }
 
-  ResourceBroker* resource_broker() { return resource_broker_.get(); }
   Scheduler* scheduler() { return scheduler_.get(); }
   StatestoreSubscriber* subscriber() { return statestore_subscriber_.get(); }
 
@@ -119,11 +115,6 @@ class ExecEnv {
   /// differently.
   bool is_fe_tests() { return is_fe_tests_; }
 
-  /// Returns true if the Llama in use is pseudo-distributed, used for development
-  /// purposes. The pseudo-distributed version has special requirements for specifying
-  /// resource locations.
-  bool is_pseudo_distributed_llama() { return is_pseudo_distributed_llama_; }
-
   /// Returns the configured defaultFs set in core-site.xml
   string default_fs() { return default_fs_; }
 
@@ -131,7 +122,6 @@ class ExecEnv {
   /// Leave protected so that subclasses can override
   boost::scoped_ptr<MetricGroup> metrics_;
   boost::scoped_ptr<DataStreamMgr> stream_mgr_;
-  boost::scoped_ptr<ResourceBroker> resource_broker_;
   boost::scoped_ptr<Scheduler> scheduler_;
   boost::scoped_ptr<StatestoreSubscriber> statestore_subscriber_;
   boost::scoped_ptr<ImpalaBackendClientCache> impalad_client_cache_;
@@ -141,7 +131,6 @@ class ExecEnv {
   boost::scoped_ptr<Webserver> webserver_;
   boost::scoped_ptr<MemTracker> mem_tracker_;
   boost::scoped_ptr<ThreadResourceMgr> thread_mgr_;
-  boost::scoped_ptr<CgroupsMgr> cgroups_mgr_;
   boost::scoped_ptr<HdfsOpThreadPool> hdfs_op_thread_pool_;
   boost::scoped_ptr<TmpFileMgr> tmp_file_mgr_;
   boost::scoped_ptr<RequestPoolService> request_pool_service_;
@@ -161,16 +150,8 @@ class ExecEnv {
   /// Address of the Impala backend server instance
   TNetworkAddress backend_address_;
 
-  /// True if the cluster has set 'yarn.scheduler.include-port-in-node-name' to true,
-  /// indicating that this cluster is pseudo-distributed. Should not be true in real
-  /// deployments.
-  bool is_pseudo_distributed_llama_;
-
   /// fs.defaultFs value set in core-site.xml
   std::string default_fs_;
-
-  /// Initialise cgroups manager, detect test RM environment and init resource broker.
-  void InitRm();
 };
 
 } // namespace impala

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/mem-pool-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/mem-pool-test.cc b/be/src/runtime/mem-pool-test.cc
index 99cec9a..604923c 100644
--- a/be/src/runtime/mem-pool-test.cc
+++ b/be/src/runtime/mem-pool-test.cc
@@ -224,8 +224,8 @@ TEST(MemPoolTest, ReturnPartial) {
 
 TEST(MemPoolTest, Limits) {
   MemTracker limit3(4 * MemPoolTest::INITIAL_CHUNK_SIZE);
-  MemTracker limit1(2 * MemPoolTest::INITIAL_CHUNK_SIZE, -1, "", &limit3);
-  MemTracker limit2(3 * MemPoolTest::INITIAL_CHUNK_SIZE, -1, "", &limit3);
+  MemTracker limit1(2 * MemPoolTest::INITIAL_CHUNK_SIZE, "", &limit3);
+  MemTracker limit2(3 * MemPoolTest::INITIAL_CHUNK_SIZE, "", &limit3);
 
   MemPool* p1 = new MemPool(&limit1);
   EXPECT_FALSE(limit1.AnyLimitExceeded());

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/mem-tracker-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/mem-tracker-test.cc b/be/src/runtime/mem-tracker-test.cc
index 87eec14..546b8ab 100644
--- a/be/src/runtime/mem-tracker-test.cc
+++ b/be/src/runtime/mem-tracker-test.cc
@@ -64,7 +64,7 @@ TEST(MemTestTest, ConsumptionMetric) {
   UIntGauge metric(md, 0);
   EXPECT_EQ(metric.value(), 0);
 
-  MemTracker t(&metric, 100, -1, "");
+  MemTracker t(&metric, 100, "");
   EXPECT_TRUE(t.has_limit());
   EXPECT_EQ(t.consumption(), 0);
 
@@ -112,8 +112,8 @@ TEST(MemTestTest, ConsumptionMetric) {
 
 TEST(MemTestTest, TrackerHierarchy) {
   MemTracker p(100);
-  MemTracker c1(80, -1, "", &p);
-  MemTracker c2(50, -1, "", &p);
+  MemTracker c1(80, "", &p);
+  MemTracker c2(50, "", &p);
 
   // everything below limits
   c1.Consume(60);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/mem-tracker.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/mem-tracker.cc b/be/src/runtime/mem-tracker.cc
index a9ceb76..a9de160 100644
--- a/be/src/runtime/mem-tracker.cc
+++ b/be/src/runtime/mem-tracker.cc
@@ -23,10 +23,8 @@
 #include <gutil/strings/substitute.h>
 
 #include "bufferpool/reservation-tracker-counters.h"
-#include "resourcebroker/resource-broker.h"
 #include "runtime/exec-env.h"
 #include "runtime/runtime-state.h"
-#include "scheduling/query-resource-mgr.h"
 #include "util/debug-util.h"
 #include "util/mem-info.h"
 #include "util/pretty-printer.h"
@@ -49,10 +47,9 @@ AtomicInt64 MemTracker::released_memory_since_gc_;
 // Name for request pool MemTrackers. '$0' is replaced with the pool name.
 const string REQUEST_POOL_MEM_TRACKER_LABEL_FORMAT = "RequestPool=$0";
 
-MemTracker::MemTracker(int64_t byte_limit, int64_t rm_reserved_limit, const string& label,
-    MemTracker* parent, bool log_usage_if_zero)
+MemTracker::MemTracker(
+    int64_t byte_limit, const string& label, MemTracker* parent, bool log_usage_if_zero)
   : limit_(byte_limit),
-    rm_reserved_limit_(rm_reserved_limit),
     label_(label),
     parent_(parent),
     consumption_(&local_counter_),
@@ -60,7 +57,6 @@ MemTracker::MemTracker(int64_t byte_limit, int64_t rm_reserved_limit, const stri
     consumption_metric_(NULL),
     auto_unregister_(false),
     log_usage_if_zero_(log_usage_if_zero),
-    query_resource_mgr_(NULL),
     num_gcs_metric_(NULL),
     bytes_freed_by_last_gc_metric_(NULL),
     bytes_over_limit_metric_(NULL),
@@ -69,11 +65,9 @@ MemTracker::MemTracker(int64_t byte_limit, int64_t rm_reserved_limit, const stri
   Init();
 }
 
-MemTracker::MemTracker(
-    RuntimeProfile* profile, int64_t byte_limit, int64_t rm_reserved_limit,
+MemTracker::MemTracker(RuntimeProfile* profile, int64_t byte_limit,
     const std::string& label, MemTracker* parent)
   : limit_(byte_limit),
-    rm_reserved_limit_(rm_reserved_limit),
     label_(label),
     parent_(parent),
     consumption_(profile->AddHighWaterMarkCounter(COUNTER_NAME, TUnit::BYTES)),
@@ -81,7 +75,6 @@ MemTracker::MemTracker(
     consumption_metric_(NULL),
     auto_unregister_(false),
     log_usage_if_zero_(true),
-    query_resource_mgr_(NULL),
     num_gcs_metric_(NULL),
     bytes_freed_by_last_gc_metric_(NULL),
     bytes_over_limit_metric_(NULL),
@@ -90,10 +83,9 @@ MemTracker::MemTracker(
   Init();
 }
 
-MemTracker::MemTracker(UIntGauge* consumption_metric,
-    int64_t byte_limit, int64_t rm_reserved_limit, const string& label)
+MemTracker::MemTracker(
+    UIntGauge* consumption_metric, int64_t byte_limit, const string& label)
   : limit_(byte_limit),
-    rm_reserved_limit_(rm_reserved_limit),
     label_(label),
     parent_(NULL),
     consumption_(&local_counter_),
@@ -101,7 +93,6 @@ MemTracker::MemTracker(UIntGauge* consumption_metric,
     consumption_metric_(consumption_metric),
     auto_unregister_(false),
     log_usage_if_zero_(true),
-    query_resource_mgr_(NULL),
     num_gcs_metric_(NULL),
     bytes_freed_by_last_gc_metric_(NULL),
     bytes_over_limit_metric_(NULL),
@@ -111,7 +102,6 @@ MemTracker::MemTracker(UIntGauge* consumption_metric,
 
 void MemTracker::Init() {
   DCHECK_GE(limit_, -1);
-  DCHECK(rm_reserved_limit_ == -1 || limit_ == -1 || rm_reserved_limit_ <= limit_);
   // populate all_trackers_ and limit_trackers_
   MemTracker* tracker = this;
   while (tracker != NULL) {
@@ -173,9 +163,8 @@ MemTracker* MemTracker::GetRequestPoolMemTracker(const string& pool_name,
   } else {
     if (parent == NULL) return NULL;
     // First time this pool_name registered, make a new object.
-    MemTracker* tracker = new MemTracker(-1, -1,
-          Substitute(REQUEST_POOL_MEM_TRACKER_LABEL_FORMAT, pool_name),
-          parent);
+    MemTracker* tracker = new MemTracker(
+        -1, Substitute(REQUEST_POOL_MEM_TRACKER_LABEL_FORMAT, pool_name), parent);
     tracker->auto_unregister_ = true;
     tracker->pool_name_ = pool_name;
     pool_to_mem_trackers_[pool_name] = tracker;
@@ -184,8 +173,7 @@ MemTracker* MemTracker::GetRequestPoolMemTracker(const string& pool_name,
 }
 
 shared_ptr<MemTracker> MemTracker::GetQueryMemTracker(
-    const TUniqueId& id, int64_t byte_limit, int64_t rm_reserved_limit, MemTracker* parent,
-    QueryResourceMgr* res_mgr) {
+    const TUniqueId& id, int64_t byte_limit, MemTracker* parent) {
   if (byte_limit != -1) {
     if (byte_limit > MemInfo::physical_mem()) {
       LOG(WARNING) << "Memory limit "
@@ -210,12 +198,11 @@ shared_ptr<MemTracker> MemTracker::GetQueryMemTracker(
   } else {
     // First time this id registered, make a new object.  Give a shared ptr to
     // the caller and put a weak ptr in the map.
-    shared_ptr<MemTracker> tracker = make_shared<MemTracker>(byte_limit,
-        rm_reserved_limit, Substitute("Query($0)", lexical_cast<string>(id)), parent);
+    shared_ptr<MemTracker> tracker = make_shared<MemTracker>(
+        byte_limit, Substitute("Query($0)", lexical_cast<string>(id)), parent);
     tracker->auto_unregister_ = true;
     tracker->query_id_ = id;
     request_to_mem_trackers_[id] = tracker;
-    if (res_mgr != NULL) tracker->SetQueryResourceMgr(res_mgr);
     return tracker;
   }
 }
@@ -278,9 +265,6 @@ string MemTracker::LogUsage(const string& prefix) const {
   ss << prefix << label_ << ":";
   if (CheckLimitExceeded()) ss << " memory limit exceeded.";
   if (limit_ > 0) ss << " Limit=" << PrettyPrinter::Print(limit_, TUnit::BYTES);
-  if (rm_reserved_limit_ > 0) {
-    ss << " RM Limit=" << PrettyPrinter::Print(rm_reserved_limit_, TUnit::BYTES);
-  }
 
   int64_t total = consumption();
   int64_t peak = consumption_->value();
@@ -358,45 +342,4 @@ void MemTracker::GcTcmalloc() {
 #endif
 }
 
-bool MemTracker::ExpandRmReservation(int64_t bytes) {
-  if (query_resource_mgr_ == NULL || rm_reserved_limit_ == -1) return false;
-  // TODO: Make this asynchronous after IO mgr changes to use TryConsume() are done.
-  lock_guard<mutex> l(resource_acquisition_lock_);
-  int64_t requested = consumption_->current_value() + bytes;
-  // Can't exceed the hard limit under any circumstance
-  if (requested >= limit_ && limit_ != -1) return false;
-  // Test to see if we can satisfy the limit anyhow; maybe a different request was already
-  // in flight.
-  if (requested < rm_reserved_limit_) return true;
-
-  int64_t bytes_allocated;
-  Status status = query_resource_mgr_->RequestMemExpansion(bytes, &bytes_allocated);
-  if (!status.ok()) {
-    LOG(INFO) << "Failed to expand memory limit by "
-              << PrettyPrinter::Print(bytes, TUnit::BYTES) << ": "
-              << status.GetDetail();
-    return false;
-  }
-
-  for (const MemTracker* tracker: limit_trackers_) {
-    if (tracker == this) continue;
-    if (tracker->consumption_->current_value() + bytes_allocated > tracker->limit_) {
-      // TODO: Allocation may be larger than needed and might exceed some parent
-      // tracker limit. IMPALA-2182.
-      VLOG_RPC << "Failed to use " << bytes_allocated << " bytes allocated over "
-               << tracker->label() << " tracker limit=" << tracker->limit_
-               << " consumption=" << tracker->consumption();
-      // Don't adjust our limit; rely on query tear-down to release the resource.
-      return false;
-    }
-  }
-
-  rm_reserved_limit_ += bytes_allocated;
-  // Resource broker might give us more than we ask for
-  if (limit_ != -1) rm_reserved_limit_ = min(rm_reserved_limit_, limit_);
-  VLOG_RPC << "Reservation bytes_allocated=" << bytes_allocated << " rm_reserved_limit="
-           << rm_reserved_limit_ << " limit=" << limit_;
-  return true;
-}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/mem-tracker.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/mem-tracker.h b/be/src/runtime/mem-tracker.h
index e3548cc..a2c3e9b 100644
--- a/be/src/runtime/mem-tracker.h
+++ b/be/src/runtime/mem-tracker.h
@@ -39,7 +39,6 @@ namespace impala {
 
 class ReservationTrackerCounters;
 class MemTracker;
-class QueryResourceMgr;
 
 /// A MemTracker tracks memory consumption; it contains an optional limit
 /// and can be arranged into a tree structure such that the consumption tracked
@@ -66,19 +65,17 @@ class MemTracker {
   /// 'label' is the label used in the usage string (LogUsage())
   /// If 'log_usage_if_zero' is false, this tracker (and its children) will not be included
   /// in LogUsage() output if consumption is 0.
-  MemTracker(int64_t byte_limit = -1, int64_t rm_reserved_limit = -1,
-      const std::string& label = std::string(), MemTracker* parent = NULL,
-      bool log_usage_if_zero = true);
+  MemTracker(int64_t byte_limit = -1, const std::string& label = std::string(),
+      MemTracker* parent = NULL, bool log_usage_if_zero = true);
 
   /// C'tor for tracker for which consumption counter is created as part of a profile.
   /// The counter is created with name COUNTER_NAME.
-  MemTracker(RuntimeProfile* profile, int64_t byte_limit, int64_t rm_reserved_limit = -1,
+  MemTracker(RuntimeProfile* profile, int64_t byte_limit,
       const std::string& label = std::string(), MemTracker* parent = NULL);
 
   /// C'tor for tracker that uses consumption_metric as the consumption value.
   /// Consume()/Release() can still be called. This is used for the process tracker.
-  MemTracker(UIntGauge* consumption_metric,
-      int64_t byte_limit = -1, int64_t rm_reserved_limit = -1,
+  MemTracker(UIntGauge* consumption_metric, int64_t byte_limit = -1,
       const std::string& label = std::string());
 
   ~MemTracker();
@@ -98,9 +95,8 @@ class MemTracker {
   /// 'parent' as the parent tracker.
   /// byte_limit and parent must be the same for all GetMemTracker() calls with the
   /// same id.
-  static std::shared_ptr<MemTracker> GetQueryMemTracker(const TUniqueId& id,
-      int64_t byte_limit, int64_t rm_reserved_limit, MemTracker* parent,
-      QueryResourceMgr* res_mgr);
+  static std::shared_ptr<MemTracker> GetQueryMemTracker(
+      const TUniqueId& id, int64_t byte_limit, MemTracker* parent);
 
   /// Returns a MemTracker object for request pool 'pool_name'. Calling this with the same
   /// 'pool_name' will return the same MemTracker object. This is used to track the local
@@ -112,14 +108,6 @@ class MemTracker {
   static MemTracker* GetRequestPoolMemTracker(const std::string& pool_name,
       MemTracker* parent);
 
-  /// Returns the minimum of limit and rm_reserved_limit
-  int64_t effective_limit() const {
-    // TODO: maybe no limit should be MAX_LONG?
-    DCHECK(rm_reserved_limit_ <= limit_ || limit_ == -1);
-    if (rm_reserved_limit_ == -1) return limit_;
-    return rm_reserved_limit_;
-  }
-
   /// Increases consumption of this tracker and its ancestors by 'bytes'.
   void Consume(int64_t bytes) {
     if (bytes <= 0) {
@@ -166,49 +154,33 @@ class MemTracker {
     if (consumption_metric_ != NULL) RefreshConsumptionFromMetric();
     if (UNLIKELY(bytes <= 0)) return true;
     int i;
-    // Walk the tracker tree top-down, to avoid expanding a limit on a child whose parent
-    // won't accommodate the change.
+    // Walk the tracker tree top-down.
     for (i = all_trackers_.size() - 1; i >= 0; --i) {
       MemTracker* tracker = all_trackers_[i];
-      int64_t limit = tracker->effective_limit();
+      const int64_t limit = tracker->limit();
       if (limit < 0) {
         tracker->consumption_->Add(bytes); // No limit at this tracker.
       } else {
-        // If TryConsume fails, we can try to GC or expand the RM reservation, but we may
-        // need to try several times if there are concurrent consumers because we don't
-        // take a lock before trying to update consumption_.
+        // If TryConsume fails, we can try to GC, but we may need to try several times if
+        // there are concurrent consumers because we don't take a lock before trying to
+        // update consumption_.
         while (true) {
           if (LIKELY(tracker->consumption_->TryAdd(bytes, limit))) break;
 
           VLOG_RPC << "TryConsume failed, bytes=" << bytes
                    << " consumption=" << tracker->consumption_->current_value()
-                   << " limit=" << limit << " attempting to GC and expand reservation";
-          // TODO: This may not be right if more than one tracker can actually change its
-          // RM reservation limit.
-          if (UNLIKELY(tracker->GcMemory(limit - bytes) &&
-                  !tracker->ExpandRmReservation(bytes))) {
+                   << " limit=" << limit << " attempting to GC";
+          if (UNLIKELY(tracker->GcMemory(limit - bytes))) {
             DCHECK_GE(i, 0);
             // Failed for this mem tracker. Roll back the ones that succeeded.
-            // TODO: this doesn't roll it back completely since the max values for
-            // the updated trackers aren't decremented. The max values are only used
-            // for error reporting so this is probably okay. Rolling those back is
-            // pretty hard; we'd need something like 2PC.
-            //
-            // TODO: This might leave us with an allocated resource that we can't use.
-            // Specifically, the RM reservation of some ancestors' trackers may have been
-            // expanded only to fail at the current tracker. This may be wasteful as
-            // subsequent TryConsume() never gets to use the reserved resources. Consider
-            // adjusting the reservation of the ancestors' trackers.
             for (int j = all_trackers_.size() - 1; j > i; --j) {
               all_trackers_[j]->consumption_->Add(-bytes);
             }
             return false;
           }
-          VLOG_RPC << "GC or expansion succeeded, TryConsume bytes=" << bytes
+          VLOG_RPC << "GC succeeded, TryConsume bytes=" << bytes
                    << " consumption=" << tracker->consumption_->current_value()
-                   << " new limit=" << tracker->effective_limit() << " prev=" << limit;
-          // Need to update the limit if the RM reservation was expanded.
-          limit = tracker->effective_limit();
+                   << " limit=" << limit;
         }
       }
     }
@@ -363,11 +335,6 @@ class MemTracker {
   /// can cause us to go way over mem limits.
   void GcTcmalloc();
 
-  /// Set the resource mgr to allow expansion of limits (if NULL, no expansion is possible)
-  void SetQueryResourceMgr(QueryResourceMgr* context) {
-    query_resource_mgr_ = context;
-  }
-
   /// Walks the MemTracker hierarchy and populates all_trackers_ and
   /// limit_trackers_
   void Init();
@@ -378,11 +345,6 @@ class MemTracker {
   static std::string LogUsage(const std::string& prefix,
       const std::list<MemTracker*>& trackers);
 
-  /// Try to expand the limit (by asking the resource broker for more memory) by at least
-  /// 'bytes'. Returns false if not possible, true if the request succeeded. May allocate
-  /// more memory than was requested.
-  bool ExpandRmReservation(int64_t bytes);
-
   /// Size, in bytes, that is considered a large value for Release() (or Consume() with
   /// a negative value). If tcmalloc is used, this can trigger it to GC.
   /// A higher value will make us call into tcmalloc less often (and therefore more
@@ -425,11 +387,6 @@ class MemTracker {
   /// there is no consumption limit.
   int64_t limit_;
 
-  /// If > -1, when RM is enabled this is the limit after which this memtracker needs to
-  /// acquire more memory from Llama.
-  /// This limit is always less than or equal to the hard limit.
-  int64_t rm_reserved_limit_;
-
   std::string label_;
   MemTracker* parent_;
 
@@ -476,14 +433,6 @@ class MemTracker {
   /// if consumption is 0.
   bool log_usage_if_zero_;
 
-  /// Lock is taken during ExpandRmReservation() to prevent concurrent acquisition of new
-  /// resources.
-  boost::mutex resource_acquisition_lock_;
-
-  /// If non-NULL, contains all the information required to expand resource reservations if
-  /// required.
-  QueryResourceMgr* query_resource_mgr_;
-
   /// The number of times the GcFunctions were called.
   IntCounter* num_gcs_metric_;
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/plan-fragment-executor.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/plan-fragment-executor.cc b/be/src/runtime/plan-fragment-executor.cc
index 1e52d08..7300f44 100644
--- a/be/src/runtime/plan-fragment-executor.cc
+++ b/be/src/runtime/plan-fragment-executor.cc
@@ -32,26 +32,21 @@
 #include "exec/hdfs-scan-node.h"
 #include "exec/hbase-table-scanner.h"
 #include "exprs/expr.h"
-#include "resourcebroker/resource-broker.h"
 #include "runtime/descriptors.h"
 #include "runtime/data-stream-mgr.h"
 #include "runtime/row-batch.h"
 #include "runtime/runtime-filter-bank.h"
 #include "runtime/mem-tracker.h"
-#include "scheduling/query-resource-mgr.h"
-#include "util/cgroups-mgr.h"
 #include "util/cpu-info.h"
 #include "util/debug-util.h"
 #include "util/container-util.h"
 #include "util/parse-util.h"
 #include "util/mem-info.h"
 #include "util/periodic-counter-updater.h"
-#include "util/llama-util.h"
 #include "util/pretty-printer.h"
 
 DEFINE_bool(serialize_batch, false, "serialize and deserialize each returned row batch");
 DEFINE_int32(status_report_interval, 5, "interval between profile reports; in seconds");
-DECLARE_bool(enable_rm);
 
 #include "common/names.h"
 
@@ -76,9 +71,6 @@ PlanFragmentExecutor::PlanFragmentExecutor(ExecEnv* exec_env,
 
 PlanFragmentExecutor::~PlanFragmentExecutor() {
   Close();
-  if (is_prepared_ && runtime_state_->query_resource_mgr() != NULL) {
-    exec_env_->resource_broker()->UnregisterQueryResourceMgr(query_id_);
-  }
   // at this point, the report thread should have been stopped
   DCHECK(!report_thread_active_);
 }
@@ -100,58 +92,15 @@ Status PlanFragmentExecutor::Prepare(const TExecPlanFragmentParams& request) {
   VLOG(2) << "fragment_instance_ctx:\n" << ThriftDebugString(fragment_instance_ctx);
 
   DCHECK(request.__isset.fragment_ctx);
-  bool request_has_reserved_resource =
-      request.fragment_instance_ctx.__isset.reserved_resource;
-  if (request_has_reserved_resource) {
-    VLOG_QUERY << "Executing fragment in reserved resource:\n"
-               << request.fragment_instance_ctx.reserved_resource;
-  }
-
-  string cgroup = "";
-  if (FLAGS_enable_rm && request_has_reserved_resource) {
-    cgroup = exec_env_->cgroups_mgr()->UniqueIdToCgroup(PrintId(query_id_, "_"));
-  }
 
   // Prepare() must not return before runtime_state_ is set if is_prepared_ was
   // set. Having runtime_state_.get() != NULL is a postcondition of this method in that
   // case. Do not call RETURN_IF_ERROR or explicitly return before this line.
-  runtime_state_.reset(new RuntimeState(request, cgroup, exec_env_));
+  runtime_state_.reset(new RuntimeState(request, exec_env_));
 
   // total_time_counter() is in the runtime_state_ so start it up now.
   SCOPED_TIMER(profile()->total_time_counter());
 
-  // Register after setting runtime_state_ to ensure proper cleanup.
-  if (FLAGS_enable_rm && !cgroup.empty() && request_has_reserved_resource) {
-    bool is_first;
-    RETURN_IF_ERROR(exec_env_->cgroups_mgr()->RegisterFragment(
-        request.fragment_instance_ctx.fragment_instance_id, cgroup, &is_first));
-    // The first fragment using cgroup sets the cgroup's CPU shares based on the reserved
-    // resource.
-    if (is_first) {
-      DCHECK(request_has_reserved_resource);
-      int32_t cpu_shares = exec_env_->cgroups_mgr()->VirtualCoresToCpuShares(
-          request.fragment_instance_ctx.reserved_resource.v_cpu_cores);
-      RETURN_IF_ERROR(exec_env_->cgroups_mgr()->SetCpuShares(cgroup, cpu_shares));
-    }
-  }
-
-  // TODO: Find the reservation id when the resource request is not set
-  if (FLAGS_enable_rm && request_has_reserved_resource) {
-    TUniqueId reservation_id;
-    reservation_id << request.fragment_instance_ctx.reserved_resource.reservation_id;
-
-    // TODO: Combine this with RegisterFragment() etc.
-    QueryResourceMgr* res_mgr;
-    bool is_first = exec_env_->resource_broker()->GetQueryResourceMgr(query_id_,
-        reservation_id, request.fragment_instance_ctx.local_resource_address, &res_mgr);
-    DCHECK(res_mgr != NULL);
-    runtime_state_->SetQueryResourceMgr(res_mgr);
-    if (is_first) {
-      runtime_state_->query_resource_mgr()->InitVcoreAcquisition(
-          request.fragment_instance_ctx.reserved_resource.v_cpu_cores);
-    }
-  }
-
   // reservation or a query option.
   int64_t bytes_limit = -1;
   if (runtime_state_->query_options().__isset.mem_limit &&
@@ -161,36 +110,14 @@ Status PlanFragmentExecutor::Prepare(const TExecPlanFragmentParams& request) {
                << PrettyPrinter::Print(bytes_limit, TUnit::BYTES);
   }
 
-  int64_t rm_reservation_size_bytes = -1;
-  if (request_has_reserved_resource &&
-      request.fragment_instance_ctx.reserved_resource.memory_mb > 0) {
-    int64_t rm_reservation_size_mb =
-      static_cast<int64_t>(request.fragment_instance_ctx.reserved_resource.memory_mb);
-    rm_reservation_size_bytes = rm_reservation_size_mb * 1024L * 1024L;
-    // Queries that use more than the hard limit will be killed, so it's not useful to
-    // have a reservation larger than the hard limit. Clamp reservation bytes limit to the
-    // hard limit (if it exists).
-    if (rm_reservation_size_bytes > bytes_limit && bytes_limit != -1) {
-      runtime_state_->LogError(ErrorMsg(TErrorCode::FRAGMENT_EXECUTOR,
-          PrettyPrinter::PrintBytes(rm_reservation_size_bytes),
-          PrettyPrinter::PrintBytes(bytes_limit)));
-      rm_reservation_size_bytes = bytes_limit;
-    }
-    VLOG_QUERY << "Using RM reservation memory limit from resource reservation: "
-               << PrettyPrinter::Print(rm_reservation_size_bytes, TUnit::BYTES);
-  }
-
   DCHECK(!fragment_instance_ctx.request_pool.empty());
-  runtime_state_->InitMemTrackers(query_id_, &fragment_instance_ctx.request_pool,
-      bytes_limit, rm_reservation_size_bytes);
+  runtime_state_->InitMemTrackers(
+      query_id_, &fragment_instance_ctx.request_pool, bytes_limit);
   RETURN_IF_ERROR(runtime_state_->CreateBlockMgr());
   runtime_state_->InitFilterBank();
 
   // Reserve one main thread from the pool
   runtime_state_->resource_pool()->AcquireThreadToken();
-  if (runtime_state_->query_resource_mgr() != NULL) {
-    runtime_state_->query_resource_mgr()->NotifyThreadUsageChange(1);
-  }
   has_thread_token_ = true;
 
   average_thread_tokens_ = profile()->AddSamplingCounter("AverageThreadTokens",
@@ -266,8 +193,8 @@ Status PlanFragmentExecutor::Prepare(const TExecPlanFragmentParams& request) {
         obj_pool(), request.fragment_ctx.fragment.output_sink,
         request.fragment_ctx.fragment.output_exprs,
         fragment_instance_ctx, row_desc(), &sink_));
-    sink_mem_tracker_.reset(new MemTracker(-1, -1, sink_->GetName(),
-        runtime_state_->instance_mem_tracker(), true));
+    sink_mem_tracker_.reset(new MemTracker(
+        -1, sink_->GetName(), runtime_state_->instance_mem_tracker(), true));
     RETURN_IF_ERROR(sink_->Prepare(runtime_state(), sink_mem_tracker_.get()));
 
     RuntimeProfile* sink_profile = sink_->profile();
@@ -565,9 +492,6 @@ void PlanFragmentExecutor::ReleaseThreadToken() {
   if (has_thread_token_) {
     has_thread_token_ = false;
     runtime_state_->resource_pool()->ReleaseThreadToken(true);
-    if (runtime_state_->query_resource_mgr() != NULL) {
-      runtime_state_->query_resource_mgr()->NotifyThreadUsageChange(-1);
-    }
     PeriodicCounterUpdater::StopSamplingCounter(average_thread_tokens_);
     PeriodicCounterUpdater::StopTimeSeriesCounter(
         thread_usage_sampled_counter_);
@@ -583,10 +507,6 @@ void PlanFragmentExecutor::Close() {
   }
   // Prepare may not have been called, which sets runtime_state_
   if (runtime_state_.get() != NULL) {
-    if (runtime_state_->query_resource_mgr() != NULL) {
-      exec_env_->cgroups_mgr()->UnregisterFragment(
-          runtime_state_->fragment_instance_id(), runtime_state_->cgroup());
-    }
     if (plan_ != NULL) plan_->Close(runtime_state_.get());
     for (DiskIoRequestContext* context: *runtime_state_->reader_contexts()) {
       runtime_state_->io_mgr()->UnregisterContext(context);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/runtime-filter-bank.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-filter-bank.cc b/be/src/runtime/runtime-filter-bank.cc
index 8d47431..a25bf8d 100644
--- a/be/src/runtime/runtime-filter-bank.cc
+++ b/be/src/runtime/runtime-filter-bank.cc
@@ -66,8 +66,8 @@ RuntimeFilterBank::RuntimeFilterBank(const TQueryCtx& query_ctx, RuntimeState* s
   default_filter_size_ =
       BitUtil::RoundUpToPowerOfTwo(min<int64_t>(default_filter_size_, max_filter_size_));
 
-  filter_mem_tracker_.reset(new MemTracker(-1, -1, "Runtime Filter Bank",
-      state->instance_mem_tracker(), false));
+  filter_mem_tracker_.reset(
+      new MemTracker(-1, "Runtime Filter Bank", state->instance_mem_tracker(), false));
 }
 
 RuntimeFilter* RuntimeFilterBank::RegisterFilter(const TRuntimeFilterDesc& filter_desc,
@@ -226,4 +226,3 @@ void RuntimeFilterBank::Close() {
   filter_mem_tracker_->Release(memory_allocated_->value());
   filter_mem_tracker_->UnregisterFromParent();
 }
-

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/runtime-state.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-state.cc b/be/src/runtime/runtime-state.cc
index 57aae58..5249076 100644
--- a/be/src/runtime/runtime-state.cc
+++ b/be/src/runtime/runtime-state.cc
@@ -67,18 +67,15 @@ static const int64_t BLOCK_MGR_MEM_MIN_REMAINING = 100 * 1024 * 1024;
 
 namespace impala {
 
-RuntimeState::RuntimeState(const TExecPlanFragmentParams& fragment_params,
-    const string& cgroup, ExecEnv* exec_env)
+RuntimeState::RuntimeState(
+    const TExecPlanFragmentParams& fragment_params, ExecEnv* exec_env)
   : obj_pool_(new ObjectPool()),
     fragment_params_(fragment_params),
-    now_(new TimestampValue(query_ctx().now_string.c_str(),
-        query_ctx().now_string.size())),
-    cgroup_(cgroup),
+    now_(new TimestampValue(
+        query_ctx().now_string.c_str(), query_ctx().now_string.size())),
     codegen_expr_(false),
-    profile_(obj_pool_.get(),
-        "Fragment " + PrintId(fragment_ctx().fragment_instance_id)),
+    profile_(obj_pool_.get(), "Fragment " + PrintId(fragment_ctx().fragment_instance_id)),
     is_cancelled_(false),
-    query_resource_mgr_(NULL),
     root_node_id_(-1) {
   Status status = Init(exec_env);
   DCHECK(status.ok()) << status.GetDetail();
@@ -92,7 +89,6 @@ RuntimeState::RuntimeState(const TQueryCtx& query_ctx)
     codegen_expr_(false),
     profile_(obj_pool_.get(), "<unnamed>"),
     is_cancelled_(false),
-    query_resource_mgr_(NULL),
     root_node_id_(-1) {
   fragment_params_.__set_query_ctx(query_ctx);
   fragment_params_.query_ctx.request.query_options.__set_batch_size(DEFAULT_BATCH_SIZE);
@@ -147,18 +143,17 @@ Status RuntimeState::Init(ExecEnv* exec_env) {
   return Status::OK();
 }
 
-void RuntimeState::InitMemTrackers(const TUniqueId& query_id, const string* pool_name,
-    int64_t query_bytes_limit, int64_t query_rm_reservation_limit_bytes) {
+void RuntimeState::InitMemTrackers(
+    const TUniqueId& query_id, const string* pool_name, int64_t query_bytes_limit) {
   MemTracker* query_parent_tracker = exec_env_->process_mem_tracker();
   if (pool_name != NULL) {
     query_parent_tracker = MemTracker::GetRequestPoolMemTracker(*pool_name,
         query_parent_tracker);
   }
   query_mem_tracker_ =
-      MemTracker::GetQueryMemTracker(query_id, query_bytes_limit,
-          query_rm_reservation_limit_bytes, query_parent_tracker, query_resource_mgr());
-  instance_mem_tracker_.reset(new MemTracker(runtime_profile(), -1, -1,
-      runtime_profile()->name(), query_mem_tracker_.get()));
+      MemTracker::GetQueryMemTracker(query_id, query_bytes_limit, query_parent_tracker);
+  instance_mem_tracker_.reset(new MemTracker(
+      runtime_profile(), -1, runtime_profile()->name(), query_mem_tracker_.get()));
 }
 
 void RuntimeState::InitFilterBank() {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/runtime-state.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-state.h b/be/src/runtime/runtime-state.h
index b5f7882..0bf9db5 100644
--- a/be/src/runtime/runtime-state.h
+++ b/be/src/runtime/runtime-state.h
@@ -65,8 +65,7 @@ typedef std::map<std::string, std::string> FileMoveMap;
 /// query and shared across all execution nodes of that query.
 class RuntimeState {
  public:
-  RuntimeState(const TExecPlanFragmentParams& fragment_params,
-      const std::string& cgroup, ExecEnv* exec_env);
+  RuntimeState(const TExecPlanFragmentParams& fragment_params, ExecEnv* exec_env);
 
   /// RuntimeState for executing expr in fe-support.
   RuntimeState(const TQueryCtx& query_ctx);
@@ -81,7 +80,7 @@ class RuntimeState {
   /// tracker (in the fifth level). If 'request_pool' is null, no request pool mem
   /// tracker is set up, i.e. query pools will have the process mem pool as the parent.
   void InitMemTrackers(const TUniqueId& query_id, const std::string* request_pool,
-      int64_t query_bytes_limit, int64_t query_rm_reservation_limit_bytes = -1);
+      int64_t query_bytes_limit);
 
   /// Initializes the runtime filter bank. Must be called after InitMemTrackers().
   void InitFilterBank();
@@ -124,7 +123,6 @@ class RuntimeState {
   const TUniqueId& fragment_instance_id() const {
     return fragment_ctx().fragment_instance_id;
   }
-  const std::string& cgroup() const { return cgroup_; }
   ExecEnv* exec_env() { return exec_env_; }
   DataStreamMgr* stream_mgr() { return exec_env_->stream_mgr(); }
   HBaseTableFactory* htable_factory() { return exec_env_->htable_factory(); }
@@ -262,9 +260,6 @@ class RuntimeState {
   /// execution doesn't continue if the query terminates abnormally.
   Status CheckQueryState();
 
-  QueryResourceMgr* query_resource_mgr() const { return query_resource_mgr_; }
-  void SetQueryResourceMgr(QueryResourceMgr* res_mgr) { query_resource_mgr_ = res_mgr; }
-
  private:
   /// Allow TestEnv to set block_mgr manually for testing.
   friend class TestEnv;
@@ -301,9 +296,6 @@ class RuntimeState {
   /// Use pointer to avoid inclusion of timestampvalue.h and avoid clang issues.
   boost::scoped_ptr<TimestampValue> now_;
 
-  /// The Impala-internal cgroup into which execution threads are assigned.
-  /// If empty, no RM is enabled.
-  std::string cgroup_;
   ExecEnv* exec_env_;
   boost::scoped_ptr<LlvmCodeGen> codegen_;
 
@@ -351,10 +343,6 @@ class RuntimeState {
   SpinLock query_status_lock_;
   Status query_status_;
 
-  /// Query-wide resource manager for resource expansion etc. Not owned by us; owned by
-  /// the ResourceBroker instead.
-  QueryResourceMgr* query_resource_mgr_;
-
   /// Reader contexts that need to be closed when the fragment is closed.
   std::vector<DiskIoRequestContext*> reader_contexts_;
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/test-env.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/test-env.cc b/be/src/runtime/test-env.cc
index a5818af..8690e39 100644
--- a/be/src/runtime/test-env.cc
+++ b/be/src/runtime/test-env.cc
@@ -70,7 +70,7 @@ RuntimeState* TestEnv::CreateRuntimeState(int64_t query_id) {
   TExecPlanFragmentParams plan_params = TExecPlanFragmentParams();
   plan_params.query_ctx.query_id.hi = 0;
   plan_params.query_ctx.query_id.lo = query_id;
-  return new RuntimeState(plan_params, "", exec_env_.get());
+  return new RuntimeState(plan_params, exec_env_.get());
 }
 
 Status TestEnv::CreateQueryState(int64_t query_id, int max_buffers, int block_size,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/scheduling/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/scheduling/CMakeLists.txt b/be/src/scheduling/CMakeLists.txt
index c5b4eb4..9cfb672 100644
--- a/be/src/scheduling/CMakeLists.txt
+++ b/be/src/scheduling/CMakeLists.txt
@@ -26,7 +26,6 @@ set(EXECUTABLE_OUTPUT_PATH "${BUILD_OUTPUT_ROOT_DIRECTORY}/scheduling")
 add_library(Scheduling STATIC
   admission-controller.cc
   backend-config.cc
-  query-resource-mgr.cc
   query-schedule.cc
   request-pool-service.cc
   simple-scheduler.cc

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/scheduling/query-resource-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/query-resource-mgr.cc b/be/src/scheduling/query-resource-mgr.cc
deleted file mode 100644
index abfe085..0000000
--- a/be/src/scheduling/query-resource-mgr.cc
+++ /dev/null
@@ -1,271 +0,0 @@
-// 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.
-
-#include "scheduling/query-resource-mgr.h"
-
-#include <boost/uuid/uuid.hpp>
-#include <boost/uuid/uuid_generators.hpp>
-#include <gutil/strings/substitute.h>
-#include <sstream>
-
-#include "runtime/exec-env.h"
-#include "resourcebroker/resource-broker.h"
-#include "util/bit-util.h"
-#include "util/cgroups-mgr.h"
-#include "util/container-util.h"
-#include "util/network-util.h"
-#include "util/promise.h"
-#include "util/time.h"
-
-#include "common/names.h"
-
-using boost::uuids::random_generator;
-using boost::uuids::uuid;
-using namespace impala;
-using namespace strings;
-
-DEFINE_int64(rm_mem_expansion_timeout_ms, 5000, "The amount of time to wait (ms) "
-    "for a memory expansion request.");
-DEFINE_double(max_vcore_oversubscription_ratio, 2.5, "(Advanced) The maximum ratio "
-    "allowed between running threads and acquired VCore resources for a query's fragments"
-    " on a single node");
-
-ResourceResolver::ResourceResolver(const unordered_set<TNetworkAddress>& unique_hosts) {
-  if (ExecEnv::GetInstance()->is_pseudo_distributed_llama()) {
-    CreateLocalLlamaNodeMapping(unique_hosts);
-  }
-}
-
-void ResourceResolver::GetResourceHostport(const TNetworkAddress& src,
-    TNetworkAddress* dest) {
-  if (ExecEnv::GetInstance()->is_pseudo_distributed_llama()) {
-    *dest = impalad_to_dn_[src];
-  } else {
-    dest->hostname = src.hostname;
-    dest->port = 0;
-  }
-}
-
-void ResourceResolver::CreateLocalLlamaNodeMapping(
-    const unordered_set<TNetworkAddress>& unique_hosts) {
-  DCHECK(ExecEnv::GetInstance()->is_pseudo_distributed_llama());
-  const vector<string>& llama_nodes =
-      ExecEnv::GetInstance()->resource_broker()->llama_nodes();
-  DCHECK(!llama_nodes.empty());
-  int llama_node_ix = 0;
-  for (const TNetworkAddress& host: unique_hosts) {
-    TNetworkAddress dn_hostport = MakeNetworkAddress(llama_nodes[llama_node_ix]);
-    impalad_to_dn_[host] = dn_hostport;
-    dn_to_impalad_[dn_hostport] = host;
-    LOG(INFO) << "Mapping Datanode " << dn_hostport << " to Impalad: " << host;
-    // Round robin the registered Llama nodes.
-    llama_node_ix = (llama_node_ix + 1) % llama_nodes.size();
-  }
-}
-
-QueryResourceMgr::QueryResourceMgr(const TUniqueId& reservation_id,
-    const TNetworkAddress& local_resource_location, const TUniqueId& query_id)
-    : reservation_id_(reservation_id), query_id_(query_id),
-      local_resource_location_(local_resource_location), exit_(false), callback_count_(0),
-      threads_running_(0), vcores_(0) {
-  max_vcore_oversubscription_ratio_ = FLAGS_max_vcore_oversubscription_ratio;
-}
-
-void QueryResourceMgr::InitVcoreAcquisition(int32_t init_vcores) {
-  LOG(INFO) << "Initialising vcore acquisition thread for query " << PrintId(query_id_)
-            << " (" << init_vcores << " initial vcores)";
-  DCHECK(acquire_vcore_thread_.get() == NULL)
-      << "Double initialisation of QueryResourceMgr::InitCpuAcquisition()";
-  vcores_ = init_vcores;
-
-  // These shared pointers to atomic values are used to communicate between the vcore
-  // acquisition thread and the class destructor. If the acquisition thread is in the
-  // middle of an Expand() call, the destructor might have to wait 5s (the default
-  // timeout) to return. This holds up query close operations. So instead check to see if
-  // the thread is in Expand(), and if so we set a synchronised flag early_exit_ which it
-  // inspects immediately after exiting Expand(), and if true, exits before touching any
-  // of the class-wide state (because the destructor may have finished before this point).
-
-  thread_in_expand_.reset(new AtomicInt32());
-  early_exit_.reset(new AtomicInt32());
-  acquire_vcore_thread_.reset(
-      new Thread("resource-mgmt", Substitute("acquire-cpu-$0", PrintId(query_id_)),
-          bind<void>(mem_fn(&QueryResourceMgr::AcquireVcoreResources), this,
-              thread_in_expand_, early_exit_)));
-}
-
-llama::TResource QueryResourceMgr::CreateResource(int64_t memory_mb, int64_t vcores) {
-  DCHECK(memory_mb > 0 || vcores > 0);
-  DCHECK(reservation_id_ != TUniqueId()) << "Expansion requires existing reservation";
-
-  unordered_set<TNetworkAddress> hosts;
-  hosts.insert(local_resource_location_);
-  ResourceResolver resolver(hosts);
-  llama::TResource res;
-  res.memory_mb = memory_mb;
-  res.v_cpu_cores = vcores;
-  TNetworkAddress res_address;
-  resolver.GetResourceHostport(local_resource_location_, &res_address);
-  res.__set_askedLocation(TNetworkAddressToString(res_address));
-
-  random_generator uuid_generator;
-  uuid id = uuid_generator();
-  res.client_resource_id.hi = *reinterpret_cast<uint64_t*>(&id.data[0]);
-  res.client_resource_id.lo = *reinterpret_cast<uint64_t*>(&id.data[8]);
-  res.enforcement = llama::TLocationEnforcement::MUST;
-  return res;
-}
-
-bool QueryResourceMgr::AboveVcoreSubscriptionThreshold() {
-  return threads_running_ > vcores_ * (max_vcore_oversubscription_ratio_ * 0.8);
-}
-
-void QueryResourceMgr::NotifyThreadUsageChange(int delta) {
-  lock_guard<mutex> l(threads_running_lock_);
-  threads_running_ += delta;
-  DCHECK(threads_running_ >= 0L);
-  if (AboveVcoreSubscriptionThreshold()) threads_changed_cv_.notify_all();
-}
-
-int32_t QueryResourceMgr::AddVcoreAvailableCb(const VcoreAvailableCb& callback) {
-  lock_guard<mutex> l(callbacks_lock_);
-  callbacks_[callback_count_] = callback;
-  callbacks_it_ = callbacks_.begin();
-  return callback_count_++;
-}
-
-void QueryResourceMgr::RemoveVcoreAvailableCb(int32_t callback_id) {
-  lock_guard<mutex> l(callbacks_lock_);
-  CallbackMap::iterator it = callbacks_.find(callback_id);
-  DCHECK(it != callbacks_.end()) << "Could not find callback with id: " << callback_id;
-  callbacks_.erase(it);
-  callbacks_it_ = callbacks_.begin();
-}
-
-Status QueryResourceMgr::RequestMemExpansion(int64_t requested_bytes,
-    int64_t* allocated_bytes) {
-  DCHECK(allocated_bytes != NULL);
-  *allocated_bytes = 0;
-  int64_t requested_mb = BitUtil::Ceil(requested_bytes, 1024L * 1024L);
-  llama::TResource res = CreateResource(max<int64_t>(1, requested_mb), 0);
-  llama::TUniqueId expansion_id;
-  llama::TAllocatedResource resource;
-  RETURN_IF_ERROR(ExecEnv::GetInstance()->resource_broker()->Expand(reservation_id_,
-      res, FLAGS_rm_mem_expansion_timeout_ms, &expansion_id, &resource));
-
-  DCHECK_EQ(resource.v_cpu_cores, 0L) << "Unexpected VCPUs returned by Llama";
-  *allocated_bytes = resource.memory_mb * 1024L * 1024L;
-  return Status::OK();
-}
-
-void QueryResourceMgr::AcquireVcoreResources(
-    shared_ptr<AtomicInt32> thread_in_expand,
-    shared_ptr<AtomicInt32> early_exit) {
-  // Take a copy because we'd like to print it in some cases after the destructor.
-  TUniqueId reservation_id = reservation_id_;
-  VLOG_QUERY << "Starting Vcore acquisition for: " << reservation_id;
-  while (!ShouldExit()) {
-    {
-      unique_lock<mutex> l(threads_running_lock_);
-      while (!AboveVcoreSubscriptionThreshold() && !ShouldExit()) {
-        threads_changed_cv_.wait(l);
-      }
-    }
-    if (ShouldExit()) break;
-
-    llama::TResource res = CreateResource(0L, 1);
-    VLOG_QUERY << "Expanding VCore allocation: " << reservation_id_;
-
-    // First signal that we are about to enter a blocking Expand() call.
-    thread_in_expand->Add(1L);
-
-    // TODO: Could cause problems if called during or after a system-wide shutdown
-    llama::TAllocatedResource resource;
-    llama::TUniqueId expansion_id;
-    Status status = ExecEnv::GetInstance()->resource_broker()->Expand(reservation_id,
-        res, -1, &expansion_id, &resource);
-    thread_in_expand->Add(-1L);
-    // If signalled to exit quickly by the destructor, exit the loop now. It's important
-    // to do so without accessing any class variables since they may no longer be valid.
-    // Need to check after setting thread_in_expand to avoid a race.
-    if (early_exit->Add(0L) != 0) {
-      VLOG_QUERY << "Fragment finished during Expand(): " << reservation_id;
-      break;
-    }
-    if (!status.ok()) {
-      VLOG_QUERY << "Could not expand CPU resources for query " << PrintId(query_id_)
-                 << ", reservation: " << PrintId(reservation_id_) << ". Error was: "
-                 << status.GetDetail();
-      // Sleep to avoid flooding the resource broker, particularly if requests are being
-      // rejected quickly (and therefore we stay oversubscribed)
-      // TODO: configurable timeout
-      SleepForMs(250);
-      continue;
-    }
-
-    DCHECK(resource.v_cpu_cores == 1)
-        << "Asked for 1 core, got: " << resource.v_cpu_cores;
-    vcores_ += resource.v_cpu_cores;
-
-    ExecEnv* exec_env = ExecEnv::GetInstance();
-    const string& cgroup =
-        exec_env->cgroups_mgr()->UniqueIdToCgroup(PrintId(query_id_, "_"));
-    int32_t num_shares = exec_env->cgroups_mgr()->VirtualCoresToCpuShares(vcores_);
-    exec_env->cgroups_mgr()->SetCpuShares(cgroup, num_shares);
-
-    // TODO: Only call one callback no matter how many VCores we just added; maybe call
-    // all of them?
-    {
-      lock_guard<mutex> l(callbacks_lock_);
-      if (callbacks_.size() != 0) {
-        callbacks_it_->second();
-        if (++callbacks_it_ == callbacks_.end()) callbacks_it_ = callbacks_.begin();
-      }
-    }
-  }
-  VLOG_QUERY << "Leaving VCore acquisition thread: " << reservation_id;
-}
-
-bool QueryResourceMgr::ShouldExit() {
-  lock_guard<mutex> l(exit_lock_);
-  return exit_;
-}
-
-void QueryResourceMgr::Shutdown() {
-  {
-    lock_guard<mutex> l(exit_lock_);
-    if (exit_) return;
-    exit_ = true;
-  }
-  {
-    lock_guard<mutex> l(callbacks_lock_);
-    callbacks_.clear();
-  }
-  threads_changed_cv_.notify_all();
-}
-
-QueryResourceMgr::~QueryResourceMgr() {
-  if (acquire_vcore_thread_.get() == NULL) return;
-  if (!ShouldExit()) Shutdown();
-  // First, set the early exit flag. Then check to see if the thread is in Expand(). If
-  // so, the acquisition thread is guaranteed to see early_exit_ == 1L once it finishes
-  // Expand(), and will exit immediately. It's therefore safe not to wait for it.
-  early_exit_->Add(1L);
-  if (thread_in_expand_->Add(0L) == 0L) {
-    acquire_vcore_thread_->Join();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/scheduling/query-resource-mgr.h
----------------------------------------------------------------------
diff --git a/be/src/scheduling/query-resource-mgr.h b/be/src/scheduling/query-resource-mgr.h
deleted file mode 100644
index 10da312..0000000
--- a/be/src/scheduling/query-resource-mgr.h
+++ /dev/null
@@ -1,227 +0,0 @@
-// 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.
-
-#ifndef SCHEDULING_QUERY_RESOURCE_MGR_H
-#define SCHEDULING_QUERY_RESOURCE_MGR_H
-
-#include "common/atomic.h"
-#include "common/status.h"
-#include "gen-cpp/Types_types.h"
-#include "gen-cpp/ResourceBrokerService.h"
-#include "gen-cpp/ImpalaInternalService.h"
-#include "gen-cpp/Frontend_types.h"
-#include "util/promise.h"
-#include "util/thread.h"
-
-#include <boost/function.hpp>
-#include <boost/thread/mutex.hpp>
-#include <boost/unordered_map.hpp>
-#include <boost/unordered_set.hpp>
-#include <string>
-
-namespace impala {
-
-class ResourceBroker;
-
-/// Utility class to map hosts to the Llama-registered resource-holding hosts
-/// (i.e. datanodes).
-class ResourceResolver {
- public:
-  ResourceResolver(const boost::unordered_set<TNetworkAddress>& unique_hosts);
-
-  /// Translates src into a network address suitable for identifying resources across
-  /// interactions with the Llama. The MiniLlama expects resources to be requested on
-  /// IP:port addresses of Hadoop DNs, whereas the regular Llama only deals with the
-  /// hostnames of Yarn NMs. For MiniLlama setups this translation uses the
-  /// impalad_to_dn_ mapping to populate dest. When using the regular Llama, this
-  /// translation sets a fixed port of 0 in dest because the Llama strips away the port
-  /// of resource locations.
-  void GetResourceHostport(const TNetworkAddress& src, TNetworkAddress* dst);
-
- private:
-  /// Impala mini clusters using the Mini Llama require translating the impalad hostports
-  /// to Hadoop DN hostports registered with the Llama during resource requests
-  /// (and then in reverse for translating granted resources to impalads).
-  /// These maps form a bi-directional hostport mapping Hadoop DN <-> impalad.
-  boost::unordered_map<TNetworkAddress, TNetworkAddress> impalad_to_dn_;
-  boost::unordered_map<TNetworkAddress, TNetworkAddress> dn_to_impalad_;
-
-  /// Called only in pseudo-distributed setups (i.e. testing only) to populate
-  /// impalad_to_dn_ and dn_to_impalad_
-  void CreateLocalLlamaNodeMapping(
-      const boost::unordered_set<TNetworkAddress>& unique_hosts);
-};
-
-/// Tracks all the state necessary to create expansion requests for all fragments of a
-/// single query on a single node. Code that might need to expand the memory reservation
-/// for this query (i.e. MemTracker) can use this class to construct expansion requests
-/// that may then be submitted to the ResourceBroker.
-//
-/// If InitCpuAcquisition() is called, this class will monitor the thread token to VCore
-/// ratio (thread consumers must use NotifyThreadUsageChange() to update the thread
-/// consumption count). If the ratio gets too high (see AboveVcoreSubscriptionThreshold()
-/// for details), we will try to acquire more VCore resources from Llama asynchronously.
-/// If the ratio passes a higher threshold (see IsVcoreOverSubscribed()), we say that the
-/// query fragments are currently oversubscribing their VCore resources.
-//
-/// Threads are typically handed to a fragment by the thread resource manager, which deals
-/// in tokens. When a fragment wants to use a token to start a thread, it should only do so
-/// if the ratio of threads to VCores (which map directly onto cgroup shares) is not too
-/// large. If it is too large - i.e. the VCores are oversubscribed - the fragment should
-/// wait to spin up a new threads until more VCore resources are acquired as above. To help
-/// with this, each fragment may register one or more callbacks with their
-/// QueryResourceMgr; when more VCore resources are acquired the callbacks are invoked in
-/// round-robin fashion. The callback should try and re-acquire the previously untaken
-/// thread token, and then a new thread may be started.
-//
-/// Only CPU-heavy threads need be managed using this class.
-//
-/// TODO: Handle reducing the number of VCores when threads finish.
-/// TODO: Consider combining more closely with ThreadResourceMgr.
-/// TODO: Add counters to RuntimeProfile to track resources.
-class QueryResourceMgr {
- public:
-  QueryResourceMgr(const TUniqueId& reservation_id,
-      const TNetworkAddress& local_resource_location, const TUniqueId& query_id);
-
-  /// Must be called only once. Starts a separate thread to monitor thread consumption,
-  /// which asks for more VCores from Llama periodically.
-  void InitVcoreAcquisition(int32_t init_vcores);
-
-  /// Should be used to check if another thread token may be acquired by this
-  /// query. Fragments may ignore this when acquiring a new CPU token, but the result will
-  /// be a larger thread:VCore ratio.
-  //
-  /// Note that this threshold is larger than the one in
-  /// AboveVcoreSubscriptionThreshold(). We want to start acquiring more VCore allocations
-  /// before we get so oversubscribed that adding new threads is considered a bad idea.
-  inline bool IsVcoreOverSubscribed() {
-    boost::lock_guard<boost::mutex> l(threads_running_lock_);
-    return threads_running_ > vcores_ * max_vcore_oversubscription_ratio_;
-  }
-
-  /// Called when thread consumption goes up or down. If the total consumption goes above a
-  /// subscription threshold, the acquisition thread will be woken to ask for more VCores.
-  void NotifyThreadUsageChange(int delta);
-
-  /// All callbacks registered here are called in round-robin fashion when more VCores are
-  /// acquired. Returns a unique ID that can be used as an argument to
-  /// RemoveVcoreAvailableCb().
-  typedef boost::function<void ()> VcoreAvailableCb;
-  int32_t AddVcoreAvailableCb(const VcoreAvailableCb& callback);
-
-  /// Removes the callback with the given ID.
-  void RemoveVcoreAvailableCb(int32_t callback_id);
-
-  /// Request an expansion of requested_bytes. If the expansion can be fulfilled within
-  /// the timeout period, the number of bytes allocated is returned in allocated_bytes
-  /// (which may be more than requested). Otherwise an error status is returned.
-  Status RequestMemExpansion(int64_t requested_bytes, int64_t* allocated_bytes);
-
-  /// Sets the exit flag for the VCore acquisiton thread, but does not block. Also clears
-  /// the set of callbacks, so that after Shutdown() has returned, no callback will be
-  /// invoked.
-  void Shutdown();
-
-  /// Waits for the VCore acquisition thread to stop.
-  ~QueryResourceMgr();
-
-  const TUniqueId& reservation_id() const { return reservation_id_; }
-
- private:
-  /// ID of the single reservation corresponding to this query
-  TUniqueId reservation_id_;
-
-  /// Query ID of the query this class manages resources for.
-  TUniqueId query_id_;
-
-  /// Network address of the local service registered with Llama. Usually corresponds to
-  /// <local-address>:0, unless a pseudo-dstributed Llama is being used (see
-  /// ResourceResolver::CreateLocalLlamaNodeMapping()).
-  TNetworkAddress local_resource_location_;
-
-  /// Used to control shutdown of AcquireCpuResources().
-  boost::mutex exit_lock_;
-  bool exit_;
-
-  /// Protects callbacks_ and callbacks_it_
-  boost::mutex callbacks_lock_;
-
-  /// List of callbacks to notify when a new VCore resource is available.
-  typedef boost::unordered_map<int32_t, VcoreAvailableCb> CallbackMap;
-  CallbackMap callbacks_;
-
-  /// Round-robin iterator to notify callbacks about new VCores one at a time.
-  CallbackMap::iterator callbacks_it_;
-
-  /// Total number of callbacks that were ever registered. Used to give each callback a
-  /// unique ID so that they can be removed.
-  int32_t callback_count_;
-
-  /// Protects threads_running_, threads_changed_cv_ and vcores_.
-  boost::mutex threads_running_lock_;
-
-  /// Waited on by AcquireCpuResources(), and notified by NotifyThreadUsageChange().
-  boost::condition_variable threads_changed_cv_;
-
-  /// The number of threads we know to be running on behalf of this query.
-  int64_t threads_running_;
-
-  /// The number of VCores acquired for this node for this query.
-  int64_t vcores_;
-
-  /// Set to FLAGS_max_vcore_oversubscription_ratio in the constructor. If the ratio of
-  /// threads to VCores exceeds this number, no more threads may be executed by this query
-  /// until more VCore resources are acquired.
-  float max_vcore_oversubscription_ratio_;
-
-  /// Runs AcquireVcoreResources() after InitVcoreAcquisition() is called.
-  boost::scoped_ptr<Thread> acquire_vcore_thread_;
-
-  /// Signals to the vcore acquisition thread that it should exit after it exits from any
-  /// pending Expand() call. Is a shared_ptr so that it will remain valid even after the
-  /// parent QueryResourceMgr has been destroyed.
-  /// TODO: Combine with ShouldExit(), and replace with AtomicBool when we have such a
-  /// thing.
-  std::shared_ptr<AtomicInt32> early_exit_;
-
-  /// Signals to the destructor that the vcore acquisition thread is currently in an
-  /// Expand() RPC. If so, the destructor does not need to wait for the acquisition thread
-  /// to exit.
-  std::shared_ptr<AtomicInt32> thread_in_expand_;
-
-  /// Creates the llama resource for the memory and/or cores specified, associated with
-  /// the reservation context.
-  llama::TResource CreateResource(int64_t memory_mb, int64_t vcores);
-
-  /// Run as a thread owned by acquire_cpu_thread_. Waits for notification from
-  /// NotifyThreadUsageChange(), then checks the subscription level to decide if more
-  /// VCores are needed, and starts a new expansion request if so.
-  void AcquireVcoreResources(std::shared_ptr<AtomicInt32 > thread_in_expand,
-      std::shared_ptr<AtomicInt32> early_exit);
-
-  /// True if thread:VCore subscription is too high, meaning more VCores are required.
-  /// Must be called holding threads_running_ lock.
-  bool AboveVcoreSubscriptionThreshold();
-
-  /// Notifies acquire_cpu_thread_ that it should terminate. Does not block.
-  bool ShouldExit();
-};
-
-}
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/scheduling/query-schedule.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/query-schedule.cc b/be/src/scheduling/query-schedule.cc
index f893f1c..b5745c4 100644
--- a/be/src/scheduling/query-schedule.cc
+++ b/be/src/scheduling/query-schedule.cc
@@ -28,7 +28,6 @@
 #include "util/uid-util.h"
 #include "util/debug-util.h"
 #include "util/parse-util.h"
-#include "util/llama-util.h"
 
 #include "common/names.h"
 
@@ -36,28 +35,13 @@ using boost::uuids::random_generator;
 using boost::uuids::uuid;
 using namespace impala;
 
-DEFINE_bool(rm_always_use_defaults, false, "If true, all queries use the same initial"
-    " resource requests regardless of their computed resource estimates. Only meaningful "
-    "if --enable_rm is set.");
-DEFINE_string(rm_default_memory, "4G", "The initial amount of memory that"
-    " a query should reserve on each node if either it does not have an available "
-    "estimate, or if --rm_always_use_defaults is set.");
-DEFINE_int32(rm_default_cpu_vcores, 2, "The initial number of virtual cores that"
-    " a query should reserve on each node if either it does not have an available "
-    "estimate, or if --rm_always_use_defaults is set.");
-
+// TODO: Remove for Impala 3.0.
+DEFINE_bool(rm_always_use_defaults, false, "Deprecated");
+DEFINE_string(rm_default_memory, "4G", "Deprecated");
+DEFINE_int32(rm_default_cpu_vcores, 2, "Deprecated");
 
 namespace impala {
 
-// Default value for the request_timeout in a reservation request. The timeout is the
-// max time in milliseconds to wait for a resource request to be fulfilled by Llama.
-// The default value of five minutes was determined to be reasonable based on
-// experiments on a 20-node cluster with TPCDS 15TB and 8 concurrent clients.
-// Over 30% of queries timed out with a reservation timeout of 1 minute but only less
-// than 5% timed out when using 5 minutes. Still, the default value is somewhat
-// arbitrary and a good value is workload dependent.
-const int64_t DEFAULT_REQUEST_TIMEOUT_MS = 5 * 60 * 1000;
-
 QuerySchedule::QuerySchedule(const TUniqueId& query_id,
     const TQueryExecRequest& request, const TQueryOptions& query_options,
     RuntimeProfile* summary_profile, RuntimeProfile::EventSequence* query_events)
@@ -97,10 +81,9 @@ int64_t QuerySchedule::GetClusterMemoryEstimate() const {
 int64_t QuerySchedule::GetPerHostMemoryEstimate() const {
   // Precedence of different estimate sources is:
   // user-supplied RM query option >
-  //   server-side defaults (if rm_always_use_defaults == true) >
   //     query option limit >
   //       estimate >
-  //         server-side defaults (if rm_always_use_defaults == false)
+  //         server-side defaults
   int64_t query_option_memory_limit = numeric_limits<int64_t>::max();
   bool has_query_option = false;
   if (query_options_.__isset.mem_limit && query_options_.mem_limit > 0) {
@@ -116,12 +99,10 @@ int64_t QuerySchedule::GetPerHostMemoryEstimate() const {
   }
 
   int64_t per_host_mem = 0L;
+  // TODO: Remove rm_initial_mem and associated logic when we're sure that clients won't
+  // be affected.
   if (query_options_.__isset.rm_initial_mem && query_options_.rm_initial_mem > 0) {
     per_host_mem = query_options_.rm_initial_mem;
-  } else if (FLAGS_rm_always_use_defaults) {
-    bool ignored;
-    per_host_mem = ParseUtil::ParseMemSpec(FLAGS_rm_default_memory,
-        &ignored, 0);
   } else if (has_query_option) {
     per_host_mem = query_option_memory_limit;
   } else if (has_estimate) {
@@ -134,115 +115,11 @@ int64_t QuerySchedule::GetPerHostMemoryEstimate() const {
   }
   // Cap the memory estimate at the amount of physical memory available. The user's
   // provided value or the estimate from planning can each be unreasonable.
-  // TODO: Get this limit from Llama (Yarn sets it).
   return min(per_host_mem, MemInfo::physical_mem());
 }
 
-int16_t QuerySchedule::GetPerHostVCores() const {
-  // Precedence of different estimate sources is:
-  // server-side defaults (if rm_always_use_defaults == true) >
-  //   computed estimates
-  //     server-side defaults (if rm_always_use_defaults == false)
-  int16_t v_cpu_cores = FLAGS_rm_default_cpu_vcores;
-  if (!FLAGS_rm_always_use_defaults && query_options_.__isset.v_cpu_cores &&
-      query_options_.v_cpu_cores > 0) {
-    v_cpu_cores = query_options_.v_cpu_cores;
-  }
-
-  return v_cpu_cores;
-}
-
-void QuerySchedule::GetResourceHostport(const TNetworkAddress& src,
-    TNetworkAddress* dst) {
-  DCHECK(dst != NULL);
-  DCHECK(resource_resolver_.get() != NULL)
-      << "resource_resolver_ is NULL, didn't call SetUniqueHosts()?";
-  resource_resolver_->GetResourceHostport(src, dst);
-}
-
 void QuerySchedule::SetUniqueHosts(const unordered_set<TNetworkAddress>& unique_hosts) {
   unique_hosts_ = unique_hosts;
-  resource_resolver_.reset(new ResourceResolver(unique_hosts_));
-}
-
-void QuerySchedule::PrepareReservationRequest(const string& pool, const string& user) {
-  reservation_request_.resources.clear();
-  reservation_request_.version = TResourceBrokerServiceVersion::V1;
-  reservation_request_.queue = pool;
-  reservation_request_.gang = true;
-  // Convert the user name to a short name (e.g. 'user1@domain' to 'user1') because
-  // Llama checks group membership based on the short name of the principal.
-  reservation_request_.user = llama::GetShortName(user);
-
-  // Set optional request timeout from query options.
-  if (query_options_.__isset.reservation_request_timeout) {
-    DCHECK_GT(query_options_.reservation_request_timeout, 0);
-    reservation_request_.__set_request_timeout(
-        query_options_.reservation_request_timeout);
-  }
-
-  // Set the reservation timeout from the query options or use a default.
-  int64_t timeout = DEFAULT_REQUEST_TIMEOUT_MS;
-  if (query_options_.__isset.reservation_request_timeout) {
-    timeout = query_options_.reservation_request_timeout;
-  }
-  reservation_request_.__set_request_timeout(timeout);
-
-  int32_t memory_mb = GetPerHostMemoryEstimate() / 1024 / 1024;
-  int32_t v_cpu_cores = GetPerHostVCores();
-  // The memory_mb and v_cpu_cores estimates may legitimately be zero,
-  // e.g., for constant selects. Do not reserve any resources in those cases.
-  if (memory_mb == 0 && v_cpu_cores == 0) return;
-
-  DCHECK(resource_resolver_.get() != NULL)
-      << "resource_resolver_ is NULL, didn't call SetUniqueHosts()?";
-  random_generator uuid_generator;
-  for (const TNetworkAddress& host: unique_hosts_) {
-    reservation_request_.resources.push_back(llama::TResource());
-    llama::TResource& resource = reservation_request_.resources.back();
-    uuid id = uuid_generator();
-    resource.client_resource_id.hi = *reinterpret_cast<uint64_t*>(&id.data[0]);
-    resource.client_resource_id.lo = *reinterpret_cast<uint64_t*>(&id.data[8]);
-    resource.enforcement = llama::TLocationEnforcement::MUST;
-
-    TNetworkAddress resource_hostport;
-    resource_resolver_->GetResourceHostport(host, &resource_hostport);
-    stringstream ss;
-    ss << resource_hostport;
-    resource.askedLocation = ss.str();
-    resource.memory_mb = memory_mb;
-    resource.v_cpu_cores = v_cpu_cores;
-  }
-}
-
-Status QuerySchedule::ValidateReservation() {
-  if (!HasReservation()) return Status("Query schedule does not have a reservation.");
-  vector<TNetworkAddress> hosts_missing_resources;
-  ResourceResolver resolver(unique_hosts_);
-  for (const FragmentExecParams& params: fragment_exec_params_) {
-    for (const TNetworkAddress& host: params.hosts) {
-      // Ignore the coordinator host which is not contained in unique_hosts_.
-      if (unique_hosts_.find(host) == unique_hosts_.end()) continue;
-      TNetworkAddress resource_hostport;
-      resolver.GetResourceHostport(host, &resource_hostport);
-      if (reservation_.allocated_resources.find(resource_hostport) ==
-          reservation_.allocated_resources.end()) {
-        hosts_missing_resources.push_back(host);
-      }
-    }
-  }
-  if (!hosts_missing_resources.empty()) {
-    stringstream ss;
-    ss << "Failed to validate reservation " << reservation_.reservation_id << "." << endl
-       << "Missing resources for hosts [";
-    for (int i = 0; i < hosts_missing_resources.size(); ++i) {
-      ss << hosts_missing_resources[i];
-      if (i + 1 !=  hosts_missing_resources.size()) ss << ", ";
-    }
-    ss << "]";
-    return Status(ss.str());
-  }
-  return Status::OK();
 }
 
 }


[7/7] incubator-impala git commit: IMPALA-4138: Fix AcquireState() for batches that change capacity

Posted by he...@apache.org.
IMPALA-4138: Fix AcquireState() for batches that change capacity

If MarkAtCapacity() is called on a row batch, it is difficult to call
AcquireState() on that batch because tuple_ptrs_size_ is not accessible
to initialise the destination batch - this is usually calculated from
capacity(), but that value is wrong for these purposes after
MarkAtCapacity().

Add RowBatch::InitialCapacity() to return the initial capacity value of
the batch.

Add row-batch-test to add initial coverage of AcquireState() API.

Change-Id: I6ceca53c406b05cd04b7d95a4f9f2eec7bc127f5
Reviewed-on: http://gerrit.cloudera.org:8080/4428
Reviewed-by: Dan Hecht <dh...@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/cd6d86b8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/cd6d86b8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/cd6d86b8

Branch: refs/heads/master
Commit: cd6d86b83078290c62c2d92b071a15ec19bbeb4f
Parents: 3aa4351
Author: Henry Robinson <he...@cloudera.com>
Authored: Thu Sep 15 12:08:52 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Wed Sep 21 03:08:02 2016 +0000

----------------------------------------------------------------------
 be/src/runtime/CMakeLists.txt    |  1 +
 be/src/runtime/row-batch-test.cc | 69 +++++++++++++++++++++++++++++++++++
 be/src/runtime/row-batch.cc      |  1 +
 be/src/runtime/row-batch.h       |  8 +++-
 4 files changed, 78 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/cd6d86b8/be/src/runtime/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/runtime/CMakeLists.txt b/be/src/runtime/CMakeLists.txt
index 6bdce48..a0b0a94 100644
--- a/be/src/runtime/CMakeLists.txt
+++ b/be/src/runtime/CMakeLists.txt
@@ -88,4 +88,5 @@ ADD_BE_TEST(buffered-tuple-stream-test)
 ADD_BE_TEST(hdfs-fs-cache-test)
 ADD_BE_TEST(tmp-file-mgr-test)
 ADD_BE_TEST(row-batch-serialize-test)
+ADD_BE_TEST(row-batch-test)
 ADD_BE_TEST(collection-value-builder-test)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/cd6d86b8/be/src/runtime/row-batch-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/row-batch-test.cc b/be/src/runtime/row-batch-test.cc
new file mode 100644
index 0000000..2a8304a
--- /dev/null
+++ b/be/src/runtime/row-batch-test.cc
@@ -0,0 +1,69 @@
+// 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.
+
+#include "testutil/death-test-util.h"
+#include "testutil/gtest-util.h"
+#include "runtime/mem-tracker.h"
+#include "runtime/row-batch.h"
+#include "testutil/desc-tbl-builder.h"
+
+#include <gtest/gtest.h>
+
+#include "common/names.h"
+
+namespace impala {
+
+TEST(RowBatchTest, AcquireStateWithMarkAtCapacity) {
+  // Test that AcquireState() can be correctly called with MarkAtCapacity() on the
+  // source batch.
+  ObjectPool pool;
+  DescriptorTblBuilder builder(&pool);
+  builder.DeclareTuple() << TYPE_INT;
+  DescriptorTbl* desc_tbl = builder.Build();
+
+  vector<bool> nullable_tuples = {false};
+  vector<TTupleId> tuple_id = {static_cast<TupleId>(0)};
+  RowDescriptor row_desc(*desc_tbl, tuple_id, nullable_tuples);
+  MemTracker tracker;
+  {
+    RowBatch src(row_desc, 1024, &tracker);
+    src.AddRow(); src.CommitLastRow();
+    // Calls MarkAtCapacity().
+    src.MarkNeedToReturn();
+
+    // Note InitialCapacity(), not capacity(). Latter will DCHECK.
+    RowBatch dest(row_desc, src.InitialCapacity(), &tracker);
+    dest.AcquireState(&src);
+  }
+
+  // Confirm the bad pattern causes an error.
+  {
+    RowBatch src(row_desc, 1024, &tracker);
+    src.AddRow(); src.CommitLastRow();
+    // Calls MarkAtCapacity().
+    src.MarkNeedToReturn();
+    RowBatch bad_dest(row_desc, src.capacity(), &tracker);
+    IMPALA_ASSERT_DEBUG_DEATH(bad_dest.AcquireState(&src), "");
+  }
+}
+
+}
+
+int main(int argc, char** argv) {
+  ::testing::InitGoogleTest(&argc, argv);
+  return RUN_ALL_TESTS();
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/cd6d86b8/be/src/runtime/row-batch.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/row-batch.cc b/be/src/runtime/row-batch.cc
index 3d076bf..daac913 100644
--- a/be/src/runtime/row-batch.cc
+++ b/be/src/runtime/row-batch.cc
@@ -83,6 +83,7 @@ RowBatch::RowBatch(const RowDescriptor& row_desc, const TRowBatch& input_batch,
     mem_tracker_(mem_tracker) {
   DCHECK(mem_tracker_ != NULL);
   tuple_ptrs_size_ = num_rows_ * input_batch.row_tuples.size() * sizeof(Tuple*);
+  DCHECK_EQ(input_batch.row_tuples.size(), row_desc.tuple_descriptors().size());
   DCHECK_GT(tuple_ptrs_size_, 0);
   // TODO: switch to Init() pattern so we can check memory limit and return Status.
   if (FLAGS_enable_partitioned_aggregation && FLAGS_enable_partitioned_hash_join) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/cd6d86b8/be/src/runtime/row-batch.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/row-batch.h b/be/src/runtime/row-batch.h
index f529c3d..75c1f44 100644
--- a/be/src/runtime/row-batch.h
+++ b/be/src/runtime/row-batch.h
@@ -279,6 +279,12 @@ class RowBatch {
   int ALWAYS_INLINE num_rows() const { return num_rows_; }
   int ALWAYS_INLINE capacity() const { return capacity_; }
 
+  // The maximum value that capacity_ ever took, before MarkCapacity() might have changed
+  // it.
+  int ALWAYS_INLINE InitialCapacity() const {
+    return tuple_ptrs_size_ / (num_tuples_per_row_ * sizeof(Tuple*));
+  }
+
   const RowDescriptor& row_desc() const { return row_desc_; }
 
   /// Max memory that this row batch can accumulate before it is considered at capacity.
@@ -351,7 +357,7 @@ class RowBatch {
 
   const int num_tuples_per_row_;
 
-  /// Array of pointers with capacity_ * num_tuples_per_row_ elements.
+  /// Array of pointers with InitialCapacity() * num_tuples_per_row_ elements.
   /// The memory ownership depends on whether legacy joins and aggs are enabled.
   ///
   /// Memory is malloc'd and owned by RowBatch:


[6/7] incubator-impala git commit: IMPALA-4170: Fix identifier quoting in COMPUTE INCREMENTAL STATS.

Posted by he...@apache.org.
IMPALA-4170: Fix identifier quoting in COMPUTE INCREMENTAL STATS.

The SQL statements generated from COMPUTE INCREMENTAL STATS
did not properly quote identifiers when incrementally updating
the stats for newly added partitions.

Our existing tests did not catch this case because the code paths
for doing the initial stats computation and the incremental stats
computation are different, in particular, the code for generating
the SQL statements.

Change-Id: I63adcc45dc964ce769107bf4139fc4566937bb96
Reviewed-on: http://gerrit.cloudera.org:8080/4479
Reviewed-by: Henry Robinson <he...@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/3aa43516
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/3aa43516
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/3aa43516

Branch: refs/heads/master
Commit: 3aa43516252fe4e75176ab62cb6ce1f04938a96e
Parents: 19de09a
Author: Alex Behm <al...@cloudera.com>
Authored: Tue Sep 20 10:19:29 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Wed Sep 21 01:24:53 2016 +0000

----------------------------------------------------------------------
 .../cloudera/impala/catalog/HdfsPartition.java  | 27 ++++++-------
 .../QueryTest/compute-stats-incremental.test    | 40 +++++++++++++++-----
 2 files changed, 42 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3aa43516/fe/src/main/java/com/cloudera/impala/catalog/HdfsPartition.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/com/cloudera/impala/catalog/HdfsPartition.java b/fe/src/main/java/com/cloudera/impala/catalog/HdfsPartition.java
index 5deb74f..f408468 100644
--- a/fe/src/main/java/com/cloudera/impala/catalog/HdfsPartition.java
+++ b/fe/src/main/java/com/cloudera/impala/catalog/HdfsPartition.java
@@ -44,11 +44,11 @@ import com.cloudera.impala.thrift.THdfsCompression;
 import com.cloudera.impala.thrift.THdfsFileBlock;
 import com.cloudera.impala.thrift.THdfsFileDesc;
 import com.cloudera.impala.thrift.THdfsPartition;
-import com.cloudera.impala.thrift.THdfsPartitionLocation;
 import com.cloudera.impala.thrift.TNetworkAddress;
 import com.cloudera.impala.thrift.TPartitionStats;
 import com.cloudera.impala.thrift.TTableStats;
 import com.cloudera.impala.util.HdfsCachingUtil;
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
@@ -57,7 +57,6 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.common.annotations.VisibleForTesting;
 
 /**
  * Query-relevant information for one table partition. Partitions are comparable
@@ -337,25 +336,23 @@ public class HdfsPartition implements Comparable<HdfsPartition> {
   /**
    * Utility method which returns a string of conjuncts of equality exprs to exactly
    * select this partition (e.g. ((month=2009) AND (year=2012)).
-   * TODO: Remove this when TODO elsewhere in this file to save and expose the list of
-   * TPartitionKeyValues has been resolved.
+   * TODO: Remove this when the TODO elsewhere in this file to save and expose the
+   * list of TPartitionKeyValues has been resolved.
    */
   public String getConjunctSql() {
-    List<String> partitionCols = Lists.newArrayList();
-    for (int i = 0; i < getTable().getNumClusteringCols(); ++i) {
-      partitionCols.add(ToSqlUtils.getIdentSql(getTable().getColumns().get(i).getName()));
+    List<String> partColSql = Lists.newArrayList();
+    for (Column partCol: getTable().getClusteringColumns()) {
+      partColSql.add(ToSqlUtils.getIdentSql(partCol.getName()));
     }
 
     List<String> conjuncts = Lists.newArrayList();
-    for (int i = 0; i < partitionCols.size(); ++i) {
-      LiteralExpr expr = getPartitionValues().get(i);
-      String sql = expr.toSql();
-      if (expr instanceof NullLiteral || sql.isEmpty()) {
-        conjuncts.add(ToSqlUtils.getIdentSql(partitionCols.get(i))
-            + " IS NULL");
+    for (int i = 0; i < partColSql.size(); ++i) {
+      LiteralExpr partVal = getPartitionValues().get(i);
+      String partValSql = partVal.toSql();
+      if (partVal instanceof NullLiteral || partValSql.isEmpty()) {
+        conjuncts.add(partColSql.get(i) + " IS NULL");
       } else {
-        conjuncts.add(ToSqlUtils.getIdentSql(partitionCols.get(i))
-            + "=" + sql);
+        conjuncts.add(partColSql.get(i) + "=" + partValSql);
       }
     }
     return "(" + Joiner.on(" AND " ).join(conjuncts) + ")";

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3aa43516/testdata/workloads/functional-query/queries/QueryTest/compute-stats-incremental.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/compute-stats-incremental.test b/testdata/workloads/functional-query/queries/QueryTest/compute-stats-incremental.test
index 382737e..5f98ee7 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/compute-stats-incremental.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/compute-stats-incremental.test
@@ -324,27 +324,47 @@ STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING
 ====
 ---- QUERY
 # Check that incremental stats queries handle partitions with keyword names
-create table incremental_keyword_part_key(col int) partitioned by
-(`date` int);
-insert into incremental_keyword_part_key partition(`date`=1) values(2);
+create table incremental_keyword_part_key(col int)
+partitioned by (`date` int, `table` string);
+insert into incremental_keyword_part_key
+partition(`date`=1, `table`='a') values(2);
 compute incremental stats incremental_keyword_part_key
-partition(`date`=1);
+partition(`date`=1, `table`='a');
 show table stats incremental_keyword_part_key;
 ---- RESULTS
-'1',1,1,'2B','NOT CACHED','NOT CACHED','TEXT','true',regex:.*
-'Total',1,1,'2B','0B','','','',''
+'1','a',1,1,'2B','NOT CACHED','NOT CACHED','TEXT','true',regex:.*
+'Total','',1,1,'2B','0B','','','',''
 ---- TYPES
-STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING
+STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING
+====
+---- QUERY
+# IMPALA-4170: Check that incremental stats queries handles
+# new partitions with keyword names.
+insert into incremental_keyword_part_key
+partition(`date`=2, `table`='b') values(3);
+insert into incremental_keyword_part_key
+partition(`date`=NULL, `table`='') values(4);
+compute incremental stats incremental_keyword_part_key;
+show table stats incremental_keyword_part_key;
+---- RESULTS
+'NULL','NULL',1,1,'2B','NOT CACHED','NOT CACHED','TEXT','true',regex:.*
+'1','a',1,1,'2B','NOT CACHED','NOT CACHED','TEXT','true',regex:.*
+'2','b',1,1,'2B','NOT CACHED','NOT CACHED','TEXT','true',regex:.*
+'Total','',3,3,'6B','0B','','','',''
+---- TYPES
+STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING
 ====
 ---- QUERY
 drop stats incremental_keyword_part_key;
 compute incremental stats incremental_keyword_part_key;
 show table stats incremental_keyword_part_key;
 ---- RESULTS
-'1',1,1,'2B','NOT CACHED','NOT CACHED','TEXT','true',regex:.*
-'Total',1,1,'2B','0B','','','',''
+'NULL','NULL',1,1,'2B','NOT CACHED','NOT CACHED','TEXT','true',regex:.*
+'1','a',1,1,'2B','NOT CACHED','NOT CACHED','TEXT','true',regex:.*
+'2','b',1,1,'2B','NOT CACHED','NOT CACHED','TEXT','true',regex:.*
+'Total','',3,3,'6B','0B','','','',''
 ---- TYPES
-STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING
+STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING
 ====
 ---- QUERY
 create table incremental_string_part_value(col int) partitioned by


[2/7] incubator-impala git commit: IMPALA-4160: Remove Llama support.

Posted by he...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/common/thrift/ResourceBrokerService.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/ResourceBrokerService.thrift b/common/thrift/ResourceBrokerService.thrift
deleted file mode 100644
index 36f1db8..0000000
--- a/common/thrift/ResourceBrokerService.thrift
+++ /dev/null
@@ -1,119 +0,0 @@
-// 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.
-
-namespace cpp impala
-namespace java com.cloudera.impala.thrift
-
-include "Status.thrift"
-include "Types.thrift"
-include "Llama.thrift"
-
-enum TResourceBrokerServiceVersion {
-   V1
-}
-
-struct TResourceBrokerRegisterRequest {
-  1: required TResourceBrokerServiceVersion version;
-  2: optional i32 client_process_id;
-  3: optional Types.TNetworkAddress notification_callback_service;
-}
-
-struct TResourceBrokerRegisterResponse {
-  1: optional Status.TStatus status;
-  2: optional Types.TUniqueId irb_handle;
-}
-
-struct TResourceBrokerUnregisterRequest {
-  1: optional TResourceBrokerServiceVersion version;
-  2: optional Types.TUniqueId irb_handle;
-}
-
-struct TResourceBrokerUnregisterResponse {
-  1: optional Status.TStatus status;
-}
-
-struct TResourceBrokerExpansionRequest {
-  1: required TResourceBrokerServiceVersion version;
-  2: optional Llama.TResource resource;
-  3: optional Types.TUniqueId reservation_id;
-  4: optional i64 request_timeout;
-}
-
-struct TResourceBrokerExpansionResponse {
-  1: optional Types.TUniqueId reservation_id;
-  2: optional map<Types.TNetworkAddress, Llama.TAllocatedResource> allocated_resources;
-}
-
-struct TResourceBrokerReservationRequest {
-  1: required TResourceBrokerServiceVersion version;
-  2: optional Types.TUniqueId irb_handle;
-  3: optional string queue;
-  4: optional list<Llama.TResource> resources;
-
-  // If true, requires a reservation response to either grant or deny all resources
-  // in this request. If false, reservation responses may deliver partially
-  // granted/denied resources.
-  5: optional bool gang;
-
-  // Max time in milliseconds the resource broker should wait for
-  // a resource request to be granted by Llama/Yarn.
-  6: optional i64 request_timeout;
-
-  // Used to allow Llama to grant or deny access to the requested queue.
-  7: optional string user;
-}
-
-struct TResourceBrokerReservationResponse {
-  1: optional Status.TStatus status;
-  2: optional Types.TUniqueId reservation_id;
-  3: optional map<Types.TNetworkAddress, Llama.TAllocatedResource> allocated_resources;
-}
-
-struct TResourceBrokerReleaseRequest {
-  1: optional TResourceBrokerServiceVersion version;
-  2: optional Types.TUniqueId irb_handle;
-  3: optional Types.TUniqueId reservation_id;
-}
-
-struct TResourceBrokerReleaseResponse {
-  1: optional Status.TStatus status;
-}
-
-service ResourceBrokerService {
-  TResourceBrokerRegisterResponse
-      Register(1: TResourceBrokerRegisterRequest request);
-  TResourceBrokerUnregisterResponse
-      Unregister(1: TResourceBrokerUnregisterRequest request);
-  TResourceBrokerReservationResponse
-      Reserve(1: TResourceBrokerReservationRequest request);
-  TResourceBrokerReleaseResponse
-      Release(1: TResourceBrokerReleaseRequest request);
-}
-
-struct TResourceBrokerNotificationPreemptionRequest {
-  1: required TResourceBrokerServiceVersion version;
-  2: optional list<string> preempted_rm_resource_ids;
-}
-
-struct TResourceBrokerNotificationPreemptionResponse {
-  1: optional Status.TStatus status;
-}
-
-service ResourceBrokerNotificationService {
-  TResourceBrokerRegisterResponse
-      Preempt(1: TResourceBrokerRegisterRequest request);
-}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/common/thrift/metrics.json
----------------------------------------------------------------------
diff --git a/common/thrift/metrics.json b/common/thrift/metrics.json
index 1d449b5..b600f80 100644
--- a/common/thrift/metrics.json
+++ b/common/thrift/metrics.json
@@ -740,216 +740,6 @@
     "key": "request-pool-service.resolve-pool-duration-ms"
   },
   {
-    "description": "The number of clients in use by the resource broker client cache.",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Client Cache Clients In Use",
-    "units": "NONE",
-    "kind": "GAUGE",
-    "key": "resource-broker.client-cache.clients-in-use"
-  },
-  {
-    "description": "The total number of clients in the resource broker client cache.",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Client Cache Total Clients",
-    "units": "NONE",
-    "kind": "GAUGE",
-    "key": "resource-broker.client-cache.total-clients"
-  },
-  {
-    "key": "resource-broker.active-llama",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Active Llama",
-    "units": "NONE",
-    "kind": "PROPERTY",
-    "description": "The address of the active Llama server."
-  },
-  {
-    "key": "resource-broker.active-llama-handle",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Active Llama Handle",
-    "units": "NONE",
-    "kind": "PROPERTY",
-    "description": "The Llama client handle provided by the active Llama server."
-  },
-  {
-    "key": "resource-broker.expansion-request-response-time",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Expansion Request Response Time",
-    "units": "TIME_S",
-    "kind": "STATS",
-    "description": "The time, in seconds, that a expansion request takes to be fulfilled by Llama"
-  },
-  {
-    "key": "resource-broker.expansion-request-rpc-time",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Expansion Request Rpc Time",
-    "units": "TIME_S",
-    "kind": "STATS",
-    "description": "The time, in seconds, that a Reserve() RPC takes to Llama"
-  },
-  {
-    "key": "resource-broker.expansion-requests-failed",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Expansion Requests Failed",
-    "units": "UNIT",
-    "kind": "COUNTER",
-    "description": "The number of expansion requests made by this Impala daemon to Llama which failed"
-  },
-  {
-    "key": "resource-broker.expansion-requests-fulfilled",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Expansion Requests Fulfilled",
-    "units": "UNIT",
-    "kind": "COUNTER",
-    "description": "The number of expansion requests made by this Impala daemon to Llama which succeeded"
-  },
-  {
-    "key": "resource-broker.expansion-requests-rejected",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Expansion Requests Rejected",
-    "units": "UNIT",
-    "kind": "COUNTER",
-    "description": "The number of expansion requests made by this Impala daemon to Llama which were rejected"
-  },
-  {
-    "key": "resource-broker.expansion-requests-timedout",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Expansion Requests Timedout",
-    "units": "UNIT",
-    "kind": "COUNTER",
-    "description": "The number of expansion requests made by this Impala daemon to Llama which timed out"
-  },
-  {
-    "key": "resource-broker.expansion-requests-total",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Expansion Requests Total",
-    "units": "UNIT",
-    "kind": "COUNTER",
-    "description": "The total number of expansion requests made by this Impala daemon to Llama"
-  },
-  {
-    "key": "resource-broker.memory-resources-in-use",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Memory Resources In Use",
-    "units": "BYTES",
-    "kind": "GAUGE",
-    "description": "The total number of bytes currently allocated to this Impala daemon by Llama"
-  },
-  {
-    "key": "resource-broker.requests-released",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Requests Released",
-    "units": "UNIT",
-    "kind": "COUNTER",
-    "description": "The number of resource-release requests received from Llama"
-  },
-  {
-    "key": "resource-broker.reservation-request-response-time",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Reservation Request Response Time",
-    "units": "TIME_S",
-    "kind": "STATS",
-    "description": "The time, in seconds, that a reservation request takes to be fulfilled by Llama"
-  },
-  {
-    "key": "resource-broker.reservation-request-rpc-time",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Reservation Request Rpc Time",
-    "units": "TIME_S",
-    "kind": "STATS",
-    "description": "The time, in seconds, that a Reserve() RPC takes to Llama"
-  },
-  {
-    "key": "resource-broker.reservation-requests-failed",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Reservation Requests Failed",
-    "units": "UNIT",
-    "kind": "COUNTER",
-    "description": "The number of reservation requests made by this Impala daemon to Llama which failed"
-  },
-  {
-    "key": "resource-broker.reservation-requests-fulfilled",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Reservation Requests Fulfilled",
-    "units": "UNIT",
-    "kind": "COUNTER",
-    "description": "The number of reservation requests made by this Impala daemon to Llama which succeeded"
-  },
-  {
-    "key": "resource-broker.reservation-requests-rejected",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Reservation Requests Rejected",
-    "units": "UNIT",
-    "kind": "COUNTER",
-    "description": "The number of reservation requests made by this Impala daemon to Llama which were rejected"
-  },
-  {
-    "key": "resource-broker.reservation-requests-timedout",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Reservation Requests Timedout",
-    "units": "UNIT",
-    "kind": "COUNTER",
-    "description": "The number of reservation requests made by this Impala daemon to Llama which timed out"
-  },
-  {
-    "key": "resource-broker.reservation-requests-total",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Reservation Requests Total",
-    "units": "UNIT",
-    "kind": "COUNTER",
-    "description": "The total number of reservation requests made by this Impala daemon to Llama"
-  },
-  {
-    "key": "resource-broker.vcpu-resources-in-use",
-    "contexts": [
-      "IMPALAD"
-    ],
-    "label": "Resource Broker Vcpu Resources In Use",
-    "units": "UNIT",
-    "kind": "GAUGE",
-    "description": "The total number of vcpus currently allocated to this Impala daemon by Llama"
-  },
-  {
     "description": "Duration (ms) of RPC calls to $0",
     "contexts": [
       "CATALOGSERVER",

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/fe/src/main/java/com/cloudera/impala/planner/Planner.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/com/cloudera/impala/planner/Planner.java b/fe/src/main/java/com/cloudera/impala/planner/Planner.java
index 0a424ad..df90df3 100644
--- a/fe/src/main/java/com/cloudera/impala/planner/Planner.java
+++ b/fe/src/main/java/com/cloudera/impala/planner/Planner.java
@@ -292,6 +292,7 @@ public class Planner {
    * Optionally excludes the requirements for unpartitioned fragments.
    * TODO: The LOG.warn() messages should eventually become Preconditions checks
    * once resource estimation is more robust.
+   * TODO: Revisit and possibly remove during MT work, particularly references to vcores.
    */
   public void computeResourceReqs(List<PlanFragment> fragments,
       boolean excludeUnpartitionedFragments,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/testdata/cluster/admin
----------------------------------------------------------------------
diff --git a/testdata/cluster/admin b/testdata/cluster/admin
index af35009..d263999 100755
--- a/testdata/cluster/admin
+++ b/testdata/cluster/admin
@@ -55,7 +55,7 @@ export KILL_CLUSTER_MARKER=IBelongToTheMiniCluster
 
 if [[ "$TARGET_FILESYSTEM" == "hdfs" ]]; then
   # The check above indicates that the regular mini-cluster is in use.
-  SUPPORTED_SERVICES=(hdfs yarn llama kms)
+  SUPPORTED_SERVICES=(hdfs yarn kms)
 else
   # Either a remote distributed file system or a local non-distributed file system is
   # in use. Currently the only service that is expected to work is Kudu, though in theory
@@ -87,9 +87,6 @@ export KUDU_WEBUI_PORT=8051   # same as default
 # easier to maintain them here.
 EMPTY_NODE_DIRS=$(echo data/dfs/{dn,nn} var/{run,lib/hadoop-hdfs,log} \
     var/{log,run}/kudu/{master,ts} var/lib/kudu/{master,ts}/{wal,data})
-if [[ "$CDH_MAJOR_VERSION" -ge 5 ]]; then
-  EMPTY_NODE_DIRS+=" /var/log/llama"
-fi
 
 EASY_ACCESS_LOG_DIR="$IMPALA_CLUSTER_LOGS_DIR"
 MINIKDC_INIT=${IMPALA_HOME}/testdata/bin/minikdc.sh
@@ -226,10 +223,7 @@ function create_cluster {
     fi
   fi
 
-  # For consistency, the first node will host all the master roles, including llama. Llama
-  # needs to know the ports of the datanodes and nodemanagers. The ports are generated
-  # below as part of setting up each node. The slave nodes are created first so the ports
-  # will be known when it is time to configure llama.
+  # For consistency, the first node will host all the master roles.
   for ((NODE_IDX=$NODE_COUNT; NODE_IDX >= 1; NODE_IDX--)); do
     NODE=${NODE_PREFIX}$NODE_IDX
     NODE_DIR=$(get_node_dir $NODE)
@@ -244,7 +238,7 @@ function create_cluster {
     if [[ $NODE_IDX -gt 1 ]]; then
       # Remove master role scripts from slave nodes
       rm -f "$NODE_DIR/etc/init.d/"{hdfs-namenode,yarn-resourcemanager} \
-            "$NODE_DIR/etc/init.d/"{llama-application,kms,kudu-master}
+            "$NODE_DIR/etc/init.d/"{kms,kudu-master}
     fi
     for EMPTY_NODE_DIR in $EMPTY_NODE_DIRS; do
       mkdir -p "$NODE_DIR/$EMPTY_NODE_DIR"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/testdata/cluster/node_templates/cdh5/etc/hadoop/conf/yarn-site.xml.tmpl
----------------------------------------------------------------------
diff --git a/testdata/cluster/node_templates/cdh5/etc/hadoop/conf/yarn-site.xml.tmpl b/testdata/cluster/node_templates/cdh5/etc/hadoop/conf/yarn-site.xml.tmpl
index 0d4b882..3a01d75 100644
--- a/testdata/cluster/node_templates/cdh5/etc/hadoop/conf/yarn-site.xml.tmpl
+++ b/testdata/cluster/node_templates/cdh5/etc/hadoop/conf/yarn-site.xml.tmpl
@@ -1,24 +1,13 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 
+<!-- TODO: Remove any Llama-related configuration. Can this file be removed entirely? -->
 <configuration>
   <property>
     <name>yarn.resourcemanager.webapp.address</name>
     <value>0.0.0.0:${YARN_WEBUI_PORT}</value>
   </property>
 
-  <!-- Min value of 0 is required by llama -->
-  <property>
-    <name>yarn.scheduler.minimum-allocation-mb</name>
-    <value>0</value>
-  </property>
-
-  <!-- Min value of 0 is required by llama -->
-  <property>
-    <name>yarn.scheduler.minimum-allocation-vcores</name>
-    <value>0</value>
-  </property>
-
   <property>
     <name>yarn.nodemanager.address</name>
     <value>127.0.0.1:${NODEMANAGER_PORT}</value>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/testdata/cluster/node_templates/cdh5/etc/init.d/llama-application
----------------------------------------------------------------------
diff --git a/testdata/cluster/node_templates/cdh5/etc/init.d/llama-application b/testdata/cluster/node_templates/cdh5/etc/init.d/llama-application
deleted file mode 100755
index ad24240..0000000
--- a/testdata/cluster/node_templates/cdh5/etc/init.d/llama-application
+++ /dev/null
@@ -1,38 +0,0 @@
-#!/bin/bash
-
-# 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.
-
-DIR=$(dirname $0)
-
-. "$DIR/common"
-. "$DIR/hdfs-common"
-
-HADOOP_LOG_DIR="$LOG_DIR/llama"
-HADOOP_CLASSPATH="$NODE_DIR/etc/llama/conf:$HADOOP_CLASSPATH"
-for JAR in $(find "$LLAMA_HOME" -name "*jar"); do
-  HADOOP_CLASSPATH="$JAR:$HADOOP_CLASSPATH"
-done
-export HADOOP_CLASSPATH
-export HADOOP_CLIENT_OPTS="$HADOOP_CLIENT_OPTS \
-    -Dllama.server.conf.dir=$NODE_DIR/etc/llama/conf"
-
-function start {
-  do_start hadoop 'com.cloudera.llama.am.LlamaAMMain'
-}
-
-$1

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/testdata/cluster/node_templates/cdh5/etc/llama/conf/llama-log4j.properties.tmpl
----------------------------------------------------------------------
diff --git a/testdata/cluster/node_templates/cdh5/etc/llama/conf/llama-log4j.properties.tmpl b/testdata/cluster/node_templates/cdh5/etc/llama/conf/llama-log4j.properties.tmpl
deleted file mode 100644
index ba62c0f..0000000
--- a/testdata/cluster/node_templates/cdh5/etc/llama/conf/llama-log4j.properties.tmpl
+++ /dev/null
@@ -1,29 +0,0 @@
-#
-# 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.
-#
-
-log4j.appender.RFA=org.apache.log4j.RollingFileAppender
-log4j.appender.RFA.File=${NODE_DIR}/var/log/llama/llama-application.log
-log4j.appender.RFA.MaxFileSize=256MB
-log4j.appender.RFA.MaxBackupIndex=20
-log4j.appender.RFA.layout=org.apache.log4j.PatternLayout
-log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
-
-log4j.logger.org.apache.hadoop.conf=ERROR, RFA
-log4j.logger.org.apache.hadoop=WARN, RFA
-log4j.logger.com.cloudera.llama=INFO, RFA
-#log4j.logger.org.apache.thrift.transport=DEBUG, RFA

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/testdata/cluster/node_templates/cdh5/etc/llama/conf/llama-site.xml.tmpl
----------------------------------------------------------------------
diff --git a/testdata/cluster/node_templates/cdh5/etc/llama/conf/llama-site.xml.tmpl b/testdata/cluster/node_templates/cdh5/etc/llama/conf/llama-site.xml.tmpl
deleted file mode 100644
index 90232c0..0000000
--- a/testdata/cluster/node_templates/cdh5/etc/llama/conf/llama-site.xml.tmpl
+++ /dev/null
@@ -1,86 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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.
--->
-<configuration>
-  <property>
-    <name>llama.am.server.thrift.http.address</name>
-    <value>0.0.0.0:${LLAMA_WEBUI_PORT}</value>
-  </property>
-
-  <!-- This is causing problems now, re-enable it later. -->
-  <property>
-    <name>llama.am.cache.enabled</name>
-    <value>false</value>
-    <description>
-      Global setting that indicates if Llama should cache allocated resources on
-      release.
-    </description>
-  </property>
-
-  <!-- This is a yarn config but Llama needs it here.  -->
-  <property>
-    <name>yarn.scheduler.include-port-in-node-name</name>
-    <value>true</value>
-  </property>
-
-  <property>
-    <name>llama.am.server.thrift.node.name.mapping.class</name>
-    <value>com.cloudera.llama.am.MiniClusterNodeMapper</value>
-    <description>
-      The NodeMapper implementation LlamaAM server uses to convert requested
-      locations into Yarn Nodes.
-      The default (and only implementation for production) drops the port
-      number if present (Impala uses DataNode addresses to request a location,
-      these addresses may contain the DataNode port number. The DataNode port
-      number is meaningless and unknown to Yarn).
-    </description>
-  </property>
-
-  <property>
-    <name>llama.minicluster.node.mapper.mapping</name>
-    <value>${LLAMA_PORT_MAPPINGS}</value>
-  </property>
-
-  <property>
-    <name>llama.am.hadoop.user.name</name>
-    <value>${USER}</value>
-    <description>
-      User name use by Llama when interacting with Yarn.
-    </description>
-  </property>
-
-<!-- BEGIN Kerberos settings -->
-  <property>
-    <name>llama.am.server.thrift.security</name>
-    <value>true</value>
-  </property>
-
-  <property>
-    <name>llama.am.server.thrift.kerberos.keytab.file</name>
-    <value>${KRB5_KTNAME}</value>
-  </property>
-
-  <property>
-    <name>llama.am.server.thrift.kerberos.server.principal.name</name>
-    <value>${MINIKDC_PRINC_LLAM}</value>
-  </property>
-
-  <property>
-    <name>llama.am.server.thrift.kerberos.notification.principal.name</name>
-    <!-- <value>${MINIKDC_PRINC_IMPALA_BE}</value> -->
-    <value>impala-be</value>
-  </property>
-<!-- END Kerberos settings -->
-
-</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/testdata/cluster/node_templates/common/etc/hadoop/conf/core-site.xml.tmpl
----------------------------------------------------------------------
diff --git a/testdata/cluster/node_templates/common/etc/hadoop/conf/core-site.xml.tmpl b/testdata/cluster/node_templates/common/etc/hadoop/conf/core-site.xml.tmpl
index 2d5cb09..4f4af77 100644
--- a/testdata/cluster/node_templates/common/etc/hadoop/conf/core-site.xml.tmpl
+++ b/testdata/cluster/node_templates/common/etc/hadoop/conf/core-site.xml.tmpl
@@ -110,16 +110,6 @@ DEFAULT</value>
     <value>*</value>
   </property>
 
-  <property>
-    <name>hadoop.proxyuser.llama.hosts</name>
-    <value>*</value>
-  </property>
-
-  <property>
-    <name>hadoop.proxyuser.llama.groups</name>
-    <value>*</value>
-  </property>
-
   <!-- END Kerberos settings -->
 
 </configuration>


[5/7] incubator-impala git commit: IMPALA-4160: Remove Llama support.

Posted by he...@apache.org.
IMPALA-4160: Remove Llama support.

Alas, poor Llama! I knew him, Impala: a system
of infinite jest, of most excellent fancy: we hath
borne him on our back a thousand times; and now, how
abhorred in my imagination it is!

Done:

* Removed QueryResourceMgr, ResourceBroker, CGroupsMgr
* Removed untested 'offline' mode and NM failure detection from
  ImpalaServer
* Removed all Llama-related Thrift files
* Removed RM-related arguments to MemTracker constructors
* Deprecated all RM-related flags, printing a warning if enable_rm is
  set
* Removed expansion logic from MemTracker
* Removed VCore logic from QuerySchedule
* Removed all reservation-related logic from Scheduler
* Removed RM metric descriptions
* Various misc. small class changes

Not done:

* Remove RM flags (--enable_rm etc.)
* Remove RM query options
* Changes to RequestPoolService (see IMPALA-4159)
* Remove estimates of VCores / memory from plan

Change-Id: Icfb14209e31f6608bb7b8a33789e00411a6447ef
Reviewed-on: http://gerrit.cloudera.org:8080/4445
Tested-by: Internal Jenkins
Reviewed-by: Henry Robinson <he...@cloudera.com>


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

Branch: refs/heads/master
Commit: 19de09ab7db4498fa3dd6e0775e32581139dd336
Parents: 3be61f9
Author: Henry Robinson <he...@cloudera.com>
Authored: Thu Sep 15 18:09:46 2016 -0700
Committer: Henry Robinson <he...@cloudera.com>
Committed: Tue Sep 20 23:50:43 2016 +0000

----------------------------------------------------------------------
 be/CMakeLists.txt                               |   4 -
 be/generated-sources/gen-cpp/CMakeLists.txt     |   8 -
 be/src/bufferpool/reservation-tracker-test.cc   |   6 +-
 be/src/exec/blocking-join-node.cc               |   8 -
 be/src/exec/data-sink.cc                        |   2 +-
 be/src/exec/exec-node.cc                        |   5 +-
 be/src/exec/hash-join-node.cc                   |   1 -
 be/src/exec/hdfs-scan-node-base.cc              |   1 -
 be/src/exec/hdfs-scan-node.cc                   |  37 -
 be/src/exec/kudu-scan-node-test.cc              |   2 +-
 be/src/exec/kudu-scan-node.cc                   |   5 -
 be/src/exec/kudu-table-sink-test.cc             |   3 +-
 be/src/exprs/expr-test.cc                       |   4 +-
 be/src/resourcebroker/CMakeLists.txt            |  31 -
 be/src/resourcebroker/resource-broker.cc        | 850 -------------------
 be/src/resourcebroker/resource-broker.h         | 424 ---------
 be/src/runtime/buffered-block-mgr-test.cc       |   4 +-
 be/src/runtime/buffered-block-mgr.cc            |   4 +-
 be/src/runtime/collection-value-builder-test.cc |   2 +-
 be/src/runtime/coordinator.cc                   |  23 +-
 be/src/runtime/data-stream-recvr.cc             |   2 +-
 be/src/runtime/data-stream-test.cc              |   8 +-
 be/src/runtime/disk-io-mgr-test.cc              |   4 +-
 be/src/runtime/disk-io-mgr.cc                   |   4 +-
 be/src/runtime/exec-env.cc                      | 196 +----
 be/src/runtime/exec-env.h                       |  19 -
 be/src/runtime/mem-pool-test.cc                 |   4 +-
 be/src/runtime/mem-tracker-test.cc              |   6 +-
 be/src/runtime/mem-tracker.cc                   |  77 +-
 be/src/runtime/mem-tracker.h                    |  81 +-
 be/src/runtime/plan-fragment-executor.cc        |  90 +-
 be/src/runtime/runtime-filter-bank.cc           |   5 +-
 be/src/runtime/runtime-state.cc                 |  25 +-
 be/src/runtime/runtime-state.h                  |  16 +-
 be/src/runtime/test-env.cc                      |   2 +-
 be/src/scheduling/CMakeLists.txt                |   1 -
 be/src/scheduling/query-resource-mgr.cc         | 271 ------
 be/src/scheduling/query-resource-mgr.h          | 227 -----
 be/src/scheduling/query-schedule.cc             | 137 +--
 be/src/scheduling/query-schedule.h              |  38 +-
 be/src/scheduling/request-pool-service.cc       |   9 +-
 be/src/scheduling/scheduler.h                   |  16 -
 be/src/scheduling/simple-scheduler-test.cc      |   4 +-
 be/src/scheduling/simple-scheduler.cc           | 171 +---
 be/src/scheduling/simple-scheduler.h            |  41 +-
 be/src/service/impala-server.cc                 |  92 +-
 be/src/service/impala-server.h                  |  24 -
 be/src/service/impalad-main.cc                  |   8 +
 be/src/service/query-exec-state.cc              |  20 -
 be/src/util/CMakeLists.txt                      |   2 -
 be/src/util/cgroups-mgr.cc                      | 238 ------
 be/src/util/cgroups-mgr.h                       | 175 ----
 be/src/util/debug-util.h                        |   1 -
 be/src/util/llama-util.cc                       | 152 ----
 be/src/util/llama-util.h                        |  75 --
 be/src/util/thread-pool.h                       |   4 -
 be/src/util/thread.cc                           |  14 -
 be/src/util/thread.h                            |  21 -
 be/src/util/uid-util.h                          |   5 +-
 bin/bootstrap_toolchain.py                      |   2 +-
 bin/create-test-configuration.sh                |   2 +-
 bin/generate_minidump_collection_testdata.py    |   1 -
 bin/start-impala-cluster.py                     |  23 +-
 common/thrift/CMakeLists.txt                    |   2 -
 common/thrift/Frontend.thrift                   |   1 +
 common/thrift/ImpalaInternalService.thrift      |   7 -
 common/thrift/Llama.thrift                      | 276 ------
 common/thrift/ResourceBrokerService.thrift      | 119 ---
 common/thrift/metrics.json                      | 210 -----
 .../com/cloudera/impala/planner/Planner.java    |   1 +
 testdata/cluster/admin                          |  12 +-
 .../cdh5/etc/hadoop/conf/yarn-site.xml.tmpl     |  13 +-
 .../cdh5/etc/init.d/llama-application           |  38 -
 .../etc/llama/conf/llama-log4j.properties.tmpl  |  29 -
 .../cdh5/etc/llama/conf/llama-site.xml.tmpl     |  86 --
 .../common/etc/hadoop/conf/core-site.xml.tmpl   |  10 -
 76 files changed, 165 insertions(+), 4376 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt
index 6df394f..2546bd0 100644
--- a/be/CMakeLists.txt
+++ b/be/CMakeLists.txt
@@ -167,7 +167,6 @@ if (DOXYGEN_FOUND)
     ${CMAKE_SOURCE_DIR}/be/src/common/
     ${CMAKE_SOURCE_DIR}/be/src/exec/
     ${CMAKE_SOURCE_DIR}/be/src/exprs/
-    ${CMAKE_SOURCE_DIR}/be/src/resourcebroker/
     ${CMAKE_SOURCE_DIR}/be/src/runtime/
     ${CMAKE_SOURCE_DIR}/be/src/scheduling/
     ${CMAKE_SOURCE_DIR}/be/src/service/
@@ -267,7 +266,6 @@ set (IMPALA_LINK_LIBS
   Exprs
   GlobalFlags
   ImpalaThrift
-  ResourceBroker
   Rpc
   Runtime
   Scheduling
@@ -295,7 +293,6 @@ if (BUILD_SHARED_LIBS)
     Statestore
     Scheduling
     Catalog
-    ResourceBroker
     ImpalaThrift
     GlobalFlags
     Common
@@ -423,7 +420,6 @@ add_subdirectory(src/catalog)
 add_subdirectory(src/codegen)
 add_subdirectory(src/common)
 add_subdirectory(src/exec)
-add_subdirectory(src/resourcebroker)
 add_subdirectory(src/exprs)
 add_subdirectory(src/runtime)
 add_subdirectory(src/scheduling)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/generated-sources/gen-cpp/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/generated-sources/gen-cpp/CMakeLists.txt b/be/generated-sources/gen-cpp/CMakeLists.txt
index ac2907e..35f7d65 100644
--- a/be/generated-sources/gen-cpp/CMakeLists.txt
+++ b/be/generated-sources/gen-cpp/CMakeLists.txt
@@ -37,10 +37,6 @@ set(SRC_FILES
   ImpalaService_constants.cpp
   ImpalaService_types.cpp
   ImpalaHiveServer2Service.cpp
-  Llama_constants.cpp
-  Llama_types.cpp
-  LlamaAMService.cpp
-  LlamaNotificationService.cpp
   beeswax_constants.cpp
   beeswax_types.cpp
   BeeswaxService.cpp
@@ -79,10 +75,6 @@ set(SRC_FILES
   NetworkTestService.cpp
   PlanNodes_constants.cpp
   PlanNodes_types.cpp
-  ResourceBrokerNotificationService.cpp
-  ResourceBrokerService_constants.cpp
-  ResourceBrokerService_types.cpp
-  ResourceBrokerService.cpp
   Results_constants.cpp
   Results_types.cpp
   Partitions_constants.cpp

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/bufferpool/reservation-tracker-test.cc
----------------------------------------------------------------------
diff --git a/be/src/bufferpool/reservation-tracker-test.cc b/be/src/bufferpool/reservation-tracker-test.cc
index e43efb8..8dd1e41 100644
--- a/be/src/bufferpool/reservation-tracker-test.cc
+++ b/be/src/bufferpool/reservation-tracker-test.cc
@@ -235,8 +235,8 @@ TEST_F(ReservationTrackerTest, MemTrackerIntegrationTwoLevel) {
   // of different code paths.
   root_.InitRootTracker(NewProfile(), MIN_BUFFER_LEN * 100);
   MemTracker root_mem_tracker;
-  MemTracker child_mem_tracker1(-1, -1, "Child 1", &root_mem_tracker);
-  MemTracker child_mem_tracker2(MIN_BUFFER_LEN * 50, -1, "Child 2", &root_mem_tracker);
+  MemTracker child_mem_tracker1(-1, "Child 1", &root_mem_tracker);
+  MemTracker child_mem_tracker2(MIN_BUFFER_LEN * 50, "Child 2", &root_mem_tracker);
   ReservationTracker child_reservations1, child_reservations2;
   child_reservations1.InitChildTracker(
       NewProfile(), &root_, &child_mem_tracker1, 500 * MIN_BUFFER_LEN);
@@ -317,7 +317,7 @@ TEST_F(ReservationTrackerTest, MemTrackerIntegrationMultiLevel) {
   reservations[0].InitRootTracker(NewProfile(), 500);
   for (int i = 1; i < HIERARCHY_DEPTH; ++i) {
     mem_trackers[i].reset(new MemTracker(
-        mem_limits[i], -1, Substitute("Child $0", i), mem_trackers[i - 1].get()));
+        mem_limits[i], Substitute("Child $0", i), mem_trackers[i - 1].get()));
     reservations[i].InitChildTracker(
         NewProfile(), &reservations[i - 1], mem_trackers[i].get(), 500);
   }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/exec/blocking-join-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/blocking-join-node.cc b/be/src/exec/blocking-join-node.cc
index 2c17d13..309bde4 100644
--- a/be/src/exec/blocking-join-node.cc
+++ b/be/src/exec/blocking-join-node.cc
@@ -25,7 +25,6 @@
 #include "runtime/row-batch.h"
 #include "runtime/runtime-state.h"
 #include "runtime/tuple-row.h"
-#include "util/cgroups-mgr.h"
 #include "util/debug-util.h"
 #include "util/runtime-profile-counters.h"
 #include "util/time.h"
@@ -196,13 +195,6 @@ Status BlockingJoinNode::ConstructBuildAndOpenProbe(RuntimeState* state,
     Thread build_thread(
         node_name_, "build thread", bind(&BlockingJoinNode::ProcessBuildInputAsync, this,
                                         state, build_sink, &build_side_status));
-    if (!state->cgroup().empty()) {
-      Status status = state->exec_env()->cgroups_mgr()->AssignThreadToCgroup(
-          build_thread, state->cgroup());
-      // If AssignThreadToCgroup() failed, we still need to wait for the build-side
-      // thread to complete before returning, so just log that error.
-      if (!status.ok()) state->LogError(status.msg());
-    }
     // Open the left child so that it may perform any initialisation in parallel.
     // Don't exit even if we see an error, we still need to wait for the build thread
     // to finish.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/exec/data-sink.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/data-sink.cc b/be/src/exec/data-sink.cc
index a6c8fcd..8c8b2dc 100644
--- a/be/src/exec/data-sink.cc
+++ b/be/src/exec/data-sink.cc
@@ -153,7 +153,7 @@ Status DataSink::Prepare(RuntimeState* state, MemTracker* mem_tracker) {
   DCHECK(mem_tracker != NULL);
   profile_ = state->obj_pool()->Add(new RuntimeProfile(state->obj_pool(), GetName()));
   mem_tracker_ = mem_tracker;
-  expr_mem_tracker_.reset(new MemTracker(-1, -1, "Exprs", mem_tracker, false));
+  expr_mem_tracker_.reset(new MemTracker(-1, "Exprs", mem_tracker, false));
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/exec/exec-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/exec-node.cc b/be/src/exec/exec-node.cc
index 59ac997..2dce0d3 100644
--- a/be/src/exec/exec-node.cc
+++ b/be/src/exec/exec-node.cc
@@ -147,10 +147,9 @@ Status ExecNode::Prepare(RuntimeState* state) {
   DCHECK(runtime_profile_.get() != NULL);
   rows_returned_counter_ =
       ADD_COUNTER(runtime_profile_, "RowsReturned", TUnit::UNIT);
-  mem_tracker_.reset(new MemTracker(
-      runtime_profile_.get(), -1, -1, runtime_profile_->name(),
+  mem_tracker_.reset(new MemTracker(runtime_profile_.get(), -1, runtime_profile_->name(),
       state->instance_mem_tracker()));
-  expr_mem_tracker_.reset(new MemTracker(-1, -1, "Exprs", mem_tracker_.get(), false));
+  expr_mem_tracker_.reset(new MemTracker(-1, "Exprs", mem_tracker_.get(), false));
 
   rows_returned_rate_ = runtime_profile()->AddDerivedCounter(
       ROW_THROUGHPUT_COUNTER, TUnit::UNIT_PER_SECOND,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/exec/hash-join-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hash-join-node.cc b/be/src/exec/hash-join-node.cc
index 946ab48..52ba1d1 100644
--- a/be/src/exec/hash-join-node.cc
+++ b/be/src/exec/hash-join-node.cc
@@ -39,7 +39,6 @@
 
 #include "common/names.h"
 
-DECLARE_string(cgroup_hierarchy_path);
 DEFINE_bool(enable_probe_side_filtering, true, "Deprecated.");
 
 using namespace impala;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/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 de1dad0..c03817b 100644
--- a/be/src/exec/hdfs-scan-node-base.cc
+++ b/be/src/exec/hdfs-scan-node-base.cc
@@ -43,7 +43,6 @@
 #include "runtime/raw-value.h"
 #include "runtime/row-batch.h"
 #include "runtime/string-buffer.h"
-#include "scheduling/query-resource-mgr.h"
 #include "util/bit-util.h"
 #include "util/container-util.h"
 #include "util/debug-util.h"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/exec/hdfs-scan-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node.cc b/be/src/exec/hdfs-scan-node.cc
index 83f6452..03f81ed 100644
--- a/be/src/exec/hdfs-scan-node.cc
+++ b/be/src/exec/hdfs-scan-node.cc
@@ -27,7 +27,6 @@
 #include "runtime/runtime-state.h"
 #include "runtime/mem-tracker.h"
 #include "runtime/row-batch.h"
-#include "scheduling/query-resource-mgr.h"
 #include "util/debug-util.h"
 #include "util/disk-info.h"
 #include "util/runtime-profile-counters.h"
@@ -154,12 +153,6 @@ Status HdfsScanNode::Prepare(RuntimeState* state) {
   SCOPED_TIMER(runtime_profile_->total_time_counter());
   RETURN_IF_ERROR(HdfsScanNodeBase::Prepare(state));
 
-  // Assign scanner thread group to cgroup, if any.
-  if (!state->cgroup().empty()) {
-    scanner_threads_.SetCgroupsMgr(state->exec_env()->cgroups_mgr());
-    scanner_threads_.SetCgroup(state->cgroup());
-  }
-
   // Compute the minimum bytes required to start a new thread. This is based on the
   // file format.
   // The higher the estimate, the less likely it is the query will fail but more likely
@@ -212,12 +205,6 @@ Status HdfsScanNode::Open(RuntimeState* state) {
   thread_avail_cb_id_ = runtime_state_->resource_pool()->AddThreadAvailableCb(
       bind<void>(mem_fn(&HdfsScanNode::ThreadTokenAvailableCb), this, _1));
 
-  if (runtime_state_->query_resource_mgr() != NULL) {
-    rm_callback_id_ = runtime_state_->query_resource_mgr()->AddVcoreAvailableCb(
-        bind<void>(mem_fn(&HdfsScanNode::ThreadTokenAvailableCb), this,
-            runtime_state_->resource_pool()));
-  }
-
   return Status::OK();
 }
 
@@ -228,9 +215,6 @@ void HdfsScanNode::Close(RuntimeState* state) {
   if (thread_avail_cb_id_ != -1) {
     state->resource_pool()->RemoveThreadAvailableCb(thread_avail_cb_id_);
   }
-  if (state->query_resource_mgr() != NULL && rm_callback_id_ != -1) {
-    state->query_resource_mgr()->RemoveVcoreAvailableCb(rm_callback_id_);
-  }
 
   scanner_threads_.JoinAll();
 
@@ -326,8 +310,6 @@ void HdfsScanNode::ThreadTokenAvailableCb(ThreadResourceMgr::ResourcePool* pool)
   //  6. Don't start up a thread if there isn't enough memory left to run it.
   //  7. Don't start up more than maximum number of scanner threads configured.
   //  8. Don't start up if there are no thread tokens.
-  //  9. Don't start up if we are running too many threads for our vcore allocation
-  //  (unless the thread is reserved, in which case it has to run).
 
   // Case 4. We have not issued the initial ranges so don't start a scanner thread.
   // Issuing ranges will call this function and we'll start the scanner threads then.
@@ -360,25 +342,12 @@ void HdfsScanNode::ThreadTokenAvailableCb(ThreadResourceMgr::ResourcePool* pool)
       break;
     }
 
-    // Case 9.
-    if (!is_reserved) {
-      if (runtime_state_->query_resource_mgr() != NULL &&
-          runtime_state_->query_resource_mgr()->IsVcoreOverSubscribed()) {
-        pool->ReleaseThreadToken(false);
-        break;
-      }
-    }
-
     COUNTER_ADD(&active_scanner_thread_counter_, 1);
     COUNTER_ADD(num_scanner_threads_started_counter_, 1);
     stringstream ss;
     ss << "scanner-thread(" << num_scanner_threads_started_counter_->value() << ")";
     scanner_threads_.AddThread(
         new Thread("hdfs-scan-node", ss.str(), &HdfsScanNode::ScannerThread, this));
-
-    if (runtime_state_->query_resource_mgr() != NULL) {
-      runtime_state_->query_resource_mgr()->NotifyThreadUsageChange(1);
-    }
   }
 }
 
@@ -411,9 +380,6 @@ void HdfsScanNode::ScannerThread() {
           // Unlock before releasing the thread token to avoid deadlock in
           // ThreadTokenAvailableCb().
           l.unlock();
-          if (runtime_state_->query_resource_mgr() != NULL) {
-            runtime_state_->query_resource_mgr()->NotifyThreadUsageChange(-1);
-          }
           runtime_state_->resource_pool()->ReleaseThreadToken(false);
           if (filter_status.ok()) {
             for (auto& ctx: filter_ctxs) {
@@ -495,9 +461,6 @@ void HdfsScanNode::ScannerThread() {
   }
 
   COUNTER_ADD(&active_scanner_thread_counter_, -1);
-  if (runtime_state_->query_resource_mgr() != NULL) {
-    runtime_state_->query_resource_mgr()->NotifyThreadUsageChange(-1);
-  }
   runtime_state_->resource_pool()->ReleaseThreadToken(false);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/exec/kudu-scan-node-test.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-scan-node-test.cc b/be/src/exec/kudu-scan-node-test.cc
index 8324628..a0eabef 100644
--- a/be/src/exec/kudu-scan-node-test.cc
+++ b/be/src/exec/kudu-scan-node-test.cc
@@ -85,7 +85,7 @@ class KuduScanNodeTest : public testing::Test {
       exec_env_->thread_mgr()->UnregisterPool(runtime_state_->resource_pool());
     }
 
-    runtime_state_.reset(new RuntimeState(TExecPlanFragmentParams(), "", exec_env_.get()));
+    runtime_state_.reset(new RuntimeState(TExecPlanFragmentParams(), exec_env_.get()));
     runtime_state_->InitMemTrackers(TUniqueId(), NULL, -1);
 
     TKuduScanNode kudu_scan_node_;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/exec/kudu-scan-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-scan-node.cc b/be/src/exec/kudu-scan-node.cc
index e171afa..827631a 100644
--- a/be/src/exec/kudu-scan-node.cc
+++ b/be/src/exec/kudu-scan-node.cc
@@ -35,7 +35,6 @@
 #include "runtime/row-batch.h"
 #include "runtime/string-value.h"
 #include "runtime/tuple-row.h"
-#include "scheduling/query-resource-mgr.h"
 #include "util/disk-info.h"
 #include "util/jni-util.h"
 #include "util/periodic-counter-updater.h"
@@ -248,10 +247,6 @@ void KuduScanNode::ThreadAvailableCb(ThreadResourceMgr::ResourcePool* pool) {
     VLOG_RPC << "Thread started: " << name;
     scanner_threads_.AddThread(new Thread("kudu-scan-node", name,
         &KuduScanNode::RunScannerThread, this, name, token));
-
-    if (runtime_state_->query_resource_mgr() != NULL) {
-      runtime_state_->query_resource_mgr()->NotifyThreadUsageChange(1);
-    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/exec/kudu-table-sink-test.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-table-sink-test.cc b/be/src/exec/kudu-table-sink-test.cc
index 96aec55..a1cbb68 100644
--- a/be/src/exec/kudu-table-sink-test.cc
+++ b/be/src/exec/kudu-table-sink-test.cc
@@ -49,8 +49,7 @@ static const int THIRD_SLOT_ID = 3;
 
 class KuduTableSinkTest : public testing::Test {
  public:
-  KuduTableSinkTest()
-      : runtime_state_(TExecPlanFragmentParams(), "", &exec_env_) {}
+  KuduTableSinkTest() : runtime_state_(TExecPlanFragmentParams(), &exec_env_) {}
 
   virtual void SetUp() {
     // Create a Kudu client and the table (this will abort the test here

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/exprs/expr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/expr-test.cc b/be/src/exprs/expr-test.cc
index 8a19603..8064de0 100644
--- a/be/src/exprs/expr-test.cc
+++ b/be/src/exprs/expr-test.cc
@@ -1025,7 +1025,7 @@ template <typename T> void TestSingleLiteralConstruction(
     const ColumnType& type, const T& value, const string& string_val) {
   ObjectPool pool;
   RowDescriptor desc;
-  RuntimeState state(TExecPlanFragmentParams(), "", NULL);
+  RuntimeState state(TExecPlanFragmentParams(), NULL);
   MemTracker tracker;
 
   Expr* expr = pool.Add(new Literal(type, value));
@@ -1041,7 +1041,7 @@ TEST_F(ExprTest, NullLiteral) {
   for (int type = TYPE_BOOLEAN; type != TYPE_DATE; ++type) {
     NullLiteral expr(static_cast<PrimitiveType>(type));
     ExprContext ctx(&expr);
-    RuntimeState state(TExecPlanFragmentParams(), "", NULL);
+    RuntimeState state(TExecPlanFragmentParams(), NULL);
     MemTracker tracker;
     EXPECT_OK(ctx.Prepare(&state, RowDescriptor(), &tracker));
     EXPECT_OK(ctx.Open(&state));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/resourcebroker/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/resourcebroker/CMakeLists.txt b/be/src/resourcebroker/CMakeLists.txt
deleted file mode 100644
index 776152c..0000000
--- a/be/src/resourcebroker/CMakeLists.txt
+++ /dev/null
@@ -1,31 +0,0 @@
-# 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.
-
-
-# where to put generated libraries
-set(LIBRARY_OUTPUT_PATH "${BUILD_OUTPUT_ROOT_DIRECTORY}/resourcebroker")
-
-# where to put generated binaries
-set(EXECUTABLE_OUTPUT_PATH "${BUILD_OUTPUT_ROOT_DIRECTORY}/resourcebroker")
-
-add_library(ResourceBroker
-  resource-broker.cc
-)
-add_dependencies(ResourceBroker thrift-deps)
-
-# TODO: Add resource broker BE test
-# ADD_BE_TEST(resource-broker-test)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/resourcebroker/resource-broker.cc
----------------------------------------------------------------------
diff --git a/be/src/resourcebroker/resource-broker.cc b/be/src/resourcebroker/resource-broker.cc
deleted file mode 100644
index 4690c59..0000000
--- a/be/src/resourcebroker/resource-broker.cc
+++ /dev/null
@@ -1,850 +0,0 @@
-// 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.
-
-#include "resourcebroker/resource-broker.h"
-
-#include <boost/algorithm/string/join.hpp>
-#include <boost/uuid/uuid.hpp>
-#include <boost/uuid/uuid_generators.hpp>
-#include <boost/uuid/uuid_io.hpp>
-#include <boost/lexical_cast.hpp>
-#include <gutil/strings/substitute.h>
-#include <thrift/Thrift.h>
-
-#include "common/status.h"
-#include "rpc/thrift-util.h"
-#include "rpc/thrift-server.h"
-#include "scheduling/query-resource-mgr.h"
-#include "scheduling/scheduler.h"
-#include "util/debug-util.h"
-#include "util/uid-util.h"
-#include "util/network-util.h"
-#include "util/llama-util.h"
-#include "util/time.h"
-#include "gen-cpp/ResourceBrokerService.h"
-#include "gen-cpp/Llama_types.h"
-
-#include "common/names.h"
-
-using boost::algorithm::join;
-using boost::algorithm::to_lower;
-using boost::uuids::random_generator;
-using namespace ::apache::thrift::server;
-using namespace ::apache::thrift;
-using namespace impala;
-using namespace strings;
-
-DECLARE_int64(llama_registration_timeout_secs);
-DECLARE_int64(llama_registration_wait_secs);
-DECLARE_int64(llama_max_request_attempts);
-
-DECLARE_int32(resource_broker_cnxn_attempts);
-DECLARE_int32(resource_broker_cnxn_retry_interval_ms);
-DECLARE_int32(resource_broker_send_timeout);
-DECLARE_int32(resource_broker_recv_timeout);
-
-static const string LLAMA_KERBEROS_SERVICE_NAME = "llama";
-
-namespace impala {
-
-// String to search for in Llama error messages to detect that Llama has restarted,
-// and hence the resource broker must re-register.
-const string LLAMA_RESTART_SEARCH_STRING = "unknown handle";
-
-class LlamaNotificationThriftIf : public llama::LlamaNotificationServiceIf {
- public:
-  LlamaNotificationThriftIf(ResourceBroker* resource_broker)
-    : resource_broker_(resource_broker) {}
-
-  virtual void AMNotification(llama::TLlamaAMNotificationResponse& response,
-      const llama::TLlamaAMNotificationRequest& request) {
-    resource_broker_->AMNotification(request, response);
-  }
-
-  virtual void NMNotification(llama::TLlamaNMNotificationResponse& response,
-      const llama::TLlamaNMNotificationRequest& request) {
-    LOG(WARNING) << "Ignoring node-manager notification. Handling not yet implemented.";
-    response.status.__set_status_code(llama::TStatusCode::OK);
-  }
-
-  virtual ~LlamaNotificationThriftIf() {}
-
- private:
-  ResourceBroker* resource_broker_;
-};
-
-ResourceBroker::ResourceBroker(const vector<TNetworkAddress>& llama_addresses,
-    const TNetworkAddress& llama_callback_address, MetricGroup* metrics) :
-    llama_addresses_(llama_addresses),
-    active_llama_addr_idx_(-1),
-    llama_callback_address_(llama_callback_address),
-    metrics_(metrics),
-    scheduler_(NULL),
-    llama_callback_thrift_iface_(new LlamaNotificationThriftIf(this)),
-    llama_client_cache_(new ClientCache<llama::LlamaAMServiceClient>(
-        FLAGS_resource_broker_cnxn_attempts,
-        FLAGS_resource_broker_cnxn_retry_interval_ms,
-        FLAGS_resource_broker_send_timeout,
-        FLAGS_resource_broker_recv_timeout,
-        LLAMA_KERBEROS_SERVICE_NAME)) {
-  DCHECK(metrics != NULL);
-  llama_client_cache_->InitMetrics(metrics, "resource-broker");
-  active_llama_metric_ = metrics->AddProperty<string>(
-      "resource-broker.active-llama", "none");
-  active_llama_handle_metric_ = metrics->AddProperty<string>(
-      "resource-broker.active-llama-handle", "none");
-
-  reservation_rpc_time_metric_ = StatsMetric<double>::CreateAndRegister(metrics,
-      "resource-broker.reservation-request-rpc-time");
-  reservation_response_time_metric_ = StatsMetric<double>::CreateAndRegister(metrics,
-      "resource-broker.reservation-request-response-time");
-  reservation_requests_total_metric_ = metrics->AddCounter<int64_t>(
-      "resource-broker.reservation-requests-total", 0);
-  reservation_requests_fulfilled_metric_ = metrics->AddCounter<int64_t>(
-      "resource-broker.reservation-requests-fulfilled", 0);
-  reservation_requests_failed_metric_ = metrics->AddCounter<int64_t>(
-      "resource-broker.reservation-requests-failed", 0);
-  reservation_requests_rejected_metric_ = metrics->AddCounter<int64_t>(
-      "resource-broker.reservation-requests-rejected", 0);
-  reservation_requests_timedout_metric_ = metrics->AddCounter<int64_t>(
-      "resource-broker.reservation-requests-timedout", 0);
-
-  expansion_rpc_time_metric_ = StatsMetric<double>::CreateAndRegister(metrics,
-      "resource-broker.expansion-request-rpc-time");
-  expansion_response_time_metric_ = StatsMetric<double>::CreateAndRegister(metrics,
-      "resource-broker.expansion-request-response-time");
-  expansion_requests_total_metric_ = metrics->AddCounter<int64_t>(
-      "resource-broker.expansion-requests-total", 0);
-  expansion_requests_fulfilled_metric_ = metrics->AddCounter<int64_t>(
-      "resource-broker.expansion-requests-fulfilled", 0);
-  expansion_requests_failed_metric_ = metrics->AddCounter<int64_t>(
-      "resource-broker.expansion-requests-failed", 0);
-  expansion_requests_rejected_metric_ = metrics->AddCounter<int64_t>(
-      "resource-broker.expansion-requests-rejected", 0);
-  expansion_requests_timedout_metric_ = metrics->AddCounter<int64_t>(
-      "resource-broker.expansion-requests-timedout", 0);
-
-  requests_released_metric_ = metrics->AddCounter<int64_t>(
-      "resource-broker.requests-released", 0);
-  allocated_memory_metric_ = metrics->AddGauge<uint64_t>(
-      "resource-broker.memory-resources-in-use", 0L);
-  allocated_vcpus_metric_ = metrics->AddGauge<uint64_t>(
-      "resource-broker.vcpu-resources-in-use", 0);
-}
-
-Status ResourceBroker::Init() {
-  // The scheduler must have been set before calling Init().
-  DCHECK(scheduler_ != NULL);
-  DCHECK(llama_callback_thrift_iface_ != NULL);
-  if (llama_addresses_.size() == 0) {
-    return Status("No Llama addresses configured (see --llama_addresses)");
-  }
-
-  boost::shared_ptr<TProcessor> llama_callback_proc(
-      new llama::LlamaNotificationServiceProcessor(llama_callback_thrift_iface_));
-  llama_callback_server_.reset(new ThriftServer("llama-callback", llama_callback_proc,
-      llama_callback_address_.port, NULL, metrics_, 5));
-  RETURN_IF_ERROR(llama_callback_server_->Start());
-
-  // Generate client id for registration with Llama, and register with LLama.
-  random_generator uuid_generator;
-  llama_client_id_ = uuid_generator();
-  RETURN_IF_ERROR(RegisterWithLlama());
-  RETURN_IF_ERROR(RefreshLlamaNodes());
-  return Status::OK();
-}
-
-Status ResourceBroker::RegisterWithLlama() {
-  // Remember the current llama_handle_ to detect if another thread has already
-  // completed the registration successfully.
-  llama::TUniqueId current_llama_handle = llama_handle_;
-
-  // Start time that this thread attempted registration. Used to limit the time that a
-  // query will wait for re-registration with the Llama to succeed.
-  int64_t start = MonotonicSeconds();
-  lock_guard<mutex> l(llama_registration_lock_);
-  if (llama_handle_ != current_llama_handle) return Status::OK();
-
-  active_llama_metric_->set_value("none");
-  active_llama_handle_metric_->set_value("none");
-
-  int llama_addr_idx = (active_llama_addr_idx_ + 1) % llama_addresses_.size();
-  int64_t now = MonotonicSeconds();
-  while (FLAGS_llama_registration_timeout_secs == -1 ||
-      (now - start) < FLAGS_llama_registration_timeout_secs) {
-    // Connect to the Llama at llama_address.
-    const TNetworkAddress& llama_address = llama_addresses_[llama_addr_idx];
-    // Client status will be ok if a Thrift connection could be successfully established
-    // for the returned client at some point in the past. Hence, there is no guarantee
-    // that the connection is still valid now and we must check for broken pipes, etc.
-    Status client_status;
-    ClientConnection<llama::LlamaAMServiceClient> llama_client(llama_client_cache_.get(),
-        llama_address, &client_status);
-    if (client_status.ok()) {
-      // Register this resource broker with Llama.
-      llama::TLlamaAMRegisterRequest request;
-      request.__set_version(llama::TLlamaServiceVersion::V1);
-      llama::TUniqueId llama_uuid;
-      UUIDToTUniqueId(llama_client_id_, &llama_uuid);
-      request.__set_client_id(llama_uuid);
-
-      llama::TNetworkAddress callback_address;
-      callback_address << llama_callback_address_;
-      request.__set_notification_callback_service(callback_address);
-      llama::TLlamaAMRegisterResponse response;
-      LOG(INFO) << "Registering Resource Broker with Llama at " << llama_address;
-      Status rpc_status =
-          llama_client.DoRpc(&llama::LlamaAMServiceClient::Register, request, &response);
-      if (rpc_status.ok()) {
-        // TODO: Is there a period where an inactive Llama may respond to RPCs?
-        // If so, then we need to keep cycling through Llamas here and not
-        // return an error.
-        RETURN_IF_ERROR(LlamaStatusToImpalaStatus(
-            response.status, "Failed to register Resource Broker with Llama."));
-        LOG(INFO) << "Received Llama client handle " << response.am_handle
-                  << ((response.am_handle == llama_handle_) ? " (same as old)" : "");
-        llama_handle_ = response.am_handle;
-        break;
-      }
-    }
-    // Cycle through the list of Llama addresses for Llama failover.
-    llama_addr_idx = (llama_addr_idx + 1) % llama_addresses_.size();
-    LOG(INFO) << "Failed to connect to Llama at " << llama_address << "." << endl
-              << "Error: " << client_status.GetDetail() << endl
-              << "Retrying to connect to Llama at "
-              << llama_addresses_[llama_addr_idx] << " in "
-              << FLAGS_llama_registration_wait_secs << "s.";
-    // Sleep to give Llama time to recover/failover before the next attempt.
-    SleepForMs(FLAGS_llama_registration_wait_secs * 1000);
-    now = MonotonicSeconds();
-  }
-  DCHECK(FLAGS_llama_registration_timeout_secs != -1);
-  if ((now - start) >= FLAGS_llama_registration_timeout_secs) {
-    return Status("Failed to (re-)register Resource Broker with Llama.");
-  }
-
-  if (llama_addr_idx != active_llama_addr_idx_) {
-    // TODO: We've switched to a different Llama (failover). Cancel all queries
-    // coordinated by this Impalad to free up physical resources that are not
-    // accounted for anymore by Yarn.
-  }
-
-  // If we reached this point, (re-)registration was successful.
-  active_llama_addr_idx_ = llama_addr_idx;
-  active_llama_metric_->set_value(lexical_cast<string>(llama_addresses_[llama_addr_idx]));
-  active_llama_handle_metric_->set_value(lexical_cast<string>(llama_handle_));
-  return Status::OK();
-}
-
-bool ResourceBroker::LlamaHasRestarted(const llama::TStatus& status) const {
-  if (status.status_code == llama::TStatusCode::OK || !status.__isset.error_msgs) {
-    return false;
-  }
-  // Check whether one of the error messages contains LLAMA_RESTART_SEARCH_STRING.
-  for (int i = 0; i < status.error_msgs.size(); ++i) {
-    string error_msg = status.error_msgs[i];
-    to_lower(error_msg);
-    if (error_msg.find(LLAMA_RESTART_SEARCH_STRING) != string::npos) {
-      LOG(INFO) << "Assuming Llama restart from error message: " << status.error_msgs[i];
-      return true;
-    }
-  }
-  return false;
-}
-
-void ResourceBroker::Close() {
-  // Close connections to all Llama addresses, not just the active one.
-  for (const TNetworkAddress& llama_address: llama_addresses_) {
-    llama_client_cache_->CloseConnections(llama_address);
-  }
-  llama_callback_server_->Join();
-}
-
-void ResourceBroker::CreateLlamaReservationRequest(
-    const TResourceBrokerReservationRequest& src,
-    llama::TLlamaAMReservationRequest& dest) {
-  dest.version = llama::TLlamaServiceVersion::V1;
-  dest.am_handle = llama_handle_;
-  dest.gang = src.gang;
-  // Queue is optional, so must be explicitly set for all versions of Thrift to work
-  // together.
-  dest.__set_queue(src.queue);
-  dest.user = src.user;
-  dest.resources = src.resources;
-  random_generator uuid_generator;
-  llama::TUniqueId request_id;
-  UUIDToTUniqueId(uuid_generator(), &request_id);
-  dest.__set_reservation_id(request_id);
-}
-
-template <class F, typename LlamaReqType, typename LlamaRespType>
-Status ResourceBroker::LlamaRpc(const F& f, LlamaReqType* request,
-    LlamaRespType* response, StatsMetric<double>* rpc_time_metric) {
-  int attempts = 0;
-  MonotonicStopWatch sw;
-  // Indicates whether to re-register with Llama before the next RPC attempt,
-  // e.g. because Llama has restarted or become unavailable.
-  bool register_with_llama = false;
-  while (attempts < FLAGS_llama_max_request_attempts) {
-    if (register_with_llama) {
-      RETURN_IF_ERROR(ReRegisterWithLlama(*request, response));
-      // Set the new Llama handle received from re-registering.
-      request->__set_am_handle(llama_handle_);
-      VLOG_RPC << "Retrying Llama RPC after re-registration: " << *request;
-      register_with_llama = false;
-    }
-    ++attempts;
-    Status rpc_status;
-    ClientConnection<llama::LlamaAMServiceClient> llama_client(llama_client_cache_.get(),
-        llama_addresses_[active_llama_addr_idx_], &rpc_status);
-    if (!rpc_status.ok()) {
-      register_with_llama = true;
-      continue;
-    }
-
-    sw.Start();
-    Status status = llama_client.DoRpc(f, *request, response);
-    if (!status.ok()) {
-      VLOG_RPC << "Error making Llama RPC: " << status.GetDetail();
-      register_with_llama = status.code() == TErrorCode::RPC_CLIENT_CONNECT_FAILURE;
-      continue;
-    }
-    if (rpc_time_metric != NULL) {
-      rpc_time_metric->Update(sw.ElapsedTime() / (1000.0 * 1000.0 * 1000.0));
-    }
-
-    // Check whether Llama has been restarted. If so, re-register with it.
-    // Break out of the loop here upon success of the RPC.
-    if (!LlamaHasRestarted(response->status)) break;
-    register_with_llama = true;
-  }
-  if (attempts >= FLAGS_llama_max_request_attempts) {
-    return Status(Substitute(
-        "Request aborted after $0 attempts due to connectivity issues with Llama.",
-        FLAGS_llama_max_request_attempts));
-  }
-  return Status::OK();
-}
-
-template <typename LlamaReqType, typename LlamaRespType>
-Status ResourceBroker::ReRegisterWithLlama(const LlamaReqType& request,
-    LlamaRespType* response) {
-  RETURN_IF_ERROR(RegisterWithLlama());
-  return RefreshLlamaNodes();
-}
-
-template <>
-Status ResourceBroker::ReRegisterWithLlama(const llama::TLlamaAMGetNodesRequest& request,
-    llama::TLlamaAMGetNodesResponse* response) {
-  return RegisterWithLlama();
-}
-
-void ResourceBroker::PendingRequest::GetResources(ResourceMap* resources) {
-  resources->clear();
-  for (const llama::TAllocatedResource& resource: allocated_resources_) {
-    TNetworkAddress host = MakeNetworkAddress(resource.location);
-    (*resources)[host] = resource;
-    VLOG_QUERY << "Getting allocated resource for reservation id "
-               << reservation_id_ << " and location " << host;
-  }
-}
-
-void ResourceBroker::PendingRequest::SetResources(
-    const vector<llama::TAllocatedResource>& resources) {
-  // TODO: Llama returns a dump of all resources that we need to manually group by
-  // reservation id. Can Llama do the grouping for us?
-  for (const llama::TAllocatedResource& resource: resources) {
-    // Ignore resources that don't belong to the given reservation id.
-    if (resource.reservation_id == request_id()) {
-      allocated_resources_.push_back(resource);
-    }
-  }
-}
-
-bool ResourceBroker::WaitForNotification(int64_t timeout, ResourceMap* resources,
-    bool* timed_out, PendingRequest* pending_request) {
-  bool request_granted;
-  if (timeout <= 0) {
-    *timed_out = false;
-    request_granted = pending_request->promise()->Get();
-  } else {
-    request_granted = pending_request->promise()->Get(timeout, timed_out);
-  }
-
-  // Remove the promise from the pending-requests map.
-  const llama::TUniqueId& res_id = pending_request->reservation_id();
-  {
-    lock_guard<mutex> l(pending_requests_lock_);
-    pending_requests_.erase(pending_request->request_id());
-    if (pending_request->is_expansion()) {
-      PendingExpansionIdsMap::iterator it = pending_expansion_ids_.find(res_id);
-      if (it == pending_expansion_ids_.end()) {
-        // If the AMNotification was received as the reservation was being cleaned up,
-        // it's possible that the pending/allocated request structures were updated
-        // before this thread was able to acquire the lock.
-        VLOG_RPC << "Didn't find reservation=" << res_id << " in pending requests";
-        return false;
-      }
-      it->second.erase(pending_request->request_id());
-    }
-  }
-
-  if (request_granted && !*timed_out) {
-    pending_request->GetResources(resources);
-    int64_t total_memory_mb = 0L;
-    int32_t total_vcpus = 0;
-    for (const ResourceMap::value_type& resource: *resources) {
-      total_memory_mb += resource.second.memory_mb;
-      total_vcpus += resource.second.v_cpu_cores;
-    }
-    {
-      lock_guard<mutex> l(allocated_requests_lock_);
-      AllocatedRequestMap::iterator it = allocated_requests_.find(res_id);
-      if (it == allocated_requests_.end()) {
-        // The reservation may have already been cleaned up. See above.
-        VLOG_RPC << "Didn't find reservation=" << res_id << " in allocated requests";
-        return false;
-      }
-      it->second.push_back(AllocatedRequest(res_id, total_memory_mb, total_vcpus,
-          pending_request->is_expansion()));
-      allocated_memory_metric_->Increment(total_memory_mb * 1024L * 1024L);
-      allocated_vcpus_metric_->Increment(total_vcpus);
-    }
-  }
-
-  return request_granted;
-}
-
-Status ResourceBroker::Expand(const TUniqueId& reservation_id,
-    const llama::TResource& resource, int64_t timeout_ms, llama::TUniqueId* expansion_id,
-    llama::TAllocatedResource* allocated_resource) {
-  llama::TLlamaAMReservationExpansionRequest ll_request;
-  llama::TLlamaAMReservationExpansionResponse ll_response;
-
-  ll_request.version = llama::TLlamaServiceVersion::V1;
-  ll_request.am_handle = llama_handle_;
-  ll_request.expansion_of << reservation_id;
-  random_generator uuid_generator;
-  llama::TUniqueId request_id;
-  UUIDToTUniqueId(uuid_generator(), &request_id);
-  ll_request.__set_expansion_id(request_id);
-  ll_request.resource = resource;
-  VLOG_RPC << "Sending expansion request for reservation_id=" << reservation_id
-           << " expansion_id=" << request_id
-           << " resource=" << resource;
-
-  PendingRequest* pending_request;
-  {
-    lock_guard<mutex> l(pending_requests_lock_);
-    PendingExpansionIdsMap::iterator it =
-        pending_expansion_ids_.find(ll_request.expansion_of);
-    // If pending_expansion_ids_ doesn't contain the reservation id then the
-    // QueryResourceMgr has already been unregistered and the reservation has been
-    // released.
-    if (it == pending_expansion_ids_.end()) {
-      return Status(Substitute("Resource expansion request (expansion id=$0, "
-          "reservation id=$1) made after reservation released.",
-          PrintId(ll_request.expansion_id), PrintId(reservation_id)));
-    }
-    it->second.insert(request_id);
-    pending_request = new PendingRequest(ll_request.expansion_of, request_id, true);
-    pending_requests_.insert(make_pair(request_id, pending_request));
-  }
-
-  MonotonicStopWatch sw;
-  sw.Start();
-  Status status = LlamaRpc(&llama::LlamaAMServiceClient::Expand, &ll_request,
-      &ll_response, expansion_rpc_time_metric_);
-  // Check the status of the response.
-  if (!status.ok()) {
-    expansion_requests_failed_metric_->Increment(1);
-    return status;
-  }
-
-  Status request_status = LlamaStatusToImpalaStatus(ll_response.status);
-  if (!request_status.ok()) {
-    expansion_requests_failed_metric_->Increment(1);
-    return request_status;
-  }
-
-  ResourceMap allocated_resources;
-  bool timed_out = false;
-  bool request_granted = WaitForNotification(timeout_ms,
-      &allocated_resources, &timed_out, pending_request);
-
-  if (timed_out) {
-    expansion_requests_timedout_metric_->Increment(1);
-    Status release_status = ReleaseRequest(request_id);
-    if (!release_status.ok()) {
-      VLOG_QUERY << "Error releasing timed out expansion request, expansion_id="
-                 << request_id << " status: " << release_status.GetDetail();
-    }
-    return Status(Substitute("Resource expansion request (expansion id=$0, "
-        "reservation id=$1) exceeded timeout of $2.",
-        PrintId(ll_request.expansion_id),
-        PrintId(reservation_id),
-        PrettyPrinter::Print(timeout_ms * 1000L * 1000L, TUnit::TIME_NS)));
-  }
-  expansion_response_time_metric_->Update(
-      sw.ElapsedTime() / (1000.0 * 1000.0 * 1000.0));
-
-  if (!request_granted) {
-    if (pending_request->is_cancelled()) {
-      return Status(Substitute("Resource expansion request (expansion id=$0, "
-          "reservation id=$1) was cancelled.", PrintId(ll_request.expansion_id),
-          PrintId(reservation_id)));
-    }
-    expansion_requests_rejected_metric_->Increment(1);
-    return Status(Substitute("Resource expansion request (expansion id=$0, "
-        "reservation id=$1) was rejected.", PrintId(ll_request.expansion_id),
-        PrintId(reservation_id)));
-  }
-
-  DCHECK_EQ(allocated_resources.size(), 1);
-  *allocated_resource = allocated_resources.begin()->second;
-  *expansion_id = request_id;
-
-  VLOG_QUERY << "Fulfilled expansion for id=" << ll_response.expansion_id
-             << " resource=" << *allocated_resource;
-  expansion_requests_fulfilled_metric_->Increment(1);
-  return Status::OK();
-}
-
-Status ResourceBroker::Reserve(const TResourceBrokerReservationRequest& request,
-    TResourceBrokerReservationResponse* response) {
-  VLOG_QUERY << "Sending reservation request: " << request;
-  reservation_requests_total_metric_->Increment(1);
-
-  llama::TLlamaAMReservationRequest ll_request;
-  llama::TLlamaAMReservationResponse ll_response;
-  CreateLlamaReservationRequest(request, ll_request);
-  const llama::TUniqueId& res_id = ll_request.reservation_id;
-
-  PendingRequest* pending_request;
-  {
-    pending_request = new PendingRequest(res_id, res_id, false);
-    lock_guard<mutex> l(pending_requests_lock_);
-    pending_requests_.insert(make_pair(pending_request->request_id(), pending_request));
-  }
-  {
-    lock_guard<mutex> l(allocated_requests_lock_);
-    DCHECK(allocated_requests_.find(res_id) == allocated_requests_.end());
-    allocated_requests_[res_id] = vector<AllocatedRequest>();
-  }
-
-  MonotonicStopWatch sw;
-  sw.Start();
-  Status status = LlamaRpc(&llama::LlamaAMServiceClient::Reserve, &ll_request,
-      &ll_response, reservation_rpc_time_metric_);
-  // Check the status of the response.
-  if (!status.ok()) {
-    reservation_requests_failed_metric_->Increment(1);
-    return status;
-  }
-  Status request_status = LlamaStatusToImpalaStatus(ll_response.status);
-  if (!request_status.ok()) {
-    reservation_requests_failed_metric_->Increment(1);
-    return request_status;
-  }
-  VLOG_RPC << "Received reservation response, waiting for notification on: " << res_id;
-
-  bool timed_out = false;
-  bool request_granted = WaitForNotification(request.request_timeout,
-      &response->allocated_resources, &timed_out, pending_request);
-
-  if (request_granted || timed_out) {
-    // Set the reservation_id to make sure it eventually gets released - even if when
-    // timed out, since the response may arrive later.
-    response->__set_reservation_id(CastTUniqueId<llama::TUniqueId, TUniqueId>(res_id));
-  }
-
-  if (timed_out) {
-    reservation_requests_timedout_metric_->Increment(1);
-    return Status(Substitute(
-        "Resource reservation request (id=$0) exceeded timeout of $1.",
-        PrintId(res_id),
-        PrettyPrinter::Print(request.request_timeout * 1000L * 1000L,
-        TUnit::TIME_NS)));
-  }
-  reservation_response_time_metric_->Update(
-      sw.ElapsedTime() / (1000.0 * 1000.0 * 1000.0));
-
-  if (!request_granted) {
-    reservation_requests_rejected_metric_->Increment(1);
-    return Status(Substitute("Resource reservation request (id=$0) was rejected.",
-        PrintId(res_id)));
-  }
-
-  response->__set_reservation_id(CastTUniqueId<llama::TUniqueId, TUniqueId>(res_id));
-  VLOG_QUERY << "Fulfilled reservation with id: " << res_id;
-  reservation_requests_fulfilled_metric_->Increment(1);
-  return Status::OK();
-}
-
-void ResourceBroker::ClearRequests(const TUniqueId& reservation_id) {
-  int64_t total_memory_bytes = 0L;
-  int32_t total_vcpus = 0L;
-  llama::TUniqueId llama_id = CastTUniqueId<TUniqueId, llama::TUniqueId>(reservation_id);
-  {
-    lock_guard<mutex> l(pending_requests_lock_);
-    PendingExpansionIdsMap::iterator it = pending_expansion_ids_.find(llama_id);
-    if (it != pending_expansion_ids_.end()) {
-      for (const llama::TUniqueId& id: it->second) {
-        PendingRequestMap::iterator request_it = pending_requests_.find(id);
-        DCHECK(request_it != pending_requests_.end());
-        if (request_it == pending_requests_.end()) continue;
-        // It is possible that the AMNotification thread set the promise and the thread
-        // waiting on the promise hasn't had a chance to acquire the
-        // pending_requests_lock_ yet to remove it from pending_requests_. We don't need
-        // to do anything because it will be released with the reservation anyway.
-        if (request_it->second->promise()->IsSet()) continue;
-        request_it->second->SetCancelled();
-        request_it->second->promise()->Set(false);
-      }
-      it->second.clear();
-      pending_expansion_ids_.erase(it);
-    }
-  }
-  {
-    lock_guard<mutex> l(allocated_requests_lock_);
-    AllocatedRequestMap::iterator it = allocated_requests_.find(llama_id);
-    if (it == allocated_requests_.end()) return;
-    for (AllocatedRequest& allocated_req: it->second) {
-      DCHECK(allocated_req.reservation_id() == llama_id);
-      total_memory_bytes += (allocated_req.memory_mb() * 1024L * 1024L);
-      total_vcpus += allocated_req.vcpus();
-    }
-    it->second.clear();
-    allocated_requests_.erase(it);
-    allocated_memory_metric_->Increment(-total_memory_bytes);
-    allocated_vcpus_metric_->Increment(-total_vcpus);
-  }
-
-  VLOG_QUERY << "Releasing "
-             << PrettyPrinter::Print(total_memory_bytes, TUnit::BYTES)
-             << " and " << total_vcpus << " cores for " << llama_id;
-}
-
-Status ResourceBroker::ReleaseRequest(const llama::TUniqueId& request_id) {
-  llama::TLlamaAMReleaseRequest llama_request;
-  llama::TLlamaAMReleaseResponse llama_response;
-  llama_request.version = llama::TLlamaServiceVersion::V1;
-  llama_request.am_handle = llama_handle_;
-  llama_request.reservation_id = request_id;
-
-  RETURN_IF_ERROR(LlamaRpc(&llama::LlamaAMServiceClient::Release,
-          &llama_request, &llama_response,reservation_rpc_time_metric_));
-  RETURN_IF_ERROR(LlamaStatusToImpalaStatus(llama_response.status));
-  return Status::OK();
-}
-
-Status ResourceBroker::ReleaseReservation(const impala::TUniqueId& reservation_id) {
-  VLOG_QUERY << "Releasing all resources for reservation: " << reservation_id;
-  llama::TUniqueId llama_id = CastTUniqueId<TUniqueId, llama::TUniqueId>(reservation_id);
-
-  ClearRequests(reservation_id);
-  RETURN_IF_ERROR(ReleaseRequest(llama_id));
-  requests_released_metric_->Increment(1);
-  return Status::OK();
-}
-
-void ResourceBroker::AMNotification(const llama::TLlamaAMNotificationRequest& request,
-    llama::TLlamaAMNotificationResponse& response) {
-  {
-    // This Impalad may have restarted, so it is possible Llama is sending notifications
-    // while this Impalad is registering with Llama.
-    lock_guard<mutex> l(llama_registration_lock_);
-    if (request.am_handle != llama_handle_) {
-      VLOG_QUERY << "Ignoring Llama AM notification with mismatched AM handle. "
-                 << "Known handle: " << llama_handle_ << ". Received handle: "
-                 << request.am_handle;
-      // Ignore all notifications with mismatched handles.
-      return;
-    }
-  }
-  // Nothing to be done for heartbeats.
-  if (request.heartbeat) return;
-  VLOG_QUERY << "Received non-heartbeat AM notification";
-
-  lock_guard<mutex> l(pending_requests_lock_);
-
-  // Process granted allocations.
-  for (const llama::TUniqueId& res_id: request.allocated_reservation_ids) {
-    PendingRequestMap::iterator it = pending_requests_.find(res_id);
-    if (it == pending_requests_.end()) {
-      VLOG_RPC << "Allocation for " << res_id << " arrived after timeout or cleanup";
-      continue;
-    }
-    if (it->second->promise()->IsSet()) {
-      // The promise should not have been set unless it was already cancelled.
-      DCHECK(it->second->is_cancelled());
-      continue;
-    }
-    LOG(INFO) << "Received allocated resource for reservation id: " << res_id;
-    it->second->SetResources(request.allocated_resources);
-    it->second->promise()->Set(true);
-  }
-
-  // Process rejected allocations.
-  for (const llama::TUniqueId& res_id: request.rejected_reservation_ids) {
-    PendingRequestMap::iterator it = pending_requests_.find(res_id);
-    if (it == pending_requests_.end()) {
-      VLOG_RPC << "Rejection for " << res_id << " arrived after timeout";
-      continue;
-    }
-    if (it->second->promise()->IsSet()) {
-      DCHECK(it->second->is_cancelled());
-      continue;
-    }
-    it->second->promise()->Set(false);
-  }
-
-  // TODO: We maybe want a thread pool for handling preemptions to avoid
-  // blocking this function on query cancellations.
-  // Process preempted reservations.
-  for (const llama::TUniqueId& res_id: request.preempted_reservation_ids) {
-    TUniqueId impala_res_id;
-    impala_res_id << res_id;
-    scheduler_->HandlePreemptedReservation(impala_res_id);
-  }
-
-  // Process preempted client resources.
-  for (const llama::TUniqueId& res_id: request.preempted_client_resource_ids) {
-    TUniqueId impala_res_id;
-    impala_res_id << res_id;
-    scheduler_->HandlePreemptedResource(impala_res_id);
-  }
-
-  // Process lost client resources.
-  for (const llama::TUniqueId& res_id: request.lost_client_resource_ids) {
-    TUniqueId impala_res_id;
-    impala_res_id << res_id;
-    scheduler_->HandlePreemptedResource(impala_res_id);
-  }
-
-  response.status.__set_status_code(llama::TStatusCode::OK);
-}
-
-void ResourceBroker::NMNotification(const llama::TLlamaNMNotificationRequest& request,
-    llama::TLlamaNMNotificationResponse& response) {
-}
-
-Status ResourceBroker::RefreshLlamaNodes() {
-  llama::TLlamaAMGetNodesRequest llama_request;
-  llama_request.__set_am_handle(llama_handle_);
-  llama_request.__set_version(llama::TLlamaServiceVersion::V1);
-  llama::TLlamaAMGetNodesResponse llama_response;
-
-  RETURN_IF_ERROR(LlamaRpc(&llama::LlamaAMServiceClient::GetNodes, &llama_request,
-      &llama_response, NULL));
-  RETURN_IF_ERROR(LlamaStatusToImpalaStatus(llama_response.status));
-  llama_nodes_ = llama_response.nodes;
-  LOG(INFO) << "Llama Nodes [" << join(llama_nodes_, ", ") << "]";
-  return Status::OK();
-}
-
-bool ResourceBroker::GetQueryResourceMgr(const TUniqueId& query_id,
-    const TUniqueId& reservation_id, const TNetworkAddress& local_resource_address,
-    QueryResourceMgr** mgr) {
-  lock_guard<mutex> l(query_resource_mgrs_lock_);
-  pair<int32_t, QueryResourceMgr*>* entry = &query_resource_mgrs_[query_id];
-  if (entry->second == NULL) {
-    entry->second =
-        new QueryResourceMgr(reservation_id, local_resource_address, query_id);
-    DCHECK_EQ(entry->first, 0);
-    // Also create the per-query entries in the allocated_resources_ and
-    // pending_expansion_ids_ map.
-    llama::TUniqueId llama_id = CastTUniqueId<TUniqueId, llama::TUniqueId>(reservation_id);
-    {
-      lock_guard<mutex> pending_lock(pending_requests_lock_);
-      DCHECK(pending_expansion_ids_.find(llama_id) == pending_expansion_ids_.end());
-      pending_expansion_ids_[llama_id] = boost::unordered_set<llama::TUniqueId>();
-    }
-    {
-      lock_guard<mutex> allocated_lock(allocated_requests_lock_);
-      if (allocated_requests_.find(llama_id) == allocated_requests_.end()) {
-        allocated_requests_[llama_id] = vector<AllocatedRequest>();
-      }
-    }
-  }
-  *mgr = entry->second;
-  // Return true if this is the first reference to this resource mgr.
-  return ++entry->first == 1L;
-}
-
-void ResourceBroker::UnregisterQueryResourceMgr(const TUniqueId& query_id) {
-  lock_guard<mutex> l(query_resource_mgrs_lock_);
-  QueryResourceMgrsMap::iterator it = query_resource_mgrs_.find(query_id);
-  DCHECK(it != query_resource_mgrs_.end())
-      << "UnregisterQueryResourceMgr() without corresponding GetQueryResourceMgr()";
-  if (--it->second.first == 0) {
-    it->second.second->Shutdown();
-    ClearRequests(it->second.second->reservation_id());
-    delete it->second.second;
-    query_resource_mgrs_.erase(it);
-  }
-}
-
-ostream& operator<<(ostream& os,
-    const map<TNetworkAddress, llama::TAllocatedResource>& resources) {
-  typedef map<TNetworkAddress, llama::TAllocatedResource> ResourceMap;
-  int count = 0;
-  for (const ResourceMap::value_type& resource: resources) {
-    os << "(" << resource.first << "," << resource.second << ")";
-    if (++count != resources.size()) os << ",";
-  }
-  return os;
-}
-
-ostream& operator<<(ostream& os, const TResourceBrokerReservationRequest& request) {
-  os << "Reservation Request("
-     << "queue=" << request.queue << " "
-     << "user=" << request.user << " "
-     << "gang=" << request.gang << " "
-     << "request_timeout=" << request.request_timeout << " "
-     << "resources=[";
-  for (int i = 0; i < request.resources.size(); ++i) {
-    os << request.resources[i];
-    if (i + 1 != request.resources.size()) os << ",";
-  }
-  os << "])";
-  return os;
-}
-
-ostream& operator<<(ostream& os, const TResourceBrokerReservationResponse& reservation) {
-  os << "Granted Reservation("
-     << "reservation id=" << reservation.reservation_id << " "
-     << "resources=[" << reservation.allocated_resources << "])";
-  return os;
-}
-
-ostream& operator<<(ostream& os, const TResourceBrokerExpansionRequest& request) {
-  os << "Expansion Request("
-     << "reservation id=" << request.reservation_id << " "
-     << "resource=" << request.resource << " "
-     << "request_timeout=" << request.request_timeout << ")";
-  return os;
-}
-
-ostream& operator<<(ostream& os, const TResourceBrokerExpansionResponse& expansion) {
-  os << "Expansion Response("
-     << "reservation id=" << expansion.reservation_id << " "
-     << "resources=[" << expansion.allocated_resources << "])";
-  return os;
-}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/resourcebroker/resource-broker.h
----------------------------------------------------------------------
diff --git a/be/src/resourcebroker/resource-broker.h b/be/src/resourcebroker/resource-broker.h
deleted file mode 100644
index b9e0bd7..0000000
--- a/be/src/resourcebroker/resource-broker.h
+++ /dev/null
@@ -1,424 +0,0 @@
-// 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.
-
-#ifndef RESOURCE_BROKER_H_
-#define RESOURCE_BROKER_H_
-
-#include <boost/unordered_map.hpp>
-#include <boost/unordered_set.hpp>
-#include <boost/scoped_ptr.hpp>
-#include <boost/uuid/uuid.hpp>
-
-#include "runtime/client-cache.h"
-#include "util/collection-metrics.h"
-#include "util/promise.h"
-#include "util/stopwatch.h"
-#include "gen-cpp/LlamaAMService.h"
-#include "gen-cpp/LlamaNotificationService.h"
-#include "gen-cpp/ResourceBrokerService_types.h"
-
-namespace impala {
-
-class QueryResourceMgr;
-class Status;
-class MetricGroup;
-class Scheduler;
-class ResourceBrokerNotificationServiceClient;
-class RuntimeProfile;
-
-/// Mediates resource-reservation requests between Impala and Yarn via the Llama service.
-/// The resource broker requests resources via the Llama's thrift interface and exposes
-/// a thrift server for the Llama to notify it of granted/denied/preempted resource
-/// reservations. The reserve/release API of the resource broker is blocking.
-/// The resource broker is configured with a list of Llama addresses that
-/// are cycled through for failover.
-/// TODO: Implement NM notification service.
-class ResourceBroker {
- public:
-  ResourceBroker(const std::vector<TNetworkAddress>& llama_addresses,
-      const TNetworkAddress& llama_callback_address, MetricGroup* metrics);
-
-  /// Register this resource broker with LLama and starts the Llama callback service.
-  /// Returns a non-OK status if the callback service failed to start (e.g., port in use)
-  /// or if registration with the Llama failed (e.g., connection to Llama failed).
-  Status Init();
-
-  /// Closes the llama_client_cache_ and joins the llama_callback_server_.
-  void Close();
-
-  /// Requests resources from Llama. Blocks until the request has been granted or denied.
-  /// TODO: Remove thrift interface
-  Status Reserve(const TResourceBrokerReservationRequest& request,
-      TResourceBrokerReservationResponse* response);
-
-  /// Requests more resources specified by 'resource' from Llama for an existing
-  /// reservation specified by the 'reservation_id'. Blocks until the request has been
-  /// granted or rejected, or no response is received within the timeout specified, in
-  /// which case a call to cancel the outstanding expansion is made and the call returns
-  /// with an error status. If timeout_ms <= 0, the call will not timeout. If the
-  /// expansion is successful, an OK status is returned and the 'expansion_id' and
-  /// 'allocated_resource' are set. An error status is returned if a timeout or an error
-  /// occurs.
-  Status Expand(const TUniqueId& reservation_id, const llama::TResource& resource,
-      int64_t timeout_ms, llama::TUniqueId* expansion_id,
-      llama::TAllocatedResource* allocated_resource);
-
-  /// Removes the record of all resource requests associated with this
-  /// 'reservation_id', updating the per-node accounting of resources and cancels any
-  /// threads waiting on pending expansions. Does not communicate this to Llama, so the
-  /// coordinator should always call ReleaseReservation() to make sure that Llama knows
-  /// the resources should be released.
-  void ClearRequests(const TUniqueId& reservation_id);
-
-  /// Releases resources acquired from Llama for this reservation and all associated
-  /// expansion requests across _all_ nodes. Should therefore only be called once per
-  /// query by the coordinator.
-  Status ReleaseReservation(const TUniqueId& reservation_id);
-
-  /// Handles asynchronous Llama Application Master (AM) notifications including
-  /// granted/denied/preempted reservations and resources.
-  void AMNotification(const llama::TLlamaAMNotificationRequest& request,
-      llama::TLlamaAMNotificationResponse& response);
-
-  /// Handles asynchronous notifications from the Llama Node Manager (NM)
-  /// auxiliary service, in particular, incoming Yarn container allocations
-  /// that are going to claim resources.
-  /// TODO: Implement once NM service is fully functional.
-  void NMNotification(const llama::TLlamaNMNotificationRequest& request,
-      llama::TLlamaNMNotificationResponse& response);
-
-  const std::vector<std::string>& llama_nodes() { return llama_nodes_; }
-
-  /// Retrieves the nodes known to Llama and stores them in llama_nodes_.
-  Status RefreshLlamaNodes();
-
-  void set_scheduler(Scheduler* scheduler) { scheduler_ = scheduler; };
-
-  /// Retrieves or creates a new QueryResourceMgr for the given query ID. Returns true if
-  /// this is the first 'checkout' of this QueryResourceMgr, false otherwise. The other
-  /// parameters are passed to the QueryResourceMgr constructor.
-  bool GetQueryResourceMgr(const TUniqueId& query_id, const TUniqueId& reservation_id,
-      const TNetworkAddress& local_resource_address, QueryResourceMgr** res_mgr);
-
-  /// Decrements the reference count for a particular QueryResourceMgr. If this is the last
-  /// reference (i.e. the ref count goes to 0), the QueryResourceMgr is deleted. It's an
-  /// error to call this with a query_id that does not have a registered QueryResourceMgr.
-  void UnregisterQueryResourceMgr(const TUniqueId& query_id);
-
- private:
-  typedef std::map<TNetworkAddress, llama::TAllocatedResource> ResourceMap;
-
-  bool has_standby_llama() { return llama_addresses_.size() > 1; }
-
-  /// Registers this resource broker with the Llama. Cycles through the list of
-  /// Llama addresses to find the active Llama which is accepting requests (if any).
-  /// Returns a non-OK status if registration with any of the Llama's did not succeed
-  /// within FLAGS_llama_registration_timeout_s seconds.
-  /// Registration with the Llama is idempotent with respect to the llama_client_id_
-  /// (see comment on llama_client_id_ for details).
-  Status RegisterWithLlama();
-
-  /// Issues the Llama RPC f where F is a thrift call taking LlamaReqType and returning
-  /// LlamaRespType. If failures occur, this function handles re-registering with Llama
-  /// if necessary and re-trying multiple times. If rpc_time_metric is non-NULL, the
-  /// metric is updated upon success of the RPC. Returns a non-OK status if the RPC
-  /// failed due to connectivity issues with the Llama. Returns OK if the RPC succeeded.
-  template <class F, typename LlamaReqType, typename LlamaRespType>
-  Status LlamaRpc(const F& f, LlamaReqType* request, LlamaRespType* response,
-      StatsMetric<double>* rpc_time_metric);
-
-  /// Re-registers with Llama to recover from the Llama being unreachable. Handles both
-  /// Llama restart and failover. This function is a template to allow specialization on
-  /// the Llama request/response type.
-  template <typename LlamaReqType, typename LlamaRespType>
-  Status ReRegisterWithLlama(const LlamaReqType& request, LlamaRespType* response);
-
-  /// Detects Llama restarts from the given return status of a Llama RPC.
-  bool LlamaHasRestarted(const llama::TStatus& status) const;
-
-  /// Sends a Llama release RPC for the reservation or expansion with the specified
-  /// request_id.
-  Status ReleaseRequest(const llama::TUniqueId& request_id);
-
-  /// Creates a Llama reservation request from a resource broker reservation request.
-  void CreateLlamaReservationRequest(const TResourceBrokerReservationRequest& src,
-      llama::TLlamaAMReservationRequest& dest);
-
-  class PendingRequest;
-  /// Wait for a reservation or expansion request to be fulfilled by the Llama via an
-  /// async call into LlamaNotificationThriftIf::AMNotification(). If timeout_ms > 0, the
-  /// call will not wait longer than timeout_ms before returning false and *timed_out set
-  /// to true. If the request is fulfilled, resources and reservation_id are populated.
-  bool WaitForNotification(int64_t timeout_ms, ResourceMap* resources, bool* timed_out,
-      PendingRequest* reservation);
-
-  /// Llama availability group.
-  std::vector<TNetworkAddress> llama_addresses_;
-
-  /// Indexes into llama_addresses_ indicating the currently active Llama.
-  /// Protected by llama_registration_lock_.
-  int active_llama_addr_idx_;
-
-  /// Address of thrift server started in this resource broker to handle
-  /// Llama notifications.
-  TNetworkAddress llama_callback_address_;
-
-  MetricGroup* metrics_;
-
-  Scheduler* scheduler_;
-
-  /// Address of the active Llama. A Llama is considered active once we have successfully
-  /// registered with it. Set to "none" while registering with the Llama.
-  StringProperty* active_llama_metric_;
-
-  /// Llama handle received from the active Llama upon registration.
-  /// Set to "none" while not registered with Llama.
-  StringProperty* active_llama_handle_metric_;
-
-  /// Accumulated statistics on the time taken to RPC a reservation request and receive
-  /// an acknowledgement from Llama.
-  StatsMetric<double>* reservation_rpc_time_metric_;
-
-  /// Accumulated statistics on the time taken to complete a reservation request
-  /// (granted or denied). The time includes the request RPC to Llama and the time
-  /// the requesting thread waits on the pending_requests_'s promise.
-  /// The metric does not include requests that timed out.
-  StatsMetric<double>* reservation_response_time_metric_;
-
-  /// Total number of reservation requests.
-  IntCounter* reservation_requests_total_metric_;
-
-  /// Number of fulfilled reservation requests.
-  IntCounter* reservation_requests_fulfilled_metric_;
-
-  /// Reservation requests that failed due to a malformed request or an internal
-  /// error in Llama.
-  IntCounter* reservation_requests_failed_metric_;
-
-  /// Number of well-formed reservation requests rejected by the central scheduler.
-  IntCounter* reservation_requests_rejected_metric_;
-
-  /// Number of well-formed reservation requests that did not get fulfilled within
-  /// the timeout period.
-  IntCounter* reservation_requests_timedout_metric_;
-
-  /// Accumulated statistics on the time taken to RPC an expansion request and receive an
-  /// acknowledgement from Llama.
-  StatsMetric<double>* expansion_rpc_time_metric_;
-
-  /// Accumulated statistics on the time taken to complete an expansion request
-  /// (granted or denied). The time includes the request RPC to Llama and the time
-  /// the requesting thread waits on the pending_requests_'s promise.
-  /// The metric does not include requests that timed out.
-  StatsMetric<double>* expansion_response_time_metric_;
-
-  /// Total number of expansion requests.
-  IntCounter* expansion_requests_total_metric_;
-
-  /// Number of fulfilled expansion requests.
-  IntCounter* expansion_requests_fulfilled_metric_;
-
-  /// Expansion requests that failed due to a malformed request or an internal
-  /// error in Llama.
-  IntCounter* expansion_requests_failed_metric_;
-
-  /// Number of well-formed expansion requests rejected by the central scheduler.
-  IntCounter* expansion_requests_rejected_metric_;
-
-  /// Number of well-formed expansion requests that did not get fulfilled within
-  /// the timeout period.
-  IntCounter* expansion_requests_timedout_metric_;
-
-  /// Total amount of memory currently allocated by Llama to this node
-  UIntGauge* allocated_memory_metric_;
-
-  /// Total number of vcpu cores currently allocated by Llama to this node
-  UIntGauge* allocated_vcpus_metric_;
-
-  /// Total number of fulfilled reservation requests that have been released.
-  IntCounter* requests_released_metric_;
-
-  /// Client id used to register with Llama. Set in Init(). Used to communicate to Llama
-  /// whether this Impalad has restarted. Registration with Llama is idempotent if the
-  /// same llama_client_id_ is passed, i.e., the same Llama handle is returned and
-  /// resource allocations are preserved. From Llama's perspective an unknown
-  /// llama_client_id_ indicates a new registration and all resources allocated by this
-  /// Impalad under a different llama_client_id_ are consider lost and will be released.
-  boost::uuids::uuid llama_client_id_;
-
-  /// Thrift API implementation which proxies Llama notifications onto this ResourceBroker.
-  boost::shared_ptr<llama::LlamaNotificationServiceIf> llama_callback_thrift_iface_;
-  boost::scoped_ptr<ThriftServer> llama_callback_server_;
-
-  /// Cache of Llama client connections.
-  boost::scoped_ptr<ClientCache<llama::LlamaAMServiceClient>> llama_client_cache_;
-
-  /// Lock to ensure that only a single registration with Llama is sent, e.g.,
-  /// when multiple concurrent requests realize that Llama has restarted.
-  boost::mutex llama_registration_lock_;
-
-  /// Handle received from Llama during registration. Set in RegisterWithLlama().
-  llama::TUniqueId llama_handle_;
-
-  /// List of nodes registered with Llama. Set in RefreshLlamaNodes().
-  std::vector<std::string> llama_nodes_;
-
-  /// A PendingRequest tracks a single reservation or expansion request that is in flight
-  /// to Llama. A new PendingRequest is created in either Expand() or Reserve(), and its
-  /// promise() is blocked on there until a response is received for that request from
-  /// Llama via AMNotification(), or until a timeout occurs.
-  //
-  /// Every request has a unique request_id which is assigned by the resource broker. Each
-  /// request is also associated with exactly one reservation, via reservation_id(). This
-  /// allows us to track which resources belong to which reservation, and to make sure that
-  /// all are correctly accounted for when the reservation is released. Each reservation ID
-  /// will belong to exactly one reservation request, and 0 or more expansion requests.
-  class PendingRequest {
-   public:
-    PendingRequest(const llama::TUniqueId& reservation_id,
-        const llama::TUniqueId& request_id, bool is_expansion)
-        : reservation_id_(reservation_id), request_id_(request_id),
-          is_expansion_(is_expansion), is_cancelled_(false) {
-      DCHECK(is_expansion || reservation_id == request_id);
-    }
-
-    /// Promise is set to true if the reservation or expansion request was granted, false
-    /// if it was rejected by Yarn. When promise()->Get() returns true,
-    /// allocated_resources_ will be populated and it will be safe to call GetResources().
-    Promise<bool>* promise() { return &promise_; }
-
-    /// Called by WaitForNotification() to populate a map of resources once the
-    /// corresponding request has returned successfully (and promise() therefore has
-    /// returned true).
-    void GetResources(ResourceMap* resources);
-
-    /// Populates allocated_resources_ from all members of resources that match the given
-    /// reservation id. Called in AMNotification().
-    void SetResources(const std::vector<llama::TAllocatedResource>& resources);
-
-    const llama::TUniqueId& request_id() const { return request_id_; }
-    const llama::TUniqueId& reservation_id() const { return reservation_id_; }
-
-    bool is_expansion() const { return is_expansion_; }
-    bool is_cancelled() const { return is_cancelled_; }
-
-    /// Sets the cancelled flag to true. Is only called before the promise is set and
-    /// while the pending_requests_lock_ is held to avoid races.
-    void SetCancelled() { is_cancelled_ = true; }
-
-   private:
-    /// Promise object that WaitForNotification() waits on and AMNotification() signals.
-    Promise<bool> promise_;
-
-    /// Filled in by AMNotification(), so that WaitForNotification() can read the set of
-    /// allocated_resources without AMNotification() having to wait (hence the copy is
-    /// deliberate, since the original copy may go out of scope).
-    std::vector<llama::TAllocatedResource> allocated_resources_;
-
-    /// The ID for the reservation associated with this request. There is always exactly
-    /// one reservation associated with every request.
-    llama::TUniqueId reservation_id_;
-
-    /// The unique ID for this request. If this is a reservation request, request_id_ ==
-    /// reservation_id_, otherwise this is generated during Expand().
-    llama::TUniqueId request_id_;
-
-    /// True if this is an expansion request, false if it is a reservation request
-    bool is_expansion_;
-
-    /// Set if the request was cancelled.
-    bool is_cancelled_;
-  };
-
-  /// Protects pending_requests_ and pending_expansion_ids_
-  boost::mutex pending_requests_lock_;
-
-  /// Map from unique request ID provided to Llama (for both reservation and expansion
-  /// requests) to PendingRequest object used to coordinate when a response is received
-  /// from Llama.
-  typedef boost::unordered_map<llama::TUniqueId, PendingRequest*> PendingRequestMap;
-  PendingRequestMap pending_requests_;
-
-  /// Map from reservation IDs to pending expansion IDs. All pending request IDs have a
-  /// PendingRequest in pending_requests_.
-  typedef boost::unordered_map<llama::TUniqueId, boost::unordered_set<llama::TUniqueId>>
-      PendingExpansionIdsMap;
-  PendingExpansionIdsMap pending_expansion_ids_;
-
-  /// An AllocatedRequest tracks resources allocated in response to one reservation or
-  /// expansion request.
-  class AllocatedRequest {
-   public:
-    AllocatedRequest(const llama::TUniqueId& reservation_id,
-        uint64_t memory_mb, uint32_t vcpus, bool is_expansion)
-        : reservation_id_(reservation_id), memory_mb_(memory_mb), vcpus_(vcpus),
-          is_expansion_(is_expansion) { }
-
-    const llama::TUniqueId reservation_id() const { return reservation_id_; }
-    uint64_t memory_mb() const { return memory_mb_; }
-    uint32_t vcpus() const { return vcpus_; }
-    bool is_expansion() const { return is_expansion_; }
-
-   private:
-    /// The reservation ID for this request. Expansions all share the same reservation ID.
-    llama::TUniqueId reservation_id_;
-
-    /// The total memory allocated to this request
-    uint64_t memory_mb_;
-
-    /// The number of VCPUs allocated to this request
-    uint32_t vcpus_;
-
-    /// True if this is an expansion request, false if it is a reservation request
-    bool is_expansion_;
-  };
-
-  /// Protectes allocated_requests_
-  boost::mutex allocated_requests_lock_;
-
-  /// Map from reservation ID to all satisfied requests - reservation and expansion -
-  /// associated with that reservation. Used only for bookkeeping so that Impala can report
-  /// on the current resource usage.
-  typedef boost::unordered_map<llama::TUniqueId, std::vector<AllocatedRequest>>
-      AllocatedRequestMap;
-  AllocatedRequestMap allocated_requests_;
-
-  /// Protects query_resource_mgrs_
-  boost::mutex query_resource_mgrs_lock_;
-  typedef boost::unordered_map<TUniqueId, std::pair<int32_t, QueryResourceMgr*>>
-      QueryResourceMgrsMap;
-
-  /// Map from query ID to a (ref_count, QueryResourceMgr*) pair, i.e. one QueryResourceMgr
-  /// per query. The refererence count is always non-zero - once it hits zero the entry in
-  /// the map is removed and the QueryResourceMgr is deleted.
-  QueryResourceMgrsMap query_resource_mgrs_;
-};
-
-std::ostream& operator<<(std::ostream& os,
-    const TResourceBrokerReservationRequest& request);
-
-std::ostream& operator<<(std::ostream& os,
-    const TResourceBrokerReservationResponse& reservation);
-
-std::ostream& operator<<(std::ostream& os,
-    const TResourceBrokerExpansionRequest& request);
-
-std::ostream& operator<<(std::ostream& os,
-    const TResourceBrokerExpansionResponse& expansion);
-}
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/buffered-block-mgr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/buffered-block-mgr-test.cc b/be/src/runtime/buffered-block-mgr-test.cc
index 5b5ee8a..169baf2 100644
--- a/be/src/runtime/buffered-block-mgr-test.cc
+++ b/be/src/runtime/buffered-block-mgr-test.cc
@@ -553,8 +553,8 @@ class BufferedBlockMgrTest : public ::testing::Test {
     const int num_threads = 8;
     thread_group workers;
     // Create a shared RuntimeState with no BufferedBlockMgr.
-    RuntimeState* shared_state = new RuntimeState(TExecPlanFragmentParams(), "",
-        test_env_->exec_env());
+    RuntimeState* shared_state =
+        new RuntimeState(TExecPlanFragmentParams(), test_env_->exec_env());
     for (int i = 0; i < num_threads; ++i) {
       thread* t = new thread(bind(
           &BufferedBlockMgrTest::CreateDestroyThread, this, shared_state));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/buffered-block-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/buffered-block-mgr.cc b/be/src/runtime/buffered-block-mgr.cc
index 90c1041..d582c31 100644
--- a/be/src/runtime/buffered-block-mgr.cc
+++ b/be/src/runtime/buffered-block-mgr.cc
@@ -1317,8 +1317,8 @@ void BufferedBlockMgr::Init(DiskIoMgr* io_mgr, RuntimeProfile* parent_profile,
   integrity_check_timer_ = ADD_TIMER(profile_.get(), "TotalIntegrityCheckTime");
 
   // Create a new mem_tracker and allocate buffers.
-  mem_tracker_.reset(new MemTracker(
-      profile(), mem_limit, -1, "Block Manager", parent_tracker));
+  mem_tracker_.reset(
+      new MemTracker(profile(), mem_limit, "Block Manager", parent_tracker));
 
   initialized_ = true;
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/collection-value-builder-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/collection-value-builder-test.cc b/be/src/runtime/collection-value-builder-test.cc
index c7843b7..b8f4b65 100644
--- a/be/src/runtime/collection-value-builder-test.cc
+++ b/be/src/runtime/collection-value-builder-test.cc
@@ -40,7 +40,7 @@ TEST(CollectionValueBuilderTest, MaxBufferSize) {
   CollectionValue coll_value;
   int64_t initial_capacity = (INT_MAX / 8) + 1;
   int64_t mem_limit = initial_capacity * 4 * 4;
-  MemTracker tracker(mem_limit, mem_limit);
+  MemTracker tracker(mem_limit);
   MemPool pool(&tracker);
   CollectionValueBuilder coll_value_builder(
       &coll_value, tuple_desc, &pool, NULL, initial_capacity);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/coordinator.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator.cc b/be/src/runtime/coordinator.cc
index 4728866..e2dd1a4 100644
--- a/be/src/runtime/coordinator.cc
+++ b/be/src/runtime/coordinator.cc
@@ -62,7 +62,6 @@
 #include "util/error-util.h"
 #include "util/hdfs-bulk-ops.h"
 #include "util/hdfs-util.h"
-#include "util/llama-util.h"
 #include "util/network-util.h"
 #include "util/pretty-printer.h"
 #include "util/summary-util.h"
@@ -486,7 +485,7 @@ Status Coordinator::Exec(QuerySchedule& schedule,
         runtime_state()->obj_pool(), request.fragments[0].output_exprs,
         output_expr_ctxs));
     MemTracker* output_expr_tracker = runtime_state()->obj_pool()->Add(new MemTracker(
-        -1, -1, "Output exprs", runtime_state()->instance_mem_tracker(), false));
+        -1, "Output exprs", runtime_state()->instance_mem_tracker(), false));
     RETURN_IF_ERROR(Expr::Prepare(
         *output_expr_ctxs, runtime_state(), row_desc(), output_expr_tracker));
   } else {
@@ -503,12 +502,12 @@ Status Coordinator::Exec(QuerySchedule& schedule,
     MemTracker* pool_tracker = MemTracker::GetRequestPoolMemTracker(
         schedule.request_pool(), exec_env_->process_mem_tracker());
     query_mem_tracker_ =
-        MemTracker::GetQueryMemTracker(query_id_, query_limit, -1, pool_tracker, NULL);
+        MemTracker::GetQueryMemTracker(query_id_, query_limit, pool_tracker);
 
     executor_.reset(NULL);
   }
-  filter_mem_tracker_.reset(new MemTracker(-1, -1, "Runtime Filter (Coordinator)",
-      query_mem_tracker(), false));
+  filter_mem_tracker_.reset(
+      new MemTracker(-1, "Runtime Filter (Coordinator)", query_mem_tracker(), false));
 
   // Initialize the execution profile structures.
   InitExecProfile(request);
@@ -1900,20 +1899,6 @@ void Coordinator::SetExecPlanFragmentParams(QuerySchedule& schedule,
   SetExecPlanDescriptorTable(fragment, rpc_params);
 
   TNetworkAddress exec_host = params.hosts[fragment_instance_idx];
-  if (schedule.HasReservation()) {
-    // The reservation has already have been validated at this point.
-    TNetworkAddress resource_hostport;
-    schedule.GetResourceHostport(exec_host, &resource_hostport);
-    map<TNetworkAddress, llama::TAllocatedResource>::const_iterator it =
-        schedule.reservation()->allocated_resources.find(resource_hostport);
-    // Only set reserved resource if we actually have one for this plan
-    // fragment. Otherwise, don't set it (usually this the coordinator fragment), and it
-    // won't participate in dynamic RM controls.
-    if (it != schedule.reservation()->allocated_resources.end()) {
-      fragment_instance_ctx.__set_reserved_resource(it->second);
-      fragment_instance_ctx.__set_local_resource_address(resource_hostport);
-    }
-  }
   FragmentScanRangeAssignment::const_iterator it =
       params.scan_range_assignment.find(exec_host);
   // Scan ranges may not always be set, so use an empty structure if so.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/data-stream-recvr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/data-stream-recvr.cc b/be/src/runtime/data-stream-recvr.cc
index 9bd51af..50103ba 100644
--- a/be/src/runtime/data-stream-recvr.cc
+++ b/be/src/runtime/data-stream-recvr.cc
@@ -295,7 +295,7 @@ DataStreamRecvr::DataStreamRecvr(DataStreamMgr* stream_mgr, MemTracker* parent_t
     is_merging_(is_merging),
     num_buffered_bytes_(0),
     profile_(profile) {
-  mem_tracker_.reset(new MemTracker(-1, -1, "DataStreamRecvr", parent_tracker));
+  mem_tracker_.reset(new MemTracker(-1, "DataStreamRecvr", parent_tracker));
   // Create one queue per sender if is_merging is true.
   int num_queues = is_merging ? num_senders : 1;
   sender_queues_.reserve(num_queues);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/19de09ab/be/src/runtime/data-stream-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/data-stream-test.cc b/be/src/runtime/data-stream-test.cc
index dae4724..f5eb783 100644
--- a/be/src/runtime/data-stream-test.cc
+++ b/be/src/runtime/data-stream-test.cc
@@ -112,9 +112,7 @@ class ImpalaTestBackend : public ImpalaInternalServiceIf {
 
 class DataStreamTest : public testing::Test {
  protected:
-  DataStreamTest()
-    : runtime_state_(TExecPlanFragmentParams(), "", &exec_env_),
-      next_val_(0) {
+  DataStreamTest() : runtime_state_(TExecPlanFragmentParams(), &exec_env_), next_val_(0) {
     // Initialize Mem trackers for use by the data stream receiver.
     exec_env_.InitForFeTests();
     runtime_state_.InitMemTrackers(TUniqueId(), NULL, -1);
@@ -482,7 +480,7 @@ class DataStreamTest : public testing::Test {
 
   void Sender(int sender_num, int channel_buffer_size,
               TPartitionType::type partition_type) {
-    RuntimeState state(TExecPlanFragmentParams(), "", &exec_env_);
+    RuntimeState state(TExecPlanFragmentParams(), &exec_env_);
     state.set_desc_tbl(desc_tbl_);
     state.InitMemTrackers(TUniqueId(), NULL, -1);
     VLOG_QUERY << "create sender " << sender_num;
@@ -596,7 +594,7 @@ TEST_F(DataStreamTest, BasicTest) {
 // TODO: Make lifecycle requirements more explicit.
 TEST_F(DataStreamTest, CloseRecvrWhileReferencesRemain) {
   scoped_ptr<RuntimeState> runtime_state(
-      new RuntimeState(TExecPlanFragmentParams(), "", &exec_env_));
+      new RuntimeState(TExecPlanFragmentParams(), &exec_env_));
   runtime_state->InitMemTrackers(TUniqueId(), NULL, -1);
 
   scoped_ptr<RuntimeProfile> profile(new RuntimeProfile(&obj_pool_, "TestReceiver"));