diff --git a/conf/nebula-metad.conf.default b/conf/nebula-metad.conf.default index 89e537b9a57..42966116ef4 100644 --- a/conf/nebula-metad.conf.default +++ b/conf/nebula-metad.conf.default @@ -52,3 +52,4 @@ --default_replica_factor=1 --heartbeat_interval_secs=10 +--agent_heartbeat_interval_secs=60 diff --git a/conf/nebula-metad.conf.production b/conf/nebula-metad.conf.production index 870ba6318c2..1aa2d116211 100644 --- a/conf/nebula-metad.conf.production +++ b/conf/nebula-metad.conf.production @@ -52,6 +52,7 @@ --default_replica_factor=1 --heartbeat_interval_secs=10 +--agent_heartbeat_interval_secs=60 ############## rocksdb Options ############## --rocksdb_wal_sync=true diff --git a/resources/gflags.json b/resources/gflags.json index e75f960e71a..59d164ea12d 100644 --- a/resources/gflags.json +++ b/resources/gflags.json @@ -23,3 +23,4 @@ "rocksdb_block_based_table_options" ] } + diff --git a/src/clients/meta/MetaClient.cpp b/src/clients/meta/MetaClient.cpp index 961d00f8162..48413c7fd21 100644 --- a/src/clients/meta/MetaClient.cpp +++ b/src/clients/meta/MetaClient.cpp @@ -12,11 +12,13 @@ #include #include +#include #include "clients/meta/FileBasedClusterIdMan.h" #include "clients/meta/stats/MetaClientStats.h" #include "common/base/Base.h" #include "common/base/MurmurHash2.h" +#include "common/base/Status.h" #include "common/conf/Configuration.h" #include "common/http/HttpClient.h" #include "common/meta/NebulaSchemaProvider.h" @@ -37,6 +39,27 @@ DEFINE_int32(meta_client_retry_interval_secs, 1, "meta client sleep interval bet DEFINE_int32(meta_client_timeout_ms, 60 * 1000, "meta client timeout"); DEFINE_string(cluster_id_path, "cluster.id", "file path saved clusterId"); DEFINE_int32(check_plan_killed_frequency, 8, "check plan killed every 1< userNameList; + for (auto& user : userRoleRet.value()) { auto rolesRet = getUserRoles(user.first).get(); if (!rolesRet.ok()) { @@ -160,11 +188,40 @@ bool MetaClient::loadUsersAndRoles() { } userRolesMap[user.first] = rolesRet.value(); userPasswordMap[user.first] = user.second; + userPasswordAttemptsRemain[user.first] = FLAGS_failed_login_attempts; + userLoginLockTime[user.first] = 0; + userNameList.emplace(user.first); } { folly::RWSpinLock::WriteHolder holder(localCacheLock_); userRolesMap_ = std::move(userRolesMap); userPasswordMap_ = std::move(userPasswordMap); + + // This method is called periodically by the heartbeat thread, but we don't want to reset the + // failed login attempts every time. Remove expired users from cache + for (auto& ele : userPasswordAttemptsRemain) { + if (userNameList.count(ele.first) == 0) { + userPasswordAttemptsRemain.erase(ele.first); + } + } + for (auto& ele : userLoginLockTime) { + if (userNameList.count(ele.first) == 0) { + userLoginLockTime.erase(ele.first); + } + } + + // If the user is not in the map, insert value with the default value + for (const auto& user : userNameList) { + if (userPasswordAttemptsRemain.count(user) == 0) { + userPasswordAttemptsRemain[user] = FLAGS_failed_login_attempts; + } + if (userLoginLockTime.count(user) == 0) { + userLoginLockTime[user] = 0; + } + } + + userPasswordAttemptsRemain_ = std::move(userPasswordAttemptsRemain); + userLoginLockTime_ = std::move(userLoginLockTime); } return true; } @@ -694,6 +751,7 @@ void MetaClient::getResponse(Request req, return; } else { LOG(ERROR) << "Send request to " << host << ", exceed retry limit"; + LOG(ERROR) << "RpcResponse exception: " << t.exception().what().c_str(); pro.setValue( Status::Error("RPC failure in MetaClient: %s", t.exception().what().c_str())); } @@ -873,8 +931,8 @@ Status MetaClient::handleResponse(const RESP& resp) { return Status::Error("list cluster failure!"); case nebula::cpp2::ErrorCode::E_LIST_CLUSTER_GET_ABS_PATH_FAILURE: return Status::Error("Failed to get the absolute path!"); - case nebula::cpp2::ErrorCode::E_GET_META_DIR_FAILURE: - return Status::Error("Failed to get meta dir!"); + case nebula::cpp2::ErrorCode::E_LIST_CLUSTER_NO_AGENT_FAILURE: + return Status::Error("There is no agent!"); case nebula::cpp2::ErrorCode::E_INVALID_JOB: return Status::Error("No valid job!"); case nebula::cpp2::ErrorCode::E_JOB_NOT_IN_SPACE: @@ -1369,6 +1427,8 @@ folly::Future> MetaClient::multiPut( cpp2::MultiPutReq req; std::vector data; + data.reserve(pairs.size()); + for (auto& element : pairs) { data.emplace_back(std::move(element)); } @@ -2353,16 +2413,73 @@ std::vector MetaClient::getRolesByUserFromCache(const std::strin return iter->second; } -bool MetaClient::authCheckFromCache(const std::string& account, const std::string& password) { +Status MetaClient::authCheckFromCache(const std::string& account, const std::string& password) { + // Check meta service status if (!ready_) { - return false; + return Status::Error("Meta Service not ready"); } + const ThreadLocalInfo& threadLocalInfo = getThreadLocalInfo(); + // Check user existence auto iter = threadLocalInfo.userPasswordMap_.find(account); if (iter == threadLocalInfo.userPasswordMap_.end()) { - return false; + return Status::Error("User not exist"); + } + + folly::RWSpinLock::WriteHolder holder(localCacheLock_); + + auto& lockedSince = userLoginLockTime_[account]; + auto& passwordAttemtRemain = userPasswordAttemptsRemain_[account]; + LOG(INFO) << "Thread id: " << std::this_thread::get_id() + << " ,passwordAttemtRemain: " << passwordAttemtRemain; + // lockedSince is non-zero means the account has been locked + if (lockedSince != 0) { + auto remainingLockTime = + (lockedSince + FLAGS_password_lock_time_in_secs) - time::WallClock::fastNowInSec(); + // If the account is still locked, there is no need to check the password + if (remainingLockTime > 0) { + return Status::Error( + "%d times consecutive incorrect passwords has been input, user name: %s has been " + "locked, try again in %ld seconds", + FLAGS_failed_login_attempts, + account.c_str(), + remainingLockTime); + } + // Clear lock state and reset attempts + lockedSince = 0; + passwordAttemtRemain = FLAGS_failed_login_attempts; } - return iter->second == password; + + if (iter->second != password) { + // By default there is no limit of login attempts if any of these 2 flags is unset + if (FLAGS_failed_login_attempts == 0 || FLAGS_password_lock_time_in_secs == 0) { + return Status::Error("Invalid password"); + } + + // If the password is not correct and passwordAttemtRemain > 0, + // Allow another attemp + if (passwordAttemtRemain > 0) { + --passwordAttemtRemain; + if (passwordAttemtRemain == 0) { + // If the remaining attemps is 0, failed to authenticate + // Block user login + lockedSince = time::WallClock::fastNowInSec(); + return Status::Error( + "%d times consecutive incorrect passwords has been input, user name: %s has been " + "locked, try again in %d seconds", + FLAGS_failed_login_attempts, + account.c_str(), + FLAGS_password_lock_time_in_secs); + } + LOG(ERROR) << "Invalid password, remaining attempts: " << passwordAttemtRemain; + return Status::Error("Invalid password, remaining attempts: %d", passwordAttemtRemain); + } + } + + // Reset password attempts + passwordAttemtRemain = FLAGS_failed_login_attempts; + lockedSince = 0; + return Status::OK(); } bool MetaClient::checkShadowAccountFromCache(const std::string& account) { @@ -3532,6 +3649,21 @@ folly::Future> MetaClient::killQuery( return future; } +folly::Future> MetaClient::getWorkerId(std::string ipAddr) { + cpp2::GetWorkerIdReq req; + req.host_ref() = std::move(ipAddr); + + folly::Promise> promise; + auto future = promise.getFuture(); + getResponse( + std::move(req), + [](auto client, auto request) { return client->future_getWorkerId(request); }, + [](cpp2::GetWorkerIdResp&& resp) -> int64_t { return std::move(resp).get_workerid(); }, + std::move(promise), + true); + return future; +} + folly::Future> MetaClient::download(const std::string& hdfsHost, int32_t hdfsPort, const std::string& hdfsPath, diff --git a/src/clients/meta/MetaClient.h b/src/clients/meta/MetaClient.h index 38ae54353ff..7103c88e6c7 100644 --- a/src/clients/meta/MetaClient.h +++ b/src/clients/meta/MetaClient.h @@ -14,6 +14,7 @@ #include #include +#include #include "common/base/Base.h" #include "common/base/ObjectPool.h" @@ -143,6 +144,10 @@ using IndexStatus = std::tuple; using UserRolesMap = std::unordered_map>; // get user password by account using UserPasswordMap = std::unordered_map; +// Mapping of user name and remaining wrong password attempts +using UserPasswordAttemptsRemain = std::unordered_map; +// Mapping of user name and the timestamp when the account is locked +using UserLoginLockTime = std::unordered_map; // config cache, get config via module and name using MetaConfigMap = @@ -199,13 +204,13 @@ struct MetaClientOptions { std::string serviceName_ = ""; // Whether to skip the config manager bool skipConfig_ = false; - // host role(graph/meta/storage) using this client + // Host role(graph/meta/storage) using this client cpp2::HostRole role_ = cpp2::HostRole::UNKNOWN; // gitInfoSHA of Host using this client std::string gitInfoSHA_{""}; - // data path list, used in storaged + // Data path list, used in storaged std::vector dataPaths_; - // install path, used in metad/graphd/storaged + // Install path, used in metad/graphd/storaged std::string rootPath_; }; @@ -593,7 +598,7 @@ class MetaClient { std::vector getRolesByUserFromCache(const std::string& user); - bool authCheckFromCache(const std::string& account, const std::string& password); + Status authCheckFromCache(const std::string& account, const std::string& password); StatusOr getTermFromCache(GraphSpaceID spaceId, PartitionID); @@ -651,6 +656,8 @@ class MetaClient { folly::Future> ingest(GraphSpaceID spaceId); + folly::Future> getWorkerId(std::string ipAddr); + HostAddr getMetaLeader() { return leader_; } @@ -659,6 +666,10 @@ class MetaClient { return heartbeatTime_; } + std::string getLocalIp() { + return options_.localHost_.toString(); + } + protected: // Return true if load succeeded. bool loadData(); @@ -815,6 +826,8 @@ class MetaClient { UserRolesMap userRolesMap_; UserPasswordMap userPasswordMap_; + UserPasswordAttemptsRemain userPasswordAttemptsRemain_; + UserLoginLockTime userLoginLockTime_; NameIndexMap tagNameIndexMap_; NameIndexMap edgeNameIndexMap_; diff --git a/src/clients/storage/StorageClient.cpp b/src/clients/storage/StorageClient.cpp index 6c1a81f5a8d..115f869883a 100644 --- a/src/clients/storage/StorageClient.cpp +++ b/src/clients/storage/StorageClient.cpp @@ -485,6 +485,7 @@ StorageRpcRespFuture StorageClient::lookupIndex( bool isEdge, int32_t tagOrEdge, const std::vector& returnCols, + std::vector orderBy, int64_t limit) { // TODO(sky) : instead of isEdge and tagOrEdge to nebula::cpp2::SchemaID for graph layer. auto space = param.space; @@ -516,6 +517,7 @@ StorageRpcRespFuture StorageClient::lookupIndex( req.indices_ref() = spec; req.common_ref() = common; req.limit_ref() = limit; + req.order_by_ref() = orderBy; } return collectResponse(param.evb, diff --git a/src/clients/storage/StorageClient.h b/src/clients/storage/StorageClient.h index 40c0dcb3aca..6b564116815 100644 --- a/src/clients/storage/StorageClient.h +++ b/src/clients/storage/StorageClient.h @@ -141,6 +141,7 @@ class StorageClient bool isEdge, int32_t tagOrEdge, const std::vector& returnCols, + std::vector orderBy, int64_t limit); StorageRpcRespFuture lookupAndTraverse( diff --git a/src/common/CMakeLists.txt b/src/common/CMakeLists.txt index a1a45aaff41..a3470a8baa0 100644 --- a/src/common/CMakeLists.txt +++ b/src/common/CMakeLists.txt @@ -27,3 +27,4 @@ nebula_add_subdirectory(utils) nebula_add_subdirectory(ssl) nebula_add_subdirectory(geo) nebula_add_subdirectory(memory) +nebula_add_subdirectory(id) diff --git a/src/common/datatypes/Duration.h b/src/common/datatypes/Duration.h index 730ab1b324d..b5d9247b3f0 100644 --- a/src/common/datatypes/Duration.h +++ b/src/common/datatypes/Duration.h @@ -125,10 +125,8 @@ struct Duration { } std::string toString() const { - std::stringstream ss; - ss << "P" << months << "M" << days() << "D" - << "T" << seconds << "S"; - return ss.str(); + return folly::sformat( + "P{}MT{}.{:0>6}000S", months, seconds + microseconds / 1000000, microseconds % 1000000); } folly::dynamic toJson() const { diff --git a/src/common/datatypes/test/ValueTest.cpp b/src/common/datatypes/test/ValueTest.cpp index 00f89ed8d99..5cfe18c76b1 100644 --- a/src/common/datatypes/test/ValueTest.cpp +++ b/src/common/datatypes/test/ValueTest.cpp @@ -1630,6 +1630,27 @@ TEST(Value, Ctor) { // Value v2(&tmp); } +TEST(Value, ToString) { + { + Duration d; + d.addYears(1); + d.addMonths(2); + d.addDays(500); + d.addSeconds(10); + d.addMicroseconds(20); + EXPECT_EQ(d.toString(), "P14MT43200010.000020000S"); + } + { + Duration d; + d.addYears(1); + d.addMonths(2); + d.addDays(500); + d.addSeconds(10); + d.addMicroseconds(20000000); + EXPECT_EQ(d.toString(), "P14MT43200030.000000000S"); + } +} + } // namespace nebula int main(int argc, char** argv) { diff --git a/src/common/expression/UUIDExpression.cpp b/src/common/expression/UUIDExpression.cpp index 02393a05c41..e9e51010196 100644 --- a/src/common/expression/UUIDExpression.cpp +++ b/src/common/expression/UUIDExpression.cpp @@ -10,36 +10,26 @@ namespace nebula { bool UUIDExpression::operator==(const Expression& rhs) const { - if (kind_ != rhs.kind()) { - return false; - } - - const auto& r = static_cast(rhs); - return field_ == r.field_; + return kind_ == rhs.kind(); } void UUIDExpression::writeTo(Encoder& encoder) const { // kind_ + UNUSED(encoder); encoder << kind_; - - // field_ - CHECK(!field_.empty()); - encoder << field_; } void UUIDExpression::resetFrom(Decoder& decoder) { - // Read field_ - field_ = decoder.readStr(); + UNUSED(decoder); } const Value& UUIDExpression::eval(ExpressionContext& ctx) { - // TODO UNUSED(ctx); return result_; } std::string UUIDExpression::toString() const { - return folly::stringPrintf("uuid(\"%s\")", field_.c_str()); + return folly::stringPrintf("uuid()"); } void UUIDExpression::accept(ExprVisitor* visitor) { diff --git a/src/common/expression/UUIDExpression.h b/src/common/expression/UUIDExpression.h index ed53879e02d..febcfc32d6a 100644 --- a/src/common/expression/UUIDExpression.h +++ b/src/common/expression/UUIDExpression.h @@ -14,8 +14,8 @@ class UUIDExpression final : public Expression { friend class Expression; public: - static UUIDExpression* make(ObjectPool* pool, const std::string& field = "") { - return pool->add(new UUIDExpression(pool, field)); + static UUIDExpression* make(ObjectPool* pool) { + return pool->add(new UUIDExpression(pool)); } bool operator==(const Expression& rhs) const override; @@ -27,18 +27,16 @@ class UUIDExpression final : public Expression { void accept(ExprVisitor* visitor) override; Expression* clone() const override { - return UUIDExpression::make(pool_, field_); + return UUIDExpression::make(pool_); } private: - explicit UUIDExpression(ObjectPool* pool, const std::string& field = "") - : Expression(pool, Kind::kUUID), field_(field) {} + explicit UUIDExpression(ObjectPool* pool) : Expression(pool, Kind::kUUID) {} void writeTo(Encoder& encoder) const override; void resetFrom(Decoder& decoder) override; private: - std::string field_; Value result_; }; diff --git a/src/common/expression/test/EncodeDecodeTest.cpp b/src/common/expression/test/EncodeDecodeTest.cpp index 090e63594ce..95f48cb7ea6 100644 --- a/src/common/expression/test/EncodeDecodeTest.cpp +++ b/src/common/expression/test/EncodeDecodeTest.cpp @@ -205,7 +205,7 @@ TEST(ExpressionEncodeDecode, TypeCastingExpression) { } TEST(ExpressionEncodeDecode, UUIDExpression) { - const auto& uuidEx = *UUIDExpression::make(&pool, "field"); + const auto& uuidEx = *UUIDExpression::make(&pool); std::string encoded = Expression::encode(uuidEx); auto decoded = Expression::decode(&pool, folly::StringPiece(encoded.data(), encoded.size())); EXPECT_EQ(uuidEx, *decoded); diff --git a/src/common/expression/test/ExpressionTest.cpp b/src/common/expression/test/ExpressionTest.cpp index 43cc6a34693..68f9417c374 100644 --- a/src/common/expression/test/ExpressionTest.cpp +++ b/src/common/expression/test/ExpressionTest.cpp @@ -142,7 +142,7 @@ TEST_F(ExpressionTest, TestExprClone) { auto fnCallExpr = FunctionCallExpression::make(&pool, "count", ArgumentList::make(&pool)); ASSERT_EQ(*fnCallExpr, *fnCallExpr->clone()); - auto uuidExpr = UUIDExpression::make(&pool, "hello"); + auto uuidExpr = UUIDExpression::make(&pool); ASSERT_EQ(*uuidExpr, *uuidExpr->clone()); auto subExpr = SubscriptExpression::make( diff --git a/src/common/graph/Response.h b/src/common/graph/Response.h index d1d6594b235..dab1d00dbf3 100644 --- a/src/common/graph/Response.h +++ b/src/common/graph/Response.h @@ -117,7 +117,7 @@ /* ListClusterInfo Failure */ \ X(E_LIST_CLUSTER_FAILURE, -2070) \ X(E_LIST_CLUSTER_GET_ABS_PATH_FAILURE, -2071) \ - X(E_GET_META_DIR_FAILURE, -2072) \ + X(E_LIST_CLUSTER_NO_AGENT_FAILURE, -2072) \ \ X(E_QUERY_NOT_FOUND, -2073) \ X(E_AGENT_HB_FAILUE, -2074) \ @@ -233,7 +233,7 @@ struct AuthResponse { if (!checkPointer(timeZoneOffsetSeconds.get(), rhs.timeZoneOffsetSeconds.get())) { return false; } - return checkPointer(timeZoneName.get(), timeZoneName.get()); + return checkPointer(timeZoneName.get(), rhs.timeZoneName.get()); } ErrorCode errorCode{ErrorCode::SUCCEEDED}; diff --git a/src/common/id/CMakeLists.txt b/src/common/id/CMakeLists.txt new file mode 100644 index 00000000000..e60a8a76ebe --- /dev/null +++ b/src/common/id/CMakeLists.txt @@ -0,0 +1,11 @@ +# Copyright (c) 2021 vesoft inc. All rights reserved. +# +# This source code is licensed under Apache 2.0 License, +# attached with Common Clause Condition 1.0, found in the LICENSES directory. + +nebula_add_library( + snowflake_obj OBJECT + Snowflake.cpp +) + +nebula_add_subdirectory(test) diff --git a/src/common/id/Snowflake.cpp b/src/common/id/Snowflake.cpp new file mode 100644 index 00000000000..021330009f9 --- /dev/null +++ b/src/common/id/Snowflake.cpp @@ -0,0 +1,64 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "common/id/Snowflake.h" + +namespace nebula { +void Snowflake::initWorkerId(meta::MetaClient* client) { + const std::string& ip = client->getLocalIp(); + auto result = client->getWorkerId(ip).get(); + if (!result.ok()) { + LOG(FATAL) << "Failed to get workerId from meta server"; + } + workerId_ = result.value(); + LOG(INFO) << "WorkerId init success: " << workerId_; +} + +int64_t Snowflake::getId() { + std::lock_guard guard(mutex_); + + int64_t timestamp = getTimestamp(); + if (timestamp < lastTimestamp_) { + LOG(ERROR) << "Clock back"; + return kFirstBitRevert & getIdByTs(timestamp); + } + + return getIdByTs(timestamp); +} + +// get snowflake id by timestamp +// update lastTimestamp_ or sequence_ +int64_t Snowflake::getIdByTs(int64_t timestamp) { + // if it is the same time, then the microsecond sequence + if (lastTimestamp_ == timestamp) { + sequence_ = (sequence_ + 1) & kMaxSequence; + // if the microsecond sequence overflow + if (sequence_ == 0) { + // block to the next millisecond, get the new timestamp + timestamp = nextTimestamp(); + } + } else { + sequence_ = 0; + } + lastTimestamp_ = timestamp; + return (timestamp - kStartStmp) << kTimestampLeft | workerId_ << kWorkerIdLeft | sequence_; +} + +int64_t Snowflake::getTimestamp() { + return std::chrono::duration_cast( + std::chrono::system_clock::now().time_since_epoch()) + .count(); +} + +// wait for the next millisecond +int64_t Snowflake::nextTimestamp() { + int64_t timestamp = getTimestamp(); + while (timestamp <= lastTimestamp_) { + timestamp = getTimestamp(); + } + return timestamp; +} + +} // namespace nebula diff --git a/src/common/id/Snowflake.h b/src/common/id/Snowflake.h new file mode 100644 index 00000000000..ae45f2e5a93 --- /dev/null +++ b/src/common/id/Snowflake.h @@ -0,0 +1,60 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#ifndef COMMON_ID_SNOWFLAKE_H_ +#define COMMON_ID_SNOWFLAKE_H_ + +#include "clients/meta/MetaClient.h" +#include "common/base/Base.h" + +namespace nebula { +class Snowflake { + FRIEND_TEST(SnowflakeTest, TestWorkerId); + FRIEND_TEST(SnowflakeTest, TestConcurrency); + friend int64_t getSequence(int64_t id); + friend int64_t getWorkerId(int64_t id); + friend int64_t getTimestamp(int64_t id); + + public: + Snowflake() = default; + + static void initWorkerId(meta::MetaClient* client); + + int64_t getId(); + + private: + static int64_t getTimestamp(); + + int64_t nextTimestamp(); + + int64_t getIdByTs(int64_t timestamp); + + std::mutex mutex_; + /* + * Snowflake id: + * timestampBit 38 bits (8.6 years) | + * workerBit 13 bits (8k workerid) | + * sequenceBit 12 bits (4 million per second) | + */ + int64_t lastTimestamp_{-1}; + static inline int64_t workerId_{0}; + int64_t sequence_{0}; + + static constexpr int64_t kStartStmp = 1577808000000; // start time: 2020-01-01 00:00:00 + static constexpr int64_t kWorkerBit = 13; // worker id bit + static constexpr int64_t kSequenceBit = 12; // sequence bit + + static constexpr int64_t kMaxWorkerId = (1 << kWorkerBit) - 1; // as worker id mask + static constexpr int64_t kMaxSequence = (1 << kSequenceBit) - 1; + + static constexpr int64_t kWorkerIdLeft = kSequenceBit; // workerId left bits + static constexpr int64_t kTimestampLeft = kSequenceBit + kWorkerBit; + + static constexpr int64_t kFirstBitRevert = 0x9000000000000000; // revert the first bit +}; + +} // namespace nebula + +#endif // COMMON_ID_SNOWFLAKE_H_ diff --git a/src/common/id/test/CMakeLists.txt b/src/common/id/test/CMakeLists.txt new file mode 100644 index 00000000000..347cd19a373 --- /dev/null +++ b/src/common/id/test/CMakeLists.txt @@ -0,0 +1,80 @@ +# Copyright (c) 2021 vesoft inc. All rights reserved. +# +# This source code is licensed under Apache 2.0 License, + +nebula_add_executable( + NAME snowflake_bm + SOURCES SnowflakeBenchmark.cpp + OBJECTS + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + LIBRARIES + follybenchmark + boost_regex + ${PROXYGEN_LIBRARIES} + ${THRIFT_LIBRARIES} +) + + +nebula_add_test( + NAME snowflake_test + SOURCES SnowflakeTest.cpp + OBJECTS + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + $ + LIBRARIES + gtest + gtest_main + ${PROXYGEN_LIBRARIES} + ${THRIFT_LIBRARIES} +) diff --git a/src/common/id/test/SnowflakeBenchmark.cpp b/src/common/id/test/SnowflakeBenchmark.cpp new file mode 100644 index 00000000000..976cec1c3ee --- /dev/null +++ b/src/common/id/test/SnowflakeBenchmark.cpp @@ -0,0 +1,84 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include + +#include +#include + +#include "common/base/Base.h" +#include "common/id/Snowflake.h" + +size_t SnowflakeTest(size_t iters) { + constexpr size_t ops = 10UL; + + nebula::Snowflake generator; + auto i = 0UL; + while (i++ < ops * iters) { + int64_t id = generator.getId(); + folly::doNotOptimizeAway(id); + } + + return iters * ops; +} + +size_t SnowflakeCurrencyTest(size_t iters, int threadNum) { + constexpr size_t ops = 100000UL; + + nebula::Snowflake generator; + std::vector threads; + threads.reserve(threadNum); + + auto proc = [&iters, &generator]() { + auto n = iters * ops; + for (auto i = 0UL; i < n; i++) { + int64_t id = generator.getId(); + folly::doNotOptimizeAway(id); + } + }; + + for (int i = 0; i < threadNum; i++) { + threads.emplace_back(std::thread(proc)); + } + + for (int i = 0; i < threadNum; i++) { + threads[i].join(); + } + + return iters * ops * threadNum; +} + +BENCHMARK_NAMED_PARAM_MULTI(SnowflakeTest, 1UL) +BENCHMARK_DRAW_LINE(); + +BENCHMARK_NAMED_PARAM_MULTI(SnowflakeCurrencyTest, 1_thread, 1) +BENCHMARK_RELATIVE_NAMED_PARAM_MULTI(SnowflakeCurrencyTest, 2_thread, 2) +BENCHMARK_RELATIVE_NAMED_PARAM_MULTI(SnowflakeCurrencyTest, 4_thread, 4) +BENCHMARK_RELATIVE_NAMED_PARAM_MULTI(SnowflakeCurrencyTest, 8_thread, 8) +BENCHMARK_RELATIVE_NAMED_PARAM_MULTI(SnowflakeCurrencyTest, 16_thread, 16) +BENCHMARK_RELATIVE_NAMED_PARAM_MULTI(SnowflakeCurrencyTest, 32_thread, 32) +BENCHMARK_RELATIVE_NAMED_PARAM_MULTI(SnowflakeCurrencyTest, 64_thread, 64) + +int main(int argc, char** argv) { + gflags::ParseCommandLineFlags(&argc, &argv, true); + folly::runBenchmarks(); + return 0; +} + +/* Intel(R) Xeon(R) CPU E5-2673 v3 @ 2.40GHz +============================================================================ +nebula/src/common/id/test/SnowflakeBenchmark.cpprelative time/iter iters/s +============================================================================ +SnowflakeTest(1UL) 68.03ns 14.70M +---------------------------------------------------------------------------- +SnowflakeCurrencyTest(1_thread) 239.12ns 4.18M +SnowflakeCurrencyTest(2_thread) 99.77% 239.66ns 4.17M +SnowflakeCurrencyTest(4_thread) 98.65% 242.39ns 4.13M +SnowflakeCurrencyTest(8_thread) 90.01% 265.67ns 3.76M +SnowflakeCurrencyTest(16_thread) 68.23% 350.46ns 2.85M +SnowflakeCurrencyTest(32_thread) 60.05% 398.22ns 2.51M +SnowflakeCurrencyTest(64_thread) 52.51% 455.38ns 2.20M +============================================================================ +*/ diff --git a/src/common/id/test/SnowflakeTest.cpp b/src/common/id/test/SnowflakeTest.cpp new file mode 100644 index 00000000000..d1ca39c3642 --- /dev/null +++ b/src/common/id/test/SnowflakeTest.cpp @@ -0,0 +1,68 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include +#include +#include + +#include +#include + +#include "common/base/Logging.h" +#include "common/id/Snowflake.h" + +namespace nebula { + +int64_t getSequence(int64_t id) { + return id & Snowflake::kMaxSequence; +} +int64_t getWorkerId(int64_t id) { + return (id >> Snowflake::kSequenceBit) & Snowflake::kMaxWorkerId; +} +int64_t getTimestamp(int64_t id) { + return id >> (Snowflake::kSequenceBit + Snowflake::kWorkerBit); +} + +TEST(SnowflakeTest, TestWorkerId) { + int64_t maxWorkerId = 1023; + + Snowflake generator; + + for (int i = 0; i < maxWorkerId; i++) { + Snowflake::workerId_ = i; + int64_t id = generator.getId(); + + ASSERT_EQ(getWorkerId(id), i); + } +} + +TEST(SnowflakeTest, TestConcurrency) { + Snowflake::workerId_ = 0; + int threadNum = 16; + int times = 1000; + + Snowflake generator; + folly::ConcurrentHashMap map; + std::vector threads; + threads.reserve(threadNum); + + auto proc = [&]() { + for (int i = 0; i < times; i++) { + int64_t id = generator.getId(); + ASSERT_TRUE(map.find(id) == map.end()) << "id: " << id; + map.insert(id, 0); + } + }; + + for (int i = 0; i < threadNum; i++) { + threads.emplace_back(std::thread(proc)); + } + + for (int i = 0; i < threadNum; i++) { + threads[i].join(); + } +} + +} // namespace nebula diff --git a/src/common/meta/GflagsManager.cpp b/src/common/meta/GflagsManager.cpp index 0588d45cde6..4661dabef95 100644 --- a/src/common/meta/GflagsManager.cpp +++ b/src/common/meta/GflagsManager.cpp @@ -53,6 +53,7 @@ std::unordered_map> GflagsManager {"minloglevel", {cpp2::ConfigMode::MUTABLE, false}}, {"v", {cpp2::ConfigMode::MUTABLE, false}}, {"heartbeat_interval_secs", {cpp2::ConfigMode::MUTABLE, false}}, + {"agent_heartbeat_interval_secs", {cpp2::ConfigMode::MUTABLE, false}}, {"meta_client_retry_times", {cpp2::ConfigMode::MUTABLE, false}}, {"slow_op_threshold_ms", {cpp2::ConfigMode::MUTABLE, false}}, {"wal_ttl", {cpp2::ConfigMode::MUTABLE, false}}, diff --git a/src/common/stats/StatsManager.cpp b/src/common/stats/StatsManager.cpp index 5911566e476..96abf5fc512 100644 --- a/src/common/stats/StatsManager.cpp +++ b/src/common/stats/StatsManager.cpp @@ -170,10 +170,13 @@ CounterId StatsManager::counterWithLabels(const CounterId& id, auto index = id.index(); CHECK(!labels.empty()); std::string newIndex; + newIndex.append(index); + newIndex.append("{"); for (auto& [k, v] : labels) { - newIndex.append(k).append("_").append(v).append("_"); + newIndex.append(k).append("=").append(v).append(","); } - newIndex.append(index); + newIndex.back() = '}'; + auto it = sm.nameMap_.find(newIndex); // Get the counter if it already exists if (it != sm.nameMap_.end()) { @@ -196,10 +199,12 @@ CounterId StatsManager::histoWithLabels(const CounterId& id, const std::vector> syste {"users", {"__users__", true}}, {"hosts", {"__hosts__", false}}, {"versions", {"__versions__", false}}, - {"machines", {"__machines__", false}}, + {"machines", {"__machines__", true}}, {"host_dirs", {"__host_dirs__", false}}, {"snapshots", {"__snapshots__", false}}, {"configs", {"__configs__", true}}, diff --git a/src/daemons/CMakeLists.txt b/src/daemons/CMakeLists.txt index 8d8d68e680f..1d476baf543 100644 --- a/src/daemons/CMakeLists.txt +++ b/src/daemons/CMakeLists.txt @@ -7,6 +7,7 @@ set(common_deps $ $ $ + $ $ $ $ diff --git a/src/graph/context/Iterator.h b/src/graph/context/Iterator.h index 7caa285f05b..1a89b12d175 100644 --- a/src/graph/context/Iterator.h +++ b/src/graph/context/Iterator.h @@ -450,11 +450,11 @@ class SequentialIter : public Iterator { } std::vector::iterator begin() { - return rows_->begin(); + return CHECK_NOTNULL(rows_)->begin(); } std::vector::iterator end() { - return rows_->end(); + return CHECK_NOTNULL(rows_)->end(); } const std::unordered_map& getColIndices() const { diff --git a/src/graph/executor/query/IndexScanExecutor.cpp b/src/graph/executor/query/IndexScanExecutor.cpp index 1e863d79155..70c5bd00831 100644 --- a/src/graph/executor/query/IndexScanExecutor.cpp +++ b/src/graph/executor/query/IndexScanExecutor.cpp @@ -47,6 +47,7 @@ folly::Future IndexScanExecutor::indexScan() { lookup->isEdge(), lookup->schemaId(), lookup->returnColumns(), + lookup->orderBy(), lookup->limit(qctx_)) .via(runner()) .thenValue([this](StorageRpcResponse &&rpcResp) { diff --git a/src/graph/optimizer/CMakeLists.txt b/src/graph/optimizer/CMakeLists.txt index d7326f5bb01..9abde4eb6fd 100644 --- a/src/graph/optimizer/CMakeLists.txt +++ b/src/graph/optimizer/CMakeLists.txt @@ -50,6 +50,13 @@ nebula_add_library( rule/PushLimitDownScanAppendVerticesRule.cpp rule/GetEdgesTransformRule.cpp rule/PushLimitDownScanEdgesAppendVerticesRule.cpp + rule/PushTopNDownIndexScanRule.cpp + rule/PushTopNDownTagIndexFullScanRule.cpp + rule/PushTopNDownTagIndexPrefixScanRule.cpp + rule/PushTopNDownTagIndexRangeScanRule.cpp + rule/PushTopNDownEdgeIndexFullScanRule.cpp + rule/PushTopNDownEdgeIndexPrefixScanRule.cpp + rule/PushTopNDownEdgeIndexRangeScanRule.cpp ) nebula_add_subdirectory(test) diff --git a/src/graph/optimizer/OptimizerUtils.cpp b/src/graph/optimizer/OptimizerUtils.cpp index 9f9fd41c78b..c5c8dad6c78 100644 --- a/src/graph/optimizer/OptimizerUtils.cpp +++ b/src/graph/optimizer/OptimizerUtils.cpp @@ -427,6 +427,7 @@ void OptimizerUtils::copyIndexScanData(const nebula::graph::IndexScan* from, to->setOrderBy(from->orderBy()); to->setLimit(from->limit(qctx)); to->setFilter(from->filter() == nullptr ? nullptr : from->filter()->clone()); + to->setYieldColumns(from->yieldColumns()); } Status OptimizerUtils::compareAndSwapBound(std::pair& a, std::pair& b) { diff --git a/src/graph/optimizer/rule/PushTopNDownEdgeIndexFullScanRule.cpp b/src/graph/optimizer/rule/PushTopNDownEdgeIndexFullScanRule.cpp new file mode 100644 index 00000000000..a64c2ae4767 --- /dev/null +++ b/src/graph/optimizer/rule/PushTopNDownEdgeIndexFullScanRule.cpp @@ -0,0 +1,110 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "graph/optimizer/rule/PushTopNDownEdgeIndexFullScanRule.h" + +#include "graph/optimizer/OptContext.h" +#include "graph/optimizer/OptGroup.h" +#include "graph/planner/plan/PlanNode.h" +#include "graph/planner/plan/Query.h" +#include "graph/planner/plan/Scan.h" + +using nebula::graph::EdgeIndexFullScan; +using nebula::graph::PlanNode; +using nebula::graph::Project; +using nebula::graph::QueryContext; +using nebula::graph::TopN; + +namespace nebula { +namespace opt { + +std::unique_ptr PushTopNDownEdgeIndexFullScanRule::kInstance = + std::unique_ptr(new PushTopNDownEdgeIndexFullScanRule()); + +PushTopNDownEdgeIndexFullScanRule::PushTopNDownEdgeIndexFullScanRule() { + RuleSet::QueryRules().addRule(this); +} + +const Pattern &PushTopNDownEdgeIndexFullScanRule::pattern() const { + static Pattern pattern = Pattern::create( + graph::PlanNode::Kind::kTopN, + {Pattern::create(graph::PlanNode::Kind::kProject, + {Pattern::create(graph::PlanNode::Kind::kEdgeIndexFullScan)})}); + return pattern; +} + +StatusOr PushTopNDownEdgeIndexFullScanRule::transform( + OptContext *octx, const MatchedResult &matched) const { + auto topNGroupNode = matched.node; + auto projectGroupNode = matched.dependencies.front().node; + auto indexScanGroupNode = matched.dependencies.front().dependencies.front().node; + + const auto topN = static_cast(topNGroupNode->node()); + const auto project = static_cast(projectGroupNode->node()); + const auto indexScan = static_cast(indexScanGroupNode->node()); + + int64_t limitRows = topN->offset() + topN->count(); + + auto &factors = topN->factors(); + auto projColNames = project->colNames(); + const auto &yieldColumns = indexScan->yieldColumns(); + + std::unordered_map namesMap; + for (auto yieldColumn : yieldColumns->columns()) { + if (yieldColumn->expr()->kind() == Expression::Kind::kEdgeProperty) { + const auto &propName = static_cast(yieldColumn->expr())->prop(); + namesMap[yieldColumn->name()] = propName; + continue; + } + return TransformResult::noTransform(); + } + std::vector orderBys; + orderBys.reserve(factors.size()); + + for (auto factor : factors) { + auto colName = projColNames[factor.first]; + auto found = namesMap.find(colName); + if (found == namesMap.end()) { + return Status::Error(); + } + storage::cpp2::OrderBy orderBy; + orderBy.prop_ref() = found->second; + orderBy.direction_ref() = factor.second == OrderFactor::OrderType::ASCEND + ? storage::cpp2::OrderDirection::ASCENDING + : storage::cpp2::OrderDirection::DESCENDING; + orderBys.emplace_back(orderBy); + } + + auto newTopN = static_cast(topN->clone()); + auto newtopNGroupNode = OptGroupNode::create(octx, newTopN, topNGroupNode->group()); + + auto newProject = static_cast(project->clone()); + auto newProjectGroup = OptGroup::create(octx); + auto newProjectGroupNode = newProjectGroup->makeGroupNode(newProject); + + auto newIndexScan = static_cast(indexScan->clone()); + newIndexScan->setLimit(limitRows); + newIndexScan->setOrderBy(orderBys); + auto newIndexScanGroup = OptGroup::create(octx); + auto newIndexScanGroupNode = newIndexScanGroup->makeGroupNode(newIndexScan); + + newtopNGroupNode->dependsOn(newProjectGroup); + newProjectGroupNode->dependsOn(newIndexScanGroup); + for (auto dep : indexScanGroupNode->dependencies()) { + newIndexScanGroupNode->dependsOn(dep); + } + + TransformResult result; + result.eraseAll = true; + result.newGroupNodes.emplace_back(newtopNGroupNode); + return result; +} + +std::string PushTopNDownEdgeIndexFullScanRule::toString() const { + return "PushTopNDownEdgeIndexFullScanRule"; +} + +} // namespace opt +} // namespace nebula diff --git a/src/graph/optimizer/rule/PushTopNDownEdgeIndexFullScanRule.h b/src/graph/optimizer/rule/PushTopNDownEdgeIndexFullScanRule.h new file mode 100644 index 00000000000..271d1045ac0 --- /dev/null +++ b/src/graph/optimizer/rule/PushTopNDownEdgeIndexFullScanRule.h @@ -0,0 +1,29 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#pragma once + +#include "graph/optimizer/OptRule.h" + +namespace nebula { +namespace opt { + +class PushTopNDownEdgeIndexFullScanRule final : public OptRule { + public: + const Pattern &pattern() const override; + + StatusOr transform(OptContext *ctx, + const MatchedResult &matched) const override; + + std::string toString() const override; + + private: + PushTopNDownEdgeIndexFullScanRule(); + + static std::unique_ptr kInstance; +}; + +} // namespace opt +} // namespace nebula diff --git a/src/graph/optimizer/rule/PushTopNDownEdgeIndexPrefixScanRule.cpp b/src/graph/optimizer/rule/PushTopNDownEdgeIndexPrefixScanRule.cpp new file mode 100644 index 00000000000..305a3f3844d --- /dev/null +++ b/src/graph/optimizer/rule/PushTopNDownEdgeIndexPrefixScanRule.cpp @@ -0,0 +1,110 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "graph/optimizer/rule/PushTopNDownEdgeIndexPrefixScanRule.h" + +#include "graph/optimizer/OptContext.h" +#include "graph/optimizer/OptGroup.h" +#include "graph/planner/plan/PlanNode.h" +#include "graph/planner/plan/Query.h" +#include "graph/planner/plan/Scan.h" + +using nebula::graph::EdgeIndexPrefixScan; +using nebula::graph::PlanNode; +using nebula::graph::Project; +using nebula::graph::QueryContext; +using nebula::graph::TopN; + +namespace nebula { +namespace opt { + +std::unique_ptr PushTopNDownEdgeIndexPrefixScanRule::kInstance = + std::unique_ptr(new PushTopNDownEdgeIndexPrefixScanRule()); + +PushTopNDownEdgeIndexPrefixScanRule::PushTopNDownEdgeIndexPrefixScanRule() { + RuleSet::QueryRules().addRule(this); +} + +const Pattern &PushTopNDownEdgeIndexPrefixScanRule::pattern() const { + static Pattern pattern = Pattern::create( + graph::PlanNode::Kind::kTopN, + {Pattern::create(graph::PlanNode::Kind::kProject, + {Pattern::create(graph::PlanNode::Kind::kEdgeIndexPrefixScan)})}); + return pattern; +} + +StatusOr PushTopNDownEdgeIndexPrefixScanRule::transform( + OptContext *octx, const MatchedResult &matched) const { + auto topNGroupNode = matched.node; + auto projectGroupNode = matched.dependencies.front().node; + auto indexScanGroupNode = matched.dependencies.front().dependencies.front().node; + + const auto topN = static_cast(topNGroupNode->node()); + const auto project = static_cast(projectGroupNode->node()); + const auto indexScan = static_cast(indexScanGroupNode->node()); + + int64_t limitRows = topN->offset() + topN->count(); + + auto &factors = topN->factors(); + auto projColNames = project->colNames(); + const auto &yieldColumns = indexScan->yieldColumns(); + + std::unordered_map namesMap; + for (auto yieldColumn : yieldColumns->columns()) { + if (yieldColumn->expr()->kind() == Expression::Kind::kEdgeProperty) { + const auto &propName = static_cast(yieldColumn->expr())->prop(); + namesMap[yieldColumn->name()] = propName; + continue; + } + return TransformResult::noTransform(); + } + std::vector orderBys; + orderBys.reserve(factors.size()); + + for (auto factor : factors) { + auto colName = projColNames[factor.first]; + auto found = namesMap.find(colName); + if (found == namesMap.end()) { + return Status::Error(); + } + storage::cpp2::OrderBy orderBy; + orderBy.prop_ref() = found->second; + orderBy.direction_ref() = factor.second == OrderFactor::OrderType::ASCEND + ? storage::cpp2::OrderDirection::ASCENDING + : storage::cpp2::OrderDirection::DESCENDING; + orderBys.emplace_back(orderBy); + } + + auto newTopN = static_cast(topN->clone()); + auto newtopNGroupNode = OptGroupNode::create(octx, newTopN, topNGroupNode->group()); + + auto newProject = static_cast(project->clone()); + auto newProjectGroup = OptGroup::create(octx); + auto newProjectGroupNode = newProjectGroup->makeGroupNode(newProject); + + auto newIndexScan = static_cast(indexScan->clone()); + newIndexScan->setLimit(limitRows); + newIndexScan->setOrderBy(orderBys); + auto newIndexScanGroup = OptGroup::create(octx); + auto newIndexScanGroupNode = newIndexScanGroup->makeGroupNode(newIndexScan); + + newtopNGroupNode->dependsOn(newProjectGroup); + newProjectGroupNode->dependsOn(newIndexScanGroup); + for (auto dep : indexScanGroupNode->dependencies()) { + newIndexScanGroupNode->dependsOn(dep); + } + + TransformResult result; + result.eraseAll = true; + result.newGroupNodes.emplace_back(newtopNGroupNode); + return result; +} + +std::string PushTopNDownEdgeIndexPrefixScanRule::toString() const { + return "PushTopNDownEdgeIndexPrefixScanRule"; +} + +} // namespace opt +} // namespace nebula diff --git a/src/graph/optimizer/rule/PushTopNDownEdgeIndexPrefixScanRule.h b/src/graph/optimizer/rule/PushTopNDownEdgeIndexPrefixScanRule.h new file mode 100644 index 00000000000..11fae1deb3d --- /dev/null +++ b/src/graph/optimizer/rule/PushTopNDownEdgeIndexPrefixScanRule.h @@ -0,0 +1,29 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#pragma once + +#include "graph/optimizer/OptRule.h" + +namespace nebula { +namespace opt { + +class PushTopNDownEdgeIndexPrefixScanRule final : public OptRule { + public: + const Pattern &pattern() const override; + + StatusOr transform(OptContext *ctx, + const MatchedResult &matched) const override; + + std::string toString() const override; + + private: + PushTopNDownEdgeIndexPrefixScanRule(); + + static std::unique_ptr kInstance; +}; + +} // namespace opt +} // namespace nebula diff --git a/src/graph/optimizer/rule/PushTopNDownEdgeIndexRangeScanRule.cpp b/src/graph/optimizer/rule/PushTopNDownEdgeIndexRangeScanRule.cpp new file mode 100644 index 00000000000..58a668885af --- /dev/null +++ b/src/graph/optimizer/rule/PushTopNDownEdgeIndexRangeScanRule.cpp @@ -0,0 +1,110 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "graph/optimizer/rule/PushTopNDownEdgeIndexRangeScanRule.h" + +#include "graph/optimizer/OptContext.h" +#include "graph/optimizer/OptGroup.h" +#include "graph/planner/plan/PlanNode.h" +#include "graph/planner/plan/Query.h" +#include "graph/planner/plan/Scan.h" + +using nebula::graph::EdgeIndexRangeScan; +using nebula::graph::PlanNode; +using nebula::graph::Project; +using nebula::graph::QueryContext; +using nebula::graph::TopN; + +namespace nebula { +namespace opt { + +std::unique_ptr PushTopNDownEdgeIndexRangeScanRule::kInstance = + std::unique_ptr(new PushTopNDownEdgeIndexRangeScanRule()); + +PushTopNDownEdgeIndexRangeScanRule::PushTopNDownEdgeIndexRangeScanRule() { + RuleSet::QueryRules().addRule(this); +} + +const Pattern &PushTopNDownEdgeIndexRangeScanRule::pattern() const { + static Pattern pattern = Pattern::create( + graph::PlanNode::Kind::kTopN, + {Pattern::create(graph::PlanNode::Kind::kProject, + {Pattern::create(graph::PlanNode::Kind::kEdgeIndexRangeScan)})}); + return pattern; +} + +StatusOr PushTopNDownEdgeIndexRangeScanRule::transform( + OptContext *octx, const MatchedResult &matched) const { + auto topNGroupNode = matched.node; + auto projectGroupNode = matched.dependencies.front().node; + auto indexScanGroupNode = matched.dependencies.front().dependencies.front().node; + + const auto topN = static_cast(topNGroupNode->node()); + const auto project = static_cast(projectGroupNode->node()); + const auto indexScan = static_cast(indexScanGroupNode->node()); + + int64_t limitRows = topN->offset() + topN->count(); + + auto &factors = topN->factors(); + auto projColNames = project->colNames(); + const auto &yieldColumns = indexScan->yieldColumns(); + + std::unordered_map namesMap; + for (auto yieldColumn : yieldColumns->columns()) { + if (yieldColumn->expr()->kind() == Expression::Kind::kEdgeProperty) { + const auto &propName = static_cast(yieldColumn->expr())->prop(); + namesMap[yieldColumn->name()] = propName; + continue; + } + return TransformResult::noTransform(); + } + std::vector orderBys; + orderBys.reserve(factors.size()); + + for (auto factor : factors) { + auto colName = projColNames[factor.first]; + auto found = namesMap.find(colName); + if (found == namesMap.end()) { + return Status::Error(); + } + storage::cpp2::OrderBy orderBy; + orderBy.prop_ref() = found->second; + orderBy.direction_ref() = factor.second == OrderFactor::OrderType::ASCEND + ? storage::cpp2::OrderDirection::ASCENDING + : storage::cpp2::OrderDirection::DESCENDING; + orderBys.emplace_back(orderBy); + } + + auto newTopN = static_cast(topN->clone()); + auto newtopNGroupNode = OptGroupNode::create(octx, newTopN, topNGroupNode->group()); + + auto newProject = static_cast(project->clone()); + auto newProjectGroup = OptGroup::create(octx); + auto newProjectGroupNode = newProjectGroup->makeGroupNode(newProject); + + auto newIndexScan = static_cast(indexScan->clone()); + newIndexScan->setLimit(limitRows); + newIndexScan->setOrderBy(orderBys); + auto newIndexScanGroup = OptGroup::create(octx); + auto newIndexScanGroupNode = newIndexScanGroup->makeGroupNode(newIndexScan); + + newtopNGroupNode->dependsOn(newProjectGroup); + newProjectGroupNode->dependsOn(newIndexScanGroup); + for (auto dep : indexScanGroupNode->dependencies()) { + newIndexScanGroupNode->dependsOn(dep); + } + + TransformResult result; + result.eraseAll = true; + result.newGroupNodes.emplace_back(newtopNGroupNode); + return result; +} + +std::string PushTopNDownEdgeIndexRangeScanRule::toString() const { + return "PushTopNDownEdgeIndexRangeScanRule"; +} + +} // namespace opt +} // namespace nebula diff --git a/src/graph/optimizer/rule/PushTopNDownEdgeIndexRangeScanRule.h b/src/graph/optimizer/rule/PushTopNDownEdgeIndexRangeScanRule.h new file mode 100644 index 00000000000..781487d9b4a --- /dev/null +++ b/src/graph/optimizer/rule/PushTopNDownEdgeIndexRangeScanRule.h @@ -0,0 +1,29 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#pragma once + +#include "graph/optimizer/OptRule.h" + +namespace nebula { +namespace opt { + +class PushTopNDownEdgeIndexRangeScanRule final : public OptRule { + public: + const Pattern &pattern() const override; + + StatusOr transform(OptContext *ctx, + const MatchedResult &matched) const override; + + std::string toString() const override; + + private: + PushTopNDownEdgeIndexRangeScanRule(); + + static std::unique_ptr kInstance; +}; + +} // namespace opt +} // namespace nebula diff --git a/src/graph/optimizer/rule/PushTopNDownIndexScanRule.cpp b/src/graph/optimizer/rule/PushTopNDownIndexScanRule.cpp new file mode 100644 index 00000000000..1589917682f --- /dev/null +++ b/src/graph/optimizer/rule/PushTopNDownIndexScanRule.cpp @@ -0,0 +1,114 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "graph/optimizer/rule/PushTopNDownIndexScanRule.h" + +#include "graph/optimizer/OptContext.h" +#include "graph/optimizer/OptGroup.h" +#include "graph/planner/plan/PlanNode.h" +#include "graph/planner/plan/Query.h" + +using nebula::graph::IndexScan; +using nebula::graph::PlanNode; +using nebula::graph::Project; +using nebula::graph::QueryContext; +using nebula::graph::TopN; + +namespace nebula { +namespace opt { + +std::unique_ptr PushTopNDownIndexScanRule::kInstance = + std::unique_ptr(new PushTopNDownIndexScanRule()); + +PushTopNDownIndexScanRule::PushTopNDownIndexScanRule() { + RuleSet::QueryRules().addRule(this); +} + +const Pattern &PushTopNDownIndexScanRule::pattern() const { + static Pattern pattern = + Pattern::create(graph::PlanNode::Kind::kTopN, + {Pattern::create(graph::PlanNode::Kind::kProject, + {Pattern::create(graph::PlanNode::Kind::kIndexScan)})}); + return pattern; +} + +StatusOr PushTopNDownIndexScanRule::transform( + OptContext *octx, const MatchedResult &matched) const { + auto topNGroupNode = matched.node; + auto projectGroupNode = matched.dependencies.front().node; + auto indexScanGroupNode = matched.dependencies.front().dependencies.front().node; + + const auto topN = static_cast(topNGroupNode->node()); + const auto project = static_cast(projectGroupNode->node()); + const auto indexScan = static_cast(indexScanGroupNode->node()); + + int64_t limitRows = topN->offset() + topN->count(); + + auto &factors = topN->factors(); + auto projColNames = project->colNames(); + const auto &yieldColumns = indexScan->yieldColumns(); + + std::unordered_map namesMap; + for (auto yieldColumn : yieldColumns->columns()) { + if (yieldColumn->expr()->kind() == Expression::Kind::kTagProperty) { + const auto &propName = static_cast(yieldColumn->expr())->prop(); + namesMap[yieldColumn->name()] = propName; + continue; + } + if (yieldColumn->expr()->kind() == Expression::Kind::kEdgeProperty) { + const auto &propName = static_cast(yieldColumn->expr())->prop(); + namesMap[yieldColumn->name()] = propName; + continue; + } + return TransformResult::noTransform(); + } + std::vector orderBys; + orderBys.reserve(factors.size()); + + for (auto factor : factors) { + auto colName = projColNames[factor.first]; + auto found = namesMap.find(colName); + if (found == namesMap.end()) { + return Status::Error(); + } + storage::cpp2::OrderBy orderBy; + orderBy.prop_ref() = found->second; + orderBy.direction_ref() = factor.second == OrderFactor::OrderType::ASCEND + ? storage::cpp2::OrderDirection::ASCENDING + : storage::cpp2::OrderDirection::DESCENDING; + orderBys.emplace_back(orderBy); + } + + auto newTopN = static_cast(topN->clone()); + auto newtopNGroupNode = OptGroupNode::create(octx, newTopN, topNGroupNode->group()); + + auto newProject = static_cast(project->clone()); + auto newProjectGroup = OptGroup::create(octx); + auto newProjectGroupNode = newProjectGroup->makeGroupNode(newProject); + + auto newIndexScan = static_cast(indexScan->clone()); + newIndexScan->setLimit(limitRows); + newIndexScan->setOrderBy(orderBys); + auto newIndexScanGroup = OptGroup::create(octx); + auto newIndexScanGroupNode = newIndexScanGroup->makeGroupNode(newIndexScan); + + newtopNGroupNode->dependsOn(newProjectGroup); + newProjectGroupNode->dependsOn(newIndexScanGroup); + for (auto dep : indexScanGroupNode->dependencies()) { + newIndexScanGroupNode->dependsOn(dep); + } + + TransformResult result; + result.eraseAll = true; + result.newGroupNodes.emplace_back(newtopNGroupNode); + return result; +} + +std::string PushTopNDownIndexScanRule::toString() const { + return "PushTopNDownIndexScanRule"; +} + +} // namespace opt +} // namespace nebula diff --git a/src/graph/optimizer/rule/PushTopNDownIndexScanRule.h b/src/graph/optimizer/rule/PushTopNDownIndexScanRule.h new file mode 100644 index 00000000000..7787320592c --- /dev/null +++ b/src/graph/optimizer/rule/PushTopNDownIndexScanRule.h @@ -0,0 +1,29 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#pragma once + +#include "graph/optimizer/OptRule.h" + +namespace nebula { +namespace opt { + +class PushTopNDownIndexScanRule final : public OptRule { + public: + const Pattern &pattern() const override; + + StatusOr transform(OptContext *ctx, + const MatchedResult &matched) const override; + + std::string toString() const override; + + private: + PushTopNDownIndexScanRule(); + + static std::unique_ptr kInstance; +}; + +} // namespace opt +} // namespace nebula diff --git a/src/graph/optimizer/rule/PushTopNDownTagIndexFullScanRule.cpp b/src/graph/optimizer/rule/PushTopNDownTagIndexFullScanRule.cpp new file mode 100644 index 00000000000..46b1eeba520 --- /dev/null +++ b/src/graph/optimizer/rule/PushTopNDownTagIndexFullScanRule.cpp @@ -0,0 +1,110 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "graph/optimizer/rule/PushTopNDownTagIndexFullScanRule.h" + +#include "graph/optimizer/OptContext.h" +#include "graph/optimizer/OptGroup.h" +#include "graph/planner/plan/PlanNode.h" +#include "graph/planner/plan/Query.h" +#include "graph/planner/plan/Scan.h" + +using nebula::graph::PlanNode; +using nebula::graph::Project; +using nebula::graph::QueryContext; +using nebula::graph::TagIndexFullScan; +using nebula::graph::TopN; + +namespace nebula { +namespace opt { + +std::unique_ptr PushTopNDownTagIndexFullScanRule::kInstance = + std::unique_ptr(new PushTopNDownTagIndexFullScanRule()); + +PushTopNDownTagIndexFullScanRule::PushTopNDownTagIndexFullScanRule() { + RuleSet::QueryRules().addRule(this); +} + +const Pattern &PushTopNDownTagIndexFullScanRule::pattern() const { + static Pattern pattern = Pattern::create( + graph::PlanNode::Kind::kTopN, + {Pattern::create(graph::PlanNode::Kind::kProject, + {Pattern::create(graph::PlanNode::Kind::kTagIndexFullScan)})}); + return pattern; +} + +StatusOr PushTopNDownTagIndexFullScanRule::transform( + OptContext *octx, const MatchedResult &matched) const { + auto topNGroupNode = matched.node; + auto projectGroupNode = matched.dependencies.front().node; + auto indexScanGroupNode = matched.dependencies.front().dependencies.front().node; + + const auto topN = static_cast(topNGroupNode->node()); + const auto project = static_cast(projectGroupNode->node()); + const auto indexScan = static_cast(indexScanGroupNode->node()); + + int64_t limitRows = topN->offset() + topN->count(); + + auto &factors = topN->factors(); + auto projColNames = project->colNames(); + const auto &yieldColumns = indexScan->yieldColumns(); + + std::unordered_map namesMap; + for (auto yieldColumn : yieldColumns->columns()) { + if (yieldColumn->expr()->kind() == Expression::Kind::kTagProperty) { + const auto &propName = static_cast(yieldColumn->expr())->prop(); + namesMap[yieldColumn->name()] = propName; + continue; + } + return TransformResult::noTransform(); + } + std::vector orderBys; + orderBys.reserve(factors.size()); + + for (auto factor : factors) { + auto colName = projColNames[factor.first]; + auto found = namesMap.find(colName); + if (found == namesMap.end()) { + return Status::Error(); + } + storage::cpp2::OrderBy orderBy; + orderBy.prop_ref() = found->second; + orderBy.direction_ref() = factor.second == OrderFactor::OrderType::ASCEND + ? storage::cpp2::OrderDirection::ASCENDING + : storage::cpp2::OrderDirection::DESCENDING; + orderBys.emplace_back(orderBy); + } + + auto newTopN = static_cast(topN->clone()); + auto newtopNGroupNode = OptGroupNode::create(octx, newTopN, topNGroupNode->group()); + + auto newProject = static_cast(project->clone()); + auto newProjectGroup = OptGroup::create(octx); + auto newProjectGroupNode = newProjectGroup->makeGroupNode(newProject); + + auto newIndexScan = static_cast(indexScan->clone()); + newIndexScan->setLimit(limitRows); + newIndexScan->setOrderBy(orderBys); + auto newIndexScanGroup = OptGroup::create(octx); + auto newIndexScanGroupNode = newIndexScanGroup->makeGroupNode(newIndexScan); + + newtopNGroupNode->dependsOn(newProjectGroup); + newProjectGroupNode->dependsOn(newIndexScanGroup); + for (auto dep : indexScanGroupNode->dependencies()) { + newIndexScanGroupNode->dependsOn(dep); + } + + TransformResult result; + result.eraseAll = true; + result.newGroupNodes.emplace_back(newtopNGroupNode); + return result; +} + +std::string PushTopNDownTagIndexFullScanRule::toString() const { + return "PushTopNDownTagIndexFullScanRule"; +} + +} // namespace opt +} // namespace nebula diff --git a/src/graph/optimizer/rule/PushTopNDownTagIndexFullScanRule.h b/src/graph/optimizer/rule/PushTopNDownTagIndexFullScanRule.h new file mode 100644 index 00000000000..763b7fbf1f8 --- /dev/null +++ b/src/graph/optimizer/rule/PushTopNDownTagIndexFullScanRule.h @@ -0,0 +1,29 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#pragma once + +#include "graph/optimizer/OptRule.h" + +namespace nebula { +namespace opt { + +class PushTopNDownTagIndexFullScanRule final : public OptRule { + public: + const Pattern &pattern() const override; + + StatusOr transform(OptContext *ctx, + const MatchedResult &matched) const override; + + std::string toString() const override; + + private: + PushTopNDownTagIndexFullScanRule(); + + static std::unique_ptr kInstance; +}; + +} // namespace opt +} // namespace nebula diff --git a/src/graph/optimizer/rule/PushTopNDownTagIndexPrefixScanRule.cpp b/src/graph/optimizer/rule/PushTopNDownTagIndexPrefixScanRule.cpp new file mode 100644 index 00000000000..044ac20b3f1 --- /dev/null +++ b/src/graph/optimizer/rule/PushTopNDownTagIndexPrefixScanRule.cpp @@ -0,0 +1,110 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "graph/optimizer/rule/PushTopNDownTagIndexPrefixScanRule.h" + +#include "graph/optimizer/OptContext.h" +#include "graph/optimizer/OptGroup.h" +#include "graph/planner/plan/PlanNode.h" +#include "graph/planner/plan/Query.h" +#include "graph/planner/plan/Scan.h" + +using nebula::graph::PlanNode; +using nebula::graph::Project; +using nebula::graph::QueryContext; +using nebula::graph::TagIndexPrefixScan; +using nebula::graph::TopN; + +namespace nebula { +namespace opt { + +std::unique_ptr PushTopNDownTagIndexPrefixScanRule::kInstance = + std::unique_ptr(new PushTopNDownTagIndexPrefixScanRule()); + +PushTopNDownTagIndexPrefixScanRule::PushTopNDownTagIndexPrefixScanRule() { + RuleSet::QueryRules().addRule(this); +} + +const Pattern &PushTopNDownTagIndexPrefixScanRule::pattern() const { + static Pattern pattern = Pattern::create( + graph::PlanNode::Kind::kTopN, + {Pattern::create(graph::PlanNode::Kind::kProject, + {Pattern::create(graph::PlanNode::Kind::kTagIndexPrefixScan)})}); + return pattern; +} + +StatusOr PushTopNDownTagIndexPrefixScanRule::transform( + OptContext *octx, const MatchedResult &matched) const { + auto topNGroupNode = matched.node; + auto projectGroupNode = matched.dependencies.front().node; + auto indexScanGroupNode = matched.dependencies.front().dependencies.front().node; + + const auto topN = static_cast(topNGroupNode->node()); + const auto project = static_cast(projectGroupNode->node()); + const auto indexScan = static_cast(indexScanGroupNode->node()); + + int64_t limitRows = topN->offset() + topN->count(); + + auto &factors = topN->factors(); + auto projColNames = project->colNames(); + const auto &yieldColumns = indexScan->yieldColumns(); + + std::unordered_map namesMap; + for (auto yieldColumn : yieldColumns->columns()) { + if (yieldColumn->expr()->kind() == Expression::Kind::kTagProperty) { + const auto &propName = static_cast(yieldColumn->expr())->prop(); + namesMap[yieldColumn->name()] = propName; + continue; + } + return TransformResult::noTransform(); + } + std::vector orderBys; + orderBys.reserve(factors.size()); + + for (auto factor : factors) { + auto colName = projColNames[factor.first]; + auto found = namesMap.find(colName); + if (found == namesMap.end()) { + return Status::Error(); + } + storage::cpp2::OrderBy orderBy; + orderBy.prop_ref() = found->second; + orderBy.direction_ref() = factor.second == OrderFactor::OrderType::ASCEND + ? storage::cpp2::OrderDirection::ASCENDING + : storage::cpp2::OrderDirection::DESCENDING; + orderBys.emplace_back(orderBy); + } + + auto newTopN = static_cast(topN->clone()); + auto newtopNGroupNode = OptGroupNode::create(octx, newTopN, topNGroupNode->group()); + + auto newProject = static_cast(project->clone()); + auto newProjectGroup = OptGroup::create(octx); + auto newProjectGroupNode = newProjectGroup->makeGroupNode(newProject); + + auto newIndexScan = static_cast(indexScan->clone()); + newIndexScan->setLimit(limitRows); + newIndexScan->setOrderBy(orderBys); + auto newIndexScanGroup = OptGroup::create(octx); + auto newIndexScanGroupNode = newIndexScanGroup->makeGroupNode(newIndexScan); + + newtopNGroupNode->dependsOn(newProjectGroup); + newProjectGroupNode->dependsOn(newIndexScanGroup); + for (auto dep : indexScanGroupNode->dependencies()) { + newIndexScanGroupNode->dependsOn(dep); + } + + TransformResult result; + result.eraseAll = true; + result.newGroupNodes.emplace_back(newtopNGroupNode); + return result; +} + +std::string PushTopNDownTagIndexPrefixScanRule::toString() const { + return "PushTopNDownTagIndexPrefixScanRule"; +} + +} // namespace opt +} // namespace nebula diff --git a/src/graph/optimizer/rule/PushTopNDownTagIndexPrefixScanRule.h b/src/graph/optimizer/rule/PushTopNDownTagIndexPrefixScanRule.h new file mode 100644 index 00000000000..b449504cd8e --- /dev/null +++ b/src/graph/optimizer/rule/PushTopNDownTagIndexPrefixScanRule.h @@ -0,0 +1,29 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#pragma once + +#include "graph/optimizer/OptRule.h" + +namespace nebula { +namespace opt { + +class PushTopNDownTagIndexPrefixScanRule final : public OptRule { + public: + const Pattern &pattern() const override; + + StatusOr transform(OptContext *ctx, + const MatchedResult &matched) const override; + + std::string toString() const override; + + private: + PushTopNDownTagIndexPrefixScanRule(); + + static std::unique_ptr kInstance; +}; + +} // namespace opt +} // namespace nebula diff --git a/src/graph/optimizer/rule/PushTopNDownTagIndexRangeScanRule.cpp b/src/graph/optimizer/rule/PushTopNDownTagIndexRangeScanRule.cpp new file mode 100644 index 00000000000..fb9733dbce7 --- /dev/null +++ b/src/graph/optimizer/rule/PushTopNDownTagIndexRangeScanRule.cpp @@ -0,0 +1,112 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "graph/optimizer/rule/PushTopNDownTagIndexRangeScanRule.h" + +#include "common/expression/AttributeExpression.h" +#include "graph/optimizer/OptContext.h" +#include "graph/optimizer/OptGroup.h" +#include "graph/planner/plan/PlanNode.h" +#include "graph/planner/plan/Query.h" +#include "graph/planner/plan/Scan.h" + +using nebula::graph::PlanNode; +using nebula::graph::Project; +using nebula::graph::QueryContext; +using nebula::graph::TagIndexRangeScan; +using nebula::graph::TopN; + +namespace nebula { +namespace opt { + +std::unique_ptr PushTopNDownTagIndexRangeScanRule::kInstance = + std::unique_ptr(new PushTopNDownTagIndexRangeScanRule()); + +PushTopNDownTagIndexRangeScanRule::PushTopNDownTagIndexRangeScanRule() { + RuleSet::QueryRules().addRule(this); +} + +const Pattern &PushTopNDownTagIndexRangeScanRule::pattern() const { + static Pattern pattern = Pattern::create( + graph::PlanNode::Kind::kTopN, + {Pattern::create(graph::PlanNode::Kind::kProject, + {Pattern::create(graph::PlanNode::Kind::kTagIndexRangeScan)})}); + return pattern; +} + +StatusOr PushTopNDownTagIndexRangeScanRule::transform( + OptContext *octx, const MatchedResult &matched) const { + auto topNGroupNode = matched.node; + auto projectGroupNode = matched.dependencies.front().node; + auto indexScanGroupNode = matched.dependencies.front().dependencies.front().node; + + const auto topN = static_cast(topNGroupNode->node()); + const auto project = static_cast(projectGroupNode->node()); + const auto indexScan = static_cast(indexScanGroupNode->node()); + + int64_t limitRows = topN->offset() + topN->count(); + + auto &factors = topN->factors(); + auto projColNames = project->colNames(); + const auto *yieldColumns = indexScan->yieldColumns(); + + std::unordered_map namesMap; + for (auto yieldColumn : yieldColumns->columns()) { + LOG(INFO) << "yieldColumn->expr()->kind()=" << yieldColumn->expr()->kind(); + if (yieldColumn->expr()->kind() == Expression::Kind::kTagProperty) { + const auto &propName = static_cast(yieldColumn->expr())->prop(); + namesMap[yieldColumn->name()] = propName; + continue; + } + return TransformResult::noTransform(); + } + std::vector orderBys; + orderBys.reserve(factors.size()); + + for (auto factor : factors) { + auto colName = projColNames[factor.first]; + auto found = namesMap.find(colName); + if (found == namesMap.end()) { + return Status::Error(); + } + storage::cpp2::OrderBy orderBy; + orderBy.prop_ref() = found->second; + orderBy.direction_ref() = factor.second == OrderFactor::OrderType::ASCEND + ? storage::cpp2::OrderDirection::ASCENDING + : storage::cpp2::OrderDirection::DESCENDING; + orderBys.emplace_back(orderBy); + } + + auto newTopN = static_cast(topN->clone()); + auto newtopNGroupNode = OptGroupNode::create(octx, newTopN, topNGroupNode->group()); + + auto newProject = static_cast(project->clone()); + auto newProjectGroup = OptGroup::create(octx); + auto newProjectGroupNode = newProjectGroup->makeGroupNode(newProject); + + auto newIndexScan = static_cast(indexScan->clone()); + newIndexScan->setLimit(limitRows); + newIndexScan->setOrderBy(orderBys); + auto newIndexScanGroup = OptGroup::create(octx); + auto newIndexScanGroupNode = newIndexScanGroup->makeGroupNode(newIndexScan); + + newtopNGroupNode->dependsOn(newProjectGroup); + newProjectGroupNode->dependsOn(newIndexScanGroup); + for (auto dep : indexScanGroupNode->dependencies()) { + newIndexScanGroupNode->dependsOn(dep); + } + + TransformResult result; + result.eraseAll = true; + result.newGroupNodes.emplace_back(newtopNGroupNode); + return result; +} + +std::string PushTopNDownTagIndexRangeScanRule::toString() const { + return "PushTopNDownTagIndexRangeScanRule"; +} + +} // namespace opt +} // namespace nebula diff --git a/src/graph/optimizer/rule/PushTopNDownTagIndexRangeScanRule.h b/src/graph/optimizer/rule/PushTopNDownTagIndexRangeScanRule.h new file mode 100644 index 00000000000..a5512fffafb --- /dev/null +++ b/src/graph/optimizer/rule/PushTopNDownTagIndexRangeScanRule.h @@ -0,0 +1,29 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#pragma once + +#include "graph/optimizer/OptRule.h" + +namespace nebula { +namespace opt { + +class PushTopNDownTagIndexRangeScanRule final : public OptRule { + public: + const Pattern &pattern() const override; + + StatusOr transform(OptContext *ctx, + const MatchedResult &matched) const override; + + std::string toString() const override; + + private: + PushTopNDownTagIndexRangeScanRule(); + + static std::unique_ptr kInstance; +}; + +} // namespace opt +} // namespace nebula diff --git a/src/graph/planner/ngql/LookupPlanner.cpp b/src/graph/planner/ngql/LookupPlanner.cpp index 45297335137..6bba68baed2 100644 --- a/src/graph/planner/ngql/LookupPlanner.cpp +++ b/src/graph/planner/ngql/LookupPlanner.cpp @@ -37,6 +37,7 @@ StatusOr LookupPlanner::transform(AstContext* astCtx) { lookupCtx->idxReturnCols, lookupCtx->schemaId, lookupCtx->isEmptyResultSet); + edgeIndexFullScan->setYieldColumns(lookupCtx->yieldExpr); plan.tail = edgeIndexFullScan; plan.root = edgeIndexFullScan; } else { @@ -48,6 +49,7 @@ StatusOr LookupPlanner::transform(AstContext* astCtx) { lookupCtx->idxReturnCols, lookupCtx->schemaId, lookupCtx->isEmptyResultSet); + tagIndexFullScan->setYieldColumns(lookupCtx->yieldExpr); plan.tail = tagIndexFullScan; plan.root = tagIndexFullScan; } diff --git a/src/graph/planner/plan/Admin.h b/src/graph/planner/plan/Admin.h index fd1acf3682d..1496d6a2ae5 100644 --- a/src/graph/planner/plan/Admin.h +++ b/src/graph/planner/plan/Admin.h @@ -92,7 +92,7 @@ class DropHosts final : public SingleDependencyNode { }; class ShowHosts final : public SingleDependencyNode { - // TODO(shylock) meta/storage/graph enumerate + // TODO(shylock) meta/storage/graph/agent enumerate public: static ShowHosts* make(QueryContext* qctx, PlanNode* dep, meta::cpp2::ListHostType type) { return qctx->objPool()->add(new ShowHosts(qctx, dep, type)); diff --git a/src/graph/planner/plan/Query.cpp b/src/graph/planner/plan/Query.cpp index deb1f89981c..5e133e82aac 100644 --- a/src/graph/planner/plan/Query.cpp +++ b/src/graph/planner/plan/Query.cpp @@ -194,6 +194,7 @@ void IndexScan::cloneMembers(const IndexScan& g) { isEdge_ = g.isEdge(); schemaId_ = g.schemaId(); isEmptyResultSet_ = g.isEmptyResultSet(); + yieldColumns_ = g.yieldColumns(); } std::unique_ptr ScanVertices::explain() const { diff --git a/src/graph/planner/plan/Query.h b/src/graph/planner/plan/Query.h index cd9a86fd683..48cdce7f335 100644 --- a/src/graph/planner/plan/Query.h +++ b/src/graph/planner/plan/Query.h @@ -507,6 +507,10 @@ class IndexScan : public Explore { return isEmptyResultSet_; } + YieldColumns* yieldColumns() const { + return yieldColumns_; + } + void setEmptyResultSet(bool isEmptyResultSet) { isEmptyResultSet_ = isEmptyResultSet; } @@ -523,6 +527,10 @@ class IndexScan : public Explore { isEdge_ = isEdge; } + void setYieldColumns(YieldColumns* yieldColumns) { + yieldColumns_ = yieldColumns; + } + PlanNode* clone() const override; std::unique_ptr explain() const override; @@ -558,6 +566,7 @@ class IndexScan : public Explore { // TODO(yee): Generate special plan for this scenario bool isEmptyResultSet_{false}; + YieldColumns* yieldColumns_; }; /** @@ -971,8 +980,8 @@ class TopN final : public SingleInputNode { static TopN* make(QueryContext* qctx, PlanNode* input, std::vector> factors = {}, - int64_t offset = -1, - int64_t count = -1) { + int64_t offset = 0, + int64_t count = 0) { return qctx->objPool()->add(new TopN(qctx, input, std::move(factors), offset, count)); } diff --git a/src/graph/scheduler/AsyncMsgNotifyBasedScheduler.cpp b/src/graph/scheduler/AsyncMsgNotifyBasedScheduler.cpp index a85b242a610..b2e1b878384 100644 --- a/src/graph/scheduler/AsyncMsgNotifyBasedScheduler.cpp +++ b/src/graph/scheduler/AsyncMsgNotifyBasedScheduler.cpp @@ -9,18 +9,20 @@ DECLARE_bool(enable_lifetime_optimize); namespace nebula { namespace graph { + AsyncMsgNotifyBasedScheduler::AsyncMsgNotifyBasedScheduler(QueryContext* qctx) : Scheduler() { qctx_ = qctx; } folly::Future AsyncMsgNotifyBasedScheduler::schedule() { + auto root = qctx_->plan()->root(); if (FLAGS_enable_lifetime_optimize) { // special for root - qctx_->plan()->root()->outputVarPtr()->userCount.store(std::numeric_limits::max(), - std::memory_order_relaxed); - analyzeLifetime(qctx_->plan()->root()); + root->outputVarPtr()->userCount.store(std::numeric_limits::max(), + std::memory_order_relaxed); + analyzeLifetime(root); } - auto executor = Executor::create(qctx_->plan()->root(), qctx_); + auto executor = Executor::create(root, qctx_); return doSchedule(executor); } @@ -78,178 +80,113 @@ folly::Future AsyncMsgNotifyBasedScheduler::doSchedule(Executor* root) c DCHECK(currentPromisesFound != promiseMap.end()); auto currentExePromises = std::move(currentPromisesFound->second); - scheduleExecutor(std::move(currentExeFutures), exe, runner, std::move(currentExePromises)); + scheduleExecutor(std::move(currentExeFutures), exe, runner) + .thenTry([this, pros = std::move(currentExePromises)](auto&& t) mutable { + if (t.hasException()) { + notifyError(pros, Status::Error(std::move(t).exception().what())); + } else { + auto v = std::move(t).value(); + if (v.ok()) { + notifyOK(pros); + } else { + notifyError(pros, v); + } + } + }); } return resultFuture; } -void AsyncMsgNotifyBasedScheduler::scheduleExecutor( - std::vector>&& futures, - Executor* exe, - folly::Executor* runner, - std::vector>&& promises) const { +folly::Future AsyncMsgNotifyBasedScheduler::scheduleExecutor( + std::vector>&& futures, Executor* exe, folly::Executor* runner) const { switch (exe->node()->kind()) { case PlanNode::Kind::kSelect: { auto select = static_cast(exe); - runSelect(std::move(futures), select, runner, std::move(promises)); - break; + return runSelect(std::move(futures), select, runner); } case PlanNode::Kind::kLoop: { auto loop = static_cast(exe); - runLoop(std::move(futures), loop, runner, std::move(promises)); - break; + return runLoop(std::move(futures), loop, runner); } case PlanNode::Kind::kArgument: { - runExecutor(std::move(futures), exe, runner, std::move(promises)); - break; + return runExecutor(std::move(futures), exe, runner); } default: { if (exe->depends().empty()) { - runLeafExecutor(exe, runner, std::move(promises)); + return runLeafExecutor(exe, runner); } else { - runExecutor(std::move(futures), exe, runner, std::move(promises)); + return runExecutor(std::move(futures), exe, runner); } - break; } } } -void AsyncMsgNotifyBasedScheduler::runSelect(std::vector>&& futures, - SelectExecutor* select, - folly::Executor* runner, - std::vector>&& promises) const { - folly::collect(futures).via(runner).thenTry([select, pros = std::move(promises), this]( - auto&& t) mutable { - if (t.hasException()) { - return notifyError(pros, Status::Error(t.exception().what())); - } - auto status = std::move(t).value(); - auto s = checkStatus(std::move(status)); - if (!s.ok()) { - return notifyError(pros, s); - } - - std::move(execute(select)) - .thenTry([select, pros = std::move(pros), this](auto&& selectTry) mutable { - if (selectTry.hasException()) { - return notifyError(pros, Status::Error(selectTry.exception().what())); - } - auto selectStatus = std::move(selectTry).value(); - if (!selectStatus.ok()) { - return notifyError(pros, selectStatus); - } - auto val = qctx_->ectx()->getValue(select->node()->outputVar()); - if (!val.isBool()) { - std::stringstream ss; - ss << "Loop produces a bad condition result: " << val << " type: " << val.type(); - return notifyError(pros, Status::Error(ss.str())); - } - - auto selectFuture = folly::makeFuture(Status::OK()); - if (val.getBool()) { - selectFuture = doSchedule(select->thenBody()); - } else { - selectFuture = doSchedule(select->elseBody()); - } - std::move(selectFuture).thenTry([pros = std::move(pros), this](auto&& bodyTry) mutable { - if (bodyTry.hasException()) { - return notifyError(pros, Status::Error(bodyTry.exception().what())); - } - auto bodyStatus = std::move(bodyTry).value(); - if (!bodyStatus.ok()) { - return notifyError(pros, bodyStatus); - } else { - return notifyOK(pros); - } - }); - }); - }); -} - -void AsyncMsgNotifyBasedScheduler::runExecutor( +folly::Future AsyncMsgNotifyBasedScheduler::runSelect( std::vector>&& futures, - Executor* exe, - folly::Executor* runner, - std::vector>&& promises) const { - folly::collect(futures).via(runner).thenTry( - [exe, pros = std::move(promises), this](auto&& t) mutable { - if (t.hasException()) { - return notifyError(pros, Status::Error(t.exception().what())); + SelectExecutor* select, + folly::Executor* runner) const { + return folly::collect(futures) + .via(runner) + .thenValue([select, this](auto&& t) mutable -> folly::Future { + NG_RETURN_IF_ERROR(checkStatus(std::move(t))); + return execute(select); + }) + .thenValue([select, this](auto&& selectStatus) mutable -> folly::Future { + NG_RETURN_IF_ERROR(selectStatus); + auto val = qctx_->ectx()->getValue(select->node()->outputVar()); + if (!val.isBool()) { + std::stringstream ss; + ss << "Loop produces a bad condition result: " << val << " type: " << val.type(); + return Status::Error(ss.str()); } - auto status = std::move(t).value(); - auto depStatus = checkStatus(std::move(status)); - if (!depStatus.ok()) { - return notifyError(pros, depStatus); + + if (val.getBool()) { + return doSchedule(select->thenBody()); } + return doSchedule(select->elseBody()); + }); +} + +folly::Future AsyncMsgNotifyBasedScheduler::runExecutor( + std::vector>&& futures, Executor* exe, folly::Executor* runner) const { + return folly::collect(futures).via(runner).thenValue( + [exe, this](auto&& t) mutable -> folly::Future { + NG_RETURN_IF_ERROR(checkStatus(std::move(t))); // Execute in current thread. - std::move(execute(exe)).thenTry([pros = std::move(pros), this](auto&& exeTry) mutable { - if (exeTry.hasException()) { - return notifyError(pros, Status::Error(exeTry.exception().what())); - } - auto exeStatus = std::move(exeTry).value(); - if (!exeStatus.ok()) { - return notifyError(pros, exeStatus); - } - return notifyOK(pros); - }); + return execute(exe); }); } -void AsyncMsgNotifyBasedScheduler::runLeafExecutor( - Executor* exe, folly::Executor* runner, std::vector>&& promises) const { - std::move(execute(exe)).via(runner).thenTry([pros = std::move(promises), this](auto&& t) mutable { - if (t.hasException()) { - return notifyError(pros, Status::Error(t.exception().what())); - } - auto s = std::move(t).value(); - if (!s.ok()) { - return notifyError(pros, s); - } - return notifyOK(pros); - }); +folly::Future AsyncMsgNotifyBasedScheduler::runLeafExecutor(Executor* exe, + folly::Executor* runner) const { + return std::move(execute(exe)).via(runner); } -void AsyncMsgNotifyBasedScheduler::runLoop(std::vector>&& futures, - LoopExecutor* loop, - folly::Executor* runner, - std::vector>&& promises) const { - folly::collect(futures).via(runner).thenTry( - [loop, runner, pros = std::move(promises), this](auto&& t) mutable { - if (t.hasException()) { - return notifyError(pros, Status::Error(t.exception().what())); +folly::Future AsyncMsgNotifyBasedScheduler::runLoop( + std::vector>&& futures, + LoopExecutor* loop, + folly::Executor* runner) const { + return folly::collect(futures) + .via(runner) + .thenValue([loop, this](auto&& t) mutable -> folly::Future { + NG_RETURN_IF_ERROR(checkStatus(std::move(t))); + return execute(loop); + }) + .thenValue([loop, runner, this](auto&& loopStatus) mutable -> folly::Future { + NG_RETURN_IF_ERROR(loopStatus); + auto val = qctx_->ectx()->getValue(loop->node()->outputVar()); + if (!val.isBool()) { + std::stringstream ss; + ss << "Loop produces a bad condition result: " << val << " type: " << val.type(); + return Status::Error(ss.str()); } - auto status = std::move(t).value(); - auto s = checkStatus(std::move(status)); - if (!s.ok()) { - return notifyError(pros, s); + if (!val.getBool()) { + return Status::OK(); } - - std::move(execute(loop)) - .thenTry([loop, runner, pros = std::move(pros), this](auto&& loopTry) mutable { - if (loopTry.hasException()) { - return notifyError(pros, Status::Error(loopTry.exception().what())); - } - auto loopStatus = std::move(loopTry).value(); - if (!loopStatus.ok()) { - return notifyError(pros, loopStatus); - } - auto val = qctx_->ectx()->getValue(loop->node()->outputVar()); - if (!val.isBool()) { - std::stringstream ss; - ss << "Loop produces a bad condition result: " << val << " type: " << val.type(); - return notifyError(pros, Status::Error(ss.str())); - } - if (val.getBool()) { - auto loopBody = loop->loopBody(); - auto scheduleFuture = doSchedule(loopBody); - std::vector> fs; - fs.emplace_back(std::move(scheduleFuture)); - runLoop(std::move(fs), loop, runner, std::move(pros)); - } else { - return notifyOK(pros); - } - }); + std::vector> fs; + fs.emplace_back(doSchedule(loop->loopBody())); + return runLoop(std::move(fs), loop, runner); }); } diff --git a/src/graph/scheduler/AsyncMsgNotifyBasedScheduler.h b/src/graph/scheduler/AsyncMsgNotifyBasedScheduler.h index 4610a5cd15e..5698d2fcce3 100644 --- a/src/graph/scheduler/AsyncMsgNotifyBasedScheduler.h +++ b/src/graph/scheduler/AsyncMsgNotifyBasedScheduler.h @@ -31,34 +31,28 @@ class AsyncMsgNotifyBasedScheduler final : public Scheduler { folly::Future doSchedule(Executor* root) const; /** - * futures: current executor will be triggered when all the futures are + * futures: current executor will be triggered when all the futures are * notified. exe: current executor runner: a thread-pool promises: the * promises will be set a value which triggers the other executors if current * executor is done working. */ - void scheduleExecutor(std::vector>&& futures, - Executor* exe, - folly::Executor* runner, - std::vector>&& promises) const; + folly::Future scheduleExecutor(std::vector>&& futures, + Executor* exe, + folly::Executor* runner) const; - void runSelect(std::vector>&& futures, - SelectExecutor* select, - folly::Executor* runner, - std::vector>&& promises) const; + folly::Future runSelect(std::vector>&& futures, + SelectExecutor* select, + folly::Executor* runner) const; - void runExecutor(std::vector>&& futures, - Executor* exe, - folly::Executor* runner, - std::vector>&& promises) const; + folly::Future runExecutor(std::vector>&& futures, + Executor* exe, + folly::Executor* runner) const; - void runLeafExecutor(Executor* exe, - folly::Executor* runner, - std::vector>&& promises) const; + folly::Future runLeafExecutor(Executor* exe, folly::Executor* runner) const; - void runLoop(std::vector>&& futures, - LoopExecutor* loop, - folly::Executor* runner, - std::vector>&& promises) const; + folly::Future runLoop(std::vector>&& futures, + LoopExecutor* loop, + folly::Executor* runner) const; Status checkStatus(std::vector&& status) const; diff --git a/src/graph/service/Authenticator.h b/src/graph/service/Authenticator.h index f7355275783..dd53b7ba177 100644 --- a/src/graph/service/Authenticator.h +++ b/src/graph/service/Authenticator.h @@ -7,6 +7,7 @@ #define GRAPH_SERVICE_AUTHENTICATOR_H_ #include "common/base/Base.h" +#include "common/base/Status.h" namespace nebula { namespace graph { @@ -15,7 +16,7 @@ class Authenticator { public: virtual ~Authenticator() {} - virtual bool NG_MUST_USE_RESULT auth(const std::string &user, const std::string &password) = 0; + virtual Status NG_MUST_USE_RESULT auth(const std::string &user, const std::string &password) = 0; }; } // namespace graph diff --git a/src/graph/service/CloudAuthenticator.cpp b/src/graph/service/CloudAuthenticator.cpp index 0729446fe07..930e7b0c911 100644 --- a/src/graph/service/CloudAuthenticator.cpp +++ b/src/graph/service/CloudAuthenticator.cpp @@ -5,6 +5,7 @@ #include "graph/service/CloudAuthenticator.h" +#include "common/base/Status.h" #include "common/encryption/Base64.h" #include "common/http/HttpClient.h" #include "graph/service/GraphFlags.h" @@ -16,13 +17,13 @@ CloudAuthenticator::CloudAuthenticator(meta::MetaClient* client) { metaClient_ = client; } -bool CloudAuthenticator::auth(const std::string& user, const std::string& password) { +Status CloudAuthenticator::auth(const std::string& user, const std::string& password) { // The shadow account on the nebula side has been created // Normal passwords and tokens use different prefixes // First, go to meta to check if the shadow account exists if (!metaClient_->checkShadowAccountFromCache(user)) { - return false; + return Status::Error("Shadow account not exist"); } // Second, use user + password authentication methods @@ -35,20 +36,20 @@ bool CloudAuthenticator::auth(const std::string& user, const std::string& passwo if (!result.ok()) { LOG(ERROR) << result.status(); - return false; + return result.status(); } try { auto json = folly::parseJson(result.value()); if (json["code"].asString().compare("0") != 0) { LOG(ERROR) << "Cloud authentication failed, user: " << user; - return false; + return Status::Error("Cloud authentication failed, user: %s", user.c_str()); } } catch (std::exception& e) { LOG(ERROR) << "Invalid json: " << e.what(); - return false; + return Status::Error("Invalid json: %s", e.what()); } - return true; + return Status::OK(); } } // namespace graph diff --git a/src/graph/service/CloudAuthenticator.h b/src/graph/service/CloudAuthenticator.h index 04e37b5ebde..35d229c7a8b 100644 --- a/src/graph/service/CloudAuthenticator.h +++ b/src/graph/service/CloudAuthenticator.h @@ -17,7 +17,7 @@ class CloudAuthenticator final : public Authenticator { public: explicit CloudAuthenticator(meta::MetaClient* client); - bool auth(const std::string& user, const std::string& password) override; + Status auth(const std::string& user, const std::string& password) override; private: meta::MetaClient* metaClient_; diff --git a/src/graph/service/GraphFlags.h b/src/graph/service/GraphFlags.h index 54777cdd003..fe3ecaec3ba 100644 --- a/src/graph/service/GraphFlags.h +++ b/src/graph/service/GraphFlags.h @@ -37,6 +37,14 @@ DECLARE_string(auth_type); DECLARE_string(cloud_http_url); DECLARE_uint32(max_allowed_statements); +// Failed login attempt +// value of failed_login_attempts is in the range from 0 to 32767. +// The deault value is 0. A value of 0 disables the option. +DECLARE_uint32(failed_login_attempts); +// value of password_lock_time_in_secs is in the range from 0 to 32767[secs]. +// The deault value is 0. A value of 0 disables the option. +DECLARE_uint32(password_lock_time_in_secs); + // optimizer DECLARE_bool(enable_optimizer); diff --git a/src/graph/service/GraphServer.cpp b/src/graph/service/GraphServer.cpp index 25e6ee6cc5e..c9254d35295 100644 --- a/src/graph/service/GraphServer.cpp +++ b/src/graph/service/GraphServer.cpp @@ -7,6 +7,7 @@ #include #include +#include "common/id/Snowflake.h" #include "graph/service/GraphFlags.h" #include "graph/service/GraphService.h" namespace nebula { @@ -39,6 +40,8 @@ bool GraphServer::start() { return false; } + nebula::Snowflake::initWorkerId(interface->metaClient_.get()); + graphThread_ = std::make_unique([&] { thriftServer_->setPort(localHost_.port); thriftServer_->setInterface(std::move(interface)); diff --git a/src/graph/service/GraphService.cpp b/src/graph/service/GraphService.cpp index 7229b548ddb..a607ffe7039 100644 --- a/src/graph/service/GraphService.cpp +++ b/src/graph/service/GraphService.cpp @@ -69,9 +69,10 @@ folly::Future GraphService::future_authenticate(const std::string& auto ctx = std::make_unique>(); auto future = ctx->future(); // check username and password failed - if (!auth(username, password)) { + auto authResult = auth(username, password); + if (!authResult.ok()) { ctx->resp().errorCode = ErrorCode::E_BAD_USERNAME_PASSWORD; - ctx->resp().errorMsg.reset(new std::string("Bad username/password")); + ctx->resp().errorMsg.reset(new std::string(authResult.toString())); ctx->finish(); stats::StatsManager::addValue(kNumAuthFailedSessions); stats::StatsManager::addValue(kNumAuthFailedSessionsBadUserNamePassword); @@ -200,9 +201,9 @@ folly::Future GraphService::future_executeJsonWithParameter( }); } -bool GraphService::auth(const std::string& username, const std::string& password) { +Status GraphService::auth(const std::string& username, const std::string& password) { if (!FLAGS_enable_authorize) { - return true; + return Status::OK(); } if (FLAGS_auth_type == "password") { @@ -214,14 +215,12 @@ bool GraphService::auth(const std::string& username, const std::string& password // There is no way to identify which one is in the graph layer, // let's check the native user's password first, then cloud user. auto pwdAuth = std::make_unique(queryEngine_->metaClient()); - if (pwdAuth->auth(username, encryption::MD5Utils::md5Encode(password))) { - return true; - } + return pwdAuth->auth(username, encryption::MD5Utils::md5Encode(password)); auto cloudAuth = std::make_unique(queryEngine_->metaClient()); return cloudAuth->auth(username, password); } LOG(WARNING) << "Unknown auth type: " << FLAGS_auth_type; - return false; + return Status::Error("Unknown auth type: %s", FLAGS_auth_type.c_str()); } folly::Future GraphService::future_verifyClientVersion( diff --git a/src/graph/service/GraphService.h b/src/graph/service/GraphService.h index bcb38993325..85ed21a6ce4 100644 --- a/src/graph/service/GraphService.h +++ b/src/graph/service/GraphService.h @@ -51,12 +51,13 @@ class GraphService final : public cpp2::GraphServiceSvIf { folly::Future future_verifyClientVersion( const cpp2::VerifyClientVersionReq& req) override; + std::unique_ptr metaClient_; + private: - bool auth(const std::string& username, const std::string& password); + Status auth(const std::string& username, const std::string& password); std::unique_ptr sessionManager_; std::unique_ptr queryEngine_; - std::unique_ptr metaClient_; }; } // namespace graph diff --git a/src/graph/service/PasswordAuthenticator.cpp b/src/graph/service/PasswordAuthenticator.cpp index 2d23587d7fc..94888aebdcf 100644 --- a/src/graph/service/PasswordAuthenticator.cpp +++ b/src/graph/service/PasswordAuthenticator.cpp @@ -12,7 +12,7 @@ PasswordAuthenticator::PasswordAuthenticator(meta::MetaClient* client) { metaClient_ = client; } -bool PasswordAuthenticator::auth(const std::string& user, const std::string& password) { +Status PasswordAuthenticator::auth(const std::string& user, const std::string& password) { return metaClient_->authCheckFromCache(user, password); } diff --git a/src/graph/service/PasswordAuthenticator.h b/src/graph/service/PasswordAuthenticator.h index 3a8bbf5abd8..7bb3e792251 100644 --- a/src/graph/service/PasswordAuthenticator.h +++ b/src/graph/service/PasswordAuthenticator.h @@ -16,7 +16,7 @@ class PasswordAuthenticator final : public Authenticator { public: explicit PasswordAuthenticator(meta::MetaClient* client); - bool auth(const std::string& user, const std::string& password) override; + Status auth(const std::string& user, const std::string& password) override; private: meta::MetaClient* metaClient_; diff --git a/src/interface/common.thrift b/src/interface/common.thrift index e9895ebb26a..a81ceaa91bf 100644 --- a/src/interface/common.thrift +++ b/src/interface/common.thrift @@ -403,7 +403,7 @@ enum ErrorCode { // ListClusterInfo Failure E_LIST_CLUSTER_FAILURE = -2070, E_LIST_CLUSTER_GET_ABS_PATH_FAILURE = -2071, - E_GET_META_DIR_FAILURE = -2072, + E_LIST_CLUSTER_NO_AGENT_FAILURE = -2072, E_QUERY_NOT_FOUND = -2073, E_AGENT_HB_FAILUE = -2074, @@ -470,6 +470,8 @@ enum ErrorCode { E_WRITE_WRITE_CONFLICT = -3073, E_CLIENT_SERVER_INCOMPATIBLE = -3061, + // get worker id + E_WORKER_ID_FAILED = -3062, E_UNKNOWN = -8000, } (cpp.enum_strict) diff --git a/src/interface/meta.thrift b/src/interface/meta.thrift index 1ee7bbacb68..b512c94ec11 100644 --- a/src/interface/meta.thrift +++ b/src/interface/meta.thrift @@ -463,6 +463,7 @@ enum ListHostType { GRAPH = 0x01, META = 0x02, STORAGE = 0x03, + AGENT = 0x04, } (cpp.enum_strict) struct ListHostsReq { @@ -506,6 +507,17 @@ struct GetPartsAllocResp { 4: optional map(cpp.template = "std::unordered_map") terms, } +// get workerid for snowflake +struct GetWorkerIdReq { + 1: binary host, +} + +struct GetWorkerIdResp { + 1: common.ErrorCode code, + 2: common.HostAddr leader, + 3: i64 workerid, +} + struct MultiPutReq { // segment is used to avoid conflict with system data. // it should be comprised of numbers and letters. @@ -1206,6 +1218,8 @@ service MetaService { GetPartsAllocResp getPartsAlloc(1: GetPartsAllocReq req); ListPartsResp listParts(1: ListPartsReq req); + GetWorkerIdResp getWorkerId(1: GetWorkerIdReq req); + ExecResp multiPut(1: MultiPutReq req); GetResp get(1: GetReq req); MultiGetResp multiGet(1: MultiGetReq req); diff --git a/src/interface/storage.thrift b/src/interface/storage.thrift index eca97a22203..1ff18fd1cd8 100644 --- a/src/interface/storage.thrift +++ b/src/interface/storage.thrift @@ -545,6 +545,7 @@ struct LookupIndexRequest { 5: optional RequestCommon common, // max row count of each partition in this response 6: optional i64 limit, + 7: optional list order_by, } diff --git a/src/meta/ActiveHostsMan.cpp b/src/meta/ActiveHostsMan.cpp index af94e214051..0097634bfae 100644 --- a/src/meta/ActiveHostsMan.cpp +++ b/src/meta/ActiveHostsMan.cpp @@ -12,6 +12,7 @@ #include "meta/processors/Common.h" DECLARE_int32(heartbeat_interval_secs); +DEFINE_int32(agent_heartbeat_interval_secs, 60, "Agent heartbeat interval in seconds"); DECLARE_uint32(expired_time_factor); namespace nebula { @@ -144,9 +145,12 @@ ErrorOr> ActiveHostsMan::getActiv } std::vector hosts; - int64_t threshold = - (expiredTTL == 0 ? FLAGS_heartbeat_interval_secs * FLAGS_expired_time_factor : expiredTTL) * - 1000; + int64_t expiredTime = + FLAGS_heartbeat_interval_secs * FLAGS_expired_time_factor; // meta/storage/graph + if (role == cpp2::HostRole::AGENT) { + expiredTime = FLAGS_agent_heartbeat_interval_secs * FLAGS_expired_time_factor; + } + int64_t threshold = (expiredTTL == 0 ? expiredTime : expiredTTL) * 1000; auto now = time::WallClock::fastNowInMilliSec(); while (iter->valid()) { auto host = MetaKeyUtils::parseHostKey(iter->key()); diff --git a/src/meta/CMakeLists.txt b/src/meta/CMakeLists.txt index 54873ae9c70..ee0025d35aa 100644 --- a/src/meta/CMakeLists.txt +++ b/src/meta/CMakeLists.txt @@ -27,6 +27,7 @@ nebula_add_library( processors/schema/ListEdgesProcessor.cpp processors/schema/DropEdgeProcessor.cpp processors/schema/SchemaUtil.cpp + processors/id/GetWorkerIdProcessor.cpp processors/index/CreateTagIndexProcessor.cpp processors/index/DropTagIndexProcessor.cpp processors/index/GetTagIndexProcessor.cpp diff --git a/src/meta/MetaServiceHandler.cpp b/src/meta/MetaServiceHandler.cpp index 997d38ed894..1ab14cd1f29 100644 --- a/src/meta/MetaServiceHandler.cpp +++ b/src/meta/MetaServiceHandler.cpp @@ -20,6 +20,7 @@ #include "meta/processors/config/ListConfigsProcessor.h" #include "meta/processors/config/RegConfigProcessor.h" #include "meta/processors/config/SetConfigProcessor.h" +#include "meta/processors/id/GetWorkerIdProcessor.h" #include "meta/processors/index/CreateEdgeIndexProcessor.h" #include "meta/processors/index/CreateTagIndexProcessor.h" #include "meta/processors/index/DropEdgeIndexProcessor.h" @@ -568,5 +569,11 @@ folly::Future MetaServiceHandler::future_verifyCl RETURN_FUTURE(processor); } +folly::Future MetaServiceHandler::future_getWorkerId( + const cpp2::GetWorkerIdReq& req) { + auto* processor = GetWorkerIdProcessor::instance(kvstore_); + RETURN_FUTURE(processor); +} + } // namespace meta } // namespace nebula diff --git a/src/meta/MetaServiceHandler.h b/src/meta/MetaServiceHandler.h index 4991bf731bc..3d9f7c7df20 100644 --- a/src/meta/MetaServiceHandler.h +++ b/src/meta/MetaServiceHandler.h @@ -239,6 +239,8 @@ class MetaServiceHandler final : public cpp2::MetaServiceSvIf { folly::Future future_verifyClientVersion( const cpp2::VerifyClientVersionReq& req) override; + folly::Future future_getWorkerId(const cpp2::GetWorkerIdReq& req) override; + private: kvstore::KVStore* kvstore_ = nullptr; ClusterID clusterId_{0}; diff --git a/src/meta/processors/BaseProcessor-inl.h b/src/meta/processors/BaseProcessor-inl.h index 27dcaea01b3..bfb203d89d6 100644 --- a/src/meta/processors/BaseProcessor-inl.h +++ b/src/meta/processors/BaseProcessor-inl.h @@ -26,9 +26,9 @@ void BaseProcessor::doPut(std::vector data) { template ErrorOr> -BaseProcessor::doPrefix(const std::string& key) { +BaseProcessor::doPrefix(const std::string& key, bool canReadFromFollower) { std::unique_ptr iter; - auto code = kvstore_->prefix(kDefaultSpaceId, kDefaultPartId, key, &iter); + auto code = kvstore_->prefix(kDefaultSpaceId, kDefaultPartId, key, &iter, canReadFromFollower); if (code != nebula::cpp2::ErrorCode::SUCCEEDED) { VLOG(2) << "Prefix Failed"; return code; diff --git a/src/meta/processors/BaseProcessor.h b/src/meta/processors/BaseProcessor.h index 5d8c1521f80..f9def2c631e 100644 --- a/src/meta/processors/BaseProcessor.h +++ b/src/meta/processors/BaseProcessor.h @@ -125,7 +125,7 @@ class BaseProcessor { void doPut(std::vector data); ErrorOr> doPrefix( - const std::string& key); + const std::string& key, bool canReadFromFollower = false); /** * General get function. diff --git a/src/meta/processors/Common.h b/src/meta/processors/Common.h index 353fddacc85..a4a436dcda1 100644 --- a/src/meta/processors/Common.h +++ b/src/meta/processors/Common.h @@ -23,6 +23,7 @@ class LockUtils { GENERATE_LOCK(lastUpdateTime); GENERATE_LOCK(space); GENERATE_LOCK(id); + GENERATE_LOCK(workerId); GENERATE_LOCK(localId); GENERATE_LOCK(tag); GENERATE_LOCK(edge); diff --git a/src/meta/processors/admin/ListClusterInfoProcessor.cpp b/src/meta/processors/admin/ListClusterInfoProcessor.cpp index ce5215e2f98..3ebd993d35f 100644 --- a/src/meta/processors/admin/ListClusterInfoProcessor.cpp +++ b/src/meta/processors/admin/ListClusterInfoProcessor.cpp @@ -97,9 +97,9 @@ void ListClusterInfoProcessor::process(const cpp2::ListClusterInfoReq& req) { agentCount++; } } - if (agentCount != 1) { + if (agentCount < 1) { LOG(ERROR) << folly::sformat("There are {} agent count is host {}", agentCount, host); - handleErrorCode(nebula::cpp2::ErrorCode::E_LIST_CLUSTER_FAILURE); + handleErrorCode(nebula::cpp2::ErrorCode::E_LIST_CLUSTER_NO_AGENT_FAILURE); onFinished(); return; } diff --git a/src/meta/processors/admin/RestoreProcessor.cpp b/src/meta/processors/admin/RestoreProcessor.cpp index ce7748ff9c3..3f3c35e6cab 100644 --- a/src/meta/processors/admin/RestoreProcessor.cpp +++ b/src/meta/processors/admin/RestoreProcessor.cpp @@ -17,7 +17,7 @@ nebula::cpp2::ErrorCode RestoreProcessor::replaceHostInPartition(const HostAddr& folly::SharedMutex::WriteHolder wHolder(LockUtils::spaceLock()); auto retCode = nebula::cpp2::ErrorCode::SUCCEEDED; const auto& spacePrefix = MetaKeyUtils::spacePrefix(); - auto iterRet = doPrefix(spacePrefix); + auto iterRet = doPrefix(spacePrefix, direct); if (!nebula::ok(iterRet)) { retCode = nebula::error(iterRet); LOG(ERROR) << "Space prefix failed, error: " << apache::thrift::util::enumNameSafe(retCode); @@ -37,7 +37,7 @@ nebula::cpp2::ErrorCode RestoreProcessor::replaceHostInPartition(const HostAddr& for (const auto& spaceId : allSpaceId) { const auto& partPrefix = MetaKeyUtils::partPrefix(spaceId); - auto iterPartRet = doPrefix(partPrefix); + auto iterPartRet = doPrefix(partPrefix, direct); if (!nebula::ok(iterPartRet)) { retCode = nebula::error(iterPartRet); LOG(ERROR) << "Part prefix failed, error: " << apache::thrift::util::enumNameSafe(retCode); @@ -87,7 +87,7 @@ nebula::cpp2::ErrorCode RestoreProcessor::replaceHostInZone(const HostAddr& ipv4 folly::SharedMutex::WriteHolder wHolder(LockUtils::spaceLock()); auto retCode = nebula::cpp2::ErrorCode::SUCCEEDED; const auto& zonePrefix = MetaKeyUtils::zonePrefix(); - auto iterRet = doPrefix(zonePrefix); + auto iterRet = doPrefix(zonePrefix, direct); if (!nebula::ok(iterRet)) { retCode = nebula::error(iterRet); LOG(ERROR) << "Zone prefix failed, error: " << apache::thrift::util::enumNameSafe(retCode); @@ -153,6 +153,10 @@ void RestoreProcessor::process(const cpp2::RestoreMetaReq& req) { auto replaceHosts = req.get_hosts(); if (!replaceHosts.empty()) { for (auto h : replaceHosts) { + if (h.get_from_host() == h.get_to_host()) { + continue; + } + auto result = replaceHostInPartition(h.get_from_host(), h.get_to_host(), true); if (result != nebula::cpp2::ErrorCode::SUCCEEDED) { LOG(ERROR) << "replaceHost in partition fails when recovered"; diff --git a/src/meta/processors/id/GetWorkerIdProcessor.cpp b/src/meta/processors/id/GetWorkerIdProcessor.cpp new file mode 100644 index 00000000000..798240ab9a0 --- /dev/null +++ b/src/meta/processors/id/GetWorkerIdProcessor.cpp @@ -0,0 +1,53 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "meta/processors/id/GetWorkerIdProcessor.h" + +namespace nebula { +namespace meta { + +void GetWorkerIdProcessor::process(const cpp2::GetWorkerIdReq& req) { + const string& ipAddr = req.get_host(); + auto result = doGet(ipAddr); + if (nebula::ok(result)) { + int64_t workerId = std::stoi(std::move(nebula::value(result))); + + handleErrorCode(nebula::cpp2::ErrorCode::SUCCEEDED); + resp_.workerid_ref() = workerId; + onFinished(); + return; + } + + folly::SharedMutex::WriteHolder wHolder(LockUtils::workerIdLock()); + auto newResult = doGet(kIdKey); + if (!nebula::ok(newResult)) { + handleErrorCode(nebula::cpp2::ErrorCode::E_WORKER_ID_FAILED); + onFinished(); + return; + } + + int64_t workerId = std::stoi(std::move(nebula::value(newResult))); + // TODO: (jackwener) limit worker, add LOG ERROR + doPut(std::vector{{ipAddr, std::to_string(workerId + 1)}}); + + handleErrorCode(nebula::cpp2::ErrorCode::SUCCEEDED); + resp_.workerid_ref() = workerId; + onFinished(); +} + +void GetWorkerIdProcessor::doPut(std::vector data) { + folly::Baton baton; + kvstore_->asyncMultiPut(kDefaultSpaceId, + kDefaultPartId, + std::move(data), + [this, &baton](nebula::cpp2::ErrorCode code) { + this->handleErrorCode(code); + baton.post(); + }); + baton.wait(); +} + +} // namespace meta +} // namespace nebula diff --git a/src/meta/processors/id/GetWorkerIdProcessor.h b/src/meta/processors/id/GetWorkerIdProcessor.h new file mode 100644 index 00000000000..1b1433d81a2 --- /dev/null +++ b/src/meta/processors/id/GetWorkerIdProcessor.h @@ -0,0 +1,42 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#ifndef META_GETWORKERIDPROCESSOR_H_ +#define META_GETWORKERIDPROCESSOR_H_ + +#include "meta/processors/BaseProcessor.h" + +namespace nebula { +namespace meta { + +class GetWorkerIdProcessor : public BaseProcessor { + public: + static GetWorkerIdProcessor* instance(kvstore::KVStore* kvstore) { + return new GetWorkerIdProcessor(kvstore); + } + + void process(const cpp2::GetWorkerIdReq& req); + + private: + explicit GetWorkerIdProcessor(kvstore::KVStore* kvstore) + : BaseProcessor(kvstore) { + // initialize worker id in kvstore just once + static bool once = [this]() { + std::vector data = {{kIdKey, "0"}}; + doPut(data); + return true; + }(); + UNUSED(once); + } + + void doPut(std::vector data); + + inline static const string kIdKey = "snowflake_worker_id"; +}; + +} // namespace meta +} // namespace nebula + +#endif // META_GETWORKERIDPROCESSOR_H_ diff --git a/src/meta/processors/parts/ListHostsProcessor.cpp b/src/meta/processors/parts/ListHostsProcessor.cpp index 23327a00fab..3495ee62628 100644 --- a/src/meta/processors/parts/ListHostsProcessor.cpp +++ b/src/meta/processors/parts/ListHostsProcessor.cpp @@ -10,6 +10,7 @@ #include "version/Version.h" DECLARE_int32(heartbeat_interval_secs); +DECLARE_int32(agent_heartbeat_interval_secs); DECLARE_uint32(expired_time_factor); DEFINE_int32(removed_threshold_sec, 24 * 60 * 60, @@ -26,6 +27,8 @@ static cpp2::HostRole toHostRole(cpp2::ListHostType type) { return cpp2::HostRole::META; case cpp2::ListHostType::STORAGE: return cpp2::HostRole::STORAGE; + case cpp2::ListHostType::AGENT: + return cpp2::HostRole::AGENT; default: return cpp2::HostRole::UNKNOWN; } @@ -133,10 +136,14 @@ nebula::cpp2::ErrorCode ListHostsProcessor::allHostsWithStatus(cpp2::HostRole ro } if (now - info.lastHBTimeInMilliSec_ < FLAGS_removed_threshold_sec * 1000) { + int64_t expiredTime = + FLAGS_heartbeat_interval_secs * FLAGS_expired_time_factor * 1000; // meta/storage/graph + if (info.role_ == cpp2::HostRole::AGENT) { + expiredTime = FLAGS_agent_heartbeat_interval_secs * FLAGS_expired_time_factor * 1000; + } // If meta didn't receive heartbeat with 2 periods, regard hosts as // offline. Same as ActiveHostsMan::getActiveHosts - if (now - info.lastHBTimeInMilliSec_ < - FLAGS_heartbeat_interval_secs * FLAGS_expired_time_factor * 1000) { + if (now - info.lastHBTimeInMilliSec_ < expiredTime) { item.status_ref() = cpp2::HostStatus::ONLINE; } else { item.status_ref() = cpp2::HostStatus::OFFLINE; diff --git a/src/parser/parser.yy b/src/parser/parser.yy index 21164635d67..582ab4dc95c 100644 --- a/src/parser/parser.yy +++ b/src/parser/parser.yy @@ -27,7 +27,6 @@ #include "common/expression/ListComprehensionExpression.h" #include "common/expression/AggregateExpression.h" #include "common/function/FunctionManager.h" - #include "common/expression/ReduceExpression.h" #include "graph/util/ParserUtil.h" #include "graph/util/ExpressionUtils.h" @@ -177,7 +176,7 @@ static constexpr size_t kCommentLengthLimit = 256; %token KW_DROP KW_REMOVE KW_SPACES KW_INGEST KW_INDEX KW_INDEXES %token KW_IF KW_NOT KW_EXISTS KW_WITH %token KW_BY KW_DOWNLOAD KW_HDFS KW_UUID KW_CONFIGS KW_FORCE -%token KW_GET KW_DECLARE KW_GRAPH KW_META KW_STORAGE +%token KW_GET KW_DECLARE KW_GRAPH KW_META KW_STORAGE KW_AGENT %token KW_TTL KW_TTL_DURATION KW_TTL_COL KW_DATA KW_STOP %token KW_FETCH KW_PROP KW_UPDATE KW_UPSERT KW_WHEN %token KW_ORDER KW_ASC KW_LIMIT KW_SAMPLE KW_OFFSET KW_ASCENDING KW_DESCENDING @@ -487,6 +486,7 @@ unreserved_keyword | KW_GRAPH { $$ = new std::string("graph"); } | KW_META { $$ = new std::string("meta"); } | KW_STORAGE { $$ = new std::string("storage"); } + | KW_AGENT { $$ = new std::string("agent"); } | KW_ALL { $$ = new std::string("all"); } | KW_ANY { $$ = new std::string("any"); } | KW_SINGLE { $$ = new std::string("single"); } @@ -698,6 +698,9 @@ expression_internal | reduce_expression { $$ = $1; } + | uuid_expression { + $$ = $1; + } ; constant_expression @@ -1109,9 +1112,8 @@ function_call_expression ; uuid_expression - : KW_UUID L_PAREN STRING R_PAREN { - $$ = UUIDExpression::make(qctx->objPool(), *$3); - delete $3; + : KW_UUID L_PAREN R_PAREN { + $$ = UUIDExpression::make(qctx->objPool()); } ; @@ -3455,6 +3457,7 @@ list_host_type : KW_GRAPH { $$ = meta::cpp2::ListHostType::GRAPH; } | KW_META { $$ = meta::cpp2::ListHostType::META; } | KW_STORAGE { $$ = meta::cpp2::ListHostType::STORAGE; } + | KW_AGENT { $$ = meta::cpp2::ListHostType::AGENT; } ; config_module_enum diff --git a/src/parser/scanner.lex b/src/parser/scanner.lex index 451a6a63f8a..af7b2ad9f58 100644 --- a/src/parser/scanner.lex +++ b/src/parser/scanner.lex @@ -220,6 +220,7 @@ LABEL_FULL_WIDTH {CN_EN_FULL_WIDTH}{CN_EN_NUM_FULL_WIDTH}* "TTL_COL" { return TokenType::KW_TTL_COL; } "GRAPH" { return TokenType::KW_GRAPH; } "META" { return TokenType::KW_META; } +"AGENT" { return TokenType::KW_AGENT; } "STORAGE" { return TokenType::KW_STORAGE; } "SHORTEST" { return TokenType::KW_SHORTEST; } "NOLOOP" { return TokenType::KW_NOLOOP; } diff --git a/src/parser/test/ParserTest.cpp b/src/parser/test/ParserTest.cpp index a40726b4525..89199e87359 100644 --- a/src/parser/test/ParserTest.cpp +++ b/src/parser/test/ParserTest.cpp @@ -1291,7 +1291,7 @@ TEST_F(ParserTest, FetchVertex) { ASSERT_TRUE(result.ok()) << result.status(); } { - std::string query = "FETCH PROP ON person uuid(\"Tom\")"; + std::string query = "FETCH PROP ON person uuid()"; auto result = parse(query); ASSERT_TRUE(result.ok()) << result.status(); } @@ -1744,7 +1744,7 @@ TEST_F(ParserTest, UnreservedKeywords) { ASSERT_TRUE(result.ok()) << result.status(); } { - std::string query = "GO FROM UUID(\"tom\") OVER guest WHERE $-.EMAIL"; + std::string query = "GO FROM UUID() OVER guest WHERE $-.EMAIL"; auto result = parse(query); ASSERT_TRUE(result.ok()) << result.status(); } @@ -1758,7 +1758,7 @@ TEST_F(ParserTest, UnreservedKeywords) { } { std::string query = - "GO FROM UUID(\"tom\") OVER like YIELD $$.tag1.EMAIL, like.users," + "GO FROM UUID() OVER like YIELD $$.tag1.EMAIL, like.users," "like._src, like._dst, like.type, $^.tag2.SPACE " "| ORDER BY $-.SPACE"; auto result = parse(query); @@ -1770,7 +1770,7 @@ TEST_F(ParserTest, UnreservedKeywords) { ASSERT_TRUE(result.ok()) << result.status(); } { - std::string query = "$var = GO FROM UUID(\"tom\") OVER like;GO FROM $var.SPACE OVER like"; + std::string query = "$var = GO FROM UUID() OVER like;GO FROM $var.SPACE OVER like"; auto result = parse(query); ASSERT_TRUE(result.ok()) << result.status(); } diff --git a/src/parser/test/ScannerTest.cpp b/src/parser/test/ScannerTest.cpp index 16b8764cce2..830e9c3bb69 100644 --- a/src/parser/test/ScannerTest.cpp +++ b/src/parser/test/ScannerTest.cpp @@ -431,6 +431,9 @@ TEST(Scanner, Basic) { CHECK_SEMANTIC_TYPE("META", TokenType::KW_META), CHECK_SEMANTIC_TYPE("Meta", TokenType::KW_META), CHECK_SEMANTIC_TYPE("meta", TokenType::KW_META), + CHECK_SEMANTIC_TYPE("AGENT", TokenType::KW_AGENT), + CHECK_SEMANTIC_TYPE("Agent", TokenType::KW_AGENT), + CHECK_SEMANTIC_TYPE("agent", TokenType::KW_AGENT), CHECK_SEMANTIC_TYPE("STORAGE", TokenType::KW_STORAGE), CHECK_SEMANTIC_TYPE("Storage", TokenType::KW_STORAGE), CHECK_SEMANTIC_TYPE("storage", TokenType::KW_STORAGE), diff --git a/src/storage/CMakeLists.txt b/src/storage/CMakeLists.txt index ddf106cae08..6b1d568b5cc 100644 --- a/src/storage/CMakeLists.txt +++ b/src/storage/CMakeLists.txt @@ -48,6 +48,7 @@ nebula_add_library( exec/IndexScanNode.cpp exec/IndexSelectionNode.cpp exec/IndexVertexScanNode.cpp + exec/IndexTopNNode.cpp kv/PutProcessor.cpp kv/GetProcessor.cpp kv/RemoveProcessor.cpp diff --git a/src/storage/exec/IndexLimitNode.h b/src/storage/exec/IndexLimitNode.h index 8811ec77a87..576d77123bd 100644 --- a/src/storage/exec/IndexLimitNode.h +++ b/src/storage/exec/IndexLimitNode.h @@ -15,10 +15,12 @@ class IndexLimitNode : public IndexNode { std::unique_ptr copy() override; std::string identify() override; + protected: + const uint64_t offset_, limit_; + private: nebula::cpp2::ErrorCode doExecute(PartitionID partId) override; Result doNext() override; - const uint64_t offset_, limit_; uint64_t currentOffset_ = 0; }; } // namespace storage diff --git a/src/storage/exec/IndexTopNNode.cpp b/src/storage/exec/IndexTopNNode.cpp new file mode 100644 index 00000000000..32cd507baed --- /dev/null +++ b/src/storage/exec/IndexTopNNode.cpp @@ -0,0 +1,129 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ +#include "storage/exec/IndexTopNNode.h" +namespace nebula { +namespace storage { +IndexTopNNode::IndexTopNNode(const IndexTopNNode& node) + : IndexLimitNode(node), orderBy_(node.orderBy_) { + name_ = "IndexTopNNode"; +} + +IndexTopNNode::IndexTopNNode(RuntimeContext* context, + uint64_t offset, + uint64_t limit, + const std::vector* orderBy) + : IndexLimitNode(context, offset, limit), orderBy_(orderBy) { + name_ = "IndexTopNNode"; +} +IndexTopNNode::IndexTopNNode(RuntimeContext* context, + uint64_t limit, + const std::vector* orderBy) + : IndexTopNNode(context, 0, limit, orderBy) {} +nebula::cpp2::ErrorCode IndexTopNNode::init(InitContext& ctx) { + DCHECK_EQ(children_.size(), 1); + for (auto iter = orderBy_->begin(); iter != orderBy_->end(); iter++) { + ctx.requiredColumns.insert(iter->get_prop()); + } + auto ret = children_[0]->init(ctx); + if (UNLIKELY(ret != ::nebula::cpp2::ErrorCode::SUCCEEDED)) { + return ret; + } + for (auto iter = orderBy_->begin(); iter != orderBy_->end(); iter++) { + auto pos = ctx.retColMap.find(iter->get_prop()); + DCHECK(pos != ctx.retColMap.end()); + colPos_[iter->get_prop()] = pos->second; + } + return ::nebula::cpp2::ErrorCode::SUCCEEDED; +} +nebula::cpp2::ErrorCode IndexTopNNode::doExecute(PartitionID partId) { + finished_ = false; + return children_[0]->execute(partId); +} + +IndexNode::Result IndexTopNNode::doNext() { + DCHECK_EQ(children_.size(), 1); + if (!finished_) { + topN(); + } + if (results_.empty()) { + return Result(); + } + auto result = results_.front(); + results_.pop_front(); + return result; +} + +void IndexTopNNode::topN() { + if (offset_ + limit_ <= 0) { + results_.emplace_back(Result()); + finished_ = true; + return; + } + auto comparator = [this](Row& lhs, Row& rhs) -> bool { + for (auto iter = orderBy_->begin(); iter != orderBy_->end(); iter++) { + auto prop = iter->get_prop(); + auto orderType = iter->get_direction(); + auto lValue = lhs[colPos_[prop]]; + auto rValue = rhs[colPos_[prop]]; + if (lValue == rValue) { + continue; + } + if (orderType == cpp2::OrderDirection::ASCENDING) { + return lValue < rValue; + } else if (orderType == cpp2::OrderDirection::DESCENDING) { + return lValue > rValue; + } + } + return false; + }; + + TopNHeap topNHeap; + topNHeap.setHeapSize(offset_ + limit_); + topNHeap.setComparator(comparator); + + auto& child = *children_[0]; + do { + auto result = child.next(); + if (!result.success()) { + results_.emplace_back(result); + finished_ = true; + return; + } + if (result.hasData()) { + topNHeap.push(std::move(result).row()); + } else { + break; + } + } while (true); + auto topNData = topNHeap.moveTopK(); + for (auto iter = topNData.begin(); iter != topNData.end(); iter++) { + results_.emplace_back(std::move(*iter)); + } + finished_ = true; +} + +std::unique_ptr IndexTopNNode::copy() { + return std::make_unique(*this); +} + +std::string IndexTopNNode::identify() { + std::stringstream ss; + for (auto iter = orderBy_->begin(); iter != orderBy_->end(); iter++) { + auto orderType = iter->get_direction(); + if (orderType == cpp2::OrderDirection::ASCENDING) { + ss << iter->get_prop() << " asc,"; + } else { + ss << iter->get_prop() << " desc,"; + } + } + if (offset_ > 0) { + return fmt::format("{}({} offset={}, limit={})", name_, ss.str(), offset_, limit_); + } else { + return fmt::format("{}({} limit={})", name_, ss.str(), limit_); + } +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/exec/IndexTopNNode.h b/src/storage/exec/IndexTopNNode.h new file mode 100644 index 00000000000..76180995ef1 --- /dev/null +++ b/src/storage/exec/IndexTopNNode.h @@ -0,0 +1,98 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ +#pragma once +#include "folly/Likely.h" +#include "storage/exec/IndexLimitNode.h" +namespace nebula { +namespace storage { +template +class TopNHeap final { + public: + ~TopNHeap() = default; + + void setHeapSize(uint64_t heapSize) { + heapSize_ = heapSize; + v_.reserve(heapSize); + } + + void setComparator(std::function comparator) { + comparator_ = comparator; + } + + void push(T&& data) { + if (v_.size() < heapSize_) { + v_.push_back(std::move(data)); + adjustUp(v_.size() - 1); + return; + } + if (comparator_(data, v_[0])) { + v_[0] = std::move(data); + adjustDown(0); + } + } + + std::vector moveTopK() { + return std::move(v_); + } + + private: + void adjustDown(size_t parent) { + size_t child = parent * 2 + 1; + size_t size = v_.size(); + while (child < size) { + if (child + 1 < size && comparator_(v_[child], v_[child + 1])) { + child += 1; + } + if (!comparator_(v_[parent], v_[child])) { + return; + } + std::swap(v_[parent], v_[child]); + parent = child; + child = parent * 2 + 1; + } + } + void adjustUp(size_t child) { + size_t parent = (child - 1) >> 1; + while (0 != child) { + if (!comparator_(v_[parent], v_[child])) { + return; + } + std::swap(v_[parent], v_[child]); + child = parent; + parent = (child - 1) >> 1; + } + } + + private: + std::function comparator_; + uint64_t heapSize_; + std::vector v_; +}; + +class IndexTopNNode : public IndexLimitNode { + public: + IndexTopNNode(const IndexTopNNode& node); + IndexTopNNode(RuntimeContext* context, + uint64_t offset, + uint64_t limit, + const std::vector* orderBy); + IndexTopNNode(RuntimeContext* context, uint64_t limit, const std::vector* orderBy); + nebula::cpp2::ErrorCode init(InitContext& ctx) override; + std::unique_ptr copy() override; + std::string identify() override; + + private: + nebula::cpp2::ErrorCode doExecute(PartitionID partId) override; + Result doNext() override; + void topN(); + const std::vector* orderBy_; + std::deque results_; + bool finished_{false}; + std::vector requiredColumns_; + Map colPos_; +}; +} // namespace storage + +} // namespace nebula diff --git a/src/storage/index/LookupProcessor.cpp b/src/storage/index/LookupProcessor.cpp index 272bfead534..21a5441d57d 100644 --- a/src/storage/index/LookupProcessor.cpp +++ b/src/storage/index/LookupProcessor.cpp @@ -17,6 +17,7 @@ #include "storage/exec/IndexNode.h" #include "storage/exec/IndexProjectionNode.h" #include "storage/exec/IndexSelectionNode.h" +#include "storage/exec/IndexTopNNode.h" #include "storage/exec/IndexVertexScanNode.h" namespace nebula { namespace storage { @@ -127,9 +128,15 @@ std::unique_ptr LookupProcessor::buildPlan(const cpp2::LookupIndexReq } if (req.limit_ref().has_value()) { auto limit = *req.get_limit(); - auto node = std::make_unique(context_.get(), limit); - node->addChild(std::move(nodes[0])); - nodes[0] = std::move(node); + if (req.order_by_ref().has_value() && req.get_order_by()->size() > 0) { + auto node = std::make_unique(context_.get(), limit, req.get_order_by()); + node->addChild(std::move(nodes[0])); + nodes[0] = std::move(node); + } else { + auto node = std::make_unique(context_.get(), limit); + node->addChild(std::move(nodes[0])); + nodes[0] = std::move(node); + } } return std::move(nodes[0]); } diff --git a/src/storage/test/CMakeLists.txt b/src/storage/test/CMakeLists.txt index 65b89befe2a..44604cfbac2 100644 --- a/src/storage/test/CMakeLists.txt +++ b/src/storage/test/CMakeLists.txt @@ -651,6 +651,26 @@ nebula_add_test( gtest ) +nebula_add_test( + NAME + index_scan_topN_test + SOURCES + IndexScanTopNTest.cpp + OBJECTS + $ + $ + $ + $ + $ + ${storage_test_deps} + LIBRARIES + ${ROCKSDB_LIBRARIES} + ${THRIFT_LIBRARIES} + ${PROXYGEN_LIBRARIES} + wangle + gtest +) + nebula_add_executable( NAME es_bulk_example @@ -741,21 +761,6 @@ nebula_add_executable( gtest ) -nebula_add_test( - NAME - chain_delete_edge_test - SOURCES - ChainDeleteEdgesTest.cpp - OBJECTS - ${storage_test_deps} - LIBRARIES - ${ROCKSDB_LIBRARIES} - ${THRIFT_LIBRARIES} - ${PROXYGEN_LIBRARIES} - wangle - gtest -) - nebula_add_executable( NAME storage_index_write_bm diff --git a/src/storage/test/GetNeighborsTest.cpp b/src/storage/test/GetNeighborsTest.cpp index cd9c5eb6b9c..c3b560732aa 100644 --- a/src/storage/test/GetNeighborsTest.cpp +++ b/src/storage/test/GetNeighborsTest.cpp @@ -1119,8 +1119,8 @@ TEST(GetNeighborsTest, TtlTest) { LOG(INFO) << "colName: " << s; } ASSERT_EQ("Tim Duncan", (*resp.vertices_ref()).rows[0].values[0].getStr()); - ASSERT_TRUE((*resp.vertices_ref()).rows[0].values[1].empty()); // stat - ASSERT_TRUE((*resp.vertices_ref()).rows[0].values[2].empty()); // expr + ASSERT_TRUE((*resp.vertices_ref()).rows[0].values[1].empty()); // stat + ASSERT_TRUE((*resp.vertices_ref()).rows[0].values[2].empty()); // expr } FLAGS_mock_ttl_col = false; } diff --git a/src/storage/test/IndexScanTopNTest.cpp b/src/storage/test/IndexScanTopNTest.cpp new file mode 100644 index 00000000000..25325fd933b --- /dev/null +++ b/src/storage/test/IndexScanTopNTest.cpp @@ -0,0 +1,565 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include +#include + +#include "codec/RowReader.h" +#include "common/base/Base.h" +#include "common/fs/TempDir.h" +#include "common/utils/NebulaKeyUtils.h" +#include "mock/AdHocIndexManager.h" +#include "mock/AdHocSchemaManager.h" +#include "mock/MockCluster.h" +#include "storage/index/LookupProcessor.h" +#include "storage/query/GetNeighborsProcessor.h" +#include "storage/test/TestUtils.h" + +namespace nebula { +namespace storage { +ObjectPool objPool; +auto pool = &objPool; + +class IndexScanTopNTest : public ::testing::Test { + protected: + GraphSpaceID spaceId = 1; + TagID tagId = 100; + EdgeType edgeType = 200; + IndexID tagIndex = 101; + IndexID edgeIndex = 201; + size_t vertexLen = 32; + const std::vector parts{1, 2, 3, 4, 5, 6}; + + private: + std::unique_ptr initKV(kvstore::KVOptions options) { + HostAddr localHost; + auto ioPool = std::make_shared(4); + auto workers = apache::thrift::concurrency::PriorityThreadManager::newPriorityThreadManager(1); + workers->setNamePrefix("executor"); + workers->start(); + localHost.host = "0.0.0.0"; + localHost.port = network::NetworkUtils::getAvailablePort(); + auto store = + std::make_unique(std::move(options), ioPool, localHost, workers); + store->init(); + return store; + } + + std::unique_ptr memPartMan() { + auto memPartMan = std::make_unique(); + // GraphSpaceID => {PartitionIDs} + auto& partsMap = memPartMan->partsMap(); + for (auto partId : parts) { + partsMap[spaceId][partId] = meta::PartHosts(); + } + return memPartMan; + } + + std::shared_ptr mockSchema() { + std::shared_ptr schema(new meta::NebulaSchemaProvider(0)); + schema->addField("col1", nebula::cpp2::PropertyType::INT64); + schema->addField("col2", nebula::cpp2::PropertyType::STRING); + return schema; + } + + std::unique_ptr memSchemaMan() { + auto schemaMan = std::make_unique(6); + schemaMan->addTagSchema(spaceId, tagId, mockSchema()); + schemaMan->addEdgeSchema(spaceId, edgeType, mockSchema()); + return schemaMan; + } + + std::vector genCols() { + std::vector cols; + meta::cpp2::ColumnDef col; + col.name = "col1"; + col.type.type_ref() = nebula::cpp2::PropertyType::INT64; + cols.emplace_back(std::move(col)); + return cols; + } + + std::unique_ptr memIndexMan(bool isEmpty = false) { + auto indexMan = std::make_unique(); + if (isEmpty) { + return indexMan; + } + indexMan->addTagIndex(spaceId, tagId, tagIndex, genCols()); + indexMan->addEdgeIndex(spaceId, edgeType, edgeIndex, genCols()); + return indexMan; + } + + protected: + bool mockData() { + auto tag = schemaMan_->getTagSchema(spaceId, tagId); + if (!tag) { + LOG(INFO) << "Space " << spaceId << ", Tag " << tagId << " invalid"; + return false; + } + + for (auto pId : parts) { + std::vector data; + for (int64_t vid = pId * 10; vid < (pId + 1) * 10; vid++) { + int64_t col1Val = vid; + // Edge and vertex have the same schema of structure, so it's good to only generate it once. + RowWriterV2 writer(tag.get()); + EXPECT_EQ(WriteResult::SUCCEEDED, writer.setValue(0, vid)); + EXPECT_EQ(WriteResult::SUCCEEDED, writer.setValue(1, "row_" + folly::to(vid))); + EXPECT_EQ(WriteResult::SUCCEEDED, writer.finish()); + auto val = std::move(writer).moveEncodedStr(); + + auto vertex = folly::to(vid); + auto edgeKey = NebulaKeyUtils::edgeKey(vertexLen, pId, vertex, edgeType, 0, vertex); + auto tagKey = NebulaKeyUtils::tagKey(vertexLen, pId, vertex, tagId); + data.emplace_back(std::move(edgeKey), val); + data.emplace_back(std::move(tagKey), std::move(val)); + if (indexMan_ != nullptr) { + auto indexItem = std::make_unique(); + indexItem->fields_ref() = genCols(); + if (indexMan_->getTagIndex(spaceId, tagIndex).ok()) { + auto vertexIndexKeys = IndexKeyUtils::vertexIndexKeys( + vertexLen, + pId, + tagIndex, + vertex, + IndexKeyUtils::encodeValues({col1Val}, indexItem.get())); + for (auto& vertexIndexKey : vertexIndexKeys) { + data.emplace_back(std::move(vertexIndexKey), ""); + } + } + if (indexMan_->getEdgeIndex(spaceId, edgeIndex).ok()) { + auto edgeIndexKeys = IndexKeyUtils::edgeIndexKeys( + vertexLen, + pId, + edgeIndex, + vertex, + 0, + vertex, + IndexKeyUtils::encodeValues({col1Val}, indexItem.get())); + for (auto& edgeIndexKey : edgeIndexKeys) { + data.emplace_back(std::move(edgeIndexKey), ""); + } + } + } + } + folly::Baton baton; + storageKV_->asyncMultiPut(spaceId, pId, std::move(data), [&](nebula::cpp2::ErrorCode code) { + EXPECT_EQ(code, nebula::cpp2::ErrorCode::SUCCEEDED); + baton.post(); + }); + baton.wait(); + } + return true; + } + + void SetUp() override { + rootPath_ = new fs::TempDir("/tmp/IndexScanTopNTest.XXXXXX"); + kvstore::KVOptions options; + schemaMan_ = memSchemaMan(); + indexMan_ = memIndexMan(); + options.partMan_ = memPartMan(); + options.schemaMan_ = schemaMan_.get(); + + std::vector paths; + paths.emplace_back(folly::stringPrintf("%s/disk1", rootPath_->path())); + + options.dataPaths_ = std::move(paths); + storageKV_ = initKV(std::move(options)); + mock::MockCluster::waitUntilAllElected(storageKV_.get(), spaceId, parts); + + storageEnv_ = std::make_unique(); + storageEnv_->schemaMan_ = schemaMan_.get(); + storageEnv_->indexMan_ = indexMan_.get(); + storageEnv_->kvstore_ = storageKV_.get(); + storageEnv_->rebuildIndexGuard_ = std::make_unique(); + storageEnv_->verticesML_ = std::make_unique(); + storageEnv_->edgesML_ = std::make_unique(); + EXPECT_TRUE(mockData()); + } + + void TearDown() override { + delete rootPath_; + } + + protected: + fs::TempDir* rootPath_; + std::unique_ptr metaClient_; + std::unique_ptr schemaMan_{nullptr}; + std::unique_ptr indexMan_{nullptr}; + std::unique_ptr storageKV_{nullptr}; + std::unique_ptr storageEnv_{nullptr}; +}; + +void verifyResult(const nebula::DataSet& expect, const nebula::DataSet& dataSet) { + ASSERT_EQ(expect.rows.size(), dataSet.rows.size()); + for (size_t i = 0; i < expect.rows.size(); i++) { + const auto& expectRow = expect.rows[i]; + const auto& actualRow = dataSet.rows[i]; + ASSERT_EQ(expectRow, actualRow); + } +} + +TEST_F(IndexScanTopNTest, LookupTagIndexTopN) { + cpp2::LookupIndexRequest req; + nebula::storage::cpp2::IndexSpec indices; + req.space_id_ref() = spaceId; + nebula::cpp2::SchemaID schemaId; + schemaId.tag_id_ref() = tagId; + indices.schema_id_ref() = schemaId; + req.parts_ref() = parts; + std::vector returnCols; + returnCols.emplace_back(kVid); + returnCols.emplace_back("col1"); + returnCols.emplace_back("col2"); + req.return_columns_ref() = std::move(returnCols); + cpp2::IndexQueryContext context1; + context1.index_id_ref() = tagIndex; + decltype(indices.contexts) contexts; + contexts.emplace_back(std::move(context1)); + indices.contexts_ref() = std::move(contexts); + req.indices_ref() = std::move(indices); + + // verify all data + { + auto* processor = LookupProcessor::instance(storageEnv_.get(), nullptr, nullptr); + auto fut = processor->getFuture(); + processor->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + EXPECT_EQ(60, resp.get_data()->rows.size()); + } + + // limit == 0 + { + nebula::storage::cpp2::OrderBy orderBy; + orderBy.prop_ref() = "col1"; + orderBy.direction_ref() = storage::cpp2::OrderDirection::ASCENDING; + std::vector orderBys; + orderBys.emplace_back(std::move(orderBy)); + req.order_by_ref() = orderBys; + req.limit_ref() = 0; + auto* processor = LookupProcessor::instance(storageEnv_.get(), nullptr, nullptr); + auto fut = processor->getFuture(); + processor->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + EXPECT_EQ(0, resp.get_data()->rows.size()); + } + + // order by col1 asc limit == 1 + { + nebula::storage::cpp2::OrderBy orderBy; + orderBy.prop_ref() = "col1"; + orderBy.direction_ref() = storage::cpp2::OrderDirection::ASCENDING; + std::vector orderBys; + orderBys.emplace_back(std::move(orderBy)); + req.order_by_ref() = orderBys; + req.limit_ref() = 1; + auto* processor = LookupProcessor::instance(storageEnv_.get(), nullptr, nullptr); + auto fut = processor->getFuture(); + processor->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + EXPECT_EQ(1 * parts.size(), resp.get_data()->rows.size()); + + nebula::DataSet expected; + expected.colNames = {kVid, "col1", "col2"}; + expected.rows.emplace_back(nebula::Row({"10", 10, "row_10"})); + expected.rows.emplace_back(nebula::Row({"20", 20, "row_20"})); + expected.rows.emplace_back(nebula::Row({"30", 30, "row_30"})); + expected.rows.emplace_back(nebula::Row({"40", 40, "row_40"})); + expected.rows.emplace_back(nebula::Row({"50", 50, "row_50"})); + expected.rows.emplace_back(nebula::Row({"60", 60, "row_60"})); + verifyResult(expected, *resp.get_data()); + } + + // limit 3 by each part through IndexScanNode->DataNode + { + nebula::storage::cpp2::OrderBy orderBy; + orderBy.prop_ref() = "col1"; + orderBy.direction_ref() = storage::cpp2::OrderDirection::ASCENDING; + std::vector orderBys; + orderBys.emplace_back(std::move(orderBy)); + req.order_by_ref() = orderBys; + req.limit_ref() = 3; + cpp2::IndexColumnHint columnHint; + columnHint.begin_value_ref() = Value(15); + columnHint.end_value_ref() = Value(64); + columnHint.column_name_ref() = "col1"; + columnHint.scan_type_ref() = cpp2::ScanType::RANGE; + std::vector columnHints; + columnHints.emplace_back(std::move(columnHint)); + req.indices_ref().value().contexts_ref().value().begin()->column_hints_ref() = + std::move(columnHints); + auto* processor = LookupProcessor::instance(storageEnv_.get(), nullptr, nullptr); + auto fut = processor->getFuture(); + processor->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + EXPECT_EQ(3 * parts.size(), resp.get_data()->rows.size()); + + nebula::DataSet expected; + expected.colNames = {kVid, "col1", "col2"}; + expected.rows.emplace_back(nebula::Row({"17", 17, "row_17"})); + expected.rows.emplace_back(nebula::Row({"15", 15, "row_15"})); + expected.rows.emplace_back(nebula::Row({"16", 16, "row_16"})); + expected.rows.emplace_back(nebula::Row({"22", 22, "row_22"})); + expected.rows.emplace_back(nebula::Row({"20", 20, "row_20"})); + expected.rows.emplace_back(nebula::Row({"21", 21, "row_21"})); + expected.rows.emplace_back(nebula::Row({"32", 32, "row_32"})); + expected.rows.emplace_back(nebula::Row({"30", 30, "row_30"})); + expected.rows.emplace_back(nebula::Row({"31", 31, "row_31"})); + expected.rows.emplace_back(nebula::Row({"42", 42, "row_42"})); + expected.rows.emplace_back(nebula::Row({"40", 40, "row_40"})); + expected.rows.emplace_back(nebula::Row({"41", 41, "row_41"})); + expected.rows.emplace_back(nebula::Row({"52", 52, "row_52"})); + expected.rows.emplace_back(nebula::Row({"50", 50, "row_50"})); + expected.rows.emplace_back(nebula::Row({"51", 51, "row_51"})); + expected.rows.emplace_back(nebula::Row({"62", 62, "row_62"})); + expected.rows.emplace_back(nebula::Row({"60", 60, "row_60"})); + expected.rows.emplace_back(nebula::Row({"61", 61, "row_61"})); + verifyResult(expected, *resp.get_data()); + } + + // limit 3 by each part through IndexScanNode->DataNode->FilterNode + { + nebula::storage::cpp2::OrderBy orderBy; + orderBy.prop_ref() = "col1"; + orderBy.direction_ref() = storage::cpp2::OrderDirection::ASCENDING; + std::vector orderBys; + orderBys.emplace_back(std::move(orderBy)); + req.order_by_ref() = orderBys; + req.limit_ref() = 3; + cpp2::IndexColumnHint columnHint; + columnHint.begin_value_ref() = Value(15); + columnHint.end_value_ref() = Value(64); + columnHint.column_name_ref() = "col1"; + columnHint.scan_type_ref() = cpp2::ScanType::RANGE; + std::vector columnHints; + columnHints.emplace_back(std::move(columnHint)); + auto expr = RelationalExpression::makeEQ( + pool, + ArithmeticExpression::makeMod(pool, + TagPropertyExpression::make(pool, "100", "col1"), + ConstantExpression::make(pool, Value(2))), + + ConstantExpression::make(pool, Value(0))); + req.indices_ref().value().contexts_ref().value().begin()->filter_ref() = expr->encode(); + req.indices_ref().value().contexts_ref().value().begin()->column_hints_ref() = + std::move(columnHints); + auto* processor = LookupProcessor::instance(storageEnv_.get(), nullptr, nullptr); + auto fut = processor->getFuture(); + processor->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + EXPECT_EQ(3 * parts.size() - 2, resp.get_data()->rows.size()); + + nebula::DataSet expected; + expected.colNames = {kVid, "col1", "col2"}; + expected.rows.emplace_back(nebula::Row({"18", 18, "row_18"})); + expected.rows.emplace_back(nebula::Row({"16", 16, "row_16"})); + expected.rows.emplace_back(nebula::Row({"24", 24, "row_24"})); + expected.rows.emplace_back(nebula::Row({"20", 20, "row_20"})); + expected.rows.emplace_back(nebula::Row({"22", 22, "row_22"})); + expected.rows.emplace_back(nebula::Row({"34", 34, "row_34"})); + expected.rows.emplace_back(nebula::Row({"30", 30, "row_30"})); + expected.rows.emplace_back(nebula::Row({"32", 32, "row_32"})); + expected.rows.emplace_back(nebula::Row({"44", 44, "row_44"})); + expected.rows.emplace_back(nebula::Row({"40", 40, "row_40"})); + expected.rows.emplace_back(nebula::Row({"42", 42, "row_42"})); + expected.rows.emplace_back(nebula::Row({"54", 54, "row_54"})); + expected.rows.emplace_back(nebula::Row({"50", 50, "row_50"})); + expected.rows.emplace_back(nebula::Row({"52", 52, "row_52"})); + expected.rows.emplace_back(nebula::Row({"62", 62, "row_62"})); + expected.rows.emplace_back(nebula::Row({"60", 60, "row_60"})); + verifyResult(expected, *resp.get_data()); + } +} + +TEST_F(IndexScanTopNTest, LookupEdgeIndexTopN) { + cpp2::LookupIndexRequest req; + nebula::storage::cpp2::IndexSpec indices; + req.space_id_ref() = spaceId; + nebula::cpp2::SchemaID schemaId; + schemaId.edge_type_ref() = edgeType; + indices.schema_id_ref() = schemaId; + req.parts_ref() = parts; + std::vector returnCols; + returnCols.emplace_back(kSrc); + returnCols.emplace_back("col1"); + returnCols.emplace_back("col2"); + req.return_columns_ref() = std::move(returnCols); + cpp2::IndexQueryContext context1; + context1.index_id_ref() = edgeIndex; + decltype(indices.contexts) contexts; + contexts.emplace_back(std::move(context1)); + indices.contexts_ref() = std::move(contexts); + req.indices_ref() = std::move(indices); + + // verify all data + { + auto* processor = LookupProcessor::instance(storageEnv_.get(), nullptr, nullptr); + auto fut = processor->getFuture(); + processor->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + EXPECT_EQ(60, resp.get_data()->rows.size()); + } + + // limit == 0 + { + nebula::storage::cpp2::OrderBy orderBy; + orderBy.prop_ref() = "col1"; + orderBy.direction_ref() = storage::cpp2::OrderDirection::ASCENDING; + std::vector orderBys; + orderBys.emplace_back(std::move(orderBy)); + req.order_by_ref() = orderBys; + req.limit_ref() = 0; + auto* processor = LookupProcessor::instance(storageEnv_.get(), nullptr, nullptr); + auto fut = processor->getFuture(); + processor->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + EXPECT_EQ(0, resp.get_data()->rows.size()); + } + + // order by col1 asc limit == 1 + { + nebula::storage::cpp2::OrderBy orderBy; + orderBy.prop_ref() = "col1"; + orderBy.direction_ref() = storage::cpp2::OrderDirection::ASCENDING; + std::vector orderBys; + orderBys.emplace_back(std::move(orderBy)); + req.order_by_ref() = orderBys; + req.limit_ref() = 1; + auto* processor = LookupProcessor::instance(storageEnv_.get(), nullptr, nullptr); + auto fut = processor->getFuture(); + processor->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + EXPECT_EQ(1 * parts.size(), resp.get_data()->rows.size()); + + nebula::DataSet expected; + expected.colNames = {kSrc, "col1", "col2"}; + expected.rows.emplace_back(nebula::Row({"10", 10, "row_10"})); + expected.rows.emplace_back(nebula::Row({"20", 20, "row_20"})); + expected.rows.emplace_back(nebula::Row({"30", 30, "row_30"})); + expected.rows.emplace_back(nebula::Row({"40", 40, "row_40"})); + expected.rows.emplace_back(nebula::Row({"50", 50, "row_50"})); + expected.rows.emplace_back(nebula::Row({"60", 60, "row_60"})); + verifyResult(expected, *resp.get_data()); + } + + // limit 3 by each part through IndexScanNode->DataNode + { + nebula::storage::cpp2::OrderBy orderBy; + orderBy.prop_ref() = "col1"; + orderBy.direction_ref() = storage::cpp2::OrderDirection::ASCENDING; + std::vector orderBys; + orderBys.emplace_back(std::move(orderBy)); + req.order_by_ref() = orderBys; + req.limit_ref() = 3; + cpp2::IndexColumnHint columnHint; + columnHint.begin_value_ref() = Value(15); + columnHint.end_value_ref() = Value(64); + columnHint.column_name_ref() = "col1"; + columnHint.scan_type_ref() = cpp2::ScanType::RANGE; + std::vector columnHints; + columnHints.emplace_back(std::move(columnHint)); + req.indices_ref().value().contexts_ref().value().begin()->column_hints_ref() = + std::move(columnHints); + auto* processor = LookupProcessor::instance(storageEnv_.get(), nullptr, nullptr); + auto fut = processor->getFuture(); + processor->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + EXPECT_EQ(3 * parts.size(), resp.get_data()->rows.size()); + + nebula::DataSet expected; + expected.colNames = {kSrc, "col1", "col2"}; + expected.rows.emplace_back(nebula::Row({"17", 17, "row_17"})); + expected.rows.emplace_back(nebula::Row({"15", 15, "row_15"})); + expected.rows.emplace_back(nebula::Row({"16", 16, "row_16"})); + expected.rows.emplace_back(nebula::Row({"22", 22, "row_22"})); + expected.rows.emplace_back(nebula::Row({"20", 20, "row_20"})); + expected.rows.emplace_back(nebula::Row({"21", 21, "row_21"})); + expected.rows.emplace_back(nebula::Row({"32", 32, "row_32"})); + expected.rows.emplace_back(nebula::Row({"30", 30, "row_30"})); + expected.rows.emplace_back(nebula::Row({"31", 31, "row_31"})); + expected.rows.emplace_back(nebula::Row({"42", 42, "row_42"})); + expected.rows.emplace_back(nebula::Row({"40", 40, "row_40"})); + expected.rows.emplace_back(nebula::Row({"41", 41, "row_41"})); + expected.rows.emplace_back(nebula::Row({"52", 52, "row_52"})); + expected.rows.emplace_back(nebula::Row({"50", 50, "row_50"})); + expected.rows.emplace_back(nebula::Row({"51", 51, "row_51"})); + expected.rows.emplace_back(nebula::Row({"62", 62, "row_62"})); + expected.rows.emplace_back(nebula::Row({"60", 60, "row_60"})); + expected.rows.emplace_back(nebula::Row({"61", 61, "row_61"})); + verifyResult(expected, *resp.get_data()); + } + + // limit 3 by each part through IndexScanNode->DataNode->FilterNode + { + nebula::storage::cpp2::OrderBy orderBy; + orderBy.prop_ref() = "col1"; + orderBy.direction_ref() = storage::cpp2::OrderDirection::ASCENDING; + std::vector orderBys; + orderBys.emplace_back(std::move(orderBy)); + req.order_by_ref() = orderBys; + req.limit_ref() = 3; + cpp2::IndexColumnHint columnHint; + columnHint.begin_value_ref() = Value(15); + columnHint.end_value_ref() = Value(64); + columnHint.column_name_ref() = "col1"; + columnHint.scan_type_ref() = cpp2::ScanType::RANGE; + std::vector columnHints; + columnHints.emplace_back(std::move(columnHint)); + auto expr = RelationalExpression::makeEQ( + pool, + ArithmeticExpression::makeMod(pool, + TagPropertyExpression::make(pool, "100", "col1"), + ConstantExpression::make(pool, Value(2))), + + ConstantExpression::make(pool, Value(0))); + req.indices_ref().value().contexts_ref().value().begin()->filter_ref() = expr->encode(); + req.indices_ref().value().contexts_ref().value().begin()->column_hints_ref() = + std::move(columnHints); + auto* processor = LookupProcessor::instance(storageEnv_.get(), nullptr, nullptr); + auto fut = processor->getFuture(); + processor->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + EXPECT_EQ(3 * parts.size() - 2, resp.get_data()->rows.size()); + + nebula::DataSet expected; + expected.colNames = {kSrc, "col1", "col2"}; + expected.rows.emplace_back(nebula::Row({"18", 18, "row_18"})); + expected.rows.emplace_back(nebula::Row({"16", 16, "row_16"})); + expected.rows.emplace_back(nebula::Row({"24", 24, "row_24"})); + expected.rows.emplace_back(nebula::Row({"20", 20, "row_20"})); + expected.rows.emplace_back(nebula::Row({"22", 22, "row_22"})); + expected.rows.emplace_back(nebula::Row({"34", 34, "row_34"})); + expected.rows.emplace_back(nebula::Row({"30", 30, "row_30"})); + expected.rows.emplace_back(nebula::Row({"32", 32, "row_32"})); + expected.rows.emplace_back(nebula::Row({"44", 44, "row_44"})); + expected.rows.emplace_back(nebula::Row({"40", 40, "row_40"})); + expected.rows.emplace_back(nebula::Row({"42", 42, "row_42"})); + expected.rows.emplace_back(nebula::Row({"54", 54, "row_54"})); + expected.rows.emplace_back(nebula::Row({"50", 50, "row_50"})); + expected.rows.emplace_back(nebula::Row({"52", 52, "row_52"})); + expected.rows.emplace_back(nebula::Row({"62", 62, "row_62"})); + expected.rows.emplace_back(nebula::Row({"60", 60, "row_60"})); + verifyResult(expected, *resp.get_data()); + } +} + +} // namespace storage +} // namespace nebula + +int main(int argc, char** argv) { + testing::InitGoogleTest(&argc, argv); + folly::init(&argc, &argv, true); + google::SetStderrLogging(google::INFO); + return RUN_ALL_TESTS(); +} diff --git a/tests/Makefile b/tests/Makefile index 0f372b6db18..5acb5635fb8 100644 --- a/tests/Makefile +++ b/tests/Makefile @@ -19,6 +19,8 @@ ENABLE_GRAPH_SSL ?= false ENABLE_META_SSL ?= false CA_SIGNED ?= false CONTAINERIZED ?= false +FAILED_LOGIN_ATTEMPTS ?= 0 +PASSWORD_LOCK_TIME_IN_SECS ?= 0 # commands gherkin_fmt = ~/.local/bin/reformat-gherkin diff --git a/tests/common/nebula_service.py b/tests/common/nebula_service.py index 82d3d40f514..1958ca15086 100644 --- a/tests/common/nebula_service.py +++ b/tests/common/nebula_service.py @@ -225,6 +225,9 @@ def _make_params(self, **kwargs): self.graphd_param['system_memory_high_watermark_ratio'] = '0.95' self.graphd_param['num_rows_to_check_memory'] = '4' self.graphd_param['session_reclaim_interval_secs'] = '2' + # Login retry + self.graphd_param['failed_login_attempts'] = '5' + self.graphd_param['password_lock_time_in_secs'] = '10' self.storaged_param = copy.copy(_params) self.storaged_param['local_config'] = 'false' diff --git a/tests/nebula-test-run.py b/tests/nebula-test-run.py index d7a83838c57..3f7ee40e77f 100755 --- a/tests/nebula-test-run.py +++ b/tests/nebula-test-run.py @@ -87,6 +87,18 @@ def init_parser(): default='false', help='run this process inside container', ) + opt_parser.add_option( + '--failed_login_attempts', + dest='failed_login_attempts', + default=0, + help='how many consecutive incorrect passwords input to a SINGLE graph service node cause the account to become locked', + ) + opt_parser.add_option( + '--password_lock_time_in_secs', + dest='password_lock_time_in_secs', + default=0, + help='how long in seconds to lock the account after too many consecutive login attempts provide an incorrect password', + ) return opt_parser diff --git a/tests/tck/conftest.py b/tests/tck/conftest.py index bfe85af7734..13b2ee1fbe3 100644 --- a/tests/tck/conftest.py +++ b/tests/tck/conftest.py @@ -14,6 +14,7 @@ from nebula2.common.ttypes import NList, NMap, Value, ErrorCode from nebula2.data.DataObject import ValueWrapper +from nebula2.Exception import AuthFailedException from pytest_bdd import given, parsers, then, when from tests.common.dataset_printer import DataSetPrinter @@ -361,6 +362,25 @@ def when_login_graphd(graph, user, password, class_fixture_variables, pytestconf class_fixture_variables["sessions"].append(sess) class_fixture_variables["pool"] = pool +# This is a workaround to test login retry because nebula-python treats +# authentication failure as exception instead of error. +@when(parse('login "{graph}" with "{user}" and "{password}" should fail:\n{msg}')) +def when_login_graphd_fail(graph, user, password, class_fixture_variables, msg): + index = parse_service_index(graph) + assert index is not None, "Invalid graph name, name is {}".format(graph) + nebula_svc = class_fixture_variables.get("cluster") + assert nebula_svc is not None, "Cannot get the cluster" + assert index < len(nebula_svc.graphd_processes) + graphd_process = nebula_svc.graphd_processes[index] + graph_ip, graph_port = graphd_process.host, graphd_process.tcp_port + pool = get_conn_pool(graph_ip, graph_port, None) + try: + sess = pool.get_session(user, password) + except AuthFailedException as e: + assert msg in e.message + except: + raise + @when(parse("executing query:\n{query}")) def executing_query(query, graph_spaces, session, request): ngql = combine_query(query) diff --git a/tests/tck/features/admin/Authentication.feature b/tests/tck/features/admin/Authentication.feature new file mode 100644 index 00000000000..5dccbe3e19f --- /dev/null +++ b/tests/tck/features/admin/Authentication.feature @@ -0,0 +1,118 @@ +# Copyright (c) 2021 vesoft inc. All rights reserved. +# +# This source code is licensed under Apache 2.0 License. +Feature: Test Authentication + + Background: + Given a nebulacluster with 1 graphd and 1 metad and 1 storaged: + """ + graphd:failed_login_attempts=5 + graphd:password_lock_time_in_secs=5 + """ + + Scenario: Test login with invalid password + When executing query: + """ + CREATE USER IF NOT EXISTS user1 WITH PASSWORD 'nebula1'; + CREATE SPACE IF NOT EXISTS root_space(vid_type=int); + USE root_space; + """ + Then the execution should be successful + And wait 3 seconds + When login "graphd[0]" with "user1" and "wrongPassword" should fail: + """ + Invalid password, remaining attempts: 4 + """ + When login "graphd[0]" with "user1" and "wrongPassword" should fail: + """ + Invalid password, remaining attempts: 3 + """ + When login "graphd[0]" with "user1" and "wrongPassword" should fail: + """ + Invalid password, remaining attempts: 2 + """ + When login "graphd[0]" with "user1" and "wrongPassword" should fail: + """ + Invalid password, remaining attempts: 1 + """ + When login "graphd[0]" with "user1" and "wrongPassword" should fail: + """ + 5 times consecutive incorrect passwords has been input, user name: user1 has been locked, try again in 5 seconds + """ + # Login with the correct password when the user is locked should fail + When login "graphd[0]" with "user1" and "nebula1" should fail: + """ + 5 times consecutive incorrect passwords has been input, user name: user1 has been locked + """ + # Wail the account to be unlocked + Then wait 6 seconds + When login "graphd[0]" with "user1" and "wrongPassword" should fail: + """ + Invalid password, remaining attempts: 4 + """ + When login "graphd[0]" with "user1" and "wrongPassword" should fail: + """ + Invalid password, remaining attempts: 3 + """ + # A successful login should resert the remaining password attempts + When login "graphd[0]" with "user1" and "nebula1" + Then the execution should be successful + When login "graphd[0]" with "user1" and "wrongPassword" should fail: + """ + Invalid password, remaining attempts: 4 + """ + + Scenario: Test login with invalid password multi users + When executing query: + """ + CREATE USER IF NOT EXISTS user1 WITH PASSWORD 'nebula1'; + CREATE USER IF NOT EXISTS user2 WITH PASSWORD 'nebula2'; + CREATE SPACE IF NOT EXISTS root_space(vid_type=int); + USE root_space; + """ + Then the execution should be successful + And wait 3 seconds + When login "graphd[0]" with "user1" and "wrongPassword" should fail: + """ + Invalid password, remaining attempts: 4 + """ + When login "graphd[0]" with "user1" and "wrongPassword" should fail: + """ + Invalid password, remaining attempts: 3 + """ + When login "graphd[0]" with "user1" and "wrongPassword" should fail: + """ + Invalid password, remaining attempts: 2 + """ + # User2 login + When login "graphd[0]" with "user2" and "wrongPassword" should fail: + """ + Invalid password, remaining attempts: 4 + """ + When login "graphd[0]" with "user2" and "wrongPassword" should fail: + """ + Invalid password, remaining attempts: 3 + """ + # User1 login + When login "graphd[0]" with "user1" and "wrongPassword" should fail: + """ + Invalid password, remaining attempts: 1 + """ + # User1 lock + When login "graphd[0]" with "user1" and "wrongPassword" should fail: + """ + 5 times consecutive incorrect passwords has been input, user name: user1 has been locked, try again in 5 seconds + """ + Then wait 6 seconds + When login "graphd[0]" with "user2" and "wrongPassword" should fail: + """ + Invalid password, remaining attempts: 2 + """ + When login "graphd[0]" with "user2" and "wrongPassword" should fail: + """ + Invalid password, remaining attempts: 1 + """ + When login "graphd[0]" with "user2" and "nebula2" + Then the execution should be successful + When login "graphd[0]" with "user1" and "nebula1" + Then the execution should be successful diff --git a/tests/tck/features/lookup/LookUpTopN.feature b/tests/tck/features/lookup/LookUpTopN.feature new file mode 100644 index 00000000000..c5aabc4463f --- /dev/null +++ b/tests/tck/features/lookup/LookUpTopN.feature @@ -0,0 +1,285 @@ +# Copyright (c) 2021 vesoft inc. All rights reserved. +# +# This source code is licensed under Apache 2.0 License. +Feature: Push TopN down IndexScan Rule + + Background: + Given a graph with space named "nba" + + Scenario: do not push topN down to IndexScan + When profiling query: + """ + LOOKUP ON player YIELD id(vertex) as id | ORDER BY $-.id | Limit 2 + """ + Then the result should be, in any order: + | id | + | "Amar'e Stoudemire" | + | "Aron Baynes" | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | {"count": "2"} | + | 6 | Project | 7 | | + | 7 | TagIndexFullScan | 0 | {"limit": "9223372036854775807" } | + | 0 | Start | | | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | | + | 6 | Project | 7 | | + | 7 | TagIndexFullScan | 0 | {"orderBy": "[]"} | + | 0 | Start | | | + When profiling query: + """ + LOOKUP ON player WHERE player.age==30 YIELD id(vertex) as id | ORDER BY $-.id | Limit 2 + """ + Then the result should be, in any order: + | id | + | "Blake Griffin" | + | "DeAndre Jordan" | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | {"count": "2"} | + | 6 | Project | 7 | | + | 7 | TagIndexPrefixScan | 0 | {"limit": "9223372036854775807" } | + | 0 | Start | | | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | | + | 6 | Project | 7 | | + | 7 | TagIndexPrefixScan | 0 | {"orderBy": "[]"} | + | 0 | Start | | | + When profiling query: + """ + LOOKUP ON player WHERE player.name > "Ti" YIELD id(vertex) as id | ORDER BY $-.id | Limit 2 + """ + Then the result should be, in any order: + | id | + | "Tiago Splitter" | + | "Tim Duncan" | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | {"count": "2"} | + | 6 | Project | 7 | | + | 7 | TagIndexRangeScan | 0 | {"limit": "9223372036854775807" } | + | 0 | Start | | | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | | + | 6 | Project | 7 | | + | 7 | TagIndexRangeScan | 0 | {"orderBy": "[]"} | + | 0 | Start | | | + When profiling query: + """ + LOOKUP ON player YIELD properties(vertex).name as name | ORDER BY $-.name | Limit 2 + """ + Then the result should be, in any order: + | name | + | "Amar'e Stoudemire" | + | "Aron Baynes" | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | {"count": "2"} | + | 6 | Project | 7 | | + | 7 | TagIndexFullScan | 0 | {"limit": "9223372036854775807" } | + | 0 | Start | | | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | | + | 6 | Project | 7 | | + | 7 | TagIndexFullScan | 0 | {"orderBy": "[]"} | + | 0 | Start | | | + When profiling query: + """ + LOOKUP ON like YIELD src(edge) as src, dst(edge) as dst, rank(edge) as rank | ORDER BY $-.src | Limit 2 + """ + Then the result should be, in any order: + | src | dst | rank | + | "Amar'e Stoudemire" | "Steve Nash" | 0 | + | "Aron Baynes" | "Tim Duncan" | 0 | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | {"count": "2"} | + | 6 | Project | 7 | | + | 7 | EdgeIndexFullScan | 0 | {"limit": "9223372036854775807" } | + | 0 | Start | | | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | | + | 6 | Project | 7 | | + | 7 | EdgeIndexFullScan | 0 | {"orderBy": "[]"} | + | 0 | Start | | | + When profiling query: + """ + LOOKUP ON like WHERE like.likeness == 90 YIELD src(edge) as src, dst(edge) as dst, rank(edge) as rank | ORDER BY $-.src | Limit 2 + """ + Then the result should be, in any order: + | src | dst | rank | + | "Amar'e Stoudemire" | "Steve Nash" | 0 | + | "Carmelo Anthony" | "Chris Paul" | 0 | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | {"count": "2"} | + | 6 | Project | 7 | | + | 7 | EdgeIndexPrefixScan | 0 | {"limit": "9223372036854775807" } | + | 0 | Start | | | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | | + | 6 | Project | 7 | | + | 7 | EdgeIndexPrefixScan | 0 | {"orderBy": "[]"} | + | 0 | Start | | | + When profiling query: + """ + LOOKUP ON like WHERE like.likeness > 90 YIELD src(edge) as src, dst(edge) as dst, rank(edge) as rank | ORDER BY $-.src | Limit 2 + """ + Then the result should be, in any order: + | src | dst | rank | + | "Dejounte Murray" | "Danny Green" | 0 | + | "Dejounte Murray" | "Chris Paul" | 0 | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | {"count": "2"} | + | 6 | Project | 7 | | + | 7 | EdgeIndexRangeScan | 0 | {"limit": "9223372036854775807" } | + | 0 | Start | | | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | | + | 6 | Project | 7 | | + | 7 | EdgeIndexRangeScan | 0 | {"orderBy": "[]"} | + | 0 | Start | | | + + Scenario: push topN down to IndexScan + When profiling query: + """ + LOOKUP ON player YIELD player.name as name | ORDER BY $-.name | Limit 2 + """ + Then the result should be, in any order: + | name | + | "Amar'e Stoudemire" | + | "Aron Baynes" | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | {"count": "2"} | + | 6 | Project | 7 | | + | 7 | TagIndexFullScan | 0 | {"limit": "2" } | + | 0 | Start | | | + When profiling query: + """ + LOOKUP ON player WHERE player.age==30 YIELD player.name as name | ORDER BY $-.name desc | Limit 2 + """ + Then the result should be, in any order: + | name | + | "Russell Westbrook" | + | "Kevin Durant" | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | {"count": "2"} | + | 6 | Project | 7 | | + | 7 | TagIndexPrefixScan | 0 | {"limit": "2" } | + | 0 | Start | | | + When profiling query: + """ + LOOKUP ON player WHERE player.age==30 YIELD player.name as name | ORDER BY $-.name | Limit 2 + """ + Then the result should be, in any order: + | name | + | "Blake Griffin" | + | "DeAndre Jordan" | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | {"count": "2"} | + | 6 | Project | 7 | | + | 7 | TagIndexPrefixScan | 0 | {"limit": "2" } | + | 0 | Start | | | + When profiling query: + """ + LOOKUP ON player WHERE player.name > "Ti" YIELD player.name as name | ORDER BY $-.name | Limit 2 + """ + Then the result should be, in any order: + | name | + | "Tiago Splitter" | + | "Tim Duncan" | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | {"count": "2"} | + | 6 | Project | 7 | | + | 7 | TagIndexRangeScan | 0 | {"limit": "2" } | + | 0 | Start | | | + When profiling query: + """ + LOOKUP ON like YIELD like.likeness as likeness | ORDER BY $-.likeness | Limit 2 + """ + Then the result should be, in any order: + | likeness | + | -1 | + | -1 | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | {"count": "2"} | + | 6 | Project | 7 | | + | 7 | EdgeIndexFullScan | 0 | {"limit": "2" } | + | 0 | Start | | | + When profiling query: + """ + LOOKUP ON like YIELD like.likeness as likeness | ORDER BY $-.likeness desc | Limit 2 + """ + Then the result should be, in any order: + | likeness | + | 100 | + | 100 | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | {"count": "2"} | + | 6 | Project | 7 | | + | 7 | EdgeIndexFullScan | 0 | {"limit": "2" } | + | 0 | Start | | | + When profiling query: + """ + LOOKUP ON like WHERE like.likeness == 90 YIELD like.likeness as likeness | ORDER BY $-.likeness | Limit 2 + """ + Then the result should be, in any order: + | likeness | + | 90 | + | 90 | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | {"count": "2"} | + | 6 | Project | 7 | | + | 7 | EdgeIndexPrefixScan | 0 | {"limit": "2" } | + | 0 | Start | | | + When profiling query: + """ + LOOKUP ON like WHERE like.likeness > 90 YIELD like.likeness as likeness | ORDER BY $-.likeness | Limit 2 + """ + Then the result should be, in any order: + | likeness | + | 95 | + | 95 | + And the execution plan should be: + | id | name | dependencies | operator info | + | 4 | DataCollect | 5 | | + | 5 | TopN | 6 | {"count": "2"} | + | 6 | Project | 7 | | + | 7 | EdgeIndexRangeScan | 0 | {"limit": "2" } | + | 0 | Start | | |