From commits-return-8569-apmail-kudu-commits-archive=kudu.apache.org@kudu.apache.org Tue Mar 10 06:44:25 2020 Return-Path: X-Original-To: apmail-kudu-commits-archive@minotaur.apache.org Delivered-To: apmail-kudu-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [207.244.88.153]) by minotaur.apache.org (Postfix) with SMTP id B015619650 for ; Tue, 10 Mar 2020 06:44:24 +0000 (UTC) Received: (qmail 44809 invoked by uid 500); 10 Mar 2020 06:44:24 -0000 Delivered-To: apmail-kudu-commits-archive@kudu.apache.org Received: (qmail 44770 invoked by uid 500); 10 Mar 2020 06:44:24 -0000 Mailing-List: contact commits-help@kudu.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@kudu.apache.org Delivered-To: mailing list commits@kudu.apache.org Received: (qmail 44756 invoked by uid 99); 10 Mar 2020 06:44:24 -0000 Received: from ec2-52-202-80-70.compute-1.amazonaws.com (HELO gitbox.apache.org) (52.202.80.70) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 10 Mar 2020 06:44:23 +0000 Received: by gitbox.apache.org (ASF Mail Server at gitbox.apache.org, from userid 33) id 9D8C08DACA; Tue, 10 Mar 2020 06:44:23 +0000 (UTC) Date: Tue, 10 Mar 2020 06:44:23 +0000 To: "commits@kudu.apache.org" Subject: [kudu] branch master updated: replace gscoped_ptr with std::unique_ptr MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 8bit Message-ID: <158382266345.16774.355571275096613088@gitbox.apache.org> From: adar@apache.org X-Git-Host: gitbox.apache.org X-Git-Repo: kudu X-Git-Refname: refs/heads/master X-Git-Reftype: branch X-Git-Oldrev: ef8696ac436776cf52181e9d94afb3c9608c9efa X-Git-Newrev: eaed5fb514e70a837cdd1ef8f39f371fddabc25a X-Git-Rev: eaed5fb514e70a837cdd1ef8f39f371fddabc25a X-Git-NotificationType: ref_changed_plus_diff X-Git-Multimail-Version: 1.5.dev Auto-Submitted: auto-generated This is an automated email from the ASF dual-hosted git repository. adar 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 eaed5fb replace gscoped_ptr with std::unique_ptr eaed5fb is described below commit eaed5fb514e70a837cdd1ef8f39f371fddabc25a Author: Adar Dembo AuthorDate: Mon Mar 9 00:06:25 2020 -0700 replace gscoped_ptr with std::unique_ptr The remaining usages are in gutil, where we've imported code wholesale which contained gscoped_ptr and I'm loathe to remove it right now. Given the high number of touched files, there are also a fair number of iwyu and clang-tidy fixups in this patch. Change-Id: I9a35557b49dba7cc6a0c131841ae3c96230bb2fc Reviewed-on: http://gerrit.cloudera.org:8080/15387 Reviewed-by: Alexey Serbin Tested-by: Adar Dembo --- docs/design-docs/cpp-client.md | 6 +- src/kudu/benchmarks/tpch/rpc_line_item_dao-test.cc | 27 +++++---- src/kudu/benchmarks/tpch/rpc_line_item_dao.cc | 53 +++++++++--------- src/kudu/benchmarks/tpch/rpc_line_item_dao.h | 21 ++++--- src/kudu/benchmarks/tpch/tpch1.cc | 14 ++--- src/kudu/benchmarks/tpch/tpch_real_world.cc | 33 ++++++----- src/kudu/cfile/binary_dict_block.h | 13 ++--- src/kudu/cfile/block_cache.cc | 1 - src/kudu/cfile/block_cache.h | 9 +-- src/kudu/cfile/bshuf_block.h | 1 + src/kudu/cfile/cfile-test-base.h | 16 ++---- src/kudu/cfile/cfile-test.cc | 13 ++--- src/kudu/cfile/cfile_reader.cc | 1 - src/kudu/cfile/cfile_reader.h | 19 +++---- src/kudu/cfile/cfile_writer.h | 17 ++---- src/kudu/cfile/encoding-test.cc | 64 +++++++++++----------- src/kudu/cfile/index-test.cc | 13 +++-- src/kudu/client/client.cc | 3 +- src/kudu/client/master_rpc.cc | 14 ++--- src/kudu/client/meta_cache.cc | 31 +++++------ src/kudu/client/predicate-test.cc | 3 +- src/kudu/client/scan_predicate.cc | 1 - src/kudu/client/scan_token-test.cc | 3 +- src/kudu/client/schema.cc | 9 +-- src/kudu/client/write_op.cc | 9 +-- src/kudu/codegen/code_cache.cc | 1 - src/kudu/codegen/code_cache.h | 12 ++-- src/kudu/codegen/code_generator.cc | 7 ++- src/kudu/codegen/codegen-test.cc | 20 ++++--- src/kudu/codegen/compilation_manager.cc | 4 +- src/kudu/codegen/compilation_manager.h | 3 +- src/kudu/consensus/consensus-test-util.h | 20 +++---- src/kudu/consensus/consensus_peers-test.cc | 8 +-- src/kudu/consensus/consensus_peers.cc | 16 +++--- src/kudu/consensus/consensus_peers.h | 21 +++---- src/kudu/consensus/consensus_queue-test.cc | 5 +- src/kudu/consensus/consensus_queue.h | 3 +- src/kudu/consensus/leader_election-test.cc | 7 +-- src/kudu/consensus/leader_election.h | 4 +- src/kudu/consensus/log-test-base.h | 7 +-- src/kudu/consensus/log-test.cc | 4 +- src/kudu/consensus/log.cc | 2 +- src/kudu/consensus/log.h | 4 +- src/kudu/consensus/log_cache-test.cc | 1 - src/kudu/consensus/peer_manager.cc | 19 ++++--- src/kudu/consensus/raft_consensus.cc | 7 +-- src/kudu/consensus/raft_consensus.h | 8 +-- src/kudu/consensus/raft_consensus_quorum-test.cc | 6 +- src/kudu/consensus/ref_counted_replicate.h | 9 +-- src/kudu/gutil/map-util.h | 2 +- src/kudu/gutil/ref_counted.h | 2 +- src/kudu/integration-tests/all_types-itest.cc | 14 ++--- src/kudu/integration-tests/alter_table-test.cc | 17 +++--- .../integration-tests/catalog_manager_tsk-itest.cc | 6 +- src/kudu/integration-tests/client-stress-test.cc | 5 +- .../consensus_peer_health_status-itest.cc | 3 +- src/kudu/integration-tests/create-table-itest.cc | 13 ++--- src/kudu/integration-tests/decimal-itest.cc | 9 ++- src/kudu/integration-tests/delete_table-itest.cc | 4 +- .../integration-tests/exactly_once_writes-itest.cc | 2 +- .../integration-tests/flex_partitioning-itest.cc | 7 +-- src/kudu/integration-tests/fuzz-itest.cc | 5 +- src/kudu/integration-tests/linked_list-test.cc | 5 +- .../integration-tests/location_assignment-itest.cc | 1 - .../integration-tests/master_replication-itest.cc | 5 +- .../integration-tests/raft_config_change-itest.cc | 2 +- .../integration-tests/raft_consensus-itest-base.cc | 5 +- src/kudu/integration-tests/raft_consensus-itest.cc | 4 +- .../raft_consensus_election-itest.cc | 2 +- .../raft_consensus_nonvoter-itest.cc | 2 +- .../raft_consensus_stress-itest.cc | 3 +- src/kudu/integration-tests/registration-test.cc | 30 +++++----- .../integration-tests/security-faults-itest.cc | 3 +- .../security-unknown-tsk-itest.cc | 3 +- src/kudu/integration-tests/tablet_copy-itest.cc | 7 ++- src/kudu/integration-tests/test_workload.cc | 43 +++++++-------- src/kudu/integration-tests/ts_itest-base.cc | 4 +- src/kudu/integration-tests/ts_itest-base.h | 7 +-- .../update_scan_delta_compact-test.cc | 9 ++- .../integration-tests/write_throttling-itest.cc | 32 +++++------ src/kudu/kserver/kserver.cc | 1 - src/kudu/master/catalog_manager.cc | 12 ++-- src/kudu/master/master-test.cc | 6 +- src/kudu/master/sys_catalog-test.cc | 6 +- src/kudu/master/ts_descriptor.h | 7 +-- src/kudu/mini-cluster/internal_mini_cluster.cc | 8 +-- src/kudu/rpc/connection.cc | 1 - src/kudu/rpc/rpc-test.cc | 3 +- src/kudu/server/glog_metrics.h | 10 ++-- src/kudu/tablet/compaction-test.cc | 41 +++++++------- src/kudu/tablet/compaction.cc | 12 ++-- src/kudu/tablet/compaction.h | 8 +-- src/kudu/tablet/delta_compaction-test.cc | 9 ++- src/kudu/tablet/delta_compaction.cc | 23 ++++---- src/kudu/tablet/delta_compaction.h | 12 ++-- src/kudu/tablet/delta_tracker.cc | 12 ++-- src/kudu/tablet/delta_tracker.h | 8 +-- src/kudu/tablet/deltafile.cc | 8 +-- src/kudu/tablet/deltafile.h | 12 ++-- src/kudu/tablet/deltamemstore-test.cc | 3 +- src/kudu/tablet/deltamemstore.cc | 11 ++-- src/kudu/tablet/deltamemstore.h | 10 +--- src/kudu/tablet/diskrowset-test.cc | 1 - src/kudu/tablet/diskrowset.cc | 25 ++++----- src/kudu/tablet/diskrowset.h | 47 ++++++++-------- src/kudu/tablet/local_tablet_writer.h | 7 +-- src/kudu/tablet/lock_manager.cc | 10 ++-- src/kudu/tablet/memrowset.cc | 17 +++--- src/kudu/tablet/memrowset.h | 7 +-- src/kudu/tablet/mock-rowsets.h | 6 +- src/kudu/tablet/multi_column_writer.cc | 3 +- src/kudu/tablet/row_op.cc | 5 +- src/kudu/tablet/row_op.h | 12 ++-- src/kudu/tablet/rowset.cc | 2 +- src/kudu/tablet/rowset.h | 5 +- src/kudu/tablet/rowset_tree.cc | 8 ++- src/kudu/tablet/rowset_tree.h | 8 +-- src/kudu/tablet/tablet-harness.h | 2 +- src/kudu/tablet/tablet-test-util.h | 3 +- src/kudu/tablet/tablet.cc | 15 +++-- src/kudu/tablet/tablet.h | 5 +- src/kudu/tablet/tablet_bootstrap-test.cc | 16 +++--- src/kudu/tablet/tablet_bootstrap.cc | 1 - src/kudu/tablet/tablet_metadata-test.cc | 11 ++-- src/kudu/tablet/tablet_metadata.cc | 7 +-- src/kudu/tablet/tablet_metadata.h | 5 +- src/kudu/tablet/tablet_random_access-test.cc | 7 +-- src/kudu/tablet/tablet_replica-test.cc | 17 +++--- src/kudu/tablet/tablet_replica.cc | 31 +++++------ src/kudu/tablet/tablet_replica.h | 5 +- .../transactions/alter_schema_transaction.cc | 6 +- .../tablet/transactions/alter_schema_transaction.h | 6 +- src/kudu/tablet/transactions/transaction.h | 6 +- src/kudu/tablet/transactions/transaction_driver.cc | 9 ++- src/kudu/tablet/transactions/transaction_driver.h | 9 ++- .../transactions/transaction_tracker-test.cc | 12 ++-- src/kudu/tablet/transactions/transaction_tracker.h | 9 +-- src/kudu/tablet/transactions/write_transaction.cc | 7 +-- src/kudu/tablet/transactions/write_transaction.h | 9 ++- src/kudu/tools/ksck_remote-test.cc | 7 +-- src/kudu/tools/ksck_remote.cc | 8 +-- src/kudu/tools/kudu-admin-test.cc | 3 +- src/kudu/tools/rebalancer_tool-test.cc | 1 - src/kudu/util/bit-stream-utils.h | 12 ++-- src/kudu/util/bit-stream-utils.inline.h | 15 ++--- src/kudu/util/rle-encoding.h | 19 ++++--- 146 files changed, 679 insertions(+), 818 deletions(-) diff --git a/docs/design-docs/cpp-client.md b/docs/design-docs/cpp-client.md index e199adb..7698114 100644 --- a/docs/design-docs/cpp-client.md +++ b/docs/design-docs/cpp-client.md @@ -54,12 +54,12 @@ void ExplicitBatchingExample() { } // Update a row. - gscoped_ptr upd = t->NewUpdate(); + unique_ptr upd = t->NewUpdate(); upd->mutable_row()->SetInt64("key", 1); upd->mutable_row()->SetInt64("val", 1 * 2 + 1); // Delete a row. - gscoped_ptr del = t->NewDelete(); + unique_ptr del = t->NewDelete(); del->mutable_row()->SetInt64("key", 2); // only specify key. // Setting a timeout on the session applies to the next Flush call. @@ -107,7 +107,7 @@ void BulkIngestExample() { CHECK_OK(session->SetFlushMode(KuduSession::AUTO_FLUSH_BACKGROUND)); for (int i = 0; i < 10000; i++) { - gscoped_ptr ins = t->NewInsertion(); + unique_ptr ins = t->NewInsertion(); ins->SetInt64("key", i); ins->SetInt64("val", i * 2); // This will start getting written in the background. diff --git a/src/kudu/benchmarks/tpch/rpc_line_item_dao-test.cc b/src/kudu/benchmarks/tpch/rpc_line_item_dao-test.cc index 9eef222..4ac9c8e 100644 --- a/src/kudu/benchmarks/tpch/rpc_line_item_dao-test.cc +++ b/src/kudu/benchmarks/tpch/rpc_line_item_dao-test.cc @@ -15,20 +15,22 @@ // specific language governing permissions and limitations // under the License. +#include "kudu/benchmarks/tpch/rpc_line_item_dao.h" + #include #include +#include #include #include #include +#include #include #include -#include "kudu/benchmarks/tpch/rpc_line_item_dao.h" #include "kudu/benchmarks/tpch/tpch-schemas.h" #include "kudu/client/row_result.h" #include "kudu/common/partial_row.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/gutil/stringprintf.h" #include "kudu/master/mini_master.h" @@ -38,14 +40,15 @@ #include "kudu/util/test_macros.h" #include "kudu/util/test_util.h" -namespace kudu { - -using client::KuduRowResult; -using cluster::InternalMiniCluster; -using cluster::InternalMiniClusterOptions; +using kudu::client::KuduRowResult; +using kudu::cluster::InternalMiniCluster; +using kudu::cluster::InternalMiniClusterOptions; using std::string; +using std::unique_ptr; using std::vector; +namespace kudu { + class RpcLineItemDAOTest : public KuduTest { public: @@ -76,8 +79,8 @@ class RpcLineItemDAOTest : public KuduTest { } protected: - gscoped_ptr cluster_; - gscoped_ptr dao_; + unique_ptr cluster_; + unique_ptr dao_; // Builds a test row to be inserted into the lineitem table. // The row's ship_date is set such that it matches the TPCH Q1 predicate. @@ -107,7 +110,7 @@ class RpcLineItemDAOTest : public KuduTest { } int CountRows() { - gscoped_ptr scanner; + unique_ptr scanner; dao_->OpenScanner(vector(), &scanner); vector rows; int count = 0; @@ -121,7 +124,7 @@ class RpcLineItemDAOTest : public KuduTest { void ScanTpch1RangeToStrings(int64_t min_orderkey, int64_t max_orderkey, vector* str_rows) { str_rows->clear(); - gscoped_ptr scanner; + unique_ptr scanner; dao_->OpenTpch1ScannerForOrderKeyRange(min_orderkey, max_orderkey, &scanner); vector rows; @@ -161,7 +164,7 @@ TEST_F(RpcLineItemDAOTest, TestUpdate) { dao_->MutateLine(boost::bind(UpdateTestRow, 1, 1, 12345, _1)); dao_->FinishWriting(); - gscoped_ptr scanner; + unique_ptr scanner; dao_->OpenScanner({ tpch::kQuantityColName }, &scanner); vector rows; while (scanner->HasMore()) { diff --git a/src/kudu/benchmarks/tpch/rpc_line_item_dao.cc b/src/kudu/benchmarks/tpch/rpc_line_item_dao.cc index adad044..fd047c4 100644 --- a/src/kudu/benchmarks/tpch/rpc_line_item_dao.cc +++ b/src/kudu/benchmarks/tpch/rpc_line_item_dao.cc @@ -17,6 +17,7 @@ #include "kudu/benchmarks/tpch/rpc_line_item_dao.h" +#include #include #include #include @@ -32,7 +33,6 @@ #include "kudu/client/schema.h" #include "kudu/client/value.h" #include "kudu/client/write_op.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/gutil/stl_util.h" #include "kudu/util/monotime.h" @@ -42,29 +42,26 @@ DEFINE_bool(tpch_cache_blocks_when_scanning, true, "Whether the scanners should cache the blocks that are read or not"); +using kudu::client::KuduInsert; +using kudu::client::KuduClientBuilder; +using kudu::client::KuduError; +using kudu::client::KuduPredicate; +using kudu::client::KuduRowResult; +using kudu::client::KuduScanner; +using kudu::client::KuduSchema; +using kudu::client::KuduSession; +using kudu::client::KuduStatusCallback; +using kudu::client::KuduTableCreator; +using kudu::client::KuduUpdate; +using kudu::client::KuduValue; using std::string; +using std::unique_ptr; using std::vector; namespace kudu { class KuduPartialRow; -using client::KuduInsert; -using client::KuduClient; -using client::KuduClientBuilder; -using client::KuduError; -using client::KuduPredicate; -using client::KuduRowResult; -using client::KuduScanner; -using client::KuduSchema; -using client::KuduSession; -using client::KuduStatusCallback; -using client::KuduStatusMemberCallback; -using client::KuduTableCreator; -using client::KuduUpdate; -using client::KuduValue; -using std::vector; - namespace { class FlushCallback : public KuduStatusCallback { @@ -137,7 +134,7 @@ void RpcLineItemDAO::Init() { .Build(&client_)); Status s = client_->OpenTable(table_name_, &client_table_); if (s.IsNotFound()) { - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); table_creator->table_name(table_name_) .schema(&schema) .num_replicas(1); @@ -166,14 +163,14 @@ void RpcLineItemDAO::Init() { } void RpcLineItemDAO::WriteLine(const boost::function &f) { - gscoped_ptr insert(client_table_->NewInsert()); + unique_ptr insert(client_table_->NewInsert()); f(insert->mutable_row()); CHECK_OK(session_->Apply(insert.release())); HandleLine(); } void RpcLineItemDAO::MutateLine(const boost::function &f) { - gscoped_ptr update(client_table_->NewUpdate()); + unique_ptr update(client_table_->NewUpdate()); f(update->mutable_row()); CHECK_OK(session_->Apply(update.release())); HandleLine(); @@ -188,12 +185,12 @@ void RpcLineItemDAO::FinishWriting() { } void RpcLineItemDAO::OpenScanner(const vector& columns, - gscoped_ptr* out_scanner) { + unique_ptr* out_scanner) { vector preds; OpenScannerImpl(columns, preds, out_scanner); } -void RpcLineItemDAO::OpenTpch1Scanner(gscoped_ptr* out_scanner) { +void RpcLineItemDAO::OpenTpch1Scanner(unique_ptr* out_scanner) { vector preds; preds.push_back(client_table_->NewComparisonPredicate( tpch::kShipDateColName, KuduPredicate::LESS_EQUAL, @@ -201,18 +198,18 @@ void RpcLineItemDAO::OpenTpch1Scanner(gscoped_ptr* out_scanner) { OpenScannerImpl(tpch::GetTpchQ1QueryColumns(), preds, out_scanner); } -void RpcLineItemDAO::OpenTpch1ScannerForOrderKeyRange(int64_t min_key, int64_t max_key, - gscoped_ptr* out_scanner) { +void RpcLineItemDAO::OpenTpch1ScannerForOrderKeyRange( + int64_t min_orderkey, int64_t max_orderkey, unique_ptr* out_scanner) { vector preds; preds.push_back(client_table_->NewComparisonPredicate( tpch::kShipDateColName, KuduPredicate::LESS_EQUAL, KuduValue::CopyString(kScanUpperBound))); preds.push_back(client_table_->NewComparisonPredicate( tpch::kOrderKeyColName, KuduPredicate::GREATER_EQUAL, - KuduValue::FromInt(min_key))); + KuduValue::FromInt(min_orderkey))); preds.push_back(client_table_->NewComparisonPredicate( tpch::kOrderKeyColName, KuduPredicate::LESS_EQUAL, - KuduValue::FromInt(max_key))); + KuduValue::FromInt(max_orderkey))); OpenScannerImpl(tpch::GetTpchQ1QueryColumns(), preds, out_scanner); } @@ -224,8 +221,8 @@ bool RpcLineItemDAO::IsTableEmpty() { void RpcLineItemDAO::OpenScannerImpl(const vector& columns, const vector& preds, - gscoped_ptr* out_scanner) { - gscoped_ptr ret(new Scanner); + unique_ptr* out_scanner) { + unique_ptr ret(new Scanner); ret->scanner_.reset(new KuduScanner(client_table_.get())); ret->scanner_->SetCacheBlocks(FLAGS_tpch_cache_blocks_when_scanning); CHECK_OK(ret->scanner_->SetProjectedColumnNames(columns)); diff --git a/src/kudu/benchmarks/tpch/rpc_line_item_dao.h b/src/kudu/benchmarks/tpch/rpc_line_item_dao.h index 7f1275f..3205be1 100644 --- a/src/kudu/benchmarks/tpch/rpc_line_item_dao.h +++ b/src/kudu/benchmarks/tpch/rpc_line_item_dao.h @@ -14,17 +14,16 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -#ifndef KUDU_TPCH_RPC_LINE_ITEM_DAO_H -#define KUDU_TPCH_RPC_LINE_ITEM_DAO_H +#pragma once #include +#include #include #include #include "kudu/client/client.h" #include "kudu/client/shared_ptr.h" #include "kudu/client/row_result.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/util/locks.h" #include "kudu/util/monotime.h" #include "kudu/util/semaphore.h" @@ -67,18 +66,19 @@ class RpcLineItemDAO { // Deletes previous scanner if one is open. // Projects only those column names listed in 'columns'. void OpenScanner(const std::vector& columns, - gscoped_ptr* scanner); + std::unique_ptr* scanner); // Calls OpenScanner with the tpch1 query parameters. - void OpenTpch1Scanner(gscoped_ptr* scanner); + void OpenTpch1Scanner(std::unique_ptr* scanner); // Opens a scanner with the TPCH Q1 projection and filter, plus range filter to only // select rows in the given order key range. void OpenTpch1ScannerForOrderKeyRange(int64_t min_orderkey, int64_t max_orderkey, - gscoped_ptr* scanner); + std::unique_ptr* scanner); bool IsTableEmpty(); - // TODO: this wrapper class is of limited utility now that we only have a single - // "DAO" implementation -- we could just return the KuduScanner to users directly. + // TODO(unknown): this wrapper class is of limited utility now that we only + // have a single "DAO" implementation -- we could just return the KuduScanner + // to users directly. class Scanner { public: ~Scanner() {} @@ -93,7 +93,7 @@ class RpcLineItemDAO { friend class RpcLineItemDAO; Scanner() {} - gscoped_ptr scanner_; + std::unique_ptr scanner_; }; private: @@ -101,7 +101,7 @@ class RpcLineItemDAO { void OpenScannerImpl(const std::vector& columns, const std::vector& preds, - gscoped_ptr* scanner); + std::unique_ptr* scanner); void HandleLine(); const std::string master_address_; @@ -124,4 +124,3 @@ class RpcLineItemDAO { }; } //namespace kudu -#endif diff --git a/src/kudu/benchmarks/tpch/tpch1.cc b/src/kudu/benchmarks/tpch/tpch1.cc index edd5da9..c4335f2 100644 --- a/src/kudu/benchmarks/tpch/tpch1.cc +++ b/src/kudu/benchmarks/tpch/tpch1.cc @@ -59,6 +59,7 @@ #include #include +#include #include #include #include @@ -74,7 +75,6 @@ #include "kudu/benchmarks/tpch/rpc_line_item_dao.h" #include "kudu/client/row_result.h" #include "kudu/codegen/compilation_manager.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/hash/city.h" #include "kudu/gutil/stringprintf.h" #include "kudu/master/mini_master.h" @@ -104,14 +104,14 @@ DEFINE_int32(tpch_max_batch_size, 1000, DEFINE_string(table_name, "lineitem", "The table name to write/read"); +using kudu::client::KuduRowResult; using std::string; +using std::unique_ptr; using std::unordered_map; using std::vector; namespace kudu { -using client::KuduRowResult; - struct Result { int32_t l_quantity; double l_extendedprice; @@ -157,7 +157,7 @@ void LoadLineItems(const string &path, RpcLineItemDAO *dao) { void WarmupScanCache(RpcLineItemDAO* dao) { // Warms up cache for the tpch1 query. - gscoped_ptr scanner; + unique_ptr scanner; dao->OpenTpch1Scanner(&scanner); codegen::CompilationManager::GetSingleton()->Wait(); } @@ -166,7 +166,7 @@ void Tpch1(RpcLineItemDAO *dao) { typedef unordered_map slice_map; typedef unordered_map slice_map_map; - gscoped_ptr scanner; + unique_ptr scanner; dao->OpenTpch1Scanner(&scanner); int matching_rows = 0; @@ -255,7 +255,7 @@ int main(int argc, char **argv) { kudu::InitGoogleLoggingSafe(argv[0]); kudu::Env* env; - gscoped_ptr cluster; + unique_ptr cluster; string master_address; if (FLAGS_use_mini_cluster) { env = kudu::Env::Default(); @@ -270,7 +270,7 @@ int main(int argc, char **argv) { master_address = FLAGS_master_address; } - gscoped_ptr dao(new kudu::RpcLineItemDAO( + unique_ptr dao(new kudu::RpcLineItemDAO( master_address, FLAGS_table_name, FLAGS_tpch_max_batch_size, /* timeout_ms = */ 5000, kudu::RpcLineItemDAO::RANGE, /* num_buckets = */ 1)); diff --git a/src/kudu/benchmarks/tpch/tpch_real_world.cc b/src/kudu/benchmarks/tpch/tpch_real_world.cc index f878229..0731516 100644 --- a/src/kudu/benchmarks/tpch/tpch_real_world.cc +++ b/src/kudu/benchmarks/tpch/tpch_real_world.cc @@ -47,8 +47,8 @@ #include #include #include +#include #include -#include #include #include @@ -63,7 +63,6 @@ #include "kudu/client/row_result.h" #include "kudu/client/schema.h" #include "kudu/common/partial_row.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/stl_util.h" #include "kudu/gutil/stringprintf.h" @@ -122,17 +121,17 @@ DEFINE_string(tpch_partition_strategy, "range", "tablets. This is less ideal, but more faithfully represents a lot of write " "workloads."); +using kudu::client::KuduRowResult; +using kudu::client::KuduSchema; +using kudu::cluster::ExternalMiniCluster; +using kudu::cluster::ExternalMiniClusterOptions; using std::string; +using std::unique_ptr; using std::vector; +using strings::Substitute; namespace kudu { -using client::KuduRowResult; -using client::KuduSchema; -using cluster::ExternalMiniCluster; -using cluster::ExternalMiniClusterOptions; -using strings::Substitute; - class TpchRealWorld { public: TpchRealWorld() @@ -147,7 +146,7 @@ class TpchRealWorld { Status Init(); - gscoped_ptr GetInittedDAO(); + unique_ptr GetInittedDAO(); void LoadLineItemsThread(int i); @@ -170,7 +169,7 @@ class TpchRealWorld { ? Substitute("$0.$1", kLineItemBase, i + 1) : kLineItemBase; } - gscoped_ptr cluster_; + unique_ptr cluster_; AtomicInt rows_inserted_; string master_addresses_; AtomicBool stop_threads_; @@ -259,7 +258,7 @@ Status TpchRealWorld::StartDbgens() { argv.emplace_back("-S"); argv.push_back(Substitute("$0", i)); } - gscoped_ptr dbgen_proc(new Subprocess(argv)); + unique_ptr dbgen_proc(new Subprocess(argv)); LOG(INFO) << "Running " << JoinStrings(argv, " "); RETURN_NOT_OK(dbgen_proc->Start()); dbgen_processes_.push_back(dbgen_proc.release()); @@ -267,7 +266,7 @@ Status TpchRealWorld::StartDbgens() { return Status::OK(); } -gscoped_ptr TpchRealWorld::GetInittedDAO() { +unique_ptr TpchRealWorld::GetInittedDAO() { // When chunking, dbgen will begin the nth chunk on the order key: // // 6000000 * SF * n / num_chunks @@ -295,7 +294,7 @@ gscoped_ptr TpchRealWorld::GetInittedDAO() { LOG(FATAL) << "Unknown partition strategy: " << FLAGS_tpch_partition_strategy; } - gscoped_ptr dao( + unique_ptr dao( new RpcLineItemDAO(master_addresses_, FLAGS_tpch_table_name, FLAGS_tpch_max_batch_size, @@ -304,12 +303,12 @@ gscoped_ptr TpchRealWorld::GetInittedDAO() { FLAGS_tpch_num_inserters, split_rows)); dao->Init(); - return std::move(dao); + return dao; } void TpchRealWorld::LoadLineItemsThread(int i) { LOG(INFO) << "Connecting to cluster at " << master_addresses_; - gscoped_ptr dao = GetInittedDAO(); + unique_ptr dao = GetInittedDAO(); LineItemTsvImporter importer(GetNthLineItemFileName(i)); boost::function f = @@ -359,7 +358,7 @@ void TpchRealWorld::MonitorDbgenThread(int i) { } void TpchRealWorld::RunQueriesThread() { - gscoped_ptr dao = GetInittedDAO(); + unique_ptr dao = GetInittedDAO(); while (!stop_threads_.Load()) { string log; if (FLAGS_tpch_load_data) { @@ -368,7 +367,7 @@ void TpchRealWorld::RunQueriesThread() { log = "querying data in cluster"; } LOG_TIMING(INFO, log) { - gscoped_ptr scanner; + unique_ptr scanner; dao->OpenTpch1Scanner(&scanner); vector rows; // We check stop_threads_ even while scanning since it can takes tens of seconds to query. diff --git a/src/kudu/cfile/binary_dict_block.h b/src/kudu/cfile/binary_dict_block.h index 796f20e..ceb044e 100644 --- a/src/kudu/cfile/binary_dict_block.h +++ b/src/kudu/cfile/binary_dict_block.h @@ -31,15 +31,13 @@ // To use other block builder/decoder, just make sure that BlockDecoder has // interface CopyNextValuesToArray(size_t*, uint8_t*). To do that, just replace // BShufBuilder/Decoder is ok. -// -// -#ifndef KUDU_CFILE_BINARY_DICT_BLOCK_H -#define KUDU_CFILE_BINARY_DICT_BLOCK_H +#pragma once #include #include #include +#include #include @@ -47,7 +45,6 @@ #include "kudu/cfile/block_encodings.h" #include "kudu/cfile/binary_plain_block.h" #include "kudu/gutil/casts.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/port.h" #include "kudu/gutil/strings/stringpiece.h" @@ -114,7 +111,7 @@ class BinaryDictBlockBuilder final : public BlockBuilder { bool finished_; const WriterOptions* options_; - gscoped_ptr data_builder_; + std::unique_ptr data_builder_; // dict_block_, dictionary_, dictionary_strings_arena_ // is related to the dictionary block (one per cfile). @@ -178,7 +175,7 @@ class BinaryDictBlockDecoder final : public BlockDecoder { // Dictionary block decoder BinaryPlainBlockDecoder* dict_decoder_; - gscoped_ptr data_decoder_; + std::unique_ptr data_decoder_; // Parent CFileIterator, each dictionary decoder in the same CFile will share // the same vocabulary, and thus, the same set of matching codewords. @@ -198,5 +195,3 @@ class BinaryDictBlockDecoder final : public BlockDecoder { MAKE_ENUM_LIMITS(kudu::cfile::DictEncodingMode, kudu::cfile::DictEncodingMode_min, kudu::cfile::DictEncodingMode_max); - -#endif // KUDU_CFILE_BINARY_DICT_BLOCK_H diff --git a/src/kudu/cfile/block_cache.cc b/src/kudu/cfile/block_cache.cc index bf93703..ab867dc 100644 --- a/src/kudu/cfile/block_cache.cc +++ b/src/kudu/cfile/block_cache.cc @@ -26,7 +26,6 @@ #include #include -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/util/block_cache_metrics.h" diff --git a/src/kudu/cfile/block_cache.h b/src/kudu/cfile/block_cache.h index e7621ec..153b3b2 100644 --- a/src/kudu/cfile/block_cache.h +++ b/src/kudu/cfile/block_cache.h @@ -14,15 +14,14 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -#ifndef KUDU_CFILE_BLOCK_CACHE_H -#define KUDU_CFILE_BLOCK_CACHE_H +#pragma once #include #include +#include #include #include "kudu/fs/block_id.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/port.h" #include "kudu/gutil/singleton.h" @@ -167,7 +166,7 @@ class BlockCache { DISALLOW_COPY_AND_ASSIGN(BlockCache); - gscoped_ptr cache_; + std::unique_ptr cache_; }; // Scoped reference to a block from the block cache. @@ -227,5 +226,3 @@ bool ValidateBlockCacheCapacity(); } // namespace cfile } // namespace kudu - -#endif diff --git a/src/kudu/cfile/bshuf_block.h b/src/kudu/cfile/bshuf_block.h index 564db76..eaeb6e0 100644 --- a/src/kudu/cfile/bshuf_block.h +++ b/src/kudu/cfile/bshuf_block.h @@ -235,6 +235,7 @@ class BShufBlockDecoder final : public BlockDecoder { num_elems_(0), compressed_size_(0), num_elems_after_padding_(0), + size_of_elem_(0), cur_idx_(0) { } diff --git a/src/kudu/cfile/cfile-test-base.h b/src/kudu/cfile/cfile-test-base.h index 7f14621..8f3a0bd 100644 --- a/src/kudu/cfile/cfile-test-base.h +++ b/src/kudu/cfile/cfile-test-base.h @@ -14,9 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. - -#ifndef KUDU_CFILE_TEST_BASE_H -#define KUDU_CFILE_TEST_BASE_H +#pragma once #include #include @@ -69,8 +67,6 @@ class DataGenerator { typedef typename DataTypeTraits::cpp_type cpp_type; DataGenerator() : - values_(NULL), - null_bitmap_(NULL), block_entries_(0), total_entries_(0) {} @@ -150,8 +146,8 @@ class DataGenerator { virtual ~DataGenerator() {} private: - gscoped_array values_; - gscoped_array null_bitmap_; + std::unique_ptr values_; + std::unique_ptr null_bitmap_; size_t block_entries_; size_t total_entries_; }; @@ -325,7 +321,7 @@ class DuplicateStringDataGenerator : public DataGenerator { char data[kItemBufferSize]; }; - gscoped_array data_buffer_; + std::unique_ptr data_buffer_; const char* format_; int num_; }; @@ -412,7 +408,7 @@ class CFileTestBase : public KuduTest { ASSERT_OK(w.Finish()); } - gscoped_ptr fs_manager_; + std::unique_ptr fs_manager_; }; @@ -567,5 +563,3 @@ void TimeReadFile(FsManager* fs_manager, const BlockId& block_id, size_t *count_ } // namespace cfile } // namespace kudu - -#endif diff --git a/src/kudu/cfile/cfile-test.cc b/src/kudu/cfile/cfile-test.cc index d8adcf0..c2c3fac 100644 --- a/src/kudu/cfile/cfile-test.cc +++ b/src/kudu/cfile/cfile-test.cc @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -27,7 +28,6 @@ #include #include -#include #include #include @@ -55,7 +55,6 @@ #include "kudu/fs/fs_manager.h" #include "kudu/fs/io_context.h" #include "kudu/gutil/casts.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/singleton.h" @@ -291,7 +290,7 @@ class TestCFile : public CFileTestBase { ASSERT_FALSE(reader->footer().incompatible_features() & IncompatibleFeatures::CHECKSUM); } - gscoped_ptr iter; + unique_ptr iter; iter.reset(IndexTreeIterator::Create(nullptr, reader.get(), reader->posidx_root())); ASSERT_OK(iter->SeekToFirst()); @@ -379,7 +378,7 @@ class TestCFile : public CFileTestBase { const fs::IOContext io_context({ "corrupted-dummy-tablet" }); opts.io_context = &io_context; RETURN_NOT_OK(CFileReader::Open(std::move(corrupt_source), std::move(opts), &reader)); - gscoped_ptr iter; + unique_ptr iter; iter.reset(IndexTreeIterator::Create(&io_context, reader.get(), reader->posidx_root())); RETURN_NOT_OK(iter->SeekToFirst()); @@ -603,7 +602,7 @@ TYPED_TEST(BitShuffleTest, TestFixedSizeReadWriteBitShuffle) { } void EncodeStringKey(const Schema &schema, const Slice& key, - gscoped_ptr *encoded_key) { + unique_ptr *encoded_key) { EncodedKeyBuilder kb(&schema); kb.AddColumnKey(&key); encoded_key->reset(kb.BuildEncodedKey()); @@ -654,7 +653,7 @@ void TestCFile::TestReadWriteStrings(EncodingType encoding, // Now try some seeks by the value instead of position ///////// - gscoped_ptr encoded_key; + unique_ptr encoded_key; bool exact; // Seek in between each key. @@ -1055,7 +1054,7 @@ TEST_P(TestCFileBothCacheMemoryTypes, TestCacheKeysAreStable) { unique_ptr reader; ASSERT_OK(CFileReader::Open(std::move(source), ReaderOptions(), &reader)); - gscoped_ptr iter; + unique_ptr iter; iter.reset(IndexTreeIterator::Create(nullptr, reader.get(), reader->posidx_root())); ASSERT_OK(iter->SeekToFirst()); diff --git a/src/kudu/cfile/cfile_reader.cc b/src/kudu/cfile/cfile_reader.cc index 5ebc193..d044a7e 100644 --- a/src/kudu/cfile/cfile_reader.cc +++ b/src/kudu/cfile/cfile_reader.cc @@ -47,7 +47,6 @@ #include "kudu/fs/error_manager.h" #include "kudu/fs/io_context.h" #include "kudu/gutil/basictypes.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/stringprintf.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/util/array_view.h" diff --git a/src/kudu/cfile/cfile_reader.h b/src/kudu/cfile/cfile_reader.h index 9df42ee..e864dfb 100644 --- a/src/kudu/cfile/cfile_reader.h +++ b/src/kudu/cfile/cfile_reader.h @@ -14,9 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. - -#ifndef KUDU_CFILE_CFILE_READER_H -#define KUDU_CFILE_CFILE_READER_H +#pragma once #include #include @@ -35,7 +33,6 @@ #include "kudu/common/rowid.h" #include "kudu/fs/block_id.h" #include "kudu/fs/block_manager.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/port.h" #include "kudu/util/compression/compression.pb.h" @@ -213,8 +210,8 @@ class CFileReader { uint8_t cfile_version_; - gscoped_ptr header_; - gscoped_ptr footer_; + std::unique_ptr header_; + std::unique_ptr footer_; const CompressionCodec* codec_; const TypeInfo *type_info_; const TypeEncodingInfo *type_encoding_info_; @@ -409,7 +406,7 @@ class CFileIterator : public ColumnIterator { struct PreparedBlock { BlockPointer dblk_ptr_; BlockHandle dblk_data_; - gscoped_ptr dblk_; + std::unique_ptr dblk_; // The rowid of the first row in this block. rowid_t first_row_idx() const { @@ -466,11 +463,11 @@ class CFileIterator : public ColumnIterator { CFileReader* reader_; - gscoped_ptr posidx_iter_; - gscoped_ptr validx_iter_; + std::unique_ptr posidx_iter_; + std::unique_ptr validx_iter_; // Decoder for the dictionary block. - gscoped_ptr dict_decoder_; + std::unique_ptr dict_decoder_; BlockHandle dict_block_handle_; // Set containing the codewords that match the predicate in a dictionary. @@ -512,5 +509,3 @@ class CFileIterator : public ColumnIterator { } // namespace cfile } // namespace kudu - -#endif diff --git a/src/kudu/cfile/cfile_writer.h b/src/kudu/cfile/cfile_writer.h index 9057138..29391bb 100644 --- a/src/kudu/cfile/cfile_writer.h +++ b/src/kudu/cfile/cfile_writer.h @@ -14,9 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. - -#ifndef KUDU_CFILE_CFILE_WRITER_H -#define KUDU_CFILE_CFILE_WRITER_H +#pragma once #include #include @@ -29,7 +27,6 @@ #include "kudu/common/rowid.h" #include "kudu/fs/block_id.h" #include "kudu/fs/block_manager.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/util/bitmap.h" #include "kudu/util/compression/compression.pb.h" @@ -234,11 +231,11 @@ class CFileWriter { // Metadata which has been added to the writer but not yet flushed. std::vector > unflushed_metadata_; - gscoped_ptr data_block_; - gscoped_ptr posidx_builder_; - gscoped_ptr validx_builder_; - gscoped_ptr null_bitmap_builder_; - gscoped_ptr block_compressor_; + std::unique_ptr data_block_; + std::unique_ptr posidx_builder_; + std::unique_ptr validx_builder_; + std::unique_ptr null_bitmap_builder_; + std::unique_ptr block_compressor_; enum State { kWriterInitialized, @@ -251,5 +248,3 @@ class CFileWriter { } // namespace cfile } // namespace kudu - -#endif diff --git a/src/kudu/cfile/encoding-test.cc b/src/kudu/cfile/encoding-test.cc index 5d6e3b7..f7eea21 100644 --- a/src/kudu/cfile/encoding-test.cc +++ b/src/kudu/cfile/encoding-test.cc @@ -44,13 +44,13 @@ #include "kudu/common/common.pb.h" #include "kudu/common/schema.h" #include "kudu/common/types.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/gutil/stringprintf.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/util/group_varint-inl.h" #include "kudu/util/hexdump.h" #include "kudu/util/int128.h" +#include "kudu/util/int128_util.h" #include "kudu/util/memory/arena.h" #include "kudu/util/random.h" #include "kudu/util/random_util.h" @@ -125,7 +125,7 @@ class TestEncoding : public KuduTest { template void TestBinarySeekByValueSmallBlock() { - gscoped_ptr opts(NewWriterOptions()); + unique_ptr opts(NewWriterOptions()); BuilderType sbb(opts.get()); // Insert "hello 0" through "hello 9" const uint kCount = 10; @@ -180,7 +180,7 @@ class TestEncoding : public KuduTest { template void TestStringSeekByValueLargeBlock() { Arena arena(1024); // TODO(todd): move to fixture? - gscoped_ptr opts(NewWriterOptions()); + unique_ptr opts(NewWriterOptions()); BinaryPrefixBlockBuilder sbb(opts.get()); const uint kCount = 1000; // Insert 'hello 000' through 'hello 999' @@ -258,7 +258,7 @@ class TestEncoding : public KuduTest { template void TestBinaryBlockRoundTrip() { - gscoped_ptr opts(NewWriterOptions()); + unique_ptr opts(NewWriterOptions()); BuilderType sbb(opts.get()); auto seed = SeedRandom(); @@ -398,7 +398,7 @@ class TestEncoding : public KuduTest { template void TestEmptyBlockEncodeDecode() { - gscoped_ptr opts(NewWriterOptions()); + unique_ptr opts(NewWriterOptions()); BlockBuilderType bb(opts.get()); Slice s = bb.Finish(0); ASSERT_GT(s.size(), 0); @@ -412,16 +412,16 @@ class TestEncoding : public KuduTest { template void TestEncodeDecodeTemplateBlockEncoder(typename TypeTraits::cpp_type* src, - uint32_t size) { + size_t size) { typedef typename TypeTraits::cpp_type CppType; const uint32_t kOrdinalPosBase = 12345; - gscoped_ptr opts(NewWriterOptions()); + unique_ptr opts(NewWriterOptions()); BlockBuilder pbb(opts.get()); pbb.Add(reinterpret_cast(src), size); Slice s = pbb.Finish(kOrdinalPosBase); - LOG(INFO)<< "Encoded size for 10k elems: " << s.size(); + LOG(INFO) << "Encoded size for 10k elems: " << s.size(); BlockDecoder pbd(s); ASSERT_OK(pbd.ParseHeader()); @@ -448,7 +448,7 @@ class TestEncoding : public KuduTest { ASSERT_EQ(0, view.nrows())<< "Should have no space left in the buffer after " << "decoding all rows"; - for (uint i = 0; i < size; i++) { + for (int i = 0; i < size; i++) { if (src[i] != decoded[i]) { FAIL()<< "Fail at index " << i << " inserted=" << src[i] << " got=" << decoded[i]; @@ -470,7 +470,7 @@ class TestEncoding : public KuduTest { // Test truncation of blocks template void TestBinaryBlockTruncation() { - gscoped_ptr opts(NewWriterOptions()); + unique_ptr opts(NewWriterOptions()); BuilderType sbb(opts.get()); const uint kCount = 10; size_t sbsize; @@ -682,80 +682,82 @@ class TestEncoding : public KuduTest { TEST_F(TestEncoding, TestPlainBlockEncoder) { const uint32_t kSize = 10000; - gscoped_ptr ints(new int32_t[kSize]); + unique_ptr ints(new int32_t[kSize]); for (int i = 0; i < kSize; i++) { ints.get()[i] = random(); } TestEncodeDecodeTemplateBlockEncoder, - PlainBlockDecoder >(ints.get(), kSize); + PlainBlockDecoder>(ints.get(), kSize); } // Test for bitshuffle block, for INT32, INT64, INT128, FLOAT, DOUBLE TEST_F(TestEncoding, TestBShufInt32BlockEncoder) { const uint32_t kSize = 10000; - gscoped_ptr ints(new int32_t[kSize]); + unique_ptr ints(new int32_t[kSize]); for (int i = 0; i < kSize; i++) { ints.get()[i] = random(); } TestEncodeDecodeTemplateBlockEncoder, - BShufBlockDecoder >(ints.get(), kSize); + BShufBlockDecoder>(ints.get(), kSize); } TEST_F(TestEncoding, TestBShufInt64BlockEncoder) { const uint32_t kSize = 10000; - gscoped_ptr ints(new int64_t[kSize]); + unique_ptr ints(new int64_t[kSize]); for (int i = 0; i < kSize; i++) { ints.get()[i] = random(); } TestEncodeDecodeTemplateBlockEncoder, - BShufBlockDecoder >(ints.get(), kSize); + BShufBlockDecoder>(ints.get(), kSize); } TEST_F(TestEncoding, TestBShufInt128BlockEncoder) { const uint32_t kSize = 10000; - gscoped_ptr ints(new int128_t[kSize]); + unique_ptr ints(new int128_t[kSize]); for (int i = 0; i < kSize; i++) { ints.get()[i] = random(); } TestEncodeDecodeTemplateBlockEncoder, - BShufBlockDecoder >(ints.get(), kSize); + BShufBlockDecoder>(ints.get(), kSize); } TEST_F(TestEncoding, TestBShufFloatBlockEncoder) { - const uint32_t kSize = 10000; + const int kSize = 10000; - gscoped_ptr floats(new float[kSize]); + unique_ptr floats(new float[kSize]); for (int i = 0; i < kSize; i++) { - floats.get()[i] = random() + static_cast(random())/INT_MAX; + floats.get()[i] = static_cast(random()) + + static_cast(random())/INT_MAX; } TestEncodeDecodeTemplateBlockEncoder, - BShufBlockDecoder >(floats.get(), kSize); + BShufBlockDecoder>(floats.get(), kSize); } TEST_F(TestEncoding, TestBShufDoubleBlockEncoder) { - const uint32_t kSize = 10000; + const int kSize = 10000; - gscoped_ptr doubles(new double[kSize]); + unique_ptr doubles(new double[kSize]); for (int i = 0; i < kSize; i++) { - doubles.get()[i] = random() + + static_cast(random())/INT_MAX; + doubles.get()[i] = static_cast(random()) + + static_cast(random())/INT_MAX; } TestEncodeDecodeTemplateBlockEncoder, - BShufBlockDecoder >(doubles.get(), kSize); + BShufBlockDecoder>(doubles.get(), kSize); } TEST_F(TestEncoding, TestRleIntBlockEncoder) { unique_ptr opts(NewWriterOptions()); RleIntBlockBuilder ibb(opts.get()); - gscoped_ptr ints(new int[10000]); + unique_ptr ints(new int[10000]); for (int i = 0; i < 10000; i++) { ints[i] = random(); } @@ -869,8 +871,8 @@ class IntEncodingTest : public TestEncoding { typedef typename TestTraits::template Classes::encoder_type encoder_type; typedef typename TestTraits::template Classes::decoder_type decoder_type; - gscoped_ptr opts(NewWriterOptions()); - gscoped_ptr ibb(new encoder_type(opts.get())); + unique_ptr opts(NewWriterOptions()); + unique_ptr ibb(new encoder_type(opts.get())); DoSeekTest(ibb.get(), num_ints, num_queries, verify); } @@ -886,8 +888,8 @@ class IntEncodingTest : public TestEncoding { typedef typename TestTraits::template Classes::encoder_type encoder_type; typedef typename TestTraits::template Classes::decoder_type decoder_type; - gscoped_ptr opts(NewWriterOptions()); - gscoped_ptr ibb(new encoder_type(opts.get())); + unique_ptr opts(NewWriterOptions()); + unique_ptr ibb(new encoder_type(opts.get())); TestIntBlockRoundTrip(ibb.get()); } }; diff --git a/src/kudu/cfile/index-test.cc b/src/kudu/cfile/index-test.cc index d1c9b03..7dcee38 100644 --- a/src/kudu/cfile/index-test.cc +++ b/src/kudu/cfile/index-test.cc @@ -18,6 +18,7 @@ #include #include #include +#include #include #include @@ -29,23 +30,25 @@ #include "kudu/common/common.pb.h" #include "kudu/common/key_encoder.h" #include "kudu/gutil/endian.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/util/faststring.h" #include "kudu/util/hexdump.h" #include "kudu/util/slice.h" #include "kudu/util/status.h" #include "kudu/util/test_macros.h" +using std::string; +using std::unique_ptr; + namespace kudu { namespace cfile { Status SearchInReaderString(const IndexBlockReader &reader, - const std::string &search_key, + const string &search_key, BlockPointer *ptr, Slice *match) { static faststring dst; - gscoped_ptr iter(reader.NewIterator()); + unique_ptr iter(reader.NewIterator()); dst.clear(); KeyEncoderTraits::Encode(search_key, &dst); Status s = iter->SeekAtOrBefore(Slice(dst)); @@ -63,7 +66,7 @@ Status SearchInReaderUint32(const IndexBlockReader &reader, static faststring dst; - gscoped_ptr iter(reader.NewIterator()); + unique_ptr iter(reader.NewIterator()); dst.clear(); KeyEncoderTraits::Encode(search_key, &dst); Status s = iter->SeekAtOrBefore(Slice(dst)); @@ -310,7 +313,7 @@ TEST(TestIndexBlock, TestIterator) { IndexBlockReader reader; ASSERT_OK(reader.Parse(s)); - gscoped_ptr iter(reader.NewIterator()); + unique_ptr iter(reader.NewIterator()); ASSERT_OK(iter->SeekToIndex(0)); ASSERT_EQ(0U, SliceAsUInt32(iter->GetCurrentKey())); ASSERT_EQ(100000U, iter->GetCurrentBlockPointer().offset()); diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc index a859a1a..8fdc40b 100644 --- a/src/kudu/client/client.cc +++ b/src/kudu/client/client.cc @@ -72,7 +72,6 @@ #include "kudu/gutil/bind.h" #include "kudu/gutil/bind_helpers.h" #include "kudu/gutil/casts.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/stl_util.h" #include "kudu/gutil/strings/numbers.h" @@ -1632,7 +1631,7 @@ void KuduScanner::Close() { // to clean up. if (!data_->next_req_.scanner_id().empty()) { CHECK(data_->proxy_); - gscoped_ptr closer(new CloseCallback); + unique_ptr closer(new CloseCallback); closer->scanner_id = data_->next_req_.scanner_id(); data_->PrepareRequest(KuduScanner::Data::CLOSE); data_->next_req_.set_close_scanner(true); diff --git a/src/kudu/client/master_rpc.cc b/src/kudu/client/master_rpc.cc index f497ead..e82cb20 100644 --- a/src/kudu/client/master_rpc.cc +++ b/src/kudu/client/master_rpc.cc @@ -20,6 +20,7 @@ #include "kudu/client/master_rpc.h" #include +#include #include #include #include @@ -32,7 +33,6 @@ #include "kudu/consensus/metadata.pb.h" #include "kudu/gutil/basictypes.h" #include "kudu/gutil/bind.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/strings/join.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/master/master.proxy.h" @@ -44,11 +44,6 @@ #include "kudu/util/status.h" #include "kudu/util/status_callback.h" -using std::pair; -using std::shared_ptr; -using std::string; -using std::vector; - using kudu::consensus::RaftPeerPB; using kudu::master::ConnectToMasterRequestPB; using kudu::master::ConnectToMasterResponsePB; @@ -62,6 +57,11 @@ using kudu::rpc::ErrorStatusPB; using kudu::rpc::Messenger; using kudu::rpc::Rpc; using kudu::rpc::RpcController; +using std::pair; +using std::shared_ptr; +using std::string; +using std::unique_ptr; +using std::vector; using strings::Substitute; namespace kudu { @@ -171,7 +171,7 @@ void ConnectToMasterRpc::SendRpcCb(const Status& status) { // will be Status::OK. // // TODO(todd): this is the most confusing code I've ever seen... - gscoped_ptr deleter(this); + unique_ptr deleter(this); Status new_status = status; rpc::RpcController* rpc = mutable_retrier()->mutable_controller(); diff --git a/src/kudu/client/meta_cache.cc b/src/kudu/client/meta_cache.cc index a00ff05..531ef76 100644 --- a/src/kudu/client/meta_cache.cc +++ b/src/kudu/client/meta_cache.cc @@ -18,6 +18,7 @@ #include "kudu/client/meta_cache.h" #include +#include #include #include #include @@ -40,7 +41,6 @@ #include "kudu/gutil/bind.h" #include "kudu/gutil/bind_helpers.h" #include "kudu/gutil/callback.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/port.h" #include "kudu/gutil/stl_util.h" @@ -57,28 +57,25 @@ #include "kudu/util/net/sockaddr.h" #include "kudu/util/pb_util.h" -using std::map; +using kudu::consensus::RaftPeerPB; +using kudu::master::ANY_REPLICA; +using kudu::master::GetTableLocationsRequestPB; +using kudu::master::GetTableLocationsResponsePB; +using kudu::master::MasterServiceProxy; +using kudu::master::TabletLocationsPB; +using kudu::master::TSInfoPB; +using kudu::rpc::BackoffType; +using kudu::rpc::CredentialsPolicy; +using kudu::tserver::TabletServerServiceProxy; using std::set; using std::shared_ptr; using std::string; +using std::unique_ptr; using std::vector; using strings::Substitute; namespace kudu { - -using consensus::RaftPeerPB; -using master::ANY_REPLICA; -using master::GetTableLocationsRequestPB; -using master::GetTableLocationsResponsePB; -using master::MasterServiceProxy; -using master::TabletLocationsPB; -using master::TSInfoPB; -using rpc::BackoffType; -using rpc::CredentialsPolicy; -using tserver::TabletServerServiceProxy; - namespace client { - namespace internal { RemoteTabletServer::RemoteTabletServer(const master::TSInfoPB& pb) @@ -91,7 +88,7 @@ void RemoteTabletServer::DnsResolutionFinished(const HostPort& hp, KuduClient* client, const StatusCallback& user_callback, const Status &result_status) { - gscoped_ptr > scoped_addrs(addrs); + unique_ptr> scoped_addrs(addrs); Status s = result_status; @@ -729,7 +726,7 @@ void LookupRpc::ResetMasterLeaderAndRetry(CredentialsPolicy creds_policy) { void LookupRpc::SendRpcCb(const Status& status) { // If we exit and haven't scheduled a retry, this object should delete // itself. - gscoped_ptr delete_me(this); + unique_ptr delete_me(this); // Check for generic errors. Status new_status = status; diff --git a/src/kudu/client/predicate-test.cc b/src/kudu/client/predicate-test.cc index 1ed3904..150775a 100644 --- a/src/kudu/client/predicate-test.cc +++ b/src/kudu/client/predicate-test.cc @@ -40,7 +40,6 @@ #include "kudu/client/value.h" #include "kudu/client/write_op.h" #include "kudu/common/partial_row.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/integral_types.h" #include "kudu/gutil/strings/escaping.h" #include "kudu/gutil/strings/substitute.h" @@ -571,7 +570,7 @@ class PredicateTest : public KuduTest { } shared_ptr client_; - gscoped_ptr cluster_; + unique_ptr cluster_; }; TEST_F(PredicateTest, TestBoolPredicates) { diff --git a/src/kudu/client/scan_predicate.cc b/src/kudu/client/scan_predicate.cc index de32e99..94abe4c 100644 --- a/src/kudu/client/scan_predicate.cc +++ b/src/kudu/client/scan_predicate.cc @@ -32,7 +32,6 @@ #include "kudu/common/scan_spec.h" #include "kudu/common/schema.h" #include "kudu/common/types.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/stl_util.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/util/block_bloom_filter.h" diff --git a/src/kudu/client/scan_token-test.cc b/src/kudu/client/scan_token-test.cc index ebfb160..c247ddc 100644 --- a/src/kudu/client/scan_token-test.cc +++ b/src/kudu/client/scan_token-test.cc @@ -42,7 +42,6 @@ #include "kudu/common/common.pb.h" #include "kudu/common/partial_row.h" #include "kudu/common/wire_protocol.pb.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/stl_util.h" #include "kudu/mini-cluster/internal_mini_cluster.h" #include "kudu/tserver/mini_tablet_server.h" @@ -141,7 +140,7 @@ class ScanTokenTest : public KuduTest { } shared_ptr client_; - gscoped_ptr cluster_; + unique_ptr cluster_; }; TEST_F(ScanTokenTest, TestScanTokens) { diff --git a/src/kudu/client/schema.cc b/src/kudu/client/schema.cc index ebb1eb3..1a9ea4c 100644 --- a/src/kudu/client/schema.cc +++ b/src/kudu/client/schema.cc @@ -35,7 +35,6 @@ #include "kudu/common/schema.h" #include "kudu/common/types.h" #include "kudu/gutil/casts.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/strings/substitute.h" @@ -62,6 +61,7 @@ MAKE_ENUM_LIMITS(kudu::client::KuduColumnSchema::DataType, kudu::client::KuduColumnSchema::BOOL); using std::string; +using std::unique_ptr; using std::unordered_map; using std::vector; using strings::Substitute; @@ -841,10 +841,11 @@ void KuduSchema::CopyFrom(const KuduSchema& other) { Status KuduSchema::Reset(const vector& columns, int key_columns) { vector cols_private; - for (const KuduColumnSchema& col : columns) { - cols_private.push_back(*col.col_); + cols_private.reserve(columns.size()); + for (const auto& col : columns) { + cols_private.emplace_back(*col.col_); } - gscoped_ptr new_schema(new Schema()); + unique_ptr new_schema(new Schema()); RETURN_NOT_OK(new_schema->Reset(cols_private, key_columns)); delete schema_; diff --git a/src/kudu/client/write_op.cc b/src/kudu/client/write_op.cc index 83d7e19..5beed13 100644 --- a/src/kudu/client/write_op.cc +++ b/src/kudu/client/write_op.cc @@ -17,6 +17,7 @@ #include "kudu/client/write_op.h" +#include #include #include @@ -29,15 +30,15 @@ #include "kudu/common/schema.h" #include "kudu/common/types.h" #include "kudu/common/wire_protocol.pb.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/util/bitmap.h" #include "kudu/util/slice.h" +using kudu::client::sp::shared_ptr; +using std::unique_ptr; + namespace kudu { namespace client { -using sp::shared_ptr; - RowOperationsPB_Type ToInternalWriteType(KuduWriteOperation::Type type) { switch (type) { case KuduWriteOperation::INSERT: return RowOperationsPB_Type_INSERT; @@ -66,7 +67,7 @@ EncodedKey* KuduWriteOperation::CreateKey() const { for (int i = 0; i < row.schema()->num_key_columns(); i++) { kb.AddColumnKey(row.cell_ptr(i)); } - gscoped_ptr key(kb.BuildEncodedKey()); + unique_ptr key(kb.BuildEncodedKey()); return key.release(); } diff --git a/src/kudu/codegen/code_cache.cc b/src/kudu/codegen/code_cache.cc index 9213a63..deec826 100644 --- a/src/kudu/codegen/code_cache.cc +++ b/src/kudu/codegen/code_cache.cc @@ -24,7 +24,6 @@ #include #include "kudu/codegen/jit_wrapper.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/gutil/ref_counted.h" #include "kudu/util/cache.h" diff --git a/src/kudu/codegen/code_cache.h b/src/kudu/codegen/code_cache.h index 0036cf4..c137ef0 100644 --- a/src/kudu/codegen/code_cache.h +++ b/src/kudu/codegen/code_cache.h @@ -14,13 +14,11 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. - -#ifndef KUDU_CODEGEN_CODE_CACHE_H -#define KUDU_CODEGEN_CODE_CACHE_H +#pragma once #include +#include -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/ref_counted.h" @@ -83,13 +81,11 @@ class CodeCache { private: class EvictionCallback; - gscoped_ptr eviction_callback_; - gscoped_ptr cache_; + std::unique_ptr eviction_callback_; + std::unique_ptr cache_; DISALLOW_COPY_AND_ASSIGN(CodeCache); }; } // namespace codegen } // namespace kudu - -#endif diff --git a/src/kudu/codegen/code_generator.cc b/src/kudu/codegen/code_generator.cc index e249d88..7e419ab 100644 --- a/src/kudu/codegen/code_generator.cc +++ b/src/kudu/codegen/code_generator.cc @@ -21,6 +21,7 @@ #include #include #include +#include #include #include @@ -40,7 +41,6 @@ #include #include "kudu/codegen/row_projector.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/once.h" #include "kudu/gutil/ref_counted.h" @@ -78,6 +78,7 @@ using llvm::Target; using llvm::TargetMachine; using llvm::Triple; using std::string; +using std::unique_ptr; namespace kudu { @@ -131,7 +132,7 @@ int DumpAsm(FuncPtr fptr, const TargetMachine& tm, std::ostream* out, int max_in MCContext context(asm_info, register_info, nullptr); - gscoped_ptr disas( + unique_ptr disas( CHECK_NOTNULL(tm.getTarget().createMCDisassembler(subtarget_info, context))); // LLVM uses these completely undocumented magic syntax constants which had @@ -142,7 +143,7 @@ int DumpAsm(FuncPtr fptr, const TargetMachine& tm, std::ostream* out, int max_in // This only has meaning for a *given* target, but at least the 0th syntax // will always be defined, so that's what we use. static const unsigned kSyntaxVariant = 0; - gscoped_ptr printer( + unique_ptr printer( CHECK_NOTNULL(tm.getTarget().createMCInstPrinter(triple, kSyntaxVariant, *asm_info, instr_info, *register_info))); diff --git a/src/kudu/codegen/codegen-test.cc b/src/kudu/codegen/codegen-test.cc index 78f0dd9..76af76a 100644 --- a/src/kudu/codegen/codegen-test.cc +++ b/src/kudu/codegen/codegen-test.cc @@ -18,8 +18,10 @@ #include #include #include +#include #include #include +#include #include #include @@ -35,7 +37,6 @@ #include "kudu/common/row.h" #include "kudu/common/rowblock.h" #include "kudu/common/schema.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/singleton.h" #include "kudu/util/logging_test_util.h" @@ -48,6 +49,7 @@ #include "kudu/util/test_util.h" using std::string; +using std::unique_ptr; using std::vector; DECLARE_bool(codegen_dump_mc); @@ -114,7 +116,7 @@ class CodegenTest : public KuduTest { template void TestProjection(const Schema* proj); // Generates a new row projector for the given projection schema. - Status Generate(const Schema* proj, gscoped_ptr* out); + Status Generate(const Schema* proj, unique_ptr* out); enum { // Base schema column indices @@ -151,9 +153,9 @@ class CodegenTest : public KuduTest { codegen::CodeGenerator generator_; Random random_; - gscoped_ptr test_rows_[kNumTestRows]; + unique_ptr test_rows_[kNumTestRows]; Arena projections_arena_; - gscoped_ptr test_rows_arena_; + unique_ptr test_rows_arena_; }; namespace { @@ -211,7 +213,7 @@ void CodegenTest::ProjectTestRows(RowProjectorType* rp, RowBlock* rb) { template void CodegenTest::TestProjection(const Schema* proj) { - gscoped_ptr with; + unique_ptr with; ASSERT_OK(Generate(proj, &with)); NoCodegenRP without(&base_, proj); ASSERT_OK(without.Init()); @@ -228,7 +230,7 @@ void CodegenTest::TestProjection(const Schema* proj) { CheckRowBlocksEqual(&rb_with, &rb_without, "Codegen", "Expected"); } -Status CodegenTest::Generate(const Schema* proj, gscoped_ptr* out) { +Status CodegenTest::Generate(const Schema* proj, unique_ptr* out) { scoped_refptr functions; RETURN_NOT_OK(generator_.CompileRowProjector(base_, *proj, &functions)); out->reset(new CodegenRP(&base_, proj, functions)); @@ -247,7 +249,7 @@ Status CodegenTest::CreatePartialSchema(const vector& col_indexes, TEST_F(CodegenTest, ObservablesTest) { // Test when not identity Schema proj = base_.CreateKeyProjection(); - gscoped_ptr with; + unique_ptr with; CHECK_OK(Generate(&proj, &with)); NoCodegenRP without(&base_, &proj); ASSERT_OK(without.Init()); @@ -258,7 +260,7 @@ TEST_F(CodegenTest, ObservablesTest) { // Test when identity Schema iproj = *&base_; - gscoped_ptr iwith; + unique_ptr iwith; CHECK_OK(Generate(&iproj, &iwith)); NoCodegenRP iwithout(&base_, &iproj); ASSERT_OK(iwithout.Init()); @@ -401,7 +403,7 @@ TEST_F(CodegenTest, TestCodeCache) { Schema projection; ASSERT_OK(CreatePartialSchema(perm, &projection)); - gscoped_ptr projector; + unique_ptr projector; if (cm->RequestRowProjector(&base_, &projection, &projector)) { num_hits++; } diff --git a/src/kudu/codegen/compilation_manager.cc b/src/kudu/codegen/compilation_manager.cc index 51263d6..2e45be4 100644 --- a/src/kudu/codegen/compilation_manager.cc +++ b/src/kudu/codegen/compilation_manager.cc @@ -34,7 +34,6 @@ #include "kudu/gutil/bind_helpers.h" #include "kudu/gutil/callback.h" #include "kudu/gutil/casts.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/ref_counted.h" #include "kudu/util/faststring.h" #include "kudu/util/flag_tags.h" @@ -46,6 +45,7 @@ #include "kudu/util/threadpool.h" using std::shared_ptr; +using std::unique_ptr; DEFINE_bool(codegen_time_compilation, false, "Whether to print time that each code " "generation request took."); @@ -176,7 +176,7 @@ Status CompilationManager::StartInstrumentation(const scoped_refptr* out) { + unique_ptr* out) { faststring key; Status s = RowProjectorFunctions::EncodeKey(*base_schema, *projection, &key); WARN_NOT_OK(s, "RowProjector compilation request failed"); diff --git a/src/kudu/codegen/compilation_manager.h b/src/kudu/codegen/compilation_manager.h index 4cca427..7aa05df 100644 --- a/src/kudu/codegen/compilation_manager.h +++ b/src/kudu/codegen/compilation_manager.h @@ -21,7 +21,6 @@ #include "kudu/codegen/code_generator.h" #include "kudu/codegen/code_cache.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/singleton.h" @@ -75,7 +74,7 @@ class CompilationManager { // Does not write to 'out' if false is returned. bool RequestRowProjector(const Schema* base_schema, const Schema* projection, - gscoped_ptr* out); + std::unique_ptr* out); // Waits for all asynchronous compilation tasks to finish. void Wait(); diff --git a/src/kudu/consensus/consensus-test-util.h b/src/kudu/consensus/consensus-test-util.h index a315028..65fd16e 100644 --- a/src/kudu/consensus/consensus-test-util.h +++ b/src/kudu/consensus/consensus-test-util.h @@ -63,11 +63,11 @@ namespace kudu { namespace consensus { -inline gscoped_ptr CreateDummyReplicate(int64_t term, - int64_t index, - const Timestamp& timestamp, - int64_t payload_size) { - gscoped_ptr msg(new ReplicateMsg); +inline std::unique_ptr CreateDummyReplicate(int64_t term, + int64_t index, + const Timestamp& timestamp, + int64_t payload_size) { + std::unique_ptr msg(new ReplicateMsg); OpId* id = msg->mutable_id(); id->set_term(term); id->set_index(index); @@ -75,7 +75,7 @@ inline gscoped_ptr CreateDummyReplicate(int64_t term, msg->set_op_type(NO_OP); msg->mutable_noop_request()->mutable_payload_for_tests()->resize(payload_size); msg->set_timestamp(timestamp.ToUint64()); - return std::move(msg); + return msg; } // Returns RaftPeerPB with given UUID and obviously-fake hostname / port combo. @@ -250,7 +250,7 @@ class DelayablePeerProxy : public TestPeerProxy { } protected: - gscoped_ptr const proxy_; + std::unique_ptr const proxy_; bool delay_response_; // Protected by lock_. CountDownLatch latch_; }; @@ -392,7 +392,7 @@ class NoOpTestPeerProxyFactory : public PeerProxyFactory { } Status NewProxy(const consensus::RaftPeerPB& peer_pb, - gscoped_ptr* proxy) override { + std::unique_ptr* proxy) override { proxy->reset(new NoOpTestPeerProxy(pool_.get(), peer_pb)); return Status::OK(); } @@ -613,7 +613,7 @@ class LocalTestPeerProxyFactory : public PeerProxyFactory { } Status NewProxy(const consensus::RaftPeerPB& peer_pb, - gscoped_ptr* proxy) override { + std::unique_ptr* proxy) override { LocalTestPeerProxy* new_proxy = new LocalTestPeerProxy(peer_pb.permanent_uuid(), pool_.get(), peers_); @@ -676,7 +676,7 @@ class TestDriver { // The commit message has the exact same type of the replicate message, but // no content. void Apply() { - gscoped_ptr msg(new CommitMsg); + std::unique_ptr msg(new CommitMsg); msg->set_op_type(round_->replicate_msg()->op_type()); msg->mutable_commited_op_id()->CopyFrom(round_->id()); CHECK_OK(log_->AsyncAppendCommit(std::move(msg), diff --git a/src/kudu/consensus/consensus_peers-test.cc b/src/kudu/consensus/consensus_peers-test.cc index 4a73948..81ec3b5 100644 --- a/src/kudu/consensus/consensus_peers-test.cc +++ b/src/kudu/consensus/consensus_peers-test.cc @@ -19,7 +19,6 @@ #include #include -#include #include #include @@ -41,7 +40,6 @@ #include "kudu/consensus/opid_util.h" #include "kudu/consensus/time_manager.h" #include "kudu/fs/fs_manager.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/gutil/ref_counted.h" #include "kudu/rpc/messenger.h" @@ -138,7 +136,7 @@ class ConsensusPeersTest : public KuduTest { peer_pb.set_member_type(RaftPeerPB::VOTER); auto proxy_ptr = new DelayablePeerProxy( raft_pool_.get(), new NoOpTestPeerProxy(raft_pool_.get(), peer_pb)); - gscoped_ptr proxy(proxy_ptr); + unique_ptr proxy(proxy_ptr); CHECK_OK(Peer::NewRemotePeer(std::move(peer_pb), kTabletId, kLeaderUuid, @@ -283,7 +281,7 @@ TEST_F(ConsensusPeersTest, TestCloseWhenRemotePeerDoesntMakeProgress) { kLeaderUuid, message_queue_.get(), raft_pool_token_.get(), - gscoped_ptr(mock_proxy), + unique_ptr(mock_proxy), messenger_, &peer)); @@ -321,7 +319,7 @@ TEST_F(ConsensusPeersTest, TestDontSendOneRpcPerWriteWhenPeerIsDown) { kLeaderUuid, message_queue_.get(), raft_pool_token_.get(), - gscoped_ptr(mock_proxy), + unique_ptr(mock_proxy), messenger_, &peer)); diff --git a/src/kudu/consensus/consensus_peers.cc b/src/kudu/consensus/consensus_peers.cc index 554ff4d..d759ca1 100644 --- a/src/kudu/consensus/consensus_peers.cc +++ b/src/kudu/consensus/consensus_peers.cc @@ -23,7 +23,6 @@ #include #include #include -#include #include #include @@ -37,7 +36,6 @@ #include "kudu/consensus/consensus_queue.h" #include "kudu/consensus/metadata.pb.h" #include "kudu/consensus/opid_util.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/port.h" #include "kudu/gutil/strings/substitute.h" @@ -109,7 +107,7 @@ Status Peer::NewRemotePeer(RaftPeerPB peer_pb, string leader_uuid, PeerMessageQueue* queue, ThreadPoolToken* raft_pool_token, - gscoped_ptr proxy, + unique_ptr proxy, shared_ptr messenger, shared_ptr* peer) { @@ -130,7 +128,7 @@ Peer::Peer(RaftPeerPB peer_pb, string leader_uuid, PeerMessageQueue* queue, ThreadPoolToken* raft_pool_token, - gscoped_ptr proxy, + unique_ptr proxy, shared_ptr messenger) : tablet_id_(std::move(tablet_id)), leader_uuid_(std::move(leader_uuid)), @@ -510,7 +508,7 @@ Peer::~Peer() { } RpcPeerProxy::RpcPeerProxy(HostPort hostport, - gscoped_ptr consensus_proxy) + unique_ptr consensus_proxy) : hostport_(std::move(hostport)), consensus_proxy_(std::move(DCHECK_NOTNULL(consensus_proxy))) { } @@ -556,7 +554,7 @@ Status CreateConsensusServiceProxyForHost( const HostPort& hostport, const shared_ptr& messenger, DnsResolver* dns_resolver, - gscoped_ptr* new_proxy) { + unique_ptr* new_proxy) { vector addrs; RETURN_NOT_OK(dns_resolver->ResolveAddresses(hostport, &addrs)); if (addrs.size() > 1) { @@ -577,10 +575,10 @@ RpcPeerProxyFactory::RpcPeerProxyFactory(shared_ptr messenger, } Status RpcPeerProxyFactory::NewProxy(const RaftPeerPB& peer_pb, - gscoped_ptr* proxy) { + unique_ptr* proxy) { HostPort hostport; RETURN_NOT_OK(HostPortFromPB(peer_pb.last_known_addr(), &hostport)); - gscoped_ptr new_proxy; + unique_ptr new_proxy; RETURN_NOT_OK(CreateConsensusServiceProxyForHost( hostport, messenger_, dns_resolver_, &new_proxy)); proxy->reset(new RpcPeerProxy(std::move(hostport), std::move(new_proxy))); @@ -594,7 +592,7 @@ Status SetPermanentUuidForRemotePeer( DCHECK(!remote_peer->has_permanent_uuid()); HostPort hostport; RETURN_NOT_OK(HostPortFromPB(remote_peer->last_known_addr(), &hostport)); - gscoped_ptr proxy; + unique_ptr proxy; RETURN_NOT_OK(CreateConsensusServiceProxyForHost( hostport, messenger, resolver, &proxy)); GetNodeInstanceRequestPB req; diff --git a/src/kudu/consensus/consensus_peers.h b/src/kudu/consensus/consensus_peers.h index 0b4f24b..887b147 100644 --- a/src/kudu/consensus/consensus_peers.h +++ b/src/kudu/consensus/consensus_peers.h @@ -14,9 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. - -#ifndef KUDU_CONSENSUS_CONSENSUS_PEERS_H_ -#define KUDU_CONSENSUS_CONSENSUS_PEERS_H_ +#pragma once #include #include @@ -30,7 +28,6 @@ #include "kudu/consensus/consensus.proxy.h" #include "kudu/consensus/metadata.pb.h" #include "kudu/consensus/ref_counted_replicate.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/rpc/response_callback.h" #include "kudu/rpc/rpc_controller.h" #include "kudu/util/locks.h" @@ -114,7 +111,7 @@ class Peer : public std::enable_shared_from_this { std::string leader_uuid, PeerMessageQueue* queue, ThreadPoolToken* raft_pool_token, - gscoped_ptr proxy, + std::unique_ptr proxy, std::shared_ptr messenger, std::shared_ptr* peer); @@ -124,7 +121,7 @@ class Peer : public std::enable_shared_from_this { std::string leader_uuid, PeerMessageQueue* queue, ThreadPoolToken* raft_pool_token, - gscoped_ptr proxy, + std::unique_ptr proxy, std::shared_ptr messenger); void SendNextRequest(bool even_if_queue_empty); @@ -161,7 +158,7 @@ class Peer : public std::enable_shared_from_this { RaftPeerPB peer_pb_; - gscoped_ptr proxy_; + std::unique_ptr proxy_; PeerMessageQueue* queue_; uint64_t failed_attempts_; @@ -241,7 +238,7 @@ class PeerProxyFactory { public: virtual Status NewProxy(const RaftPeerPB& peer_pb, - gscoped_ptr* proxy) = 0; + std::unique_ptr* proxy) = 0; virtual ~PeerProxyFactory() {} @@ -252,7 +249,7 @@ class PeerProxyFactory { class RpcPeerProxy : public PeerProxy { public: RpcPeerProxy(HostPort hostport, - gscoped_ptr consensus_proxy); + std::unique_ptr consensus_proxy); void UpdateAsync(const ConsensusRequestPB& request, ConsensusResponsePB* response, @@ -278,7 +275,7 @@ class RpcPeerProxy : public PeerProxy { private: const HostPort hostport_; - gscoped_ptr consensus_proxy_; + std::unique_ptr consensus_proxy_; }; // PeerProxyFactory implementation that generates RPCPeerProxies @@ -289,7 +286,7 @@ class RpcPeerProxyFactory : public PeerProxyFactory { ~RpcPeerProxyFactory() = default; Status NewProxy(const RaftPeerPB& peer_pb, - gscoped_ptr* proxy) override; + std::unique_ptr* proxy) override; const std::shared_ptr& messenger() const override { return messenger_; @@ -310,5 +307,3 @@ Status SetPermanentUuidForRemotePeer( } // namespace consensus } // namespace kudu - -#endif /* KUDU_CONSENSUS_CONSENSUS_PEERS_H_ */ diff --git a/src/kudu/consensus/consensus_queue-test.cc b/src/kudu/consensus/consensus_queue-test.cc index b77c8db..c929c1d 100644 --- a/src/kudu/consensus/consensus_queue-test.cc +++ b/src/kudu/consensus/consensus_queue-test.cc @@ -51,7 +51,6 @@ #include "kudu/consensus/ref_counted_replicate.h" #include "kudu/consensus/time_manager.h" #include "kudu/fs/fs_manager.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/gutil/ref_counted.h" #include "kudu/util/async_util.h" @@ -240,11 +239,11 @@ class ConsensusQueueTest : public KuduTest { MetricRegistry metric_registry_; scoped_refptr metric_entity_server_; scoped_refptr metric_entity_tablet_; - gscoped_ptr fs_manager_; + unique_ptr fs_manager_; scoped_refptr log_; unique_ptr raft_pool_; unique_ptr time_manager_; - gscoped_ptr queue_; + unique_ptr queue_; scoped_refptr registry_; unique_ptr clock_; atomic quiescing_; diff --git a/src/kudu/consensus/consensus_queue.h b/src/kudu/consensus/consensus_queue.h index def0ba3..384f3a5 100644 --- a/src/kudu/consensus/consensus_queue.h +++ b/src/kudu/consensus/consensus_queue.h @@ -33,7 +33,6 @@ #include "kudu/consensus/metadata.pb.h" #include "kudu/consensus/opid.pb.h" #include "kudu/consensus/ref_counted_replicate.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/threading/thread_collision_warner.h" #include "kudu/util/locks.h" @@ -438,7 +437,7 @@ class PeerMessageQueue { Mode mode; // The currently-active raft config. Only set if in LEADER mode. - gscoped_ptr active_config; + std::unique_ptr active_config; std::string ToString() const; }; diff --git a/src/kudu/consensus/leader_election-test.cc b/src/kudu/consensus/leader_election-test.cc index e6e601c..101ec13 100644 --- a/src/kudu/consensus/leader_election-test.cc +++ b/src/kudu/consensus/leader_election-test.cc @@ -35,7 +35,6 @@ #include "kudu/consensus/metadata.pb.h" #include "kudu/consensus/raft_consensus.h" #include "kudu/gutil/casts.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/stl_util.h" @@ -92,7 +91,7 @@ class FromMapPeerProxyFactory : public PeerProxyFactory { } Status NewProxy(const RaftPeerPB& peer_pb, - gscoped_ptr* proxy) override { + unique_ptr* proxy) override { PeerProxy* proxy_ptr = FindPtrOrNull(*proxy_map_, peer_pb.permanent_uuid()); if (!proxy_ptr) return Status::NotFound("no proxy for peer"); proxy->reset(proxy_ptr); @@ -146,11 +145,11 @@ class LeaderElectionTest : public KuduTest { RaftConfigPB config_; ProxyMap proxies_; - gscoped_ptr proxy_factory_; + unique_ptr proxy_factory_; unique_ptr pool_; CountDownLatch latch_; - gscoped_ptr result_; + unique_ptr result_; }; void LeaderElectionTest::ElectionCallback(const ElectionResult& result) { diff --git a/src/kudu/consensus/leader_election.h b/src/kudu/consensus/leader_election.h index c42e7c6..c7fe8f8 100644 --- a/src/kudu/consensus/leader_election.h +++ b/src/kudu/consensus/leader_election.h @@ -14,7 +14,6 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. - #pragma once #include @@ -28,7 +27,6 @@ #include "kudu/consensus/consensus_peers.h" #include "kudu/consensus/metadata.pb.h" #include "kudu/consensus/raft_consensus.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/ref_counted.h" #include "kudu/rpc/rpc_controller.h" @@ -162,7 +160,7 @@ class LeaderElection : public RefCountedThreadSafe { struct VoterState { std::string peer_uuid; - gscoped_ptr proxy; + std::unique_ptr proxy; // If constructing the proxy failed (e.g. due to a DNS resolution issue) // then 'proxy' will be NULL, and 'proxy_status' will contain the error. diff --git a/src/kudu/consensus/log-test-base.h b/src/kudu/consensus/log-test-base.h index 1ae43ae..b7728bd 100644 --- a/src/kudu/consensus/log-test-base.h +++ b/src/kudu/consensus/log-test-base.h @@ -36,7 +36,6 @@ #include "kudu/consensus/metadata.pb.h" #include "kudu/consensus/opid_util.h" #include "kudu/fs/fs_manager.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/stl_util.h" #include "kudu/gutil/stringprintf.h" #include "kudu/gutil/strings/substitute.h" @@ -279,7 +278,7 @@ class LogTestBase : public KuduTest { int rs_id, int dms_id, bool sync = APPEND_SYNC) { - gscoped_ptr commit(new consensus::CommitMsg); + std::unique_ptr commit(new consensus::CommitMsg); commit->set_op_type(consensus::WRITE_OP); commit->mutable_commited_op_id()->CopyFrom(original_opid); @@ -300,7 +299,7 @@ class LogTestBase : public KuduTest { // indicating that the associated writes failed due to // "NotFound" errors. Status AppendCommitWithNotFoundOpResults(const consensus::OpId& original_opid) { - gscoped_ptr commit(new consensus::CommitMsg); + std::unique_ptr commit(new consensus::CommitMsg); commit->set_op_type(consensus::WRITE_OP); commit->mutable_commited_op_id()->CopyFrom(original_opid); @@ -314,7 +313,7 @@ class LogTestBase : public KuduTest { return AppendCommit(std::move(commit)); } - Status AppendCommit(gscoped_ptr commit, + Status AppendCommit(std::unique_ptr commit, bool sync = APPEND_SYNC) { if (sync) { Synchronizer s; diff --git a/src/kudu/consensus/log-test.cc b/src/kudu/consensus/log-test.cc index 3800554..f3f2b2f 100644 --- a/src/kudu/consensus/log-test.cc +++ b/src/kudu/consensus/log-test.cc @@ -24,7 +24,6 @@ #include #include #include -#include #include #include @@ -43,7 +42,6 @@ #include "kudu/consensus/log_util.h" #include "kudu/consensus/opid.pb.h" #include "kudu/consensus/opid_util.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/stl_util.h" #include "kudu/gutil/strings/substitute.h" @@ -928,7 +926,7 @@ void LogTest::AppendTestSequence(const vector& seq) { } case TestLogSequenceElem::COMMIT: { - gscoped_ptr commit(new CommitMsg); + unique_ptr commit(new CommitMsg); commit->set_op_type(NO_OP); commit->mutable_commited_op_id()->CopyFrom(e.id); Synchronizer s; diff --git a/src/kudu/consensus/log.cc b/src/kudu/consensus/log.cc index ed6057a..48e4be6 100644 --- a/src/kudu/consensus/log.cc +++ b/src/kudu/consensus/log.cc @@ -875,7 +875,7 @@ Status Log::AsyncAppendReplicates(const vector& replicates, return AsyncAppend(std::move(batch), callback); } -Status Log::AsyncAppendCommit(gscoped_ptr commit_msg, +Status Log::AsyncAppendCommit(unique_ptr commit_msg, const StatusCallback& callback) { MAYBE_FAULT(FLAGS_fault_crash_before_append_commit); diff --git a/src/kudu/consensus/log.h b/src/kudu/consensus/log.h index cd2907b..ceeba63 100644 --- a/src/kudu/consensus/log.h +++ b/src/kudu/consensus/log.h @@ -37,7 +37,6 @@ #include "kudu/consensus/opid.pb.h" #include "kudu/consensus/ref_counted_replicate.h" #include "kudu/gutil/callback.h" // IWYU pragma: keep -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/ref_counted.h" #include "kudu/util/blocking_queue.h" @@ -302,10 +301,9 @@ class Log : public RefCountedThreadSafe { // Append the given commit message, asynchronously. // // Returns a bad status if the log is already shut down. - Status AsyncAppendCommit(gscoped_ptr commit_msg, + Status AsyncAppendCommit(std::unique_ptr commit_msg, const StatusCallback& callback); - // Blocks the current thread until all the entries in the log queue // are flushed and fsynced (if fsync of log entries is enabled). Status WaitUntilAllFlushed(); diff --git a/src/kudu/consensus/log_cache-test.cc b/src/kudu/consensus/log_cache-test.cc index 7057555..3972696 100644 --- a/src/kudu/consensus/log_cache-test.cc +++ b/src/kudu/consensus/log_cache-test.cc @@ -44,7 +44,6 @@ #include "kudu/consensus/ref_counted_replicate.h" #include "kudu/fs/fs_manager.h" #include "kudu/gutil/bind.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/strings/substitute.h" diff --git a/src/kudu/consensus/peer_manager.cc b/src/kudu/consensus/peer_manager.cc index 370f130..4608e25 100644 --- a/src/kudu/consensus/peer_manager.cc +++ b/src/kudu/consensus/peer_manager.cc @@ -20,7 +20,6 @@ #include #include #include -#include #include #include @@ -28,7 +27,6 @@ #include "kudu/consensus/consensus_peers.h" #include "kudu/consensus/log.h" #include "kudu/consensus/metadata.pb.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/port.h" #include "kudu/gutil/strings/substitute.h" @@ -36,13 +34,16 @@ using kudu::log::Log; using kudu::pb_util::SecureShortDebugString; +using std::shared_ptr; +using std::string; +using std::unique_ptr; using strings::Substitute; namespace kudu { namespace consensus { -PeerManager::PeerManager(std::string tablet_id, - std::string local_uuid, +PeerManager::PeerManager(string tablet_id, + string local_uuid, PeerProxyFactory* peer_proxy_factory, PeerMessageQueue* queue, ThreadPoolToken* raft_pool_token, @@ -73,11 +74,11 @@ Status PeerManager::UpdateRaftConfig(const RaftConfigPB& config) { } VLOG(1) << GetLogPrefix() << "Adding remote peer. Peer: " << SecureShortDebugString(peer_pb); - gscoped_ptr peer_proxy; + unique_ptr peer_proxy; RETURN_NOT_OK_PREPEND(peer_proxy_factory_->NewProxy(peer_pb, &peer_proxy), "Could not obtain a remote proxy to the peer."); - std::shared_ptr remote_peer; + shared_ptr remote_peer; RETURN_NOT_OK(Peer::NewRemotePeer(peer_pb, tablet_id_, local_uuid_, @@ -107,8 +108,8 @@ void PeerManager::SignalRequest(bool force_if_queue_empty) { } } -Status PeerManager::StartElection(const std::string& uuid) { - std::shared_ptr peer; +Status PeerManager::StartElection(const string& uuid) { + shared_ptr peer; { std::lock_guard lock(lock_); peer = FindPtrOrNull(peers_, uuid); @@ -130,7 +131,7 @@ void PeerManager::Close() { } } -std::string PeerManager::GetLogPrefix() const { +string PeerManager::GetLogPrefix() const { return Substitute("T $0 P $1: ", tablet_id_, local_uuid_); } diff --git a/src/kudu/consensus/raft_consensus.cc b/src/kudu/consensus/raft_consensus.cc index bf10e63..2aeffb6 100644 --- a/src/kudu/consensus/raft_consensus.cc +++ b/src/kudu/consensus/raft_consensus.cc @@ -25,7 +25,6 @@ #include #include #include -#include #include #include @@ -644,7 +643,7 @@ void RaftConsensus::EndLeaderTransferPeriod() { } scoped_refptr RaftConsensus::NewRound( - gscoped_ptr replicate_msg, + unique_ptr replicate_msg, ConsensusReplicatedCallback replicated_cb) { return make_scoped_refptr(new ConsensusRound(this, std::move(replicate_msg), @@ -2818,7 +2817,7 @@ void RaftConsensus::NonTxRoundReplicationFinished(ConsensusRound* round, VLOG_WITH_PREFIX_UNLOCKED(1) << "Committing " << op_type_str << " with op id " << round->id(); round_handler_->FinishConsensusOnlyRound(round); - gscoped_ptr commit_msg(new CommitMsg); + unique_ptr commit_msg(new CommitMsg); commit_msg->set_op_type(round->replicate_msg()->op_type()); *commit_msg->mutable_commited_op_id() = round->id(); @@ -3223,7 +3222,7 @@ ConsensusBootstrapInfo::~ConsensusBootstrapInfo() { //////////////////////////////////////////////////////////////////////// ConsensusRound::ConsensusRound(RaftConsensus* consensus, - gscoped_ptr replicate_msg, + unique_ptr replicate_msg, ConsensusReplicatedCallback replicated_cb) : consensus_(consensus), replicate_msg_(new RefCountedReplicate(replicate_msg.release())), diff --git a/src/kudu/consensus/raft_consensus.h b/src/kudu/consensus/raft_consensus.h index 99c5638..1a1cef3 100644 --- a/src/kudu/consensus/raft_consensus.h +++ b/src/kudu/consensus/raft_consensus.h @@ -14,7 +14,6 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. - #pragma once #include @@ -38,7 +37,6 @@ #include "kudu/consensus/opid.pb.h" #include "kudu/consensus/ref_counted_replicate.h" #include "kudu/gutil/callback.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/port.h" #include "kudu/gutil/ref_counted.h" @@ -55,7 +53,7 @@ namespace kudu { typedef std::lock_guard Lock; -typedef gscoped_ptr ScopedLock; +typedef std::unique_ptr ScopedLock; class Status; class ThreadPool; @@ -218,7 +216,7 @@ class RaftConsensus : public std::enable_shared_from_this, // (and later on the CommitMsg). ConsensusRound will also point to and // increase the reference count for the provided callbacks. scoped_refptr NewRound( - gscoped_ptr replicate_msg, + std::unique_ptr replicate_msg, ConsensusReplicatedCallback replicated_cb); // Called by a Leader to replicate an entry to the state machine. @@ -1005,7 +1003,7 @@ class ConsensusRound : public RefCountedThreadSafe { // Ctor used for leader transactions. Leader transactions can and must specify the // callbacks prior to initiating the consensus round. ConsensusRound(RaftConsensus* consensus, - gscoped_ptr replicate_msg, + std::unique_ptr replicate_msg, ConsensusReplicatedCallback replicated_cb); // Ctor used for follower/learner transactions. These transactions do not use the diff --git a/src/kudu/consensus/raft_consensus_quorum-test.cc b/src/kudu/consensus/raft_consensus_quorum-test.cc index 2fb196b..ff21436 100644 --- a/src/kudu/consensus/raft_consensus_quorum-test.cc +++ b/src/kudu/consensus/raft_consensus_quorum-test.cc @@ -20,7 +20,6 @@ #include #include #include -#include #include #include #include @@ -57,7 +56,6 @@ #include "kudu/fs/fs_manager.h" #include "kudu/gutil/bind.h" #include "kudu/gutil/casts.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/strings/strcat.h" @@ -270,7 +268,7 @@ class RaftConsensusQuorumTest : public KuduTest { Status AppendDummyMessage(int peer_idx, scoped_refptr* round) { - gscoped_ptr msg(new ReplicateMsg()); + unique_ptr msg(new ReplicateMsg()); msg->set_op_type(NO_OP); msg->mutable_noop_request(); msg->set_timestamp(clock_.Now().ToUint64()); @@ -302,7 +300,7 @@ class RaftConsensusQuorumTest : public KuduTest { commit_callback = Bind(&DoNothingStatusCB); } - gscoped_ptr msg(new CommitMsg()); + unique_ptr msg(new CommitMsg()); msg->set_op_type(NO_OP); msg->mutable_commited_op_id()->CopyFrom(round->id()); CHECK_OK(logs_[peer_idx]->AsyncAppendCommit(std::move(msg), commit_callback)); diff --git a/src/kudu/consensus/ref_counted_replicate.h b/src/kudu/consensus/ref_counted_replicate.h index 712ae14..b2fa844 100644 --- a/src/kudu/consensus/ref_counted_replicate.h +++ b/src/kudu/consensus/ref_counted_replicate.h @@ -14,13 +14,10 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. - -#ifndef KUDU_CONSENSUS_REF_COUNTED_REPLICATE_H_ -#define KUDU_CONSENSUS_REF_COUNTED_REPLICATE_H_ +#pragma once #include "kudu/consensus/consensus.pb.h" #include "kudu/gutil/ref_counted.h" -#include "kudu/gutil/gscoped_ptr.h" namespace kudu { namespace consensus { @@ -35,7 +32,7 @@ class RefCountedReplicate : public RefCountedThreadSafe { } private: - gscoped_ptr msg_; + std::unique_ptr msg_; }; typedef scoped_refptr ReplicateRefPtr; @@ -46,5 +43,3 @@ inline ReplicateRefPtr make_scoped_refptr_replicate(ReplicateMsg* replicate) { } // namespace consensus } // namespace kudu - -#endif /* KUDU_CONSENSUS_REF_COUNTED_REPLICATE_H_ */ diff --git a/src/kudu/gutil/map-util.h b/src/kudu/gutil/map-util.h index 7b6d7bb..ae6d2d1 100644 --- a/src/kudu/gutil/map-util.h +++ b/src/kudu/gutil/map-util.h @@ -738,7 +738,7 @@ void ReverseMap(const Collection& collection, // delete EraseKeyReturnValuePtr(&my_map, "abc"); // // Use returned value: -// gscoped_ptr value_ptr(EraseKeyReturnValuePtr(&my_map, "abc")); +// unique_ptr value_ptr(EraseKeyReturnValuePtr(&my_map, "abc")); // if (value_ptr.get()) // value_ptr->DoSomething(); // diff --git a/src/kudu/gutil/ref_counted.h b/src/kudu/gutil/ref_counted.h index b064a42..8183a74 100644 --- a/src/kudu/gutil/ref_counted.h +++ b/src/kudu/gutil/ref_counted.h @@ -326,7 +326,7 @@ class scoped_refptr { swap(&r.ptr_); } - // Like gscoped_ptr::reset(), drops a reference on the currently held object + // Like unique_ptr::reset(), drops a reference on the currently held object // (if any), and adds a reference to the passed-in object (if not NULL). void reset(T* p = NULL) { *this = p; diff --git a/src/kudu/integration-tests/all_types-itest.cc b/src/kudu/integration-tests/all_types-itest.cc index aa242d7..f77d4c1 100644 --- a/src/kudu/integration-tests/all_types-itest.cc +++ b/src/kudu/integration-tests/all_types-itest.cc @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -37,7 +38,6 @@ #include "kudu/common/common.pb.h" #include "kudu/common/partial_row.h" #include "kudu/common/types.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/mathlimits.h" #include "kudu/gutil/port.h" #include "kudu/gutil/stringprintf.h" @@ -56,16 +56,16 @@ DEFINE_int32(num_rows_per_tablet, 100, "The number of rows to be inserted into each tablet"); +using kudu::cluster::ExternalMiniCluster; +using kudu::cluster::ExternalMiniClusterOptions; +using kudu::client::sp::shared_ptr; using std::string; +using std::unique_ptr; using std::vector; namespace kudu { namespace client { -using cluster::ExternalMiniCluster; -using cluster::ExternalMiniClusterOptions; -using sp::shared_ptr; - static const int kNumTabletServers = 3; static const int kNumTablets = 3; static const int kMaxBatchSize = 8 * 1024 * 1024; @@ -409,7 +409,7 @@ class AllTypesItest : public KuduTest { Status CreateTable() { CreateAllTypesSchema(); vector split_rows = setup_.GenerateSplitRows(schema_); - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); for (const KuduPartialRow* row : split_rows) { split_rows_.push_back(*row); @@ -632,7 +632,7 @@ class AllTypesItest : public KuduTest { KuduSchema schema_; vector split_rows_; shared_ptr client_; - gscoped_ptr cluster_; + unique_ptr cluster_; shared_ptr table_; }; diff --git a/src/kudu/integration-tests/alter_table-test.cc b/src/kudu/integration-tests/alter_table-test.cc index 8cab86c..5798f99 100644 --- a/src/kudu/integration-tests/alter_table-test.cc +++ b/src/kudu/integration-tests/alter_table-test.cc @@ -46,7 +46,6 @@ #include "kudu/common/schema.h" #include "kudu/common/wire_protocol.h" #include "kudu/consensus/raft_consensus.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/stl_util.h" @@ -146,7 +145,7 @@ class AlterTableTest : public KuduTest { .Build(&client_)); // Add a table, make sure it reports itself. - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); CHECK_OK(table_creator->table_name(kTableName) .schema(&schema_) .set_range_partition_columns({ "c0" }) @@ -225,7 +224,7 @@ class AlterTableTest : public KuduTest { const string& column_name, int32_t default_value, const MonoDelta& timeout) { - gscoped_ptr table_alterer(client_->NewTableAlterer(table_name)); + unique_ptr table_alterer(client_->NewTableAlterer(table_name)); table_alterer->AddColumn(column_name)->Type(KuduColumnSchema::INT32)-> NotNull()->Default(KuduValue::FromInt(default_value)); return table_alterer->timeout(timeout)->Alter(); @@ -259,7 +258,7 @@ class AlterTableTest : public KuduTest { CHECK_OK(row->SetInt32(0, i * 100)); split_rows.push_back(row); } - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wdeprecated-declarations" return table_creator->table_name(table_name) @@ -291,7 +290,7 @@ class AlterTableTest : public KuduTest { static const char *kTableName; - gscoped_ptr cluster_; + unique_ptr cluster_; shared_ptr client_; KuduSchema schema_; @@ -375,7 +374,7 @@ TEST_F(AlterTableTest, TestAddNullableColumnWithoutDefault) { ASSERT_OK(tablet_replica_->tablet()->Flush()); { - gscoped_ptr table_alterer(client_->NewTableAlterer(kTableName)); + unique_ptr table_alterer(client_->NewTableAlterer(kTableName)); table_alterer->AddColumn("new")->Type(KuduColumnSchema::INT32); ASSERT_OK(table_alterer->Alter()); } @@ -395,7 +394,7 @@ TEST_F(AlterTableTest, TestRenamePrimaryKeyColumn) { ASSERT_OK(tablet_replica_->tablet()->Flush()); { - gscoped_ptr table_alterer(client_->NewTableAlterer(kTableName)); + unique_ptr table_alterer(client_->NewTableAlterer(kTableName)); table_alterer->AlterColumn("c0")->RenameTo("primaryKeyRenamed"); table_alterer->AlterColumn("c1")->RenameTo("secondColumn"); ASSERT_OK(table_alterer->Alter()); @@ -410,7 +409,7 @@ TEST_F(AlterTableTest, TestRenamePrimaryKeyColumn) { EXPECT_EQ("(int32 primaryKeyRenamed=16777216, int32 secondColumn=1)", rows[1]); { - gscoped_ptr table_alterer(client_->NewTableAlterer(kTableName)); + unique_ptr table_alterer(client_->NewTableAlterer(kTableName)); table_alterer->AlterColumn("primaryKeyRenamed")->RenameTo("pk"); table_alterer->AlterColumn("secondColumn")->RenameTo("sc"); ASSERT_OK(table_alterer->Alter()); @@ -622,7 +621,7 @@ void AlterTableTest::InsertRows(int start_row, int num_rows) { // Insert a bunch of rows with the current schema for (int i = start_row; i < start_row + num_rows; i++) { - gscoped_ptr insert(table->NewInsert()); + unique_ptr insert(table->NewInsert()); // Endian-swap the key so that we spew inserts randomly // instead of just a sequential write pattern. This way // compactions may actually be triggered. diff --git a/src/kudu/integration-tests/catalog_manager_tsk-itest.cc b/src/kudu/integration-tests/catalog_manager_tsk-itest.cc index d3e7dab..ac196ea 100644 --- a/src/kudu/integration-tests/catalog_manager_tsk-itest.cc +++ b/src/kudu/integration-tests/catalog_manager_tsk-itest.cc @@ -17,13 +17,14 @@ #include #include -#include #include +#include #include #include #include #include #include +#include #include #include @@ -37,7 +38,6 @@ #include "kudu/common/partial_row.h" #include "kudu/consensus/consensus.pb.h" #include "kudu/consensus/consensus.proxy.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/master/sys_catalog.h" #include "kudu/mini-cluster/external_mini_cluster.h" @@ -121,7 +121,7 @@ class CatalogManagerTskITest : public KuduTest { // Create a table. auto schema = KuduSchema::FromSchema(CreateKeyValueTestSchema()); - gscoped_ptr table_creator(client->NewTableCreator()); + unique_ptr table_creator(client->NewTableCreator()); ASSERT_OK(table_creator->table_name(kTableName) .set_range_partition_columns({ "key" }) diff --git a/src/kudu/integration-tests/client-stress-test.cc b/src/kudu/integration-tests/client-stress-test.cc index 102de8b..139f4a7 100644 --- a/src/kudu/integration-tests/client-stress-test.cc +++ b/src/kudu/integration-tests/client-stress-test.cc @@ -18,6 +18,7 @@ #include #include +#include #include #include #include @@ -33,7 +34,6 @@ #include "kudu/client/scan_predicate.h" #include "kudu/client/shared_ptr.h" #include "kudu/client/value.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/strings/join.h" @@ -58,6 +58,7 @@ METRIC_DECLARE_counter(follower_memory_pressure_rejections); using strings::Substitute; using std::set; using std::string; +using std::unique_ptr; using std::vector; namespace kudu { @@ -111,7 +112,7 @@ class ClientStressTest : public KuduTest { return ExternalMiniClusterOptions(); } - gscoped_ptr cluster_; + unique_ptr cluster_; }; // Stress test a case where most of the operations are expected to time out. diff --git a/src/kudu/integration-tests/consensus_peer_health_status-itest.cc b/src/kudu/integration-tests/consensus_peer_health_status-itest.cc index 097787d..7139640 100644 --- a/src/kudu/integration-tests/consensus_peer_health_status-itest.cc +++ b/src/kudu/integration-tests/consensus_peer_health_status-itest.cc @@ -15,9 +15,11 @@ // specific language governing permissions and limitations // under the License. +#include #include #include #include +#include #include #include @@ -26,7 +28,6 @@ #include "kudu/consensus/consensus.pb.h" #include "kudu/consensus/metadata.pb.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/integration-tests/cluster_itest_util.h" #include "kudu/integration-tests/raft_consensus-itest-base.h" diff --git a/src/kudu/integration-tests/create-table-itest.cc b/src/kudu/integration-tests/create-table-itest.cc index c7b91f1..5421a78 100644 --- a/src/kudu/integration-tests/create-table-itest.cc +++ b/src/kudu/integration-tests/create-table-itest.cc @@ -38,7 +38,6 @@ #include "kudu/common/partial_row.h" #include "kudu/common/schema.h" #include "kudu/common/wire_protocol-test-util.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/mathlimits.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/strings/substitute.h" @@ -110,7 +109,7 @@ TEST_F(CreateTableITest, TestCreateWhenMajorityOfReplicasFailCreation) { // Try to create a single-tablet table. // This won't succeed because we can't create enough replicas to get // a quorum. - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); auto client_schema = KuduSchema::FromSchema(GetSimpleTestSchema()); ASSERT_OK(table_creator->table_name(kTableName) .schema(&client_schema) @@ -196,7 +195,7 @@ TEST_F(CreateTableITest, TestSpreadReplicasEvenly) { const int kNumTablets = 20; NO_FATALS(StartCluster({}, {}, kNumServers)); - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); auto client_schema = KuduSchema::FromSchema(GetSimpleTestSchema()); ASSERT_OK(table_creator->table_name(kTableName) .schema(&client_schema) @@ -280,7 +279,7 @@ TEST_F(CreateTableITest, TestSpreadReplicasEvenlyWithDimension) { int32_t range_lower_bound, int32_t range_upper_bound, const string& dimension_label) -> Status { - gscoped_ptr table_creator(client->NewTableCreator()); + unique_ptr table_creator(client->NewTableCreator()); unique_ptr lower_bound(client_schema->NewRow()); RETURN_NOT_OK(lower_bound->SetInt32("key2", range_lower_bound)); unique_ptr upper_bound(client_schema->NewRow()); @@ -301,7 +300,7 @@ TEST_F(CreateTableITest, TestSpreadReplicasEvenlyWithDimension) { int32_t range_lower_bound, int32_t range_upper_bound, const string& dimension_label) -> Status { - gscoped_ptr table_alterer(client->NewTableAlterer(table_name)); + unique_ptr table_alterer(client->NewTableAlterer(table_name)); unique_ptr lower_bound(client_schema->NewRow()); RETURN_NOT_OK(lower_bound->SetInt32("key2", range_lower_bound)); unique_ptr upper_bound(client_schema->NewRow()); @@ -399,7 +398,7 @@ static void LookUpRandomKeysLoop(const std::shared_ptr r(client_schema.NewRow()); + unique_ptr r(client_schema.NewRow()); while (!quit->Load()) { master::GetTableLocationsRequestPB req; @@ -467,7 +466,7 @@ TEST_F(CreateTableITest, TestCreateTableWithDeadTServers) { Schema schema(GetSimpleTestSchema()); auto client_schema = KuduSchema::FromSchema(GetSimpleTestSchema()); - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); // Don't bother waiting for table creation to finish; it'll never happen // because all of the tservers are dead. diff --git a/src/kudu/integration-tests/decimal-itest.cc b/src/kudu/integration-tests/decimal-itest.cc index 5fdbd11..e0eccbc 100644 --- a/src/kudu/integration-tests/decimal-itest.cc +++ b/src/kudu/integration-tests/decimal-itest.cc @@ -16,8 +16,8 @@ // under the License. #include +#include #include -#include #include #include @@ -30,14 +30,13 @@ #include "kudu/client/write_op.h" #include "kudu/common/partial_row.h" #include "kudu/integration-tests/external_mini_cluster-itest-base.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/util/decimal_util.h" #include "kudu/util/int128.h" #include "kudu/util/status.h" #include "kudu/util/test_macros.h" using std::string; -using std::vector; +using std::unique_ptr; namespace kudu { namespace client { @@ -89,7 +88,7 @@ TEST_F(DecimalItest, TestDecimalTypes) { ASSERT_OK(builder.Build(&schema)); // Create Table - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); ASSERT_OK(table_creator->table_name(kTableName) .schema(&schema) .num_replicas(kNumServers) @@ -99,7 +98,7 @@ TEST_F(DecimalItest, TestDecimalTypes) { ASSERT_OK(client_->OpenTable(kTableName, &table)); // Alter Default Value - gscoped_ptr table_alterer(client_->NewTableAlterer(kTableName)); + unique_ptr table_alterer(client_->NewTableAlterer(kTableName)); table_alterer->AlterColumn("alteredDefault")->Default(KuduValue::FromDecimal(456789, 2)); ASSERT_OK(table_alterer->Alter()); diff --git a/src/kudu/integration-tests/delete_table-itest.cc b/src/kudu/integration-tests/delete_table-itest.cc index ce0fdd9..7086b86 100644 --- a/src/kudu/integration-tests/delete_table-itest.cc +++ b/src/kudu/integration-tests/delete_table-itest.cc @@ -19,6 +19,7 @@ #include #include +#include #include #include #include @@ -46,7 +47,6 @@ #include "kudu/consensus/consensus.pb.h" #include "kudu/consensus/metadata.pb.h" #include "kudu/gutil/basictypes.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/strings/split.h" #include "kudu/gutil/strings/substitute.h" @@ -1381,7 +1381,7 @@ TEST_P(DeleteTableTombstonedParamTest, TestTabletTombstone) { KuduPartialRow* split_row = client_schema.NewRow(); ASSERT_OK(split_row->SetInt32(0, numeric_limits::max() / kNumTablets)); split_rows.push_back(split_row); - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wdeprecated-declarations" ASSERT_OK(table_creator->table_name(TestWorkload::kDefaultTableName) diff --git a/src/kudu/integration-tests/exactly_once_writes-itest.cc b/src/kudu/integration-tests/exactly_once_writes-itest.cc index 7ab92c4..53b19d1 100644 --- a/src/kudu/integration-tests/exactly_once_writes-itest.cc +++ b/src/kudu/integration-tests/exactly_once_writes-itest.cc @@ -19,6 +19,7 @@ #include #include #include +#include #include #include @@ -30,7 +31,6 @@ #include "kudu/common/wire_protocol-test-util.h" #include "kudu/common/wire_protocol.h" #include "kudu/common/wire_protocol.pb.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/strings/substitute.h" diff --git a/src/kudu/integration-tests/flex_partitioning-itest.cc b/src/kudu/integration-tests/flex_partitioning-itest.cc index 4eeaa68..68ecc8e 100644 --- a/src/kudu/integration-tests/flex_partitioning-itest.cc +++ b/src/kudu/integration-tests/flex_partitioning-itest.cc @@ -40,7 +40,6 @@ #include "kudu/client/write_op.h" #include "kudu/common/common.pb.h" #include "kudu/common/partial_row.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/stl_util.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/integration-tests/cluster_itest_util.h" @@ -209,7 +208,7 @@ class FlexPartitioningITest : public KuduTest, KuduSchema schema; ASSERT_OK(b.Build(&schema)); - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); table_creator->table_name(kTableName) .schema(&schema) .num_replicas(1); @@ -316,7 +315,7 @@ class FlexPartitioningITest : public KuduTest, Random random_; - gscoped_ptr cluster_; + unique_ptr cluster_; shared_ptr client_; shared_ptr table_; @@ -339,7 +338,7 @@ Status FlexPartitioningITest::InsertRows(const RangePartitionOptions& range_part int count = 0; for (const auto& bound : bounds) { for (int32_t i = bound.first[0]; i < bound.second[0]; i++) { - gscoped_ptr insert(table_->NewInsert()); + unique_ptr insert(table_->NewInsert()); GenerateDataForRow(table_->schema(), i, &random_, insert->mutable_row()); inserted_rows_.emplace_back(new KuduPartialRow(*insert->mutable_row())); RETURN_NOT_OK(session->Apply(insert.release())); diff --git a/src/kudu/integration-tests/fuzz-itest.cc b/src/kudu/integration-tests/fuzz-itest.cc index fd3a904..774ba1e 100644 --- a/src/kudu/integration-tests/fuzz-itest.cc +++ b/src/kudu/integration-tests/fuzz-itest.cc @@ -48,7 +48,6 @@ #include "kudu/common/partial_row.h" #include "kudu/common/schema.h" #include "kudu/gutil/casts.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/strings/join.h" @@ -276,7 +275,7 @@ class FuzzTest : public KuduTest { .default_admin_operation_timeout(MonoDelta::FromSeconds(60)) .Build(&client_)); // Add a table, make sure it reports itself. - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); CHECK_OK(table_creator->table_name(kTableName) .schema(&schema_) .set_range_partition_columns({ "key" }) @@ -667,7 +666,7 @@ class FuzzTest : public KuduTest { int update_multiplier); KuduSchema schema_; - gscoped_ptr cluster_; + unique_ptr cluster_; shared_ptr client_; shared_ptr session_; shared_ptr table_; diff --git a/src/kudu/integration-tests/linked_list-test.cc b/src/kudu/integration-tests/linked_list-test.cc index fb4918c..8aa545a 100644 --- a/src/kudu/integration-tests/linked_list-test.cc +++ b/src/kudu/integration-tests/linked_list-test.cc @@ -31,6 +31,7 @@ // either zero or one times, and no link_to refers to a missing key. #include +#include #include #include #include @@ -44,7 +45,6 @@ #include "kudu/client/shared_ptr.h" #include "kudu/common/wire_protocol.pb.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/integration-tests/cluster_itest_util.h" #include "kudu/integration-tests/linked_list-test-util.h" @@ -82,6 +82,7 @@ using kudu::itest::WaitForReplicasReportedToMaster; using kudu::itest::WaitForServersToAgree; using kudu::master::VOTER_REPLICA; using std::string; +using std::unique_ptr; using std::vector; namespace kudu { @@ -156,7 +157,7 @@ class LinkedListTest : public tserver::TabletServerIntegrationTestBase { protected: shared_ptr client_; - gscoped_ptr tester_; + unique_ptr tester_; }; TEST_F(LinkedListTest, TestLoadAndVerify) { diff --git a/src/kudu/integration-tests/location_assignment-itest.cc b/src/kudu/integration-tests/location_assignment-itest.cc index ce1d891..b4510ba 100644 --- a/src/kudu/integration-tests/location_assignment-itest.cc +++ b/src/kudu/integration-tests/location_assignment-itest.cc @@ -31,7 +31,6 @@ #include "kudu/client/client.h" #include "kudu/client/shared_ptr.h" #include "kudu/common/wire_protocol.pb.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/stl_util.h" #include "kudu/gutil/strings/substitute.h" diff --git a/src/kudu/integration-tests/master_replication-itest.cc b/src/kudu/integration-tests/master_replication-itest.cc index d527feb..d61ae91 100644 --- a/src/kudu/integration-tests/master_replication-itest.cc +++ b/src/kudu/integration-tests/master_replication-itest.cc @@ -33,7 +33,6 @@ #include "kudu/common/partial_row.h" #include "kudu/common/wire_protocol.pb.h" #include "kudu/consensus/replica_management.pb.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/strings/substitute.h" @@ -171,7 +170,7 @@ class MasterReplicationTest : public KuduTest { b.AddColumn("int_val")->Type(KuduColumnSchema::INT32)->NotNull(); b.AddColumn("string_val")->Type(KuduColumnSchema::STRING)->NotNull(); CHECK_OK(b.Build(&schema)); - gscoped_ptr table_creator(client->NewTableCreator()); + unique_ptr table_creator(client->NewTableCreator()); return table_creator->table_name(table_name) .set_range_partition_columns({ "key" }) .schema(&schema) @@ -180,7 +179,7 @@ class MasterReplicationTest : public KuduTest { protected: InternalMiniClusterOptions opts_; - gscoped_ptr cluster_; + unique_ptr cluster_; }; // Basic test. Verify that: diff --git a/src/kudu/integration-tests/raft_config_change-itest.cc b/src/kudu/integration-tests/raft_config_change-itest.cc index 38b881c..5d540fe 100644 --- a/src/kudu/integration-tests/raft_config_change-itest.cc +++ b/src/kudu/integration-tests/raft_config_change-itest.cc @@ -16,6 +16,7 @@ // under the License. #include +#include #include #include #include @@ -34,7 +35,6 @@ #include "kudu/consensus/consensus.pb.h" #include "kudu/consensus/metadata.pb.h" #include "kudu/consensus/quorum_util.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/integration-tests/cluster_itest_util.h" diff --git a/src/kudu/integration-tests/raft_consensus-itest-base.cc b/src/kudu/integration-tests/raft_consensus-itest-base.cc index ef28531..fe238c7 100644 --- a/src/kudu/integration-tests/raft_consensus-itest-base.cc +++ b/src/kudu/integration-tests/raft_consensus-itest-base.cc @@ -19,6 +19,7 @@ #include #include +#include #include #include #include @@ -37,7 +38,6 @@ #include "kudu/common/wire_protocol.h" #include "kudu/consensus/consensus.pb.h" #include "kudu/consensus/opid.pb.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/stringprintf.h" @@ -83,6 +83,7 @@ using kudu::itest::TServerDetails; using kudu::pb_util::SecureDebugString; using kudu::rpc::RpcController; using std::string; +using std::unique_ptr; using std::vector; namespace kudu { @@ -151,7 +152,7 @@ void RaftConsensusITestBase::InsertTestRowsRemoteThread( uint64_t last_row_in_batch = first_row_in_batch + count / num_batches; for (int j = first_row_in_batch; j < last_row_in_batch; j++) { - gscoped_ptr insert(table->NewInsert()); + unique_ptr insert(table->NewInsert()); KuduPartialRow* row = insert->mutable_row(); CHECK_OK(row->SetInt32(0, j)); CHECK_OK(row->SetInt32(1, j * 2)); diff --git a/src/kudu/integration-tests/raft_consensus-itest.cc b/src/kudu/integration-tests/raft_consensus-itest.cc index ef14ce6..8c22508 100644 --- a/src/kudu/integration-tests/raft_consensus-itest.cc +++ b/src/kudu/integration-tests/raft_consensus-itest.cc @@ -45,7 +45,6 @@ #include "kudu/consensus/opid_util.h" #include "kudu/consensus/quorum_util.h" #include "kudu/gutil/basictypes.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/stl_util.h" @@ -147,6 +146,7 @@ using kudu::rpc::RpcController; using kudu::server::SetFlagRequestPB; using kudu::server::SetFlagResponsePB; using std::string; +using std::unique_ptr; using std::unordered_map; using std::vector; using strings::Substitute; @@ -321,7 +321,7 @@ void RaftConsensusITest::InsertPayloadIgnoreErrors(int start_row, CHECK_OK(session->SetFlushMode(KuduSession::MANUAL_FLUSH)); string payload(payload_size, 'x'); for (int i = 0; i < num_rows; i++) { - gscoped_ptr insert(table->NewInsert()); + unique_ptr insert(table->NewInsert()); KuduPartialRow* row = insert->mutable_row(); CHECK_OK(row->SetInt32(0, i + start_row)); CHECK_OK(row->SetInt32(1, 0)); diff --git a/src/kudu/integration-tests/raft_consensus_election-itest.cc b/src/kudu/integration-tests/raft_consensus_election-itest.cc index 287f8f6..c78c3f7 100644 --- a/src/kudu/integration-tests/raft_consensus_election-itest.cc +++ b/src/kudu/integration-tests/raft_consensus_election-itest.cc @@ -17,6 +17,7 @@ #include #include +#include #include #include #include @@ -33,7 +34,6 @@ #include "kudu/consensus/metadata.pb.h" #include "kudu/consensus/opid.pb.h" #include "kudu/consensus/opid_util.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/integration-tests/cluster_itest_util.h" diff --git a/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc b/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc index 6bf794e..1392f20 100644 --- a/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc +++ b/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc @@ -17,6 +17,7 @@ #include #include +#include #include #include #include @@ -36,7 +37,6 @@ #include "kudu/consensus/consensus.pb.h" #include "kudu/consensus/metadata.pb.h" #include "kudu/consensus/quorum_util.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/stl_util.h" #include "kudu/gutil/strings/substitute.h" diff --git a/src/kudu/integration-tests/raft_consensus_stress-itest.cc b/src/kudu/integration-tests/raft_consensus_stress-itest.cc index fee4cfd..7191542 100644 --- a/src/kudu/integration-tests/raft_consensus_stress-itest.cc +++ b/src/kudu/integration-tests/raft_consensus_stress-itest.cc @@ -18,17 +18,16 @@ #include #include #include +#include #include #include #include #include #include -#include #include #include -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/integration-tests/cluster_itest_util.h" #include "kudu/integration-tests/cluster_verifier.h" diff --git a/src/kudu/integration-tests/registration-test.cc b/src/kudu/integration-tests/registration-test.cc index e5f6916..bae2843 100644 --- a/src/kudu/integration-tests/registration-test.cc +++ b/src/kudu/integration-tests/registration-test.cc @@ -21,6 +21,7 @@ #include #include #include +#include #include #include @@ -32,7 +33,6 @@ #include "kudu/common/schema.h" #include "kudu/common/wire_protocol.h" #include "kudu/common/wire_protocol.pb.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/gutil/walltime.h" @@ -69,25 +69,25 @@ METRIC_DECLARE_counter(rows_inserted); METRIC_DECLARE_counter(rows_updated); using boost::none; +using kudu::cluster::InternalMiniCluster; +using kudu::master::CatalogManager; +using kudu::master::CreateTableRequestPB; +using kudu::master::CreateTableResponsePB; +using kudu::master::GetTableLocationsResponsePB; +using kudu::master::IsCreateTableDoneRequestPB; +using kudu::master::IsCreateTableDoneResponsePB; +using kudu::master::MiniMaster; +using kudu::master::TSDescriptor; +using kudu::master::TabletLocationsPB; +using kudu::pb_util::SecureShortDebugString; +using kudu::tserver::MiniTabletServer; using std::shared_ptr; using std::string; +using std::unique_ptr; using std::vector; namespace kudu { -using cluster::InternalMiniCluster; -using master::CatalogManager; -using master::CreateTableRequestPB; -using master::CreateTableResponsePB; -using master::GetTableLocationsResponsePB; -using master::IsCreateTableDoneRequestPB; -using master::IsCreateTableDoneResponsePB; -using master::MiniMaster; -using master::TSDescriptor; -using master::TabletLocationsPB; -using kudu::pb_util::SecureShortDebugString; -using tserver::MiniTabletServer; - void CreateTableForTesting(MiniMaster* mini_master, const string& table_name, const Schema& schema, @@ -218,7 +218,7 @@ class RegistrationTest : public KuduTest { } protected: - gscoped_ptr cluster_; + unique_ptr cluster_; Schema schema_; int64_t setup_time_; }; diff --git a/src/kudu/integration-tests/security-faults-itest.cc b/src/kudu/integration-tests/security-faults-itest.cc index 9fb5fab..b64aac6 100644 --- a/src/kudu/integration-tests/security-faults-itest.cc +++ b/src/kudu/integration-tests/security-faults-itest.cc @@ -34,7 +34,6 @@ #include "kudu/client/shared_ptr.h" #include "kudu/client/write_op.h" #include "kudu/common/partial_row.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/mini-cluster/external_mini_cluster.h" #include "kudu/security/test/mini_kdc.h" @@ -124,7 +123,7 @@ class SecurityComponentsFaultsITest : public KuduTest { // Create a table. KuduSchema schema = KuduSchema::FromSchema(CreateKeyValueTestSchema()); - gscoped_ptr table_creator(client->NewTableCreator()); + unique_ptr table_creator(client->NewTableCreator()); RETURN_NOT_OK(table_creator->table_name(kTableName) .set_range_partition_columns({ "key" }) diff --git a/src/kudu/integration-tests/security-unknown-tsk-itest.cc b/src/kudu/integration-tests/security-unknown-tsk-itest.cc index 97b530c..6beffbf 100644 --- a/src/kudu/integration-tests/security-unknown-tsk-itest.cc +++ b/src/kudu/integration-tests/security-unknown-tsk-itest.cc @@ -37,7 +37,6 @@ #include "kudu/client/shared_ptr.h" #include "kudu/client/write_op.h" #include "kudu/common/partial_row.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/stl_util.h" #include "kudu/gutil/walltime.h" #include "kudu/integration-tests/test_workload.h" @@ -217,7 +216,7 @@ TEST_F(SecurityUnknownTskTest, ErrorUnavailableCommonOperations) { // calls to the tablet server. The table should consists of multiple tablets // hosted by all available tablet servers, so the insert or scan requests are // sent to all avaialble tablet servers. - gscoped_ptr table_creator(client->NewTableCreator()); + unique_ptr table_creator(client->NewTableCreator()); shared_ptr table; ASSERT_OK(table_creator->table_name(table_name) .set_range_partition_columns({ "key" }) diff --git a/src/kudu/integration-tests/tablet_copy-itest.cc b/src/kudu/integration-tests/tablet_copy-itest.cc index 53ec383..f3183c0 100644 --- a/src/kudu/integration-tests/tablet_copy-itest.cc +++ b/src/kudu/integration-tests/tablet_copy-itest.cc @@ -17,6 +17,7 @@ #include #include +#include #include #include #include @@ -48,7 +49,6 @@ #include "kudu/consensus/opid_util.h" #include "kudu/fs/fs_manager.h" #include "kudu/gutil/basictypes.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/strings/substitute.h" @@ -128,6 +128,7 @@ using std::mutex; using std::set; using std::string; using std::thread; +using std::unique_ptr; using std::unordered_map; using std::vector; using strings::Substitute; @@ -501,7 +502,7 @@ TEST_F(TabletCopyITest, TestDeleteTabletDuringTabletCopy) { ASSERT_OK(env_->CreateDir(testbase)); opts.wal_root = JoinPathSegments(testbase, "wals"); opts.data_roots.push_back(JoinPathSegments(testbase, "data-0")); - gscoped_ptr fs_manager(new FsManager(env_, opts)); + unique_ptr fs_manager(new FsManager(env_, opts)); ASSERT_OK(fs_manager->CreateInitialFileSystemLayout()); ASSERT_OK(fs_manager->Open()); scoped_refptr cmeta_manager( @@ -642,7 +643,7 @@ TEST_F(TabletCopyITest, TestConcurrentTabletCopys) { } #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wdeprecated-declarations" - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); ASSERT_OK(table_creator->table_name(TestWorkload::kDefaultTableName) .split_rows(splits) .schema(&client_schema) diff --git a/src/kudu/integration-tests/test_workload.cc b/src/kudu/integration-tests/test_workload.cc index 43d3797..441e22b 100644 --- a/src/kudu/integration-tests/test_workload.cc +++ b/src/kudu/integration-tests/test_workload.cc @@ -28,7 +28,6 @@ #include "kudu/client/write_op.h" #include "kudu/common/partial_row.h" #include "kudu/common/wire_protocol-test-util.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/mathlimits.h" #include "kudu/gutil/port.h" #include "kudu/gutil/stl_util.h" @@ -38,25 +37,25 @@ #include "kudu/util/status.h" #include "kudu/util/test_util.h" -namespace kudu { - -using client::KuduClient; -using client::KuduColumnSchema; -using client::KuduDelete; -using client::KuduError; -using client::KuduInsert; -using client::KuduScanBatch; -using client::KuduScanner; -using client::KuduSchema; -using client::KuduSession; -using client::KuduTable; -using client::KuduTableCreator; -using client::KuduUpdate; -using client::sp::shared_ptr; -using cluster::MiniCluster; - +using kudu::client::KuduClient; +using kudu::client::KuduColumnSchema; +using kudu::client::KuduDelete; +using kudu::client::KuduError; +using kudu::client::KuduInsert; +using kudu::client::KuduScanBatch; +using kudu::client::KuduScanner; +using kudu::client::KuduSchema; +using kudu::client::KuduSession; +using kudu::client::KuduTable; +using kudu::client::KuduTableCreator; +using kudu::client::KuduUpdate; +using kudu::client::sp::shared_ptr; +using kudu::cluster::MiniCluster; +using std::unique_ptr; using std::vector; +namespace kudu { + const char* const TestWorkload::kDefaultTableName = "test-workload"; TestWorkload::TestWorkload(MiniCluster* cluster) @@ -152,12 +151,12 @@ void TestWorkload::WriteThread() { { for (int i = 0; i < write_batch_size_; i++) { if (write_pattern_ == UPDATE_ONE_ROW) { - gscoped_ptr update(table->NewUpdate()); + unique_ptr update(table->NewUpdate()); KuduPartialRow* row = update->mutable_row(); GenerateDataForRow(schema_, 0, &rng_, row); CHECK_OK(session->Apply(update.release())); } else { - gscoped_ptr insert(table->NewInsert()); + unique_ptr insert(table->NewInsert()); KuduPartialRow* row = insert->mutable_row(); int32_t key; if (write_pattern_ == INSERT_SEQUENTIAL_ROWS) { @@ -196,7 +195,7 @@ void TestWorkload::WriteThread() { // Write delete row to cluster. if (write_pattern_ == INSERT_RANDOM_ROWS_WITH_DELETE) { for (auto key : keys) { - gscoped_ptr op(table->NewDelete()); + unique_ptr op(table->NewDelete()); KuduPartialRow* row = op->mutable_row(); WriteValueToColumn(schema_, 0, key, row); CHECK_OK(session->Apply(op.release())); @@ -314,7 +313,7 @@ void TestWorkload::Setup() { } // Create the table. - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wdeprecated-declarations" Status s = table_creator->table_name(table_name_) diff --git a/src/kudu/integration-tests/ts_itest-base.cc b/src/kudu/integration-tests/ts_itest-base.cc index ae5d60d..2013e5e 100644 --- a/src/kudu/integration-tests/ts_itest-base.cc +++ b/src/kudu/integration-tests/ts_itest-base.cc @@ -28,7 +28,6 @@ #include #include -#include #include #include @@ -73,6 +72,7 @@ using kudu::cluster::LocationInfo; using std::pair; using std::set; using std::string; +using std::unique_ptr; using std::unordered_multimap; using std::unordered_set; using std::vector; @@ -531,7 +531,7 @@ void TabletServerIntegrationTestBase::CreateTable(const string& table_id) { // The tests here make extensive use of server schemas, but we need // a client schema to create the table. client::KuduSchema client_schema(client::KuduSchema::FromSchema(schema_)); - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); ASSERT_OK(table_creator->table_name(table_id) .schema(&client_schema) .set_range_partition_columns({ "key" }) diff --git a/src/kudu/integration-tests/ts_itest-base.h b/src/kudu/integration-tests/ts_itest-base.h index 1594a21..4270555 100644 --- a/src/kudu/integration-tests/ts_itest-base.h +++ b/src/kudu/integration-tests/ts_itest-base.h @@ -14,16 +14,15 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. - #pragma once #include +#include #include #include #include #include "kudu/client/shared_ptr.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/integration-tests/cluster_itest_util.h" #include "kudu/integration-tests/mini_cluster_fs_inspector.h" #include "kudu/mini-cluster/external_mini_cluster.h" @@ -157,8 +156,8 @@ class TabletServerIntegrationTestBase : public TabletServerTestBase { std::vector GetServersWithoutReplica(const std::string& tablet_id) const; protected: - gscoped_ptr cluster_; - gscoped_ptr inspect_; + std::unique_ptr cluster_; + std::unique_ptr inspect_; // Maps server uuid to TServerDetails itest::TabletServerMap tablet_servers_; diff --git a/src/kudu/integration-tests/update_scan_delta_compact-test.cc b/src/kudu/integration-tests/update_scan_delta_compact-test.cc index 13d4970..f16af36 100644 --- a/src/kudu/integration-tests/update_scan_delta_compact-test.cc +++ b/src/kudu/integration-tests/update_scan_delta_compact-test.cc @@ -23,7 +23,6 @@ #include #include -#include #include #include @@ -35,7 +34,6 @@ #include "kudu/client/shared_ptr.h" #include "kudu/client/write_op.h" #include "kudu/common/partial_row.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/strings/strcat.h" @@ -81,6 +79,7 @@ using kudu::client::sp::shared_ptr; using kudu::cluster::InternalMiniCluster; using kudu::cluster::InternalMiniClusterOptions; using std::string; +using std::unique_ptr; using std::vector; namespace kudu { @@ -107,7 +106,7 @@ class UpdateScanDeltaCompactionTest : public KuduTest { void CreateTable() { NO_FATALS(InitCluster()); - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); ASSERT_OK(table_creator->table_name(kTableName) .schema(&schema_) .set_range_partition_columns({ "key" }) @@ -210,7 +209,7 @@ void UpdateScanDeltaCompactionTest::InsertBaseData() { LOG_TIMING(INFO, "Insert") { for (int64_t key = 0; key < FLAGS_row_count; key++) { - gscoped_ptr insert(table_->NewInsert()); + unique_ptr insert(table_->NewInsert()); MakeRow(key, 0, insert->mutable_row()); ASSERT_OK(session->Apply(insert.release())); ASSERT_OK(WaitForLastBatchAndFlush(key, &last_s, &last_s_cb, session)); @@ -272,7 +271,7 @@ void UpdateScanDeltaCompactionTest::UpdateRows(CountDownLatch* stop_latch) { last_s_cb.Run(Status::OK()); LOG_TIMING(INFO, "Update") { for (int64_t key = 0; key < FLAGS_row_count && stop_latch->count() > 0; key++) { - gscoped_ptr update(table_->NewUpdate()); + unique_ptr update(table_->NewUpdate()); MakeRow(key, iteration, update->mutable_row()); CHECK_OK(session->Apply(update.release())); CHECK_OK(WaitForLastBatchAndFlush(key, &last_s, &last_s_cb, session)); diff --git a/src/kudu/integration-tests/write_throttling-itest.cc b/src/kudu/integration-tests/write_throttling-itest.cc index 3ce5344..f477c3a 100644 --- a/src/kudu/integration-tests/write_throttling-itest.cc +++ b/src/kudu/integration-tests/write_throttling-itest.cc @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +#include #include #include #include @@ -28,31 +29,28 @@ #include "kudu/client/shared_ptr.h" #include "kudu/client/write_op.h" #include "kudu/common/partial_row.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/integration-tests/external_mini_cluster-itest-base.h" #include "kudu/util/monotime.h" #include "kudu/util/status.h" #include "kudu/util/test_macros.h" -using std::string; -using std::vector; - DEFINE_int32(throttling_test_time, 3, "Number of seconds to run write throttling test"); +using kudu::client::KuduColumnSchema; +using kudu::client::KuduInsert; +using kudu::client::KuduSchema; +using kudu::client::KuduSchemaBuilder; +using kudu::client::KuduSession; +using kudu::client::KuduTable; +using kudu::client::KuduTableCreator; +using std::string; +using std::unique_ptr; +using std::vector; + namespace kudu { namespace tablet { -using client::KuduClient; -using client::KuduClientBuilder; -using client::KuduColumnSchema; -using client::KuduInsert; -using client::KuduSchema; -using client::KuduSchemaBuilder; -using client::KuduSession; -using client::KuduTable; -using client::KuduTableCreator; - class WriteThrottlingTest : public ExternalMiniClusterITestBase { protected: WriteThrottlingTest() { @@ -63,7 +61,7 @@ class WriteThrottlingTest : public ExternalMiniClusterITestBase { } void CreateTable() { - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); ASSERT_OK(table_creator->table_name(kTableName) .schema(&schema_) .set_range_partition_columns({ "key" }) @@ -96,7 +94,7 @@ TEST_F(WriteThrottlingTest, ThrottleWriteRpcPerSec) { for (int t = 0; t < FLAGS_throttling_test_time; t++) { MonoTime begin = MonoTime::Now(); for (int i = 0; i < TARGET_QPS; i++) { - gscoped_ptr insert(table_->NewInsert()); + unique_ptr insert(table_->NewInsert()); KuduPartialRow* row = insert->mutable_row(); CHECK_OK(row->SetInt64("key", t * TARGET_QPS + i)); CHECK_OK(row->SetStringNoCopy("string_val", string_val)); @@ -128,7 +126,7 @@ TEST_F(WriteThrottlingTest, ThrottleWriteBytesPerSec) { for (int t = 0; t < FLAGS_throttling_test_time; t++) { MonoTime begin = MonoTime::Now(); for (int i = 0; i < TARGET_QPS; i++) { - gscoped_ptr insert(table_->NewInsert()); + unique_ptr insert(table_->NewInsert()); KuduPartialRow* row = insert->mutable_row(); CHECK_OK(row->SetInt64("key", t * TARGET_QPS + i)); CHECK_OK(row->SetStringNoCopy("string_val", string_val)); diff --git a/src/kudu/kserver/kserver.cc b/src/kudu/kserver/kserver.cc index c2b1db9..c645240 100644 --- a/src/kudu/kserver/kserver.cc +++ b/src/kudu/kserver/kserver.cc @@ -28,7 +28,6 @@ #include #include "kudu/fs/fs_manager.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/integral_types.h" #include "kudu/gutil/strings/numbers.h" #include "kudu/gutil/strings/substitute.h" diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc index 8a2d8a5..562516e 100644 --- a/src/kudu/master/catalog_manager.cc +++ b/src/kudu/master/catalog_manager.cc @@ -54,7 +54,6 @@ #include #include #include -#include #include #include #include @@ -87,7 +86,6 @@ #include "kudu/gutil/basictypes.h" #include "kudu/gutil/bind.h" #include "kudu/gutil/bind_helpers.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/port.h" @@ -3168,7 +3166,7 @@ class PickLeaderReplica : public TSPicker { class RetryingTSRpcTask : public MonitoredTask { public: RetryingTSRpcTask(Master *master, - gscoped_ptr replica_picker, + unique_ptr replica_picker, scoped_refptr table) : master_(master), replica_picker_(std::move(replica_picker)), @@ -3238,7 +3236,7 @@ class RetryingTSRpcTask : public MonitoredTask { void RpcCallback(); Master * const master_; - const gscoped_ptr replica_picker_; + const unique_ptr replica_picker_; const scoped_refptr table_; MonoTime start_ts_; @@ -3413,7 +3411,7 @@ class RetrySpecificTSRpcTask : public RetryingTSRpcTask { const string& permanent_uuid, const scoped_refptr& table) : RetryingTSRpcTask(master, - gscoped_ptr(new PickSpecificUUID(permanent_uuid)), + unique_ptr(new PickSpecificUUID(permanent_uuid)), table), permanent_uuid_(permanent_uuid) { } @@ -3606,7 +3604,7 @@ class AsyncAlterTable : public RetryingTSRpcTask { AsyncAlterTable(Master *master, scoped_refptr tablet) : RetryingTSRpcTask(master, - gscoped_ptr(new PickLeaderReplica(tablet)), + unique_ptr(new PickLeaderReplica(tablet)), tablet->table()), tablet_(std::move(tablet)) { } @@ -3706,7 +3704,7 @@ AsyncChangeConfigTask::AsyncChangeConfigTask(Master* master, ConsensusStatePB cstate, consensus::ChangeConfigType change_config_type) : RetryingTSRpcTask(master, - gscoped_ptr(new PickLeaderReplica(tablet)), + unique_ptr(new PickLeaderReplica(tablet)), tablet->table()), tablet_(std::move(tablet)), cstate_(std::move(cstate)), diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc index 18bd30d..79bd17a 100644 --- a/src/kudu/master/master-test.cc +++ b/src/kudu/master/master-test.cc @@ -50,7 +50,6 @@ #include "kudu/consensus/replica_management.pb.h" #include "kudu/generated/version_defines.h" #include "kudu/gutil/dynamic_annotations.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/strings/split.h" @@ -103,6 +102,7 @@ using std::pair; using std::shared_ptr; using std::string; using std::thread; +using std::unique_ptr; using std::unordered_map; using std::unordered_set; using std::vector; @@ -161,9 +161,9 @@ class MasterTest : public KuduTest { const vector>& bounds); shared_ptr client_messenger_; - gscoped_ptr mini_master_; + unique_ptr mini_master_; Master* master_; - gscoped_ptr proxy_; + unique_ptr proxy_; }; TEST_F(MasterTest, TestPingServer) { diff --git a/src/kudu/master/sys_catalog-test.cc b/src/kudu/master/sys_catalog-test.cc index 6fbb942..a10349c 100644 --- a/src/kudu/master/sys_catalog-test.cc +++ b/src/kudu/master/sys_catalog-test.cc @@ -27,7 +27,6 @@ #include "kudu/common/common.pb.h" #include "kudu/common/schema.h" #include "kudu/common/wire_protocol.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/ref_counted.h" #include "kudu/master/catalog_manager.h" #include "kudu/master/master.h" @@ -54,6 +53,7 @@ using kudu::security::DataFormat; using kudu::security::PrivateKey; using std::shared_ptr; using std::string; +using std::unique_ptr; using std::vector; namespace google { @@ -90,9 +90,9 @@ class SysCatalogTest : public KuduTest { } shared_ptr client_messenger_; - gscoped_ptr mini_master_; + unique_ptr mini_master_; Master* master_; - gscoped_ptr proxy_; + unique_ptr proxy_; }; class TestTableLoader : public TableVisitor { diff --git a/src/kudu/master/ts_descriptor.h b/src/kudu/master/ts_descriptor.h index 9c4199a..5b8c8ee 100644 --- a/src/kudu/master/ts_descriptor.h +++ b/src/kudu/master/ts_descriptor.h @@ -14,8 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -#ifndef KUDU_MASTER_TS_DESCRIPTOR_H -#define KUDU_MASTER_TS_DESCRIPTOR_H +#pragma once #include #include @@ -31,7 +30,6 @@ #include "kudu/common/wire_protocol.pb.h" #include "kudu/gutil/basictypes.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/map-util.h" #include "kudu/util/locks.h" @@ -204,7 +202,7 @@ class TSDescriptor : public enable_make_shared { // The tablet server's location, as determined by the master at registration. boost::optional location_; - gscoped_ptr registration_; + std::unique_ptr registration_; std::shared_ptr ts_admin_proxy_; std::shared_ptr consensus_proxy_; @@ -218,4 +216,3 @@ typedef std::vector> TSDescriptorVector; } // namespace master } // namespace kudu -#endif /* KUDU_MASTER_TS_DESCRIPTOR_H */ diff --git a/src/kudu/mini-cluster/internal_mini_cluster.cc b/src/kudu/mini-cluster/internal_mini_cluster.cc index 80dd51c..830eb43 100644 --- a/src/kudu/mini-cluster/internal_mini_cluster.cc +++ b/src/kudu/mini-cluster/internal_mini_cluster.cc @@ -26,7 +26,6 @@ #include "kudu/client/client.h" #include "kudu/common/wire_protocol.pb.h" #include "kudu/fs/fs_manager.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/master/catalog_manager.h" #include "kudu/master/master.h" @@ -199,9 +198,10 @@ Status InternalMiniCluster::AddTabletServer() { } string bind_ip = GetBindIpForDaemonWithType(MiniCluster::TSERVER, new_idx, opts_.bind_mode); - gscoped_ptr tablet_server(new MiniTabletServer(GetTabletServerFsRoot(new_idx), - HostPort(bind_ip, ts_rpc_port), - opts_.num_data_dirs)); + unique_ptr tablet_server(new MiniTabletServer( + GetTabletServerFsRoot(new_idx), + HostPort(bind_ip, ts_rpc_port), + opts_.num_data_dirs)); // set the master addresses tablet_server->options()->master_addresses = master_rpc_addrs(); diff --git a/src/kudu/rpc/connection.cc b/src/kudu/rpc/connection.cc index 508cb35..7dc0c03 100644 --- a/src/kudu/rpc/connection.cc +++ b/src/kudu/rpc/connection.cc @@ -32,7 +32,6 @@ #include #include -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/strings/human_readable.h" #include "kudu/gutil/strings/substitute.h" diff --git a/src/kudu/rpc/rpc-test.cc b/src/kudu/rpc/rpc-test.cc index 4258de2..cb78577 100644 --- a/src/kudu/rpc/rpc-test.cc +++ b/src/kudu/rpc/rpc-test.cc @@ -34,7 +34,6 @@ #include #include "kudu/gutil/casts.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/stl_util.h" @@ -1377,7 +1376,7 @@ TEST_P(TestRpc, TestCancellationAsync) { RpcController controller; // The payload to be used during the RPC. - gscoped_array payload(new uint8_t[TEST_PAYLOAD_SIZE]); + unique_ptr payload(new uint8_t[TEST_PAYLOAD_SIZE]); // Used to generate sleep time between invoking RPC and requesting cancellation. Random rand(SeedRandom()); diff --git a/src/kudu/server/glog_metrics.h b/src/kudu/server/glog_metrics.h index 9a7bf32..e78f4f9 100644 --- a/src/kudu/server/glog_metrics.h +++ b/src/kudu/server/glog_metrics.h @@ -14,10 +14,10 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -#ifndef KUDU_SERVER_GLOG_METRICS_H -#define KUDU_SERVER_GLOG_METRICS_H +#pragma once + +#include -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/ref_counted.h" namespace google { @@ -38,14 +38,12 @@ class ScopedGLogMetrics { ~ScopedGLogMetrics(); private: - gscoped_ptr sink_; + std::unique_ptr sink_; }; - // Registers glog-related metrics. // This can be called multiple times on different entities, though the resulting // metrics will be identical, since the GLog tracking is process-wide. void RegisterGLogMetrics(const scoped_refptr& entity); } // namespace kudu -#endif /* KUDU_SERVER_GLOG_METRICS_H */ diff --git a/src/kudu/tablet/compaction-test.cc b/src/kudu/tablet/compaction-test.cc index 974fa18..451e4a4 100644 --- a/src/kudu/tablet/compaction-test.cc +++ b/src/kudu/tablet/compaction-test.cc @@ -50,7 +50,6 @@ #include "kudu/fs/fs_manager.h" #include "kudu/fs/log_block_manager.h" #include "kudu/gutil/casts.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/tablet/diskrowset.h" @@ -310,13 +309,13 @@ class TestCompaction : public KuduRowSetTest { } } - Status BuildCompactionInput(const MvccSnapshot& merge_snap, - const vector >& rowsets, - const Schema& projection, - gscoped_ptr* out) { + static Status BuildCompactionInput(const MvccSnapshot& merge_snap, + const vector >& rowsets, + const Schema& projection, + unique_ptr* out) { vector > merge_inputs; for (const shared_ptr &rs : rowsets) { - gscoped_ptr input; + unique_ptr input; RETURN_NOT_OK(CompactionInput::Create(*rs, &projection, merge_snap, nullptr, &input)); merge_inputs.push_back(shared_ptr(input.release())); } @@ -331,7 +330,7 @@ class TestCompaction : public KuduRowSetTest { const Schema& projection, int64_t roll_threshold, vector >* result_rowsets) { MvccSnapshot merge_snap(mvcc_); - gscoped_ptr compact_input; + unique_ptr compact_input; RETURN_NOT_OK(BuildCompactionInput(merge_snap, rowsets, projection, &compact_input)); DoFlushAndReopen(compact_input.get(), projection, merge_snap, roll_threshold, result_rowsets); @@ -356,7 +355,7 @@ class TestCompaction : public KuduRowSetTest { vector >* result_rowsets) { MvccSnapshot snap(mvcc_); vector > rowset_metas; - gscoped_ptr input(CompactionInput::Create(mrs, &projection, snap)); + unique_ptr input(CompactionInput::Create(mrs, &projection, snap)); DoFlushAndReopen(input.get(), projection, snap, roll_threshold, result_rowsets); } @@ -459,7 +458,7 @@ class TestCompaction : public KuduRowSetTest { LOG_TIMING(INFO, "compacting " + std::string((OVERLAP_INPUTS ? "with overlap" : "without overlap"))) { MvccSnapshot merge_snap(mvcc_); - gscoped_ptr compact_input; + unique_ptr compact_input; ASSERT_OK(BuildCompactionInput(merge_snap, rowsets, schema_, &compact_input)); // Use a low target row size to increase the number of resulting rowsets. RollingDiskRowSetWriter rdrsw(tablet()->metadata(), schema_, @@ -505,7 +504,7 @@ TEST_F(TestCompaction, TestMemRowSetInput) { // and mutations. vector out; MvccSnapshot snap(mvcc_); - gscoped_ptr input(CompactionInput::Create(*mrs, &schema_, snap)); + unique_ptr input(CompactionInput::Create(*mrs, &schema_, snap)); IterateInput(input.get(), &out); ASSERT_EQ(10, out.size()); EXPECT_EQ(R"(RowIdxInBlock: 0; Base: (string key="hello 00000000", int32 val=0, )" @@ -569,7 +568,7 @@ TEST_F(TestCompaction, TestRowSetInput) { // Check compaction input vector out; - gscoped_ptr input; + unique_ptr input; ASSERT_OK(CompactionInput::Create(*rs, &schema_, MvccSnapshot(mvcc_), nullptr, &input)); IterateInput(input.get(), &out); ASSERT_EQ(10, out.size()); @@ -638,7 +637,7 @@ TEST_F(TestCompaction, TestDuplicatedGhostRowsMerging) { // Now compact all the drs and make sure we don't get duplicated keys on the output CompactAndReopenNoRoll(all_rss, schema_, &result); - gscoped_ptr input; + unique_ptr input; ASSERT_OK(CompactionInput::Create(*result, &schema_, MvccSnapshot::CreateSnapshotIncludingAllTransactions(), @@ -803,7 +802,7 @@ TEST_F(TestCompaction, TestDuplicatedRowsRandomCompaction) { vector> inputs; for (auto& row_set : row_sets) { - gscoped_ptr ci; + unique_ptr ci; CHECK_OK(row_set->NewCompactionInput(&schema_, all_snap, nullptr, &ci)); inputs.push_back(shared_ptr(ci.release())); } @@ -824,7 +823,7 @@ TEST_F(TestCompaction, TestDuplicatedRowsRandomCompaction) { } vector out; - gscoped_ptr ci; + unique_ptr ci; CHECK_OK(row_sets[0]->NewCompactionInput(&schema_, all_snap, nullptr, &ci)); IterateInput(ci.get(), &out); @@ -873,17 +872,17 @@ TEST_F(TestCompaction, TestMRSCompactionDoesntOutputUnobservableRows) { MvccSnapshot all_snap = MvccSnapshot::CreateSnapshotIncludingAllTransactions(); - gscoped_ptr rs1_input; + unique_ptr rs1_input; ASSERT_OK(CompactionInput::Create(*rs1, &schema_, all_snap, nullptr, &rs1_input)); - gscoped_ptr rs2_input; + unique_ptr rs2_input; ASSERT_OK(CompactionInput::Create(*rs2, &schema_, all_snap, nullptr, &rs2_input)); vector> to_merge; to_merge.push_back(shared_ptr(rs1_input.release())); to_merge.push_back(shared_ptr(rs2_input.release())); - gscoped_ptr merged(CompactionInput::Merge(to_merge, &schema_)); + unique_ptr merged(CompactionInput::Merge(to_merge, &schema_)); // Make sure the unobservable version of the row that was inserted and deleted in the MRS // in the same transaction doesn't show up in the compaction input. @@ -919,7 +918,7 @@ TEST_F(TestCompaction, TestOneToOne) { // Catch the updates that came in after the snapshot flush was made. MvccSnapshot snap2(mvcc_); - gscoped_ptr input(CompactionInput::Create(*mrs, &schema_, snap2)); + unique_ptr input(CompactionInput::Create(*mrs, &schema_, snap2)); // Add some more updates which come into the new rowset while the "reupdate" is happening. UpdateRows(rs.get(), 1000, 0, 3); @@ -941,7 +940,7 @@ TEST_F(TestCompaction, TestOneToOne) { // And compact (1 input to 1 output) MvccSnapshot snap3(mvcc_); - gscoped_ptr compact_input; + unique_ptr compact_input; ASSERT_OK(CompactionInput::Create(*rs, &schema_, snap3, nullptr, &compact_input)); DoFlushAndReopen(compact_input.get(), schema_, snap3, kLargeRollThreshold, nullptr); } @@ -979,7 +978,7 @@ TEST_F(TestCompaction, TestKUDU102) { shared_ptr(CompactionInput::Create(*mrs, &schema_, snap2))); merge_inputs.push_back( shared_ptr(CompactionInput::Create(*mrs_b, &schema_, snap2))); - gscoped_ptr input(CompactionInput::Merge(merge_inputs, &schema_)); + unique_ptr input(CompactionInput::Merge(merge_inputs, &schema_)); string dummy_name = ""; @@ -1036,7 +1035,7 @@ TEST_F(TestCompaction, TestMergeMRS) { shared_ptr(CompactionInput::Create(*mrs_a, &schema_, snap))); merge_inputs.push_back( shared_ptr(CompactionInput::Create(*mrs_b, &schema_, snap))); - gscoped_ptr input(CompactionInput::Merge(merge_inputs, &schema_)); + unique_ptr input(CompactionInput::Merge(merge_inputs, &schema_)); vector out; IterateInput(input.get(), &out); diff --git a/src/kudu/tablet/compaction.cc b/src/kudu/tablet/compaction.cc index ecebbf8..1550447 100644 --- a/src/kudu/tablet/compaction.cc +++ b/src/kudu/tablet/compaction.cc @@ -179,7 +179,7 @@ class MemRowSetCompactionInput : public CompactionInput { private: DISALLOW_COPY_AND_ASSIGN(MemRowSetCompactionInput); - gscoped_ptr row_block_; + unique_ptr row_block_; unique_ptr iter_; @@ -415,7 +415,7 @@ class MergeCompactionInput : public CompactionInput { : schema_(schema), num_dup_rows_(0) { for (const shared_ptr &input : inputs) { - gscoped_ptr state(new MergeState); + unique_ptr state(new MergeState); state->input = input; states_.push_back(state.release()); } @@ -850,7 +850,7 @@ Status CompactionInput::Create(const DiskRowSet &rowset, const Schema* projection, const MvccSnapshot &snap, const IOContext* io_context, - gscoped_ptr* out) { + unique_ptr* out) { CHECK(projection->has_column_ids()); unique_ptr base_cwise(rowset.base_data_->NewIterator(projection, io_context)); @@ -902,7 +902,7 @@ Status RowSetsInCompaction::CreateCompactionInput(const MvccSnapshot &snap, vector > inputs; for (const shared_ptr &rs : rowsets_) { - gscoped_ptr input; + unique_ptr input; RETURN_NOT_OK_PREPEND(rs->NewCompactionInput(schema, snap, io_context, &input), Substitute("Could not create compaction input for rowset $0", rs->ToString())); @@ -910,7 +910,7 @@ Status RowSetsInCompaction::CreateCompactionInput(const MvccSnapshot &snap, } if (inputs.size() == 1) { - out->swap(inputs[0]); + *out = std::move(inputs[0]); } else { out->reset(CompactionInput::Merge(inputs, schema)); } @@ -1315,7 +1315,7 @@ Status ReupdateMissedDeltas(const IOContext* io_context, } DeltaTracker* cur_tracker = cur_drs->delta_tracker(); - gscoped_ptr result(new OperationResultPB); + unique_ptr result(new OperationResultPB); DCHECK_LT(idx_in_delta_tracker, num_rows); Status s = cur_tracker->Update(mut->timestamp(), idx_in_delta_tracker, diff --git a/src/kudu/tablet/compaction.h b/src/kudu/tablet/compaction.h index e6fc30a..0c101c1 100644 --- a/src/kudu/tablet/compaction.h +++ b/src/kudu/tablet/compaction.h @@ -14,8 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -#ifndef KUDU_TABLET_COMPACTION_H -#define KUDU_TABLET_COMPACTION_H +#pragma once #include #include @@ -28,7 +27,6 @@ #include "kudu/common/rowblock.h" #include "kudu/common/timestamp.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/tablet/rowset.h" #include "kudu/util/status.h" @@ -109,7 +107,7 @@ class CompactionInput { const Schema* projection, const MvccSnapshot &snap, const fs::IOContext* io_context, - gscoped_ptr* out); + std::unique_ptr* out); // Create an input which reads from the given memrowset, yielding base rows and updates // prior to the given snapshot. @@ -262,5 +260,3 @@ std::string CompactionInputRowToString(const CompactionInputRow& input_row); } // namespace tablet } // namespace kudu - -#endif diff --git a/src/kudu/tablet/delta_compaction-test.cc b/src/kudu/tablet/delta_compaction-test.cc index 2d7fa51..a602b11 100644 --- a/src/kudu/tablet/delta_compaction-test.cc +++ b/src/kudu/tablet/delta_compaction-test.cc @@ -36,7 +36,6 @@ #include "kudu/fs/block_id.h" #include "kudu/fs/block_manager.h" #include "kudu/fs/fs_manager.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/tablet/delta_iterator_merger.h" #include "kudu/tablet/delta_key.h" @@ -79,7 +78,7 @@ class TestDeltaCompaction : public KuduTest { return builder.Build(); } - Status GetDeltaFileWriter(gscoped_ptr* dfw, + Status GetDeltaFileWriter(unique_ptr* dfw, BlockId* block_id) const { unique_ptr block; RETURN_NOT_OK(fs_manager_->CreateNewBlock({}, &block)); @@ -108,7 +107,7 @@ class TestDeltaCompaction : public KuduTest { protected: int64_t deltafile_idx_; Schema schema_; - gscoped_ptr fs_manager_; + unique_ptr fs_manager_; }; TEST_F(TestDeltaCompaction, TestMergeMultipleSchemas) { @@ -135,7 +134,7 @@ TEST_F(TestDeltaCompaction, TestMergeMultipleSchemas) { for (const Schema& schema : schemas) { // Write the Deltas BlockId block_id; - gscoped_ptr dfw; + unique_ptr dfw; ASSERT_OK(GetDeltaFileWriter(&dfw, &block_id)); // Generate N updates with the new schema, some of them are on existing @@ -200,7 +199,7 @@ TEST_F(TestDeltaCompaction, TestMergeMultipleSchemas) { opts.projection = &merge_schema; unique_ptr merge_iter; ASSERT_OK(DeltaIteratorMerger::Create(inputs, opts, &merge_iter)); - gscoped_ptr dfw; + unique_ptr dfw; BlockId block_id; ASSERT_OK(GetDeltaFileWriter(&dfw, &block_id)); ASSERT_OK(WriteDeltaIteratorToFile(merge_iter.get(), diff --git a/src/kudu/tablet/delta_compaction.cc b/src/kudu/tablet/delta_compaction.cc index d6dbb9e..21eefdd 100644 --- a/src/kudu/tablet/delta_compaction.cc +++ b/src/kudu/tablet/delta_compaction.cc @@ -52,22 +52,19 @@ #include "kudu/util/memory/arena.h" #include "kudu/util/trace.h" +using kudu::fs::BlockCreationTransaction; +using kudu::fs::BlockManager; +using kudu::fs::CreateBlockOptions; +using kudu::fs::IOContext; +using kudu::fs::WritableBlock; using std::shared_ptr; - -namespace kudu { - -using fs::BlockCreationTransaction; -using fs::BlockManager; -using fs::CreateBlockOptions; -using fs::IOContext; -using fs::WritableBlock; using std::string; using std::unique_ptr; using std::vector; using strings::Substitute; +namespace kudu { namespace tablet { - namespace { const size_t kRowsPerBlock = 100; // Number of rows per block of columns @@ -267,11 +264,11 @@ Status MajorDeltaCompaction::FlushRowSetAndDeltas(const IOContext* io_context) { Status MajorDeltaCompaction::OpenBaseDataWriter() { CHECK(!base_data_writer_); - gscoped_ptr w(new MultiColumnWriter(fs_manager_, - &partial_schema_, - tablet_id_)); + unique_ptr w(new MultiColumnWriter(fs_manager_, + &partial_schema_, + tablet_id_)); RETURN_NOT_OK(w->Open()); - base_data_writer_.swap(w); + base_data_writer_ = std::move(w); return Status::OK(); } diff --git a/src/kudu/tablet/delta_compaction.h b/src/kudu/tablet/delta_compaction.h index f790504..9bc5caa 100644 --- a/src/kudu/tablet/delta_compaction.h +++ b/src/kudu/tablet/delta_compaction.h @@ -14,8 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -#ifndef KUDU_TABLET_DELTA_COMPACTION_H -#define KUDU_TABLET_DELTA_COMPACTION_H +#pragma once #include #include @@ -24,7 +23,6 @@ #include "kudu/common/schema.h" #include "kudu/fs/block_id.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/tablet/compaction.h" #include "kudu/tablet/delta_store.h" #include "kudu/util/status.h" @@ -129,12 +127,12 @@ class MajorDeltaCompaction { const std::string tablet_id_; // Outputs: - gscoped_ptr base_data_writer_; + std::unique_ptr base_data_writer_; // The following two may not be initialized if we don't need to write a delta file. - gscoped_ptr new_redo_delta_writer_; + std::unique_ptr new_redo_delta_writer_; BlockId new_redo_delta_block_; - gscoped_ptr new_undo_delta_writer_; + std::unique_ptr new_undo_delta_writer_; BlockId new_undo_delta_block_; size_t redo_delta_mutations_written_; @@ -149,5 +147,3 @@ class MajorDeltaCompaction { } // namespace tablet } // namespace kudu - -#endif diff --git a/src/kudu/tablet/delta_tracker.cc b/src/kudu/tablet/delta_tracker.cc index 0c0b0d7..00d47d7 100644 --- a/src/kudu/tablet/delta_tracker.cc +++ b/src/kudu/tablet/delta_tracker.cc @@ -77,13 +77,13 @@ Status DeltaTracker::Open(const shared_ptr& rowset_metadata, LogAnchorRegistry* log_anchor_registry, const TabletMemTrackers& mem_trackers, const IOContext* io_context, - gscoped_ptr* delta_tracker) { - gscoped_ptr local_dt( + unique_ptr* delta_tracker) { + unique_ptr local_dt( new DeltaTracker(rowset_metadata, log_anchor_registry, mem_trackers)); RETURN_NOT_OK(local_dt->DoOpen(io_context)); - delta_tracker->swap(local_dt); + *delta_tracker = std::move(local_dt); return Status::OK(); } @@ -166,7 +166,7 @@ Status DeltaTracker::CreateAndInitDMSUnlocked(const fs::IOContext* io_context) { &dms)); RETURN_NOT_OK(dms->Init(io_context)); - dms_.swap(dms); + dms_ = std::move(dms); dms_exists_.Store(true); return Status::OK(); } @@ -718,7 +718,7 @@ Status DeltaTracker::FlushDMS(DeltaMemStore* dms, Substitute("Unable to start writing to delta block $0", block_id.ToString())); - gscoped_ptr stats; + unique_ptr stats; RETURN_NOT_OK(dms->FlushToFile(&dfw, &stats)); RETURN_NOT_OK(dfw.Finish()); const auto bytes_written = dfw.written_size(); @@ -779,7 +779,7 @@ Status DeltaTracker::Flush(const IOContext* io_context, MetadataFlushType flush_ count = dms_exists_.Load() ? dms_->Count() : 0; // Swap the DeltaMemStore and dms_ is null now. - old_dms.swap(dms_); + old_dms = std::move(dms_); dms_exists_.Store(false); if (count == 0) { diff --git a/src/kudu/tablet/delta_tracker.h b/src/kudu/tablet/delta_tracker.h index a4ea0a1..5ae97a2 100644 --- a/src/kudu/tablet/delta_tracker.h +++ b/src/kudu/tablet/delta_tracker.h @@ -14,8 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -#ifndef KUDU_TABLET_DELTATRACKER_H -#define KUDU_TABLET_DELTATRACKER_H +#pragma once #include #include @@ -26,7 +25,6 @@ #include #include "kudu/common/rowid.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/tablet/cfile_set.h" #include "kudu/tablet/delta_key.h" @@ -87,7 +85,7 @@ class DeltaTracker { log::LogAnchorRegistry* log_anchor_registry, const TabletMemTrackers& mem_trackers, const fs::IOContext* io_context, - gscoped_ptr* delta_tracker); + std::unique_ptr* delta_tracker); Status WrapIterator(const std::shared_ptr &base, const RowIteratorOptions& opts, @@ -384,5 +382,3 @@ class DeltaTracker { } // namespace tablet } // namespace kudu - -#endif diff --git a/src/kudu/tablet/deltafile.cc b/src/kudu/tablet/deltafile.cc index 26a3495..227429a 100644 --- a/src/kudu/tablet/deltafile.cc +++ b/src/kudu/tablet/deltafile.cc @@ -24,7 +24,6 @@ #include #include -#include #include "kudu/cfile/binary_plain_block.h" #include "kudu/cfile/cfile_reader.h" @@ -41,7 +40,6 @@ #include "kudu/fs/block_id.h" #include "kudu/fs/block_manager.h" #include "kudu/fs/fs_manager.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/stringprintf.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/tablet/delta_relevancy.h" @@ -235,7 +233,7 @@ Status DeltaFileReader::OpenNoInit(unique_ptr block, RETURN_NOT_OK(CFileReader::OpenNoInit(std::move(block), std::move(options), &cf_reader)); - gscoped_ptr df_reader( + unique_ptr df_reader( new DeltaFileReader(std::move(cf_reader), delta_type)); if (!FLAGS_cfile_lazy_open) { RETURN_NOT_OK(df_reader->Init(io_context)); @@ -280,9 +278,9 @@ Status DeltaFileReader::ReadDeltaStats() { if (!deltastats_pb.ParseFromString(filestats_pb_buf)) { return Status::Corruption("unable to parse the delta stats protobuf"); } - gscoped_ptrstats(new DeltaStats()); + unique_ptr stats(new DeltaStats()); RETURN_NOT_OK(stats->InitFromPB(deltastats_pb)); - delta_stats_.swap(stats); + delta_stats_ = std::move(stats); return Status::OK(); } diff --git a/src/kudu/tablet/deltafile.h b/src/kudu/tablet/deltafile.h index 56632ff..68222be 100644 --- a/src/kudu/tablet/deltafile.h +++ b/src/kudu/tablet/deltafile.h @@ -14,8 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -#ifndef KUDU_TABLET_DELTAFILE_H -#define KUDU_TABLET_DELTAFILE_H +#pragma once #include #include @@ -32,7 +31,6 @@ #include "kudu/cfile/cfile_reader.h" #include "kudu/cfile/cfile_writer.h" #include "kudu/common/rowid.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/port.h" #include "kudu/gutil/strings/substitute.h" @@ -211,7 +209,7 @@ class DeltaFileReader : public DeltaStore, Status ReadDeltaStats(); std::shared_ptr reader_; - gscoped_ptr delta_stats_; + std::unique_ptr delta_stats_; // The type of this delta, i.e. UNDO or REDO. const DeltaType delta_type_; @@ -269,7 +267,7 @@ class DeltaFileIterator : public DeltaIterator { // The block decoder, to avoid having to re-parse the block header // on every ApplyUpdates() call - gscoped_ptr decoder_; + std::unique_ptr decoder_; // The first row index for which there is an update in this delta block. rowid_t first_updated_idx_; @@ -318,7 +316,7 @@ class DeltaFileIterator : public DeltaIterator { DeltaPreparer> preparer_; - gscoped_ptr index_iter_; + std::unique_ptr index_iter_; bool prepared_; bool exhausted_; @@ -337,5 +335,3 @@ class DeltaFileIterator : public DeltaIterator { } // namespace tablet } // namespace kudu - -#endif diff --git a/src/kudu/tablet/deltamemstore-test.cc b/src/kudu/tablet/deltamemstore-test.cc index 5fde7c4..51099e5 100644 --- a/src/kudu/tablet/deltamemstore-test.cc +++ b/src/kudu/tablet/deltamemstore-test.cc @@ -48,7 +48,6 @@ #include "kudu/consensus/opid_util.h" #include "kudu/fs/block_manager.h" #include "kudu/fs/fs_manager.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/tablet/delta_key.h" #include "kudu/tablet/delta_stats.h" @@ -212,7 +211,7 @@ TEST_F(TestDeltaMemStore, TestUpdateCount) { ASSERT_OK(fs.CreateNewBlock({}, &block)); DeltaFileWriter dfw(std::move(block)); ASSERT_OK(dfw.Start()); - gscoped_ptr stats; + unique_ptr stats; dms_->FlushToFile(&dfw, &stats); ASSERT_EQ(n_rows / 2, stats->update_count_for_col_id(schema_.column_id(kIntColumn))); diff --git a/src/kudu/tablet/deltamemstore.cc b/src/kudu/tablet/deltamemstore.cc index e356655..b78a108 100644 --- a/src/kudu/tablet/deltamemstore.cc +++ b/src/kudu/tablet/deltamemstore.cc @@ -127,13 +127,14 @@ Status DeltaMemStore::Update(Timestamp timestamp, } Status DeltaMemStore::FlushToFile(DeltaFileWriter *dfw, - gscoped_ptr* stats_ret) { - gscoped_ptr stats(new DeltaStats()); + unique_ptr* stats_ret) { + unique_ptr stats(new DeltaStats()); - gscoped_ptr iter(tree_.NewIterator()); + unique_ptr iter(tree_.NewIterator()); iter->SeekToStart(); while (iter->IsValid()) { - Slice key_slice, val; + Slice key_slice; + Slice val; iter->GetCurrentEntry(&key_slice, &val); DeltaKey key; RETURN_NOT_OK(key.DecodeFrom(&key_slice)); @@ -165,7 +166,7 @@ Status DeltaMemStore::CheckRowDeleted(rowid_t row_idx, bool exact; - gscoped_ptr iter(tree_.NewIterator()); + unique_ptr iter(tree_.NewIterator()); if (!iter->SeekAtOrBefore(key_slice, &exact)) { return Status::OK(); } diff --git a/src/kudu/tablet/deltamemstore.h b/src/kudu/tablet/deltamemstore.h index 687f45e..e287b68 100644 --- a/src/kudu/tablet/deltamemstore.h +++ b/src/kudu/tablet/deltamemstore.h @@ -14,8 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -#ifndef KUDU_TABLET_DELTAMEMSTORE_H -#define KUDU_TABLET_DELTAMEMSTORE_H +#pragma once #include #include @@ -26,7 +25,6 @@ #include "kudu/common/rowid.h" #include "kudu/consensus/log_anchor_registry.h" #include "kudu/gutil/atomicops.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/port.h" #include "kudu/tablet/concurrent_btree.h" @@ -105,7 +103,7 @@ class DeltaMemStore : public DeltaStore, // Flush the DMS to the given file writer. // Returns statistics in *stats. Status FlushToFile(DeltaFileWriter *dfw, - gscoped_ptr* stats); + std::unique_ptr* stats); // Create an iterator for applying deltas from this DMS. // @@ -241,7 +239,7 @@ class DMSIterator : public DeltaIterator { DeltaPreparer preparer_; - gscoped_ptr iter_; + std::unique_ptr iter_; bool initted_; @@ -251,5 +249,3 @@ class DMSIterator : public DeltaIterator { } // namespace tablet } // namespace kudu - -#endif diff --git a/src/kudu/tablet/diskrowset-test.cc b/src/kudu/tablet/diskrowset-test.cc index bc6f3f7..947c5a5 100644 --- a/src/kudu/tablet/diskrowset-test.cc +++ b/src/kudu/tablet/diskrowset-test.cc @@ -46,7 +46,6 @@ #include "kudu/consensus/opid_util.h" #include "kudu/fs/block_id.h" #include "kudu/fs/io_context.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/stringprintf.h" #include "kudu/gutil/strings/stringpiece.h" diff --git a/src/kudu/tablet/diskrowset.cc b/src/kudu/tablet/diskrowset.cc index 64ec1e1..e7c3d59 100644 --- a/src/kudu/tablet/diskrowset.cc +++ b/src/kudu/tablet/diskrowset.cc @@ -39,7 +39,6 @@ #include "kudu/common/types.h" #include "kudu/fs/block_manager.h" #include "kudu/fs/fs_manager.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/tablet/cfile_set.h" #include "kudu/tablet/compaction.h" @@ -368,8 +367,8 @@ Status RollingDiskRowSetWriter::RollWriter() { cur_redo_ds_block_id_ = redo_data_block->id(); cur_undo_writer_.reset(new DeltaFileWriter(std::move(undo_data_block))); cur_redo_writer_.reset(new DeltaFileWriter(std::move(redo_data_block))); - cur_undo_delta_stats.reset(new DeltaStats()); - cur_redo_delta_stats.reset(new DeltaStats()); + cur_undo_delta_stats_.reset(new DeltaStats()); + cur_redo_delta_stats_.reset(new DeltaStats()); row_idx_in_cur_drs_ = 0; can_roll_ = false; @@ -403,7 +402,7 @@ Status RollingDiskRowSetWriter::AppendUndoDeltas(rowid_t row_idx_in_block, return AppendDeltas(row_idx_in_block, undo_delta_head, row_idx, cur_undo_writer_.get(), - cur_undo_delta_stats.get()); + cur_undo_delta_stats_.get()); } Status RollingDiskRowSetWriter::AppendRedoDeltas(rowid_t row_idx_in_block, @@ -412,7 +411,7 @@ Status RollingDiskRowSetWriter::AppendRedoDeltas(rowid_t row_idx_in_block, return AppendDeltas(row_idx_in_block, redo_delta_head, row_idx, cur_redo_writer_.get(), - cur_redo_delta_stats.get()); + cur_redo_delta_stats_.get()); } template @@ -450,8 +449,8 @@ Status RollingDiskRowSetWriter::FinishCurrentWriter() { RETURN_NOT_OK(writer_status); CHECK_GT(cur_writer_->written_count(), 0); - cur_undo_writer_->WriteDeltaStats(*cur_undo_delta_stats); - cur_redo_writer_->WriteDeltaStats(*cur_redo_delta_stats); + cur_undo_writer_->WriteDeltaStats(*cur_undo_delta_stats_); + cur_redo_writer_->WriteDeltaStats(*cur_redo_delta_stats_); // Commit the UNDO block. Status::Aborted() indicates that there // were no UNDOs written. @@ -460,7 +459,7 @@ Status RollingDiskRowSetWriter::FinishCurrentWriter() { RETURN_NOT_OK(s); cur_drs_metadata_->CommitUndoDeltaDataBlock(cur_undo_ds_block_id_); } else { - DCHECK_EQ(cur_undo_delta_stats->min_timestamp(), Timestamp::kMax); + DCHECK_EQ(cur_undo_delta_stats_->min_timestamp(), Timestamp::kMax); } // Same for the REDO block. @@ -469,7 +468,7 @@ Status RollingDiskRowSetWriter::FinishCurrentWriter() { RETURN_NOT_OK(s); cur_drs_metadata_->CommitRedoDeltaDataBlock(0, 0, cur_redo_ds_block_id_); } else { - DCHECK_EQ(cur_redo_delta_stats->min_timestamp(), Timestamp::kMax); + DCHECK_EQ(cur_redo_delta_stats_->min_timestamp(), Timestamp::kMax); } written_size_ += cur_writer_->written_size(); @@ -583,7 +582,7 @@ Status DiskRowSet::MajorCompactDeltaStoresWithColumnIds(const vector& RETURN_NOT_OK(delta_tracker()->CheckWritableUnlocked()); // TODO(todd): do we need to lock schema or anything here? - gscoped_ptr compaction; + unique_ptr compaction; RETURN_NOT_OK(NewMajorDeltaCompaction(col_ids, std::move(history_gc_opts), io_context, &compaction)); @@ -633,7 +632,7 @@ Status DiskRowSet::MajorCompactDeltaStoresWithColumnIds(const vector& Status DiskRowSet::NewMajorDeltaCompaction(const vector& col_ids, HistoryGcOpts history_gc_opts, const IOContext* io_context, - gscoped_ptr* out) const { + unique_ptr* out) const { DCHECK(open_); shared_lock l(component_lock_); @@ -675,7 +674,7 @@ Status DiskRowSet::NewRowIterator(const RowIteratorOptions& opts, Status DiskRowSet::NewCompactionInput(const Schema* projection, const MvccSnapshot &snap, const IOContext* io_context, - gscoped_ptr* out) const { + unique_ptr* out) const { return CompactionInput::Create(*this, projection, snap, io_context, out); } @@ -895,7 +894,7 @@ Status DiskRowSet::DeleteAncientUndoDeltas(Timestamp ancient_history_mark, Status DiskRowSet::DebugDump(vector *lines) { // Using CompactionInput to dump our data is an easy way of seeing all the // rows and deltas. - gscoped_ptr input; + unique_ptr input; RETURN_NOT_OK(NewCompactionInput(&rowset_metadata_->tablet_schema(), MvccSnapshot::CreateSnapshotIncludingAllTransactions(), nullptr, &input)); diff --git a/src/kudu/tablet/diskrowset.h b/src/kudu/tablet/diskrowset.h index 52689a4..045826b 100644 --- a/src/kudu/tablet/diskrowset.h +++ b/src/kudu/tablet/diskrowset.h @@ -35,7 +35,6 @@ #include "kudu/common/schema.h" #include "kudu/fs/block_id.h" #include "kudu/fs/fs_manager.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/strings/substitute.h" #include "kudu/tablet/delta_key.h" @@ -147,9 +146,9 @@ class DiskRowSetWriter { bool finished_; rowid_t written_count_; - gscoped_ptr col_writer_; - gscoped_ptr bloom_writer_; - gscoped_ptr ad_hoc_index_writer_; + std::unique_ptr col_writer_; + std::unique_ptr bloom_writer_; + std::unique_ptr ad_hoc_index_writer_; // The last encoded key written. faststring last_encoded_key_; @@ -184,20 +183,20 @@ class RollingDiskRowSetWriter { // 'live_row_count' means the number of live rows in this input block. Status AppendBlock(const RowBlock &block, int live_row_count = 0); - // Appends a sequence of REDO deltas for the same row to the current - // redo delta file. 'row_idx_in_next_block' is the positional index after - // the last written block. The 'row_idx_in_drs' out parameter will be set - // with the row index from the start of the DiskRowSet currently being written. - Status AppendRedoDeltas(rowid_t row_idx_in_next_block, - Mutation* redo_deltas, + // Appends a sequence of REDO deltas for the same row to the current redo + // delta file. 'row_idx_in_block' is the positional index after the last + // written block. The 'row_idx_in_drs' out parameter will be set with the row + // index from the start of the DiskRowSet currently being written. + Status AppendRedoDeltas(rowid_t row_idx_in_block, + Mutation* redo_delta_head, rowid_t* row_idx_in_drs); - // Appends a sequence of UNDO deltas for the same row to the current - // undo delta file. 'row_idx_in_next_block' is the positional index after - // the last written block. The 'row_idx_in_drs' out parameter will be set - // with the row index from the start of the DiskRowSet currently being written. - Status AppendUndoDeltas(rowid_t row_idx_in_next_block, - Mutation* undo_deltas, + // Appends a sequence of UNDO deltas for the same row to the current undo + // delta file. 'row_idx_in_block' is the positional index after the last + // written block. The 'row_idx_in_drs' out parameter will be set with the row + // index from the start of the DiskRowSet currently being written. + Status AppendUndoDeltas(rowid_t row_idx_in_block, + Mutation* undo_delta_head, rowid_t* row_idx_in_drs); // Try to roll the output, if we've passed the configured threshold. This will @@ -248,14 +247,14 @@ class RollingDiskRowSetWriter { const BloomFilterSizing bloom_sizing_; const size_t target_rowset_size_; - gscoped_ptr cur_writer_; + std::unique_ptr cur_writer_; // A delta writer to store the undos for each DRS - gscoped_ptr cur_undo_writer_; - gscoped_ptr cur_undo_delta_stats; + std::unique_ptr cur_undo_writer_; + std::unique_ptr cur_undo_delta_stats_; // a delta writer to store the redos for each DRS - gscoped_ptr cur_redo_writer_; - gscoped_ptr cur_redo_delta_stats; + std::unique_ptr cur_redo_writer_; + std::unique_ptr cur_redo_delta_stats_; BlockId cur_undo_ds_block_id_; BlockId cur_redo_ds_block_id_; @@ -363,7 +362,7 @@ class DiskRowSet : public RowSet { virtual Status NewCompactionInput(const Schema* projection, const MvccSnapshot &snap, const fs::IOContext* io_context, - gscoped_ptr* out) const override; + std::unique_ptr* out) const override; // Gets the number of rows in this rowset, checking 'num_rows_' first. If not // yet set, consults the base data and stores the result in 'num_rows_'. @@ -463,7 +462,7 @@ class DiskRowSet : public RowSet { Status NewMajorDeltaCompaction(const std::vector& col_ids, HistoryGcOpts history_gc_opts, const fs::IOContext* io_context, - gscoped_ptr* out) const; + std::unique_ptr* out) const; // Major compacts all the delta files for the specified columns. Status MajorCompactDeltaStoresWithColumnIds(const std::vector& col_ids, @@ -481,7 +480,7 @@ class DiskRowSet : public RowSet { // Base data for this rowset. mutable rw_spinlock component_lock_; std::shared_ptr base_data_; - gscoped_ptr delta_tracker_; + std::unique_ptr delta_tracker_; // Number of rows in the rowset. This may be unset (-1) if the rows in the // underlying cfile set have not been counted yet. diff --git a/src/kudu/tablet/local_tablet_writer.h b/src/kudu/tablet/local_tablet_writer.h index dc5e938..33fde5f 100644 --- a/src/kudu/tablet/local_tablet_writer.h +++ b/src/kudu/tablet/local_tablet_writer.h @@ -14,9 +14,9 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -#ifndef KUDU_TABLET_LOCAL_TABLET_WRITER_H -#define KUDU_TABLET_LOCAL_TABLET_WRITER_H +#pragma once +#include #include #include "kudu/common/partial_row.h" @@ -144,7 +144,7 @@ class LocalTabletWriter { TxResultPB result_; tserver::WriteRequestPB req_; - gscoped_ptr tx_state_; + std::unique_ptr tx_state_; DISALLOW_COPY_AND_ASSIGN(LocalTabletWriter); }; @@ -152,4 +152,3 @@ class LocalTabletWriter { } // namespace tablet } // namespace kudu -#endif /* KUDU_TABLET_LOCAL_TABLET_WRITER_H */ diff --git a/src/kudu/tablet/lock_manager.cc b/src/kudu/tablet/lock_manager.cc index e914994..ac6ecf6 100644 --- a/src/kudu/tablet/lock_manager.cc +++ b/src/kudu/tablet/lock_manager.cc @@ -18,6 +18,7 @@ #include "kudu/tablet/lock_manager.h" #include +#include #include #include #include @@ -26,7 +27,6 @@ #include "kudu/gutil/atomicops.h" #include "kudu/gutil/dynamic_annotations.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/hash/city.h" #include "kudu/gutil/port.h" #include "kudu/gutil/walltime.h" @@ -38,6 +38,8 @@ #include "kudu/util/trace.h" using base::subtle::NoBarrier_Load; +using std::string; +using std::unique_ptr; namespace kudu { namespace tablet { @@ -64,7 +66,7 @@ class LockEntry { return key_hash_ == hash && key_ == key; } - std::string ToString() const { + string ToString() const { return KUDU_REDACT(key_.ToDebugString()); } @@ -165,7 +167,7 @@ class LockTable { // number of buckets in the table uint64_t size_; // table buckets - gscoped_array buckets_; + unique_ptr buckets_; // number of items in the table base::subtle::Atomic64 item_count_; }; @@ -244,7 +246,7 @@ void LockTable::Resize() { return; // Allocate a new bucket list - gscoped_array new_buckets(new Bucket[new_size]); + unique_ptr new_buckets(new Bucket[new_size]); size_t new_mask = new_size - 1; // Copy entries diff --git a/src/kudu/tablet/memrowset.cc b/src/kudu/tablet/memrowset.cc index 49d04b7..8a1db29 100644 --- a/src/kudu/tablet/memrowset.cc +++ b/src/kudu/tablet/memrowset.cc @@ -19,7 +19,6 @@ #include #include -#include #include #include @@ -310,7 +309,7 @@ Status MemRowSet::NewRowIterator(const RowIteratorOptions& opts, Status MemRowSet::NewCompactionInput(const Schema* projection, const MvccSnapshot& snap, const IOContext* /*io_context*/, - gscoped_ptr* out) const { + unique_ptr* out) const { out->reset(CompactionInput::Create(*this, projection, snap)); return Status::OK(); } @@ -342,7 +341,7 @@ typedef MemRowSet::Iterator::MRSRowProjector MRSRowProjector; template class MRSRowProjectorImpl : public MRSRowProjector { public: - explicit MRSRowProjectorImpl(gscoped_ptr actual) + explicit MRSRowProjectorImpl(unique_ptr actual) : actual_(std::move(actual)) {} Status Init() override { return actual_->Init(); } @@ -362,26 +361,26 @@ class MRSRowProjectorImpl : public MRSRowProjector { } private: - gscoped_ptr actual_; + unique_ptr actual_; }; // If codegen is enabled, then generates a codegen::RowProjector; // otherwise makes a regular one. -gscoped_ptr GenerateAppropriateProjector( +unique_ptr GenerateAppropriateProjector( const Schema* base, const Schema* projection) { // Attempt code-generated implementation if (FLAGS_mrs_use_codegen) { - gscoped_ptr actual; + unique_ptr actual; if (codegen::CompilationManager::GetSingleton()->RequestRowProjector( base, projection, &actual)) { - return gscoped_ptr( + return unique_ptr( new MRSRowProjectorImpl(std::move(actual))); } } // Proceed with default implementation - gscoped_ptr actual(new RowProjector(base, projection)); - return gscoped_ptr( + unique_ptr actual(new RowProjector(base, projection)); + return unique_ptr( new MRSRowProjectorImpl(std::move(actual))); } diff --git a/src/kudu/tablet/memrowset.h b/src/kudu/tablet/memrowset.h index f632cc7..ecd8e03 100644 --- a/src/kudu/tablet/memrowset.h +++ b/src/kudu/tablet/memrowset.h @@ -36,7 +36,6 @@ #include "kudu/common/timestamp.h" #include "kudu/consensus/log_anchor_registry.h" #include "kudu/gutil/atomicops.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/tablet/concurrent_btree.h" #include "kudu/tablet/rowset.h" @@ -331,7 +330,7 @@ class MemRowSet : public RowSet, virtual Status NewCompactionInput(const Schema* projection, const MvccSnapshot& snap, const fs::IOContext* io_context, - gscoped_ptr* out) const override; + std::unique_ptr* out) const override; // Return the Schema for the rows in this memrowset. const Schema &schema() const { @@ -592,7 +591,7 @@ class MemRowSet::Iterator : public RowwiseIterator { ApplyStatus* apply_status); const std::shared_ptr memrowset_; - gscoped_ptr iter_; + std::unique_ptr iter_; const RowIteratorOptions opts_; @@ -600,7 +599,7 @@ class MemRowSet::Iterator : public RowwiseIterator { // Relies on the MRSRowProjector interface to abstract from the two // different implementations of the RowProjector, which may change // at runtime (using vs. not using code generation). - const gscoped_ptr projector_; + const std::unique_ptr projector_; DeltaProjector delta_projector_; // The index of the first IS_DELETED virtual column in the projection schema, diff --git a/src/kudu/tablet/mock-rowsets.h b/src/kudu/tablet/mock-rowsets.h index 719b748..b86153f 100644 --- a/src/kudu/tablet/mock-rowsets.h +++ b/src/kudu/tablet/mock-rowsets.h @@ -14,8 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -#ifndef KUDU_TABLET_MOCK_ROWSETS_H -#define KUDU_TABLET_MOCK_ROWSETS_H +#pragma once #include #include @@ -62,7 +61,7 @@ class MockRowSet : public RowSet { virtual Status NewCompactionInput(const Schema* /*projection*/, const MvccSnapshot& /*snap*/, const fs::IOContext* /*io_context*/, - gscoped_ptr* /*out*/) const OVERRIDE { + std::unique_ptr* /*out*/) const OVERRIDE { LOG(FATAL) << "Unimplemented"; return Status::OK(); } @@ -237,4 +236,3 @@ class MockMemRowSet : public MockRowSet { } // namespace tablet } // namespace kudu -#endif /* KUDU_TABLET_MOCK_ROWSETS_H */ diff --git a/src/kudu/tablet/multi_column_writer.cc b/src/kudu/tablet/multi_column_writer.cc index 6e39b0f..667aa3f 100644 --- a/src/kudu/tablet/multi_column_writer.cc +++ b/src/kudu/tablet/multi_column_writer.cc @@ -30,7 +30,6 @@ #include "kudu/fs/block_id.h" #include "kudu/fs/block_manager.h" #include "kudu/fs/fs_manager.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/stl_util.h" #include "kudu/gutil/strings/substitute.h" @@ -85,7 +84,7 @@ Status MultiColumnWriter::Open() { BlockId block_id(block->id()); // Create the CFile writer itself. - gscoped_ptr writer(new CFileWriter( + unique_ptr writer(new CFileWriter( std::move(opts), col.type_info(), col.is_nullable(), diff --git a/src/kudu/tablet/row_op.cc b/src/kudu/tablet/row_op.cc index efe28e0..989ebfa 100644 --- a/src/kudu/tablet/row_op.cc +++ b/src/kudu/tablet/row_op.cc @@ -17,7 +17,7 @@ #include "kudu/tablet/row_op.h" -#include +#include #include #include @@ -28,6 +28,7 @@ #include "kudu/util/status.h" using kudu::pb_util::SecureDebugString; +using std::unique_ptr; namespace kudu { @@ -58,7 +59,7 @@ void RowOp::SetErrorIgnored() { error_ignored = true; } -void RowOp::SetMutateSucceeded(gscoped_ptr result) { +void RowOp::SetMutateSucceeded(unique_ptr result) { DCHECK(!this->result) << SecureDebugString(*result); this->result = std::move(result); } diff --git a/src/kudu/tablet/row_op.h b/src/kudu/tablet/row_op.h index ade5ef9..c2bbf16 100644 --- a/src/kudu/tablet/row_op.h +++ b/src/kudu/tablet/row_op.h @@ -14,13 +14,12 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -#ifndef KUDU_TABLET_ROW_OP_H -#define KUDU_TABLET_ROW_OP_H +#pragma once +#include #include #include "kudu/common/row_operations.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/tablet/lock_manager.h" #include "kudu/tablet/rowset.h" #include "kudu/tablet/tablet.pb.h" @@ -43,7 +42,7 @@ struct RowOp { void SetFailed(const Status& s); void SetInsertSucceeded(int mrs_id); void SetErrorIgnored(); - void SetMutateSucceeded(gscoped_ptr result); + void SetMutateSucceeded(std::unique_ptr result); // Sets the result of a skipped operation on bootstrap. // TODO(dralves) Currently this performs a copy. Might be avoided with some refactoring. // see TODO(dralves) in TabletBoostrap::ApplyOperations(). @@ -78,7 +77,7 @@ struct RowOp { // The key probe structure contains the row key in both key-encoded and // ContiguousRow formats, bloom probe structure, etc. This is set during // the "prepare" phase. - gscoped_ptr key_probe; + std::unique_ptr key_probe; // The row lock which has been acquired for this row. Set during the "prepare" // phase. @@ -102,10 +101,9 @@ struct RowOp { RowSet* present_in_rowset = nullptr; // The result of the operation. - gscoped_ptr result; + std::unique_ptr result; }; } // namespace tablet } // namespace kudu -#endif /* KUDU_TABLET_ROW_OP_H */ diff --git a/src/kudu/tablet/rowset.cc b/src/kudu/tablet/rowset.cc index 3566fb1..2247e2d 100644 --- a/src/kudu/tablet/rowset.cc +++ b/src/kudu/tablet/rowset.cc @@ -138,7 +138,7 @@ Status DuplicatingRowSet::NewRowIterator(const RowIteratorOptions& opts, Status DuplicatingRowSet::NewCompactionInput(const Schema* /*projection*/, const MvccSnapshot& /*snap*/, const IOContext* /*io_context*/, - gscoped_ptr* /*out*/) const { + unique_ptr* /*out*/) const { LOG(FATAL) << "duplicating rowsets do not act as compaction input"; return Status::OK(); } diff --git a/src/kudu/tablet/rowset.h b/src/kudu/tablet/rowset.h index 9e96e2c..524572a 100644 --- a/src/kudu/tablet/rowset.h +++ b/src/kudu/tablet/rowset.h @@ -32,7 +32,6 @@ #include "kudu/common/row.h" #include "kudu/common/rowid.h" #include "kudu/common/timestamp.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/port.h" #include "kudu/tablet/mvcc.h" @@ -157,7 +156,7 @@ class RowSet { virtual Status NewCompactionInput(const Schema* projection, const MvccSnapshot &snap, const fs::IOContext* io_context, - gscoped_ptr* out) const = 0; + std::unique_ptr* out) const = 0; // Count the number of rows in this rowset. virtual Status CountRows(const fs::IOContext* io_context, rowid_t *count) const = 0; @@ -404,7 +403,7 @@ class DuplicatingRowSet : public RowSet { virtual Status NewCompactionInput(const Schema* projection, const MvccSnapshot &snap, const fs::IOContext* io_context, - gscoped_ptr* out) const OVERRIDE; + std::unique_ptr* out) const OVERRIDE; Status CountRows(const fs::IOContext* io_context, rowid_t *count) const OVERRIDE; diff --git a/src/kudu/tablet/rowset_tree.cc b/src/kudu/tablet/rowset_tree.cc index dc3a25b..0ff9dea 100644 --- a/src/kudu/tablet/rowset_tree.cc +++ b/src/kudu/tablet/rowset_tree.cc @@ -34,9 +34,10 @@ #include "kudu/util/interval_tree-inl.h" #include "kudu/util/slice.h" -using std::vector; using std::shared_ptr; using std::string; +using std::unique_ptr; +using std::vector; namespace kudu { namespace tablet { @@ -137,9 +138,10 @@ Status RowSetTree::Reset(const RowSetVector &rowsets) { // Iterate over each of the provided RowSets, fetching their // bounds and adding them to the local vectors. for (const shared_ptr &rs : rowsets) { - gscoped_ptr rsit(new RowSetWithBounds()); + unique_ptr rsit(new RowSetWithBounds()); rsit->rowset = rs.get(); - string min_key, max_key; + string min_key; + string max_key; Status s = rs->GetBounds(&min_key, &max_key); if (s.IsNotSupported()) { // This rowset is a MemRowSet, for which the bounds change as more diff --git a/src/kudu/tablet/rowset_tree.h b/src/kudu/tablet/rowset_tree.h index 7c566de..1245aee 100644 --- a/src/kudu/tablet/rowset_tree.h +++ b/src/kudu/tablet/rowset_tree.h @@ -14,17 +14,16 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -#ifndef KUDU_TABLET_ROWSET_MANAGER_H -#define KUDU_TABLET_ROWSET_MANAGER_H +#pragma once #include #include +#include #include #include #include -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/tablet/rowset.h" #include "kudu/util/slice.h" @@ -112,7 +111,7 @@ class RowSetTree { private: // Interval tree of the rowsets. Used to efficiently find rowsets which might contain // a probe row. - gscoped_ptr > tree_; + std::unique_ptr> tree_; // Ordered map of all the interval endpoints, holding the implicit contiguous // intervals @@ -142,4 +141,3 @@ class RowSetTree { } // namespace tablet } // namespace kudu -#endif diff --git a/src/kudu/tablet/tablet-harness.h b/src/kudu/tablet/tablet-harness.h index c3197e9..346155d 100644 --- a/src/kudu/tablet/tablet-harness.h +++ b/src/kudu/tablet/tablet-harness.h @@ -158,7 +158,7 @@ class TabletHarness { std::unique_ptr clock_; Schema schema_; - gscoped_ptr fs_manager_; + std::unique_ptr fs_manager_; std::shared_ptr tablet_; }; diff --git a/src/kudu/tablet/tablet-test-util.h b/src/kudu/tablet/tablet-test-util.h index 5e0bcc9..937221b 100644 --- a/src/kudu/tablet/tablet-test-util.h +++ b/src/kudu/tablet/tablet-test-util.h @@ -52,7 +52,6 @@ #include "kudu/fs/block_id.h" #include "kudu/fs/block_manager.h" #include "kudu/fs/fs_manager.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/port.h" #include "kudu/gutil/strings/join.h" @@ -173,7 +172,7 @@ class KuduTabletTest : public KuduTest { const Schema client_schema_; const TabletHarness::Options::ClockType clock_type_; - gscoped_ptr harness_; + std::unique_ptr harness_; }; class KuduRowSetTest : public KuduTabletTest { diff --git a/src/kudu/tablet/tablet.cc b/src/kudu/tablet/tablet.cc index d2a158c..32541cf 100644 --- a/src/kudu/tablet/tablet.cc +++ b/src/kudu/tablet/tablet.cc @@ -22,7 +22,6 @@ #include #include #include -#include #include #include #include @@ -531,7 +530,7 @@ void Tablet::AssignTimestampAndStartTransactionForTests(WriteTransactionState* t } void Tablet::StartTransaction(WriteTransactionState* tx_state) { - gscoped_ptr mvcc_tx; + unique_ptr mvcc_tx; DCHECK(tx_state->has_timestamp()); mvcc_tx.reset(new ScopedTransaction(&mvcc_, tx_state->timestamp())); tx_state->SetMvccTx(std::move(mvcc_tx)); @@ -690,7 +689,7 @@ Status Tablet::ApplyUpsertAsUpdate(const IOContext* io_context, // were unset (eg because the table only _has_ primary keys, or because // the rest are intended to be set to their defaults), we need to // avoid doing anything. - gscoped_ptr result(new OperationResultPB()); + unique_ptr result(new OperationResultPB()); if (enc.is_empty()) { upsert->SetMutateSucceeded(std::move(result)); return Status::OK(); @@ -764,7 +763,7 @@ Status Tablet::MutateRowUnlocked(const IOContext* io_context, DCHECK(mutate->checked_present); DCHECK(mutate->valid); - gscoped_ptr result(new OperationResultPB()); + unique_ptr result(new OperationResultPB()); const TabletComponents* comps = DCHECK_NOTNULL(tx_state->tablet_components()); Timestamp ts = tx_state->timestamp(); @@ -1460,19 +1459,19 @@ void Tablet::RegisterMaintenanceOps(MaintenanceManager* maint_mgr) { } vector maintenance_ops; - gscoped_ptr rs_compact_op(new CompactRowSetsOp(this)); + unique_ptr rs_compact_op(new CompactRowSetsOp(this)); maint_mgr->RegisterOp(rs_compact_op.get()); maintenance_ops.push_back(rs_compact_op.release()); - gscoped_ptr minor_delta_compact_op(new MinorDeltaCompactionOp(this)); + unique_ptr minor_delta_compact_op(new MinorDeltaCompactionOp(this)); maint_mgr->RegisterOp(minor_delta_compact_op.get()); maintenance_ops.push_back(minor_delta_compact_op.release()); - gscoped_ptr major_delta_compact_op(new MajorDeltaCompactionOp(this)); + unique_ptr major_delta_compact_op(new MajorDeltaCompactionOp(this)); maint_mgr->RegisterOp(major_delta_compact_op.get()); maintenance_ops.push_back(major_delta_compact_op.release()); - gscoped_ptr undo_delta_block_gc_op(new UndoDeltaBlockGCOp(this)); + unique_ptr undo_delta_block_gc_op(new UndoDeltaBlockGCOp(this)); maint_mgr->RegisterOp(undo_delta_block_gc_op.get()); maintenance_ops.push_back(undo_delta_block_gc_op.release()); diff --git a/src/kudu/tablet/tablet.h b/src/kudu/tablet/tablet.h index 9c6d810..6950cc0 100644 --- a/src/kudu/tablet/tablet.h +++ b/src/kudu/tablet/tablet.h @@ -32,7 +32,6 @@ #include "kudu/common/iterator.h" #include "kudu/common/schema.h" #include "kudu/fs/io_context.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/integral_types.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/port.h" @@ -722,7 +721,7 @@ class Tablet { TabletMemTrackers mem_trackers_; scoped_refptr metric_entity_; - gscoped_ptr metrics_; + std::unique_ptr metrics_; std::unique_ptr throttler_; @@ -734,7 +733,7 @@ class Tablet { MvccManager mvcc_; LockManager lock_manager_; - gscoped_ptr compaction_policy_; + std::unique_ptr compaction_policy_; // Lock protecting the selection of rowsets for compaction. // Only one thread may run the compaction selection algorithm at a time diff --git a/src/kudu/tablet/tablet_bootstrap-test.cc b/src/kudu/tablet/tablet_bootstrap-test.cc index 46229f7..c3fd891 100644 --- a/src/kudu/tablet/tablet_bootstrap-test.cc +++ b/src/kudu/tablet/tablet_bootstrap-test.cc @@ -22,7 +22,6 @@ #include #include #include -#include #include #include @@ -57,7 +56,6 @@ #include "kudu/consensus/ref_counted_replicate.h" #include "kudu/fs/data_dirs.h" #include "kudu/fs/fs_manager.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/gutil/ref_counted.h" #include "kudu/rpc/result_tracker.h" @@ -528,7 +526,7 @@ TEST_F(BootstrapTest, TestOutOfOrderCommits) { ASSERT_OK(AppendReplicateBatch(replicate)); // Now commit the mutate before the insert (in the log). - gscoped_ptr mutate_commit(new consensus::CommitMsg); + unique_ptr mutate_commit(new consensus::CommitMsg); mutate_commit->set_op_type(consensus::WRITE_OP); mutate_commit->mutable_commited_op_id()->CopyFrom(mutate_opid); TxResultPB* result = mutate_commit->mutable_result(); @@ -538,7 +536,7 @@ TEST_F(BootstrapTest, TestOutOfOrderCommits) { ASSERT_OK(AppendCommit(std::move(mutate_commit))); - gscoped_ptr insert_commit(new consensus::CommitMsg); + unique_ptr insert_commit(new consensus::CommitMsg); insert_commit->set_op_type(consensus::WRITE_OP); insert_commit->mutable_commited_op_id()->CopyFrom(insert_opid); result = insert_commit->mutable_result(); @@ -592,7 +590,7 @@ TEST_F(BootstrapTest, TestMissingCommitMessage) { ASSERT_OK(AppendReplicateBatch(replicate)); // Now commit the mutate before the insert (in the log). - gscoped_ptr mutate_commit(new consensus::CommitMsg); + unique_ptr mutate_commit(new consensus::CommitMsg); mutate_commit->set_op_type(consensus::WRITE_OP); mutate_commit->mutable_commited_op_id()->CopyFrom(mutate_opid); TxResultPB* result = mutate_commit->mutable_result(); @@ -644,14 +642,14 @@ TEST_F(BootstrapTest, TestConsensusOnlyOperationOutOfOrderTimestamp) { // Now commit in OpId order. // NO_OP... - gscoped_ptr mutate_commit(new consensus::CommitMsg); + unique_ptr mutate_commit(new consensus::CommitMsg); mutate_commit->set_op_type(consensus::NO_OP); *mutate_commit->mutable_commited_op_id() = noop_replicate->get()->id(); ASSERT_OK(AppendCommit(std::move(mutate_commit))); // ...and WRITE_OP... - mutate_commit = gscoped_ptr(new consensus::CommitMsg); + mutate_commit = unique_ptr(new consensus::CommitMsg); mutate_commit->set_op_type(consensus::WRITE_OP); *mutate_commit->mutable_commited_op_id() = write_replicate->get()->id(); TxResultPB* result = mutate_commit->mutable_result(); @@ -719,13 +717,13 @@ TEST_F(BootstrapTest, TestKudu2509) { ASSERT_OK(AppendReplicateBatch(replicate)); // Now commit the mutate before the insert (in the log). - gscoped_ptr mutate_commit(new consensus::CommitMsg); + unique_ptr mutate_commit(new consensus::CommitMsg); mutate_commit->set_op_type(consensus::WRITE_OP); mutate_commit->mutable_commited_op_id()->CopyFrom(mutate_opid); mutate_commit->mutable_result()->add_ops()->add_mutated_stores()->set_mrs_id(1); ASSERT_OK(AppendCommit(std::move(mutate_commit))); - gscoped_ptr insert_commit(new consensus::CommitMsg); + unique_ptr insert_commit(new consensus::CommitMsg); insert_commit->set_op_type(consensus::WRITE_OP); insert_commit->mutable_commited_op_id()->CopyFrom(insert_opid); insert_commit->mutable_result()->add_ops()->add_mutated_stores()->set_mrs_id(1); diff --git a/src/kudu/tablet/tablet_bootstrap.cc b/src/kudu/tablet/tablet_bootstrap.cc index 85f5c5c..b4d980f 100644 --- a/src/kudu/tablet/tablet_bootstrap.cc +++ b/src/kudu/tablet/tablet_bootstrap.cc @@ -55,7 +55,6 @@ #include "kudu/fs/fs_manager.h" #include "kudu/fs/io_context.h" #include "kudu/gutil/bind.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/port.h" diff --git a/src/kudu/tablet/tablet_metadata-test.cc b/src/kudu/tablet/tablet_metadata-test.cc index 1e69ce7..7ef0433 100644 --- a/src/kudu/tablet/tablet_metadata-test.cc +++ b/src/kudu/tablet/tablet_metadata-test.cc @@ -33,7 +33,6 @@ #include "kudu/common/schema.h" #include "kudu/common/wire_protocol-test-util.h" #include "kudu/fs/block_id.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/gutil/ref_counted.h" #include "kudu/tablet/local_tablet_writer.h" @@ -70,14 +69,14 @@ class TestTabletMetadata : public KuduTabletTest { } void BuildPartialRow(int key, int intval, const char* strval, - gscoped_ptr* row); + unique_ptr* row); protected: - gscoped_ptr writer_; + unique_ptr writer_; }; void TestTabletMetadata::BuildPartialRow(int key, int intval, const char* strval, - gscoped_ptr* row) { + unique_ptr* row) { row->reset(new KuduPartialRow(&client_schema_)); CHECK_OK((*row)->SetInt32(0, key)); CHECK_OK((*row)->SetInt32(1, intval)); @@ -89,7 +88,7 @@ TEST_F(TestTabletMetadata, TestLoadFromSuperBlock) { TabletMetadata* meta = harness_->tablet()->metadata(); // Write some data to the tablet and flush. - gscoped_ptr row; + unique_ptr row; BuildPartialRow(0, 0, "foo", &row); ASSERT_OK(writer_->Insert(*row)); ASSERT_OK(harness_->tablet()->Flush()); @@ -168,7 +167,7 @@ TEST_F(TestTabletMetadata, TestOnDiskSize) { ASSERT_GT(initial_size, 0); // Write some data to the tablet and flush. - gscoped_ptr row; + unique_ptr row; BuildPartialRow(0, 0, "foo", &row); writer_->Insert(*row); ASSERT_OK(harness_->tablet()->Flush()); diff --git a/src/kudu/tablet/tablet_metadata.cc b/src/kudu/tablet/tablet_metadata.cc index 129da86..754f53c 100644 --- a/src/kudu/tablet/tablet_metadata.cc +++ b/src/kudu/tablet/tablet_metadata.cc @@ -21,7 +21,6 @@ #include #include #include -#include #include #include @@ -368,7 +367,7 @@ Status TabletMetadata::LoadFromSuperBlock(const TabletSuperBlockPB& superblock) table_name_ = superblock.table_name(); uint32_t schema_version = superblock.schema_version(); - gscoped_ptr schema(new Schema()); + unique_ptr schema(new Schema()); RETURN_NOT_OK_PREPEND(SchemaFromPB(superblock.schema(), schema.get()), "Failed to parse Schema from superblock " + SecureShortDebugString(superblock)); @@ -757,12 +756,12 @@ RowSetMetadata *TabletMetadata::GetRowSetForTests(int64_t id) { } void TabletMetadata::SetSchema(const Schema& schema, uint32_t version) { - gscoped_ptr new_schema(new Schema(schema)); + unique_ptr new_schema(new Schema(schema)); std::lock_guard l(data_lock_); SetSchemaUnlocked(std::move(new_schema), version); } -void TabletMetadata::SetSchemaUnlocked(gscoped_ptr new_schema, uint32_t version) { +void TabletMetadata::SetSchemaUnlocked(unique_ptr new_schema, uint32_t version) { DCHECK(new_schema->has_column_ids()); Schema* old_schema = schema_; diff --git a/src/kudu/tablet/tablet_metadata.h b/src/kudu/tablet/tablet_metadata.h index fae2a3c..384a9ca 100644 --- a/src/kudu/tablet/tablet_metadata.h +++ b/src/kudu/tablet/tablet_metadata.h @@ -30,7 +30,6 @@ #include "kudu/fs/block_id.h" #include "kudu/gutil/atomicops.h" #include "kudu/gutil/callback.h" // IWYU pragma: keep -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/ref_counted.h" #include "kudu/tablet/metadata.pb.h" @@ -311,7 +310,7 @@ class TabletMetadata : public RefCountedThreadSafe { // Constructor for loading an existing tablet. TabletMetadata(FsManager* fs_manager, std::string tablet_id); - void SetSchemaUnlocked(gscoped_ptr schema, uint32_t version); + void SetSchemaUnlocked(std::unique_ptr schema, uint32_t version); Status LoadFromDisk(); @@ -370,7 +369,7 @@ class TabletMetadata : public RefCountedThreadSafe { int64_t last_durable_mrs_id_; // The current schema version. This is owned by this class. - // We don't use gscoped_ptr so that we can do an atomic swap. + // We don't use unique_ptr so that we can do an atomic swap. Schema* schema_; uint32_t schema_version_; std::string table_name_; diff --git a/src/kudu/tablet/tablet_random_access-test.cc b/src/kudu/tablet/tablet_random_access-test.cc index 3452742..b02af28 100644 --- a/src/kudu/tablet/tablet_random_access-test.cc +++ b/src/kudu/tablet/tablet_random_access-test.cc @@ -37,7 +37,6 @@ #include "kudu/common/scan_spec.h" #include "kudu/common/schema.h" #include "kudu/common/wire_protocol.pb.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/gutil/ref_counted.h" #include "kudu/tablet/key_value_test_schema.h" @@ -227,7 +226,7 @@ class TestRandomAccess : public KuduTabletTest { const optional& old_row, vector* ops) { - gscoped_ptr row(new KuduPartialRow(&client_schema_)); + unique_ptr row(new KuduPartialRow(&client_schema_)); CHECK_OK(row->SetInt32(0, key)); optional ret = ExpectedKeyValueRow(); ret->key = key; @@ -275,7 +274,7 @@ class TestRandomAccess : public KuduTabletTest { // Adds a delete of the given row to 'ops', returning an empty string (indicating that // the row no longer exists). optional DeleteRow(int key, vector* ops) { - gscoped_ptr row(new KuduPartialRow(&client_schema_)); + unique_ptr row(new KuduPartialRow(&client_schema_)); CHECK_OK(row->SetInt32(0, key)); ops->push_back(LocalTabletWriter::Op(RowOperationsPB::DELETE, row.release())); return boost::none; @@ -330,7 +329,7 @@ class TestRandomAccess : public KuduTabletTest { // operations. This stops the compact/flush thread. CountDownLatch done_; - gscoped_ptr writer_; + unique_ptr writer_; }; TEST_F(TestRandomAccess, Test) { diff --git a/src/kudu/tablet/tablet_replica-test.cc b/src/kudu/tablet/tablet_replica-test.cc index 7cba80d..e6e9c19 100644 --- a/src/kudu/tablet/tablet_replica-test.cc +++ b/src/kudu/tablet/tablet_replica-test.cc @@ -49,7 +49,6 @@ #include "kudu/gutil/bind.h" #include "kudu/gutil/bind_helpers.h" #include "kudu/gutil/callback.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/ref_counted.h" #include "kudu/rpc/messenger.h" @@ -343,7 +342,7 @@ class TabletReplicaTest : public KuduTabletTest { // Execute insert requests and roll log after each one. Status ExecuteInsertsAndRollLogs(int num_inserts) { for (int i = 0; i < num_inserts; i++) { - gscoped_ptr req(new WriteRequestPB()); + unique_ptr req(new WriteRequestPB()); RETURN_NOT_OK(GenerateSequentialInsertRequest(GetTestSchema(), req.get())); RETURN_NOT_OK(ExecuteWriteAndRollLog(tablet_replica_.get(), *req)); } @@ -354,7 +353,7 @@ class TabletReplicaTest : public KuduTabletTest { // Execute delete requests and roll log after each one. Status ExecuteDeletesAndRollLogs(int num_deletes) { for (int i = 0; i < num_deletes; i++) { - gscoped_ptr req(new WriteRequestPB()); + unique_ptr req(new WriteRequestPB()); CHECK_OK(GenerateSequentialDeleteRequest(req.get())); CHECK_OK(ExecuteWriteAndRollLog(tablet_replica_.get(), *req)); } @@ -413,7 +412,7 @@ class DelayedApplyTransaction : public WriteTransaction { apply_continue_(DCHECK_NOTNULL(apply_continue)) { } - virtual Status Apply(gscoped_ptr* commit_msg) override { + virtual Status Apply(unique_ptr* commit_msg) override { apply_started_->CountDown(); LOG(INFO) << "Delaying apply..."; apply_continue_->Wait(); @@ -579,8 +578,8 @@ TEST_F(TabletReplicaTest, TestActiveTransactionPreventsLogGC) { CountDownLatch rpc_latch(1); CountDownLatch apply_started(1); CountDownLatch apply_continue(1); - gscoped_ptr req(new WriteRequestPB()); - gscoped_ptr resp(new WriteResponsePB()); + unique_ptr req(new WriteRequestPB()); + unique_ptr resp(new WriteResponsePB()); { // Long-running mutation. ASSERT_OK(GenerateSequentialDeleteRequest(req.get())); @@ -592,14 +591,14 @@ TEST_F(TabletReplicaTest, TestActiveTransactionPreventsLogGC) { tx_state->set_completion_callback(unique_ptr( new LatchTransactionCompletionCallback(&rpc_latch, resp.get()))); - gscoped_ptr transaction( + unique_ptr transaction( new DelayedApplyTransaction(&apply_started, &apply_continue, std::move(tx_state))); scoped_refptr driver; - ASSERT_OK(tablet_replica_->NewLeaderTransactionDriver(transaction.PassAs(), - &driver)); + ASSERT_OK(tablet_replica_->NewLeaderTransactionDriver(std::move(transaction), + &driver)); ASSERT_OK(driver->ExecuteAsync()); apply_started.Wait(); diff --git a/src/kudu/tablet/tablet_replica.cc b/src/kudu/tablet/tablet_replica.cc index 41b4324..b02b137 100644 --- a/src/kudu/tablet/tablet_replica.cc +++ b/src/kudu/tablet/tablet_replica.cc @@ -23,7 +23,6 @@ #include #include #include -#include #include #include @@ -428,10 +427,10 @@ Status TabletReplica::SubmitWrite(unique_ptr state) { RETURN_NOT_OK(CheckRunning()); state->SetResultTracker(result_tracker_); - gscoped_ptr transaction(new WriteTransaction(std::move(state), + unique_ptr transaction(new WriteTransaction(std::move(state), consensus::LEADER)); scoped_refptr driver; - RETURN_NOT_OK(NewLeaderTransactionDriver(transaction.PassAs(), + RETURN_NOT_OK(NewLeaderTransactionDriver(std::move(transaction), &driver)); return driver->ExecuteAsync(); } @@ -439,10 +438,10 @@ Status TabletReplica::SubmitWrite(unique_ptr state) { Status TabletReplica::SubmitAlterSchema(unique_ptr state) { RETURN_NOT_OK(CheckRunning()); - gscoped_ptr transaction( + unique_ptr transaction( new AlterSchemaTransaction(std::move(state), consensus::LEADER)); scoped_refptr driver; - RETURN_NOT_OK(NewLeaderTransactionDriver(transaction.PassAs(), &driver)); + RETURN_NOT_OK(NewLeaderTransactionDriver(std::move(transaction), &driver)); return driver->ExecuteAsync(); } @@ -618,7 +617,7 @@ Status TabletReplica::StartFollowerTransaction(const scoped_refptrreplicate_msg(); DCHECK(replicate_msg->has_timestamp()); - gscoped_ptr transaction; + unique_ptr transaction; switch (replicate_msg->op_type()) { case WRITE_OP: { @@ -703,7 +702,7 @@ void TabletReplica::FinishConsensusOnlyRound(ConsensusRound* round) { } } -Status TabletReplica::NewLeaderTransactionDriver(gscoped_ptr transaction, +Status TabletReplica::NewLeaderTransactionDriver(unique_ptr transaction, scoped_refptr* driver) { scoped_refptr tx_driver = new TransactionDriver( &txn_tracker_, @@ -713,12 +712,12 @@ Status TabletReplica::NewLeaderTransactionDriver(gscoped_ptr transa apply_pool_, &txn_order_verifier_); RETURN_NOT_OK(tx_driver->Init(std::move(transaction), consensus::LEADER)); - driver->swap(tx_driver); + *driver = std::move(tx_driver); return Status::OK(); } -Status TabletReplica::NewReplicaTransactionDriver(gscoped_ptr transaction, +Status TabletReplica::NewReplicaTransactionDriver(unique_ptr transaction, scoped_refptr* driver) { scoped_refptr tx_driver = new TransactionDriver( &txn_tracker_, @@ -728,7 +727,7 @@ Status TabletReplica::NewReplicaTransactionDriver(gscoped_ptr trans apply_pool_, &txn_order_verifier_); RETURN_NOT_OK(tx_driver->Init(std::move(transaction), consensus::REPLICA)); - driver->swap(tx_driver); + *driver = std::move(tx_driver); return Status::OK(); } @@ -746,15 +745,15 @@ void TabletReplica::RegisterMaintenanceOps(MaintenanceManager* maint_mgr) { vector maintenance_ops; - gscoped_ptr mrs_flush_op(new FlushMRSOp(this)); + unique_ptr mrs_flush_op(new FlushMRSOp(this)); maint_mgr->RegisterOp(mrs_flush_op.get()); maintenance_ops.push_back(mrs_flush_op.release()); - gscoped_ptr dms_flush_op(new FlushDeltaMemStoresOp(this)); + unique_ptr dms_flush_op(new FlushDeltaMemStoresOp(this)); maint_mgr->RegisterOp(dms_flush_op.get()); maintenance_ops.push_back(dms_flush_op.release()); - gscoped_ptr log_gc(new LogGCOp(this)); + unique_ptr log_gc(new LogGCOp(this)); maint_mgr->RegisterOp(log_gc.get()); maintenance_ops.push_back(log_gc.release()); @@ -762,7 +761,7 @@ void TabletReplica::RegisterMaintenanceOps(MaintenanceManager* maint_mgr) { { std::lock_guard l(lock_); DCHECK(maintenance_ops_.empty()); - maintenance_ops_.swap(maintenance_ops); + maintenance_ops_ = std::move(maintenance_ops); tablet = tablet_; } tablet->RegisterMaintenanceOps(maint_mgr); @@ -780,7 +779,7 @@ void TabletReplica::UnregisterMaintenanceOps() { vector maintenance_ops; { std::lock_guard l(lock_); - maintenance_ops.swap(maintenance_ops_); + maintenance_ops = std::move(maintenance_ops_); } for (MaintenanceOp* op : maintenance_ops) { op->Unregister(); @@ -857,7 +856,7 @@ void TabletReplica::UpdateTabletStats(vector* dirty_tablets) { if (consensus::RaftPeerPB_Role_LEADER == role) { dirty_tablets->emplace_back(tablet_id()); } - stats_pb_.Swap(&pb); + stats_pb_ = std::move(pb); } } diff --git a/src/kudu/tablet/tablet_replica.h b/src/kudu/tablet/tablet_replica.h index abc9079..a7e115c 100644 --- a/src/kudu/tablet/tablet_replica.h +++ b/src/kudu/tablet/tablet_replica.h @@ -31,7 +31,6 @@ #include "kudu/consensus/raft_consensus.h" #include "kudu/fs/fs_manager.h" #include "kudu/gutil/callback.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/ref_counted.h" #include "kudu/tablet/metadata.pb.h" @@ -259,10 +258,10 @@ class TabletReplica : public RefCountedThreadSafe, // Convenience method to return the permanent_uuid of this peer. std::string permanent_uuid() const { return tablet_->metadata()->fs_manager()->uuid(); } - Status NewLeaderTransactionDriver(gscoped_ptr transaction, + Status NewLeaderTransactionDriver(std::unique_ptr transaction, scoped_refptr* driver); - Status NewReplicaTransactionDriver(gscoped_ptr transaction, + Status NewReplicaTransactionDriver(std::unique_ptr transaction, scoped_refptr* driver); // Tells the tablet's log to garbage collect. diff --git a/src/kudu/tablet/transactions/alter_schema_transaction.cc b/src/kudu/tablet/transactions/alter_schema_transaction.cc index a964342..76519c2 100644 --- a/src/kudu/tablet/transactions/alter_schema_transaction.cc +++ b/src/kudu/tablet/transactions/alter_schema_transaction.cc @@ -84,7 +84,7 @@ AlterSchemaTransaction::AlterSchemaTransaction(unique_ptr* replicate_msg) { +void AlterSchemaTransaction::NewReplicateMsg(unique_ptr* replicate_msg) { replicate_msg->reset(new ReplicateMsg); (*replicate_msg)->set_op_type(ALTER_SCHEMA_OP); (*replicate_msg)->mutable_alter_schema_request()->CopyFrom(*state()->request()); @@ -94,7 +94,7 @@ Status AlterSchemaTransaction::Prepare() { TRACE("PREPARE ALTER-SCHEMA: Starting"); // Decode schema - gscoped_ptr schema(new Schema); + unique_ptr schema(new Schema); Status s = SchemaFromPB(state_->request()->schema(), schema.get()); if (!s.ok()) { state_->completion_callback()->set_error(s, TabletServerErrorPB::INVALID_SCHEMA); @@ -118,7 +118,7 @@ Status AlterSchemaTransaction::Start() { return Status::OK(); } -Status AlterSchemaTransaction::Apply(gscoped_ptr* commit_msg) { +Status AlterSchemaTransaction::Apply(unique_ptr* commit_msg) { TRACE("APPLY ALTER-SCHEMA: Starting"); Tablet* tablet = state_->tablet_replica()->tablet(); diff --git a/src/kudu/tablet/transactions/alter_schema_transaction.h b/src/kudu/tablet/transactions/alter_schema_transaction.h index 69b9c31..4342ac2 100644 --- a/src/kudu/tablet/transactions/alter_schema_transaction.h +++ b/src/kudu/tablet/transactions/alter_schema_transaction.h @@ -14,7 +14,6 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. - #pragma once #include @@ -26,7 +25,6 @@ #include "kudu/common/common.pb.h" #include "kudu/consensus/consensus.pb.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/tablet/tablet.pb.h" #include "kudu/tablet/transactions/transaction.h" @@ -134,7 +132,7 @@ class AlterSchemaTransaction : public Transaction { AlterSchemaTransactionState* state() override { return state_.get(); } const AlterSchemaTransactionState* state() const override { return state_.get(); } - void NewReplicateMsg(gscoped_ptr* replicate_msg) override; + void NewReplicateMsg(std::unique_ptr* replicate_msg) override; // Executes a Prepare for the alter schema transaction. Status Prepare() override; @@ -143,7 +141,7 @@ class AlterSchemaTransaction : public Transaction { Status Start() override; // Executes an Apply for the alter schema transaction - Status Apply(gscoped_ptr* commit_msg) override; + Status Apply(std::unique_ptr* commit_msg) override; // Actually commits the transaction. void Finish(TransactionResult result) override; diff --git a/src/kudu/tablet/transactions/transaction.h b/src/kudu/tablet/transactions/transaction.h index ba4b64d..1a3125d 100644 --- a/src/kudu/tablet/transactions/transaction.h +++ b/src/kudu/tablet/transactions/transaction.h @@ -18,6 +18,7 @@ #include #include +#include #include #include @@ -29,7 +30,6 @@ #include "kudu/consensus/consensus.pb.h" #include "kudu/consensus/opid.pb.h" #include "kudu/consensus/raft_consensus.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/port.h" #include "kudu/gutil/ref_counted.h" #include "kudu/rpc/result_tracker.h" @@ -101,7 +101,7 @@ class Transaction { TransactionType tx_type() const { return tx_type_; } // Builds the ReplicateMsg for this transaction. - virtual void NewReplicateMsg(gscoped_ptr* replicate_msg) = 0; + virtual void NewReplicateMsg(std::unique_ptr* replicate_msg) = 0; // Executes the prepare phase of this transaction, the actual actions // of this phase depend on the transaction type, but usually are limited @@ -123,7 +123,7 @@ class Transaction { // Executes the Apply() phase of the transaction, the actual actions of // this phase depend on the transaction type, but usually this is the // method where data-structures are changed. - virtual Status Apply(gscoped_ptr* commit_msg) = 0; + virtual Status Apply(std::unique_ptr* commit_msg) = 0; // Executed after the transaction has been applied and the commit message has // been appended to the log (though it might not be durable yet), or if the diff --git a/src/kudu/tablet/transactions/transaction_driver.cc b/src/kudu/tablet/transactions/transaction_driver.cc index 2f4d9c0..c9d26e5 100644 --- a/src/kudu/tablet/transactions/transaction_driver.cc +++ b/src/kudu/tablet/transactions/transaction_driver.cc @@ -21,7 +21,6 @@ #include #include #include -#include #include #include @@ -29,10 +28,10 @@ #include #include "kudu/clock/clock.h" -#include "kudu/consensus/raft_consensus.h" #include "kudu/common/common.pb.h" #include "kudu/common/timestamp.h" #include "kudu/consensus/log.h" +#include "kudu/consensus/raft_consensus.h" #include "kudu/consensus/time_manager.h" #include "kudu/gutil/bind.h" #include "kudu/gutil/bind_helpers.h" @@ -125,7 +124,7 @@ TransactionDriver::TransactionDriver(TransactionTracker *txn_tracker, } } -Status TransactionDriver::Init(gscoped_ptr transaction, +Status TransactionDriver::Init(unique_ptr transaction, DriverType type) { // If the tablet has been stopped, the replica is likely shutting down soon. // Prevent further transacions from starting. @@ -165,7 +164,7 @@ Status TransactionDriver::Init(gscoped_ptr transaction, } } else { DCHECK_EQ(type, consensus::LEADER); - gscoped_ptr replicate_msg; + unique_ptr replicate_msg; transaction_->NewReplicateMsg(&replicate_msg); if (consensus_) { // sometimes NULL in tests // A raw pointer is required to avoid a refcount cycle. @@ -517,7 +516,7 @@ void TransactionDriver::ApplyTask() { scoped_refptr ref(this); { - gscoped_ptr commit_msg; + unique_ptr commit_msg; Status s = transaction_->Apply(&commit_msg); if (PREDICT_FALSE(!s.ok())) { LOG(WARNING) << Substitute("Did not Apply transaction $0: $1", diff --git a/src/kudu/tablet/transactions/transaction_driver.h b/src/kudu/tablet/transactions/transaction_driver.h index b661c02..05b680d 100644 --- a/src/kudu/tablet/transactions/transaction_driver.h +++ b/src/kudu/tablet/transactions/transaction_driver.h @@ -14,16 +14,15 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. - #pragma once +#include #include #include #include "kudu/consensus/consensus.pb.h" #include "kudu/consensus/opid.pb.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/ref_counted.h" #include "kudu/gutil/walltime.h" @@ -34,9 +33,9 @@ #include "kudu/util/trace.h" namespace kudu { -class Timestamp; class ThreadPool; class ThreadPoolToken; +class Timestamp; namespace log { class Log; @@ -234,7 +233,7 @@ class TransactionDriver : public RefCountedThreadSafe { // Perform any non-constructor initialization. Sets the transaction // that will be executed. - Status Init(gscoped_ptr transaction, + Status Init(std::unique_ptr transaction, consensus::DriverType type); // Returns the OpId of the transaction being executed or an uninitialized @@ -375,7 +374,7 @@ class TransactionDriver : public RefCountedThreadSafe { mutable simple_spinlock opid_lock_; // The transaction to be executed by this driver. - gscoped_ptr transaction_; + std::unique_ptr transaction_; // Trace object for tracing any transactions started by this driver. scoped_refptr trace_; diff --git a/src/kudu/tablet/transactions/transaction_tracker-test.cc b/src/kudu/tablet/transactions/transaction_tracker-test.cc index d7b412e..f1481a8 100644 --- a/src/kudu/tablet/transactions/transaction_tracker-test.cc +++ b/src/kudu/tablet/transactions/transaction_tracker-test.cc @@ -31,7 +31,6 @@ #include "kudu/consensus/consensus.pb.h" #include "kudu/gutil/casts.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/ref_counted.h" #include "kudu/tablet/transactions/transaction.h" #include "kudu/tablet/transactions/transaction_driver.h" @@ -57,6 +56,7 @@ METRIC_DECLARE_counter(transaction_memory_limit_rejections); using std::pair; using std::shared_ptr; +using std::unique_ptr; using std::vector; namespace kudu { @@ -80,7 +80,7 @@ class TransactionTrackerTest : public KuduTest, state_(state) { } - void NewReplicateMsg(gscoped_ptr* replicate_msg) override { + void NewReplicateMsg(unique_ptr* replicate_msg) override { replicate_msg->reset(new consensus::ReplicateMsg()); } TransactionState* state() override { return state_.get(); } @@ -88,14 +88,14 @@ class TransactionTrackerTest : public KuduTest, Status Prepare() override { return Status::OK(); } Status Start() override { return Status::OK(); } - Status Apply(gscoped_ptr* /* commit_msg */) override { + Status Apply(unique_ptr* /* commit_msg */) override { return Status::OK(); } std::string ToString() const override { return "NoOp"; } private: - gscoped_ptr state_; + unique_ptr state_; }; TransactionTrackerTest() @@ -116,8 +116,8 @@ class TransactionTrackerTest : public KuduTest, nullptr, nullptr, nullptr)); - gscoped_ptr tx(new NoOpTransaction(new NoOpTransactionState)); - RETURN_NOT_OK(driver->Init(tx.PassAs(), consensus::LEADER)); + unique_ptr tx(new NoOpTransaction(new NoOpTransactionState)); + RETURN_NOT_OK(driver->Init(std::move(tx), consensus::LEADER)); local_drivers.push_back(driver); } diff --git a/src/kudu/tablet/transactions/transaction_tracker.h b/src/kudu/tablet/transactions/transaction_tracker.h index 93e1b9e..96ac8fd 100644 --- a/src/kudu/tablet/transactions/transaction_tracker.h +++ b/src/kudu/tablet/transactions/transaction_tracker.h @@ -14,16 +14,13 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. - -#ifndef KUDU_TABLET_TRANSACTION_TRACKER_H_ -#define KUDU_TABLET_TRANSACTION_TRACKER_H_ +#pragma once #include #include #include #include -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/ref_counted.h" #include "kudu/tablet/transactions/transaction_driver.h" @@ -103,7 +100,7 @@ class TransactionTracker { ScopedRefPtrEqualToFunctor > TxnMap; TxnMap pending_txns_; - gscoped_ptr metrics_; + std::unique_ptr metrics_; std::shared_ptr mem_tracker_; @@ -112,5 +109,3 @@ class TransactionTracker { } // namespace tablet } // namespace kudu - -#endif // KUDU_TABLET_TRANSACTION_TRACKER_H_ diff --git a/src/kudu/tablet/transactions/write_transaction.cc b/src/kudu/tablet/transactions/write_transaction.cc index 2bfbbde..7350e92 100644 --- a/src/kudu/tablet/transactions/write_transaction.cc +++ b/src/kudu/tablet/transactions/write_transaction.cc @@ -22,7 +22,6 @@ #include #include #include -#include #include #include @@ -138,7 +137,7 @@ WriteTransaction::WriteTransaction(unique_ptr state, Driv start_time_ = MonoTime::Now(); } -void WriteTransaction::NewReplicateMsg(gscoped_ptr* replicate_msg) { +void WriteTransaction::NewReplicateMsg(unique_ptr* replicate_msg) { replicate_msg->reset(new ReplicateMsg); (*replicate_msg)->set_op_type(WRITE_OP); (*replicate_msg)->mutable_write_request()->CopyFrom(*state()->request()); @@ -221,7 +220,7 @@ void WriteTransaction::UpdatePerRowErrors() { // FIXME: Since this is called as a void in a thread-pool callback, // it seems pointless to return a Status! -Status WriteTransaction::Apply(gscoped_ptr* commit_msg) { +Status WriteTransaction::Apply(unique_ptr* commit_msg) { TRACE_EVENT0("txn", "WriteTransaction::Apply"); TRACE("APPLY: Starting."); @@ -320,7 +319,7 @@ WriteTransactionState::WriteTransactionState(TabletReplica* tablet_replica, } } -void WriteTransactionState::SetMvccTx(gscoped_ptr mvcc_tx) { +void WriteTransactionState::SetMvccTx(unique_ptr mvcc_tx) { DCHECK(!mvcc_tx_) << "Mvcc transaction already started/set."; mvcc_tx_ = std::move(mvcc_tx); } diff --git a/src/kudu/tablet/transactions/write_transaction.h b/src/kudu/tablet/transactions/write_transaction.h index b6b6f5c..23c829a 100644 --- a/src/kudu/tablet/transactions/write_transaction.h +++ b/src/kudu/tablet/transactions/write_transaction.h @@ -29,7 +29,6 @@ #include "kudu/common/row_operations.h" #include "kudu/common/wire_protocol.pb.h" #include "kudu/consensus/consensus.pb.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/ref_counted.h" #include "kudu/tablet/rowset.h" @@ -150,7 +149,7 @@ class WriteTransactionState : public TransactionState { // This must be called exactly once, after the timestamp was acquired. // This also copies the timestamp from the MVCC transaction into the // WriteTransactionState object. - void SetMvccTx(gscoped_ptr mvcc_tx); + void SetMvccTx(std::unique_ptr mvcc_tx); // Set the Tablet components that this transaction will write into. // Called exactly once at the beginning of Apply, before applying its @@ -255,7 +254,7 @@ class WriteTransactionState : public TransactionState { ProbeStats* stats_array_ = nullptr; // The MVCC transaction, set up during PREPARE phase - gscoped_ptr mvcc_tx_; + std::unique_ptr mvcc_tx_; // The tablet components, acquired at the same time as mvcc_tx_ is set. scoped_refptr tablet_components_; @@ -281,7 +280,7 @@ class WriteTransaction : public Transaction { WriteTransactionState* state() override { return state_.get(); } const WriteTransactionState* state() const override { return state_.get(); } - void NewReplicateMsg(gscoped_ptr* replicate_msg) override; + void NewReplicateMsg(std::unique_ptr* replicate_msg) override; // Executes a Prepare for a write transaction. // @@ -316,7 +315,7 @@ class WriteTransaction : public Transaction { // are placed in the queue (but not necessarily in the same order of the // original requests) which is already a requirement of the consensus // algorithm. - Status Apply(gscoped_ptr* commit_msg) override; + Status Apply(std::unique_ptr* commit_msg) override; // If result == COMMITTED, commits the mvcc transaction and updates // the metrics, if result == ABORTED aborts the mvcc transaction. diff --git a/src/kudu/tools/ksck_remote-test.cc b/src/kudu/tools/ksck_remote-test.cc index 0649060..ace10e2 100644 --- a/src/kudu/tools/ksck_remote-test.cc +++ b/src/kudu/tools/ksck_remote-test.cc @@ -21,11 +21,11 @@ #include #include #include +#include #include #include #include -#include #include #include @@ -34,7 +34,6 @@ #include "kudu/client/shared_ptr.h" #include "kudu/client/write_op.h" #include "kudu/common/partial_row.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/macros.h" #include "kudu/gutil/port.h" #include "kudu/gutil/ref_counted.h" @@ -126,7 +125,7 @@ class RemoteKsckTest : public KuduTest { ASSERT_OK(mini_cluster_->CreateClient(nullptr, &client_)); // Create one table. - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wdeprecated-declarations" ASSERT_OK(table_creator->table_name(kTableName) @@ -182,7 +181,7 @@ class RemoteKsckTest : public KuduTest { } for (uint64_t i = 0; continue_writing.Load(); i++) { - gscoped_ptr insert(table->NewInsert()); + unique_ptr insert(table->NewInsert()); GenerateDataForRow(table->schema(), i, &random_, insert->mutable_row()); status = session->Apply(insert.release()); if (!status.ok()) { diff --git a/src/kudu/tools/ksck_remote.cc b/src/kudu/tools/ksck_remote.cc index 0f3a1da..1afd91a 100644 --- a/src/kudu/tools/ksck_remote.cc +++ b/src/kudu/tools/ksck_remote.cc @@ -43,7 +43,6 @@ #include "kudu/consensus/consensus.proxy.h" #include "kudu/consensus/metadata.pb.h" #include "kudu/gutil/basictypes.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/stl_util.h" #include "kudu/gutil/strings/substitute.h" @@ -98,6 +97,7 @@ using kudu::rpc::MessengerBuilder; using kudu::rpc::RpcController; using std::shared_ptr; using std::string; +using std::unique_ptr; using std::vector; using strings::Substitute; @@ -387,7 +387,7 @@ class ChecksumStepper { } void HandleResponse() { - gscoped_ptr deleter(this); + unique_ptr deleter(this); Status s = rpc_.status(); if (s.ok() && resp_.has_error()) { s = StatusFromPB(resp_.error().status()); @@ -453,7 +453,7 @@ class ChecksumStepper { LOG(FATAL) << "Unknown type"; break; } - gscoped_ptr handler(new ChecksumCallbackHandler(this)); + unique_ptr handler(new ChecksumCallbackHandler(this)); rpc::ResponseCallback cb = boost::bind(&ChecksumCallbackHandler::Run, handler.get()); proxy_->ChecksumAsync(req_, &resp_, &rpc_, cb); ignore_result(handler.release()); @@ -486,7 +486,7 @@ void RemoteKsckTabletServer::RunTabletChecksumScanAsync( const Schema& schema, const KsckChecksumOptions& options, shared_ptr manager) { - gscoped_ptr stepper( + unique_ptr stepper( new ChecksumStepper(tablet_id, schema, uuid(), options, manager, ts_proxy_)); stepper->Start(); ignore_result(stepper.release()); // Deletes self on callback. diff --git a/src/kudu/tools/kudu-admin-test.cc b/src/kudu/tools/kudu-admin-test.cc index bb68170..c40021e 100644 --- a/src/kudu/tools/kudu-admin-test.cc +++ b/src/kudu/tools/kudu-admin-test.cc @@ -49,7 +49,6 @@ #include "kudu/consensus/opid.pb.h" #include "kudu/consensus/quorum_util.h" #include "kudu/gutil/basictypes.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/stl_util.h" #include "kudu/gutil/strings/join.h" @@ -1672,7 +1671,7 @@ TEST_F(AdminCliTest, TestListTablesDetail) { // Add another table to test multiple tables output. const string kAnotherTableId = "TestAnotherTable"; auto client_schema = KuduSchema::FromSchema(schema_); - gscoped_ptr table_creator(client_->NewTableCreator()); + unique_ptr table_creator(client_->NewTableCreator()); ASSERT_OK(table_creator->table_name(kAnotherTableId) .schema(&client_schema) .set_range_partition_columns({ "key" }) diff --git a/src/kudu/tools/rebalancer_tool-test.cc b/src/kudu/tools/rebalancer_tool-test.cc index 86dffc6..53de5ea 100644 --- a/src/kudu/tools/rebalancer_tool-test.cc +++ b/src/kudu/tools/rebalancer_tool-test.cc @@ -42,7 +42,6 @@ #include "kudu/consensus/consensus.pb.h" #include "kudu/consensus/consensus.proxy.h" #include "kudu/consensus/quorum_util.h" -#include "kudu/gutil/gscoped_ptr.h" #include "kudu/gutil/map-util.h" #include "kudu/gutil/stl_util.h" #include "kudu/gutil/strings/join.h" diff --git a/src/kudu/util/bit-stream-utils.h b/src/kudu/util/bit-stream-utils.h index c6aeb01..a772c7c 100644 --- a/src/kudu/util/bit-stream-utils.h +++ b/src/kudu/util/bit-stream-utils.h @@ -14,8 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -#ifndef IMPALA_UTIL_BIT_STREAM_UTILS_H -#define IMPALA_UTIL_BIT_STREAM_UTILS_H +#pragma once #include "kudu/gutil/port.h" #include "kudu/util/bit-util.h" @@ -93,7 +92,12 @@ class BitReader { // 'buffer' is the buffer to read from. The buffer's length is 'buffer_len'. BitReader(const uint8_t* buffer, int buffer_len); - BitReader() : buffer_(NULL), max_bytes_(0) {} + BitReader() + : buffer_(nullptr), + max_bytes_(0), + buffered_values_(0), + byte_offset_(0), + bit_offset_(0) {} // Gets the next value from the buffer. Returns true if 'v' could be read or false if // there are not enough bytes left. num_bits must be <= 32. @@ -146,5 +150,3 @@ class BitReader { }; } // namespace kudu - -#endif diff --git a/src/kudu/util/bit-stream-utils.inline.h b/src/kudu/util/bit-stream-utils.inline.h index d168bda..f9186f5 100644 --- a/src/kudu/util/bit-stream-utils.inline.h +++ b/src/kudu/util/bit-stream-utils.inline.h @@ -14,8 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -#ifndef IMPALA_UTIL_BIT_STREAM_UTILS_INLINE_H -#define IMPALA_UTIL_BIT_STREAM_UTILS_INLINE_H +#pragma once #include @@ -90,11 +89,11 @@ inline void BitWriter::PutVlqInt(int32_t v) { inline BitReader::BitReader(const uint8_t* buffer, int buffer_len) - : buffer_(buffer), - max_bytes_(buffer_len), - buffered_values_(0), - byte_offset_(0), - bit_offset_(0) { + : buffer_(buffer), + max_bytes_(buffer_len), + buffered_values_(0), + byte_offset_(0), + bit_offset_(0) { int num_bytes = std::min(8, max_bytes_); memcpy(&buffered_values_, buffer_ + byte_offset_, num_bytes); } @@ -207,5 +206,3 @@ inline bool BitReader::GetVlqInt(int32_t* v) { } } // namespace kudu - -#endif diff --git a/src/kudu/util/rle-encoding.h b/src/kudu/util/rle-encoding.h index f3cf2c1..4a00148 100644 --- a/src/kudu/util/rle-encoding.h +++ b/src/kudu/util/rle-encoding.h @@ -84,17 +84,22 @@ class RleDecoder { // Create a decoder object. buffer/buffer_len is the decoded data. // bit_width is the width of each value (before encoding). RleDecoder(const uint8_t* buffer, int buffer_len, int bit_width) - : bit_reader_(buffer, buffer_len), - bit_width_(bit_width), - current_value_(0), - repeat_count_(0), - literal_count_(0), - rewind_state_(CANT_REWIND) { + : bit_reader_(buffer, buffer_len), + bit_width_(bit_width), + current_value_(0), + repeat_count_(0), + literal_count_(0), + rewind_state_(CANT_REWIND) { DCHECK_GE(bit_width_, 1); DCHECK_LE(bit_width_, 64); } - RleDecoder() {} + RleDecoder() + : bit_width_(0), + current_value_(0), + repeat_count_(0), + literal_count_(0), + rewind_state_(CANT_REWIND) {} // Skip n values, and returns the number of non-zero entries skipped. size_t Skip(size_t to_skip);