You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2021/02/09 05:05:31 UTC

[kudu] branch master updated: [util] remove AutoReleasePool and cleanup related code

This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
     new 9592762  [util] remove AutoReleasePool and cleanup related code
9592762 is described below

commit 9592762f175c068ba6c1b8e4458871fb78153e68
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Sun Feb 7 00:06:32 2021 -0800

    [util] remove AutoReleasePool and cleanup related code
    
    This patch removes AutoReleasePool class and changes just two places
    where it was used to store heap-allocated objects.  With this patch,
    objects of appropriate type are wrapped into std::unique_ptr and
    stored in std::deque container.  Along the way, I did other minor
    code cleanup.
    
    The motivation for this change is reducing number of memory allocations
    and removing unneeded complexity in the code.
    
    Change-Id: I8adb5d08ec37e716baf524cb6c5f52366553883c
    Reviewed-on: http://gerrit.cloudera.org:8080/17036
    Tested-by: Alexey Serbin <as...@cloudera.com>
    Reviewed-by: Hao Hao <ha...@cloudera.com>
---
 src/kudu/client/client.cc              | 10 ++--
 src/kudu/client/scan_configuration.cc  | 14 ++---
 src/kudu/client/scan_configuration.h   | 18 +++----
 src/kudu/tablet/ops/alter_schema_op.cc |  3 +-
 src/kudu/tablet/ops/alter_schema_op.h  |  2 +-
 src/kudu/tablet/ops/op.cc              | 25 +++++----
 src/kudu/tablet/ops/op.h               | 24 ++++-----
 src/kudu/util/auto_release_pool.h      | 99 ----------------------------------
 8 files changed, 49 insertions(+), 146 deletions(-)

diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 18fbced..0ba3f26 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -22,6 +22,7 @@
 #include <cstdlib>
 #include <functional>
 #include <map>
+#include <memory>
 #include <mutex>
 #include <ostream>
 #include <set>
@@ -1625,12 +1626,12 @@ Status KuduScanner::SetTimeoutMillis(int millis) {
 }
 
 Status KuduScanner::AddConjunctPredicate(KuduPredicate* pred) {
+  // Take ownership even if returning non-OK status.
+  unique_ptr<KuduPredicate> p(pred);
   if (data_->open_) {
-    // Take ownership even if we return a bad status.
-    delete pred;
     return Status::IllegalState("Predicate must be set before Open()");
   }
-  return data_->mutable_configuration()->AddConjunctPredicate(pred);
+  return data_->mutable_configuration()->AddConjunctPredicate(std::move(p));
 }
 
 Status KuduScanner::AddLowerBound(const KuduPartialRow& key) {
@@ -2043,7 +2044,8 @@ Status KuduScanTokenBuilder::IncludeTabletMetadata(bool include_metadata) {
 }
 
 Status KuduScanTokenBuilder::AddConjunctPredicate(KuduPredicate* pred) {
-  return data_->mutable_configuration()->AddConjunctPredicate(pred);
+  unique_ptr<KuduPredicate> p(pred);
+  return data_->mutable_configuration()->AddConjunctPredicate(std::move(p));
 }
 
 Status KuduScanTokenBuilder::AddLowerBound(const KuduPartialRow& key) {
diff --git a/src/kudu/client/scan_configuration.cc b/src/kudu/client/scan_configuration.cc
index 7871c50..470b126 100644
--- a/src/kudu/client/scan_configuration.cc
+++ b/src/kudu/client/scan_configuration.cc
@@ -90,10 +90,11 @@ Status ScanConfiguration::SetProjectedColumnIndexes(const vector<int>& col_index
   return CreateProjection(cols);
 }
 
-Status ScanConfiguration::AddConjunctPredicate(KuduPredicate* pred) {
-  // Take ownership even if we return a bad status.
-  pool_.Add(pred);
-  return pred->data_->AddToScanSpec(&spec_, &arena_);
+Status ScanConfiguration::AddConjunctPredicate(unique_ptr<KuduPredicate> pred) {
+  // Take ownership even if returning non-OK status.
+  auto* pred_raw_ptr = pred.get();
+  predicates_pool_.emplace_back(std::move(pred));
+  return pred_raw_ptr->data_->AddToScanSpec(&spec_, &arena_);
 }
 
 void ScanConfiguration::AddConjunctPredicate(ColumnPredicate pred) {
@@ -248,9 +249,10 @@ void ScanConfiguration::OptimizeScanSpec() {
 }
 
 Status ScanConfiguration::CreateProjection(const vector<ColumnSchema>& cols) {
-  unique_ptr<Schema> s(new Schema());
+  unique_ptr<Schema> s(new Schema);
   RETURN_NOT_OK(s->Reset(cols, 0));
-  projection_ = pool_.Add(s.release());
+  projection_ = s.get();
+  schemas_pool_.push_back(std::move(s));
   client_projection_ = KuduSchema::FromSchema(*projection_);
   return Status::OK();
 }
diff --git a/src/kudu/client/scan_configuration.h b/src/kudu/client/scan_configuration.h
index f45e166..39d34b6 100644
--- a/src/kudu/client/scan_configuration.h
+++ b/src/kudu/client/scan_configuration.h
@@ -18,16 +18,19 @@
 #pragma once
 
 #include <cstdint>
+#include <deque>
+#include <memory>
 #include <string>
 #include <vector>
 
 #include <glog/logging.h>
 
 #include "kudu/client/client.h"
+#include "kudu/client/scan_predicate.h"
 #include "kudu/client/schema.h"
 #include "kudu/common/scan_spec.h"
+#include "kudu/common/schema.h"
 #include "kudu/gutil/port.h"
-#include "kudu/util/auto_release_pool.h"
 #include "kudu/util/memory/arena.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/slice.h"
@@ -36,14 +39,10 @@
 namespace kudu {
 
 class ColumnPredicate;
-class ColumnSchema;
 class KuduPartialRow;
-class Schema;
 
 namespace client {
 
-class KuduPredicate;
-
 // A configuration object which holds Kudu scan options.
 //
 // Unless otherwise specified, the method documentation matches the
@@ -57,7 +56,7 @@ class ScanConfiguration {
 
   Status SetProjectedColumnIndexes(const std::vector<int>& col_indexes) WARN_UNUSED_RESULT;
 
-  Status AddConjunctPredicate(KuduPredicate* pred) WARN_UNUSED_RESULT;
+  Status AddConjunctPredicate(std::unique_ptr<KuduPredicate> pred) WARN_UNUSED_RESULT;
 
   void AddConjunctPredicate(ColumnPredicate pred);
 
@@ -231,9 +230,10 @@ class ScanConfiguration {
   // Manages interior allocations for the scan spec and copied bounds.
   Arena arena_;
 
-  // Manages objects which need to live for the lifetime of the configuration,
-  // such as schemas, predicates, and keys.
-  AutoReleasePool pool_;
+  // Two containers below are to manage objects which need to live
+  // for the lifetime of the configuration, such as schemas and predicates.
+  std::deque<std::unique_ptr<Schema>> schemas_pool_;
+  std::deque<std::unique_ptr<KuduPredicate>> predicates_pool_;
 
   uint64_t row_format_flags_;
 };
diff --git a/src/kudu/tablet/ops/alter_schema_op.cc b/src/kudu/tablet/ops/alter_schema_op.cc
index 1695216..1c73ea2 100644
--- a/src/kudu/tablet/ops/alter_schema_op.cc
+++ b/src/kudu/tablet/ops/alter_schema_op.cc
@@ -34,6 +34,7 @@
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/tablet/tablet.h"
+#include "kudu/tablet/tablet.pb.h"
 #include "kudu/tablet/tablet_replica.h"
 #include "kudu/tserver/tserver.pb.h"
 #include "kudu/util/pb_util.h"
@@ -105,7 +106,7 @@ Status AlterSchemaOp::Prepare() {
   Tablet* tablet = state_->tablet_replica()->tablet();
   RETURN_NOT_OK(tablet->CreatePreparedAlterSchema(state(), schema.get()));
 
-  state_->AddToAutoReleasePool(schema.release());
+  state_->AddToAutoReleasePool(std::move(schema));
 
   TRACE("PREPARE ALTER-SCHEMA: finished");
   return s;
diff --git a/src/kudu/tablet/ops/alter_schema_op.h b/src/kudu/tablet/ops/alter_schema_op.h
index d95c80c..9bccd2a 100644
--- a/src/kudu/tablet/ops/alter_schema_op.h
+++ b/src/kudu/tablet/ops/alter_schema_op.h
@@ -26,7 +26,6 @@
 #include "kudu/common/common.pb.h"
 #include "kudu/consensus/consensus.pb.h"
 #include "kudu/gutil/macros.h"
-#include "kudu/tablet/tablet.pb.h"
 #include "kudu/tablet/ops/op.h"
 #include "kudu/tserver/tserver_admin.pb.h"
 #include "kudu/util/status.h"
@@ -38,6 +37,7 @@ class rw_semaphore;
 
 namespace tablet {
 
+class OperationResultPB;
 class TabletReplica;
 
 // Op Context for the AlterSchema operation.
diff --git a/src/kudu/tablet/ops/op.cc b/src/kudu/tablet/ops/op.cc
index 2f247c9..27c4464 100644
--- a/src/kudu/tablet/ops/op.cc
+++ b/src/kudu/tablet/ops/op.cc
@@ -17,8 +17,11 @@
 
 #include "kudu/tablet/ops/op.h"
 
+#include "kudu/common/schema.h"
 #include "kudu/rpc/result_tracker.h"
 
+using kudu::tserver::TabletServerErrorPB;
+
 namespace kudu {
 namespace tablet {
 
@@ -42,11 +45,11 @@ OpState::~OpState() {
 }
 
 OpCompletionCallback::OpCompletionCallback()
-    : code_(tserver::TabletServerErrorPB::UNKNOWN_ERROR) {
+    : code_(TabletServerErrorPB::UNKNOWN_ERROR) {
 }
 
 void OpCompletionCallback::set_error(const Status& status,
-                                     tserver::TabletServerErrorPB::Code code) {
+                                     TabletServerErrorPB::Code code) {
   status_ = status;
   code_ = code;
 }
@@ -63,7 +66,7 @@ const Status& OpCompletionCallback::status() const {
   return status_;
 }
 
-const tserver::TabletServerErrorPB::Code OpCompletionCallback::error_code() const {
+TabletServerErrorPB::Code OpCompletionCallback::error_code() const {
   return code_;
 }
 
@@ -72,14 +75,14 @@ void OpCompletionCallback::OpCompleted() {}
 OpCompletionCallback::~OpCompletionCallback() {}
 
 OpMetrics::OpMetrics()
-  : successful_inserts(0),
-    insert_ignore_errors(0),
-    successful_upserts(0),
-    successful_updates(0),
-    update_ignore_errors(0),
-    successful_deletes(0),
-    delete_ignore_errors(0),
-    commit_wait_duration_usec(0) {
+    : successful_inserts(0),
+      insert_ignore_errors(0),
+      successful_upserts(0),
+      successful_updates(0),
+      update_ignore_errors(0),
+      successful_deletes(0),
+      delete_ignore_errors(0),
+      commit_wait_duration_usec(0) {
 }
 
 void OpMetrics::Reset() {
diff --git a/src/kudu/tablet/ops/op.h b/src/kudu/tablet/ops/op.h
index 5463df6..c94837c 100644
--- a/src/kudu/tablet/ops/op.h
+++ b/src/kudu/tablet/ops/op.h
@@ -19,6 +19,7 @@
 #include <atomic>
 #include <cstddef>
 #include <cstdint>
+#include <deque>
 #include <memory>
 #include <string>
 #include <utility>
@@ -32,12 +33,10 @@
 #include "kudu/consensus/consensus.pb.h"
 #include "kudu/consensus/opid.pb.h"
 #include "kudu/consensus/raft_consensus.h"
-#include "kudu/gutil/port.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/rpc/result_tracker.h"
 #include "kudu/rpc/rpc_header.pb.h"
 #include "kudu/tserver/tserver.pb.h"
-#include "kudu/util/auto_release_pool.h"
 #include "kudu/util/countdown_latch.h"
 #include "kudu/util/memory/arena.h"
 #include "kudu/util/status.h"
@@ -49,6 +48,9 @@ class Message;
 }
 
 namespace kudu {
+
+class Schema;
+
 namespace tablet {
 class OpCompletionCallback;
 class OpState;
@@ -207,15 +209,8 @@ class OpState {
   }
 
   // Sets a heap object to be managed by this op's AutoReleasePool.
-  template<class T>
-  T* AddToAutoReleasePool(T* t) {
-    return pool_.Add(t);
-  }
-
-  // Sets an array heap object to be managed by this op's AutoReleasePool.
-  template<class T>
-  T* AddArrayToAutoReleasePool(T* t) {
-    return pool_.AddArray(t);
+  void AddToAutoReleasePool(std::unique_ptr<Schema> t) {
+    schemas_pool_.emplace_back(std::move(t));
   }
 
   // Return the arena associated with this op.
@@ -288,7 +283,7 @@ class OpState {
   // Optional callback to be called once the op completes.
   std::unique_ptr<OpCompletionCallback> completion_clbk_;
 
-  AutoReleasePool pool_;
+  std::deque<std::unique_ptr<Schema>> schemas_pool_;
 
   // This operation's timestamp.
   // This is only set once during the operation lifecycle, using external synchronization.
@@ -325,7 +320,6 @@ class OpState {
 // which avoids callers having to keep checking for NULL.
 class OpCompletionCallback {
  public:
-
   OpCompletionCallback();
 
   // Allows to set an error for this op and a mapping to a server level code.
@@ -338,7 +332,7 @@ class OpCompletionCallback {
 
   const Status& status() const;
 
-  const tserver::TabletServerErrorPB::Code error_code() const;
+  tserver::TabletServerErrorPB::Code error_code() const;
 
   // Subclasses should override this.
   virtual void OpCompleted();
@@ -364,7 +358,7 @@ class LatchOpCompletionCallback : public OpCompletionCallback {
       response_(DCHECK_NOTNULL(response)) {
   }
 
-  virtual void OpCompleted() OVERRIDE {
+  void OpCompleted() override {
     if (!status_.ok()) {
       StatusToPB(status_, response_->mutable_error()->mutable_status());
       response_->mutable_error()->set_code(code_);
diff --git a/src/kudu/util/auto_release_pool.h b/src/kudu/util/auto_release_pool.h
deleted file mode 100644
index eaed9c2..0000000
--- a/src/kudu/util/auto_release_pool.h
+++ /dev/null
@@ -1,99 +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.
-//
-// Simple pool of objects that will be deallocated when the pool is
-// destroyed
-
-#ifndef KUDU_UTIL_AUTO_RELEASE_POOL_H
-#define KUDU_UTIL_AUTO_RELEASE_POOL_H
-
-#include <vector>
-
-#include "kudu/gutil/spinlock.h"
-
-namespace kudu {
-
-// Thread-safe.
-class AutoReleasePool {
- public:
-  AutoReleasePool(): objects_() { }
-
-  ~AutoReleasePool() {
-    for (auto& object : objects_) {
-      delete object;
-    }
-  }
-
-  template <class T>
-  T *Add(T *t) {
-    base::SpinLockHolder l(&lock_);
-    objects_.push_back(new SpecificElement<T>(t));
-    return t;
-  }
-
-  // Add an array-allocated object to the pool. This is identical to
-  // Add() except that it will be freed with 'delete[]' instead of 'delete'.
-  template<class T>
-  T* AddArray(T *t) {
-    base::SpinLockHolder l(&lock_);
-    objects_.push_back(new SpecificArrayElement<T>(t));
-    return t;
-  }
-
-  // Donate all objects in this pool to another pool.
-  void DonateAllTo(AutoReleasePool* dst) {
-    base::SpinLockHolder l(&lock_);
-    base::SpinLockHolder l_them(&dst->lock_);
-
-    dst->objects_.reserve(dst->objects_.size() + objects_.size());
-    dst->objects_.insert(dst->objects_.end(), objects_.begin(), objects_.end());
-    objects_.clear();
-  }
-
- private:
-  struct GenericElement {
-    virtual ~GenericElement() {}
-  };
-
-  template <class T>
-  struct SpecificElement : GenericElement {
-    explicit SpecificElement(T *t): t(t) {}
-    ~SpecificElement() {
-      delete t;
-    }
-
-    T *t;
-  };
-
-  template <class T>
-  struct SpecificArrayElement : GenericElement {
-    explicit SpecificArrayElement(T *t): t(t) {}
-    ~SpecificArrayElement() {
-      delete [] t;
-    }
-
-    T *t;
-  };
-
-  typedef std::vector<GenericElement *> ElementVector;
-  ElementVector objects_;
-  base::SpinLock lock_;
-};
-
-
-} // namespace kudu
-#endif