From dbea9c772426c5fb2ad915765eae27a2b2a59d1f Mon Sep 17 00:00:00 2001 From: CBS <56461666+bright-starry-sky@users.noreply.github.com> Date: Wed, 15 Sep 2021 09:37:29 +0800 Subject: [PATCH] Lookup limit push down (#2796) * lookup limit * add testcase * addressed comments * add test case * comment typo --- src/clients/storage/GraphStorageClient.cpp | 10 +- src/interface/storage.thrift | 5 +- src/storage/exec/IndexEdgeNode.h | 11 +- src/storage/exec/IndexScanNode.h | 10 +- src/storage/exec/IndexVertexNode.h | 11 +- src/storage/index/LookupBaseProcessor-inl.h | 40 ++- src/storage/index/LookupBaseProcessor.h | 1 + src/storage/test/CMakeLists.txt | 19 ++ src/storage/test/DeleteTagsTest.cpp | 5 +- src/storage/test/IndexScanLimitTest.cpp | 342 ++++++++++++++++++++ src/storage/test/IndexWithTTLTest.cpp | 29 +- src/storage/test/KillQueryTest.cpp | 10 +- src/storage/test/LookupIndexTest.cpp | 229 +++++++------ 13 files changed, 588 insertions(+), 134 deletions(-) create mode 100644 src/storage/test/IndexScanLimitTest.cpp diff --git a/src/clients/storage/GraphStorageClient.cpp b/src/clients/storage/GraphStorageClient.cpp index 2c0ead60f3f..33f258275da 100644 --- a/src/clients/storage/GraphStorageClient.cpp +++ b/src/clients/storage/GraphStorageClient.cpp @@ -492,11 +492,18 @@ folly::SemiFuture> GraphStorageClient: int32_t tagOrEdge, const std::vector& returnCols, folly::EventBase* evb) { + // TODO(sky) : instead of isEdge and tagOrEdge to nebula::cpp2::SchemaID for graph layer. auto status = getHostParts(space); if (!status.ok()) { return folly::makeFuture>( std::runtime_error(status.status().toString())); } + nebula::cpp2::SchemaID schemaId; + if (isEdge) { + schemaId.set_edge_type(tagOrEdge); + } else { + schemaId.set_tag_id(tagOrEdge); + } auto& clusters = status.value(); std::unordered_map requests; @@ -510,8 +517,7 @@ folly::SemiFuture> GraphStorageClient: cpp2::IndexSpec spec; spec.set_contexts(contexts); - spec.set_is_edge(isEdge); - spec.set_tag_or_edge_id(tagOrEdge); + spec.set_schema_id(schemaId); req.set_indices(spec); req.set_common(common); } diff --git a/src/interface/storage.thrift b/src/interface/storage.thrift index d3afad1ee22..067756ee89f 100644 --- a/src/interface/storage.thrift +++ b/src/interface/storage.thrift @@ -524,8 +524,7 @@ struct IndexQueryContext { struct IndexSpec { // In order to union multiple indices, multiple index hints are allowed 1: required list contexts, - 2: required bool is_edge, - 3: required i32 tag_or_edge_id, + 2: common.SchemaID schema_id, } @@ -537,6 +536,8 @@ struct LookupIndexRequest { // Support kVid and kTag for vertex, kSrc, kType, kRank and kDst for edge. 4: optional list return_columns, 5: optional RequestCommon common, + // max row count of each partition in this response + 6: optional i64 limit, } diff --git a/src/storage/exec/IndexEdgeNode.h b/src/storage/exec/IndexEdgeNode.h index 2810fc45f75..6ce73afe1eb 100644 --- a/src/storage/exec/IndexEdgeNode.h +++ b/src/storage/exec/IndexEdgeNode.h @@ -21,11 +21,13 @@ class IndexEdgeNode final : public RelNode { IndexEdgeNode(RuntimeContext* context, IndexScanNode* indexScanNode, const std::vector>& schemas, - const std::string& schemaName) + const std::string& schemaName, + int64_t limit) : context_(context), indexScanNode_(indexScanNode), schemas_(schemas), - schemaName_(schemaName) { + schemaName_(schemaName), + limit_(limit) { RelNode::name_ = "IndexEdgeNode"; } @@ -40,7 +42,11 @@ class IndexEdgeNode final : public RelNode { data_.clear(); std::vector edges; auto* iter = static_cast(indexScanNode_->iterator()); + int64_t count = 0; while (iter && iter->valid()) { + if (limit_ > -1 && count++ == limit_) { + break; + } if (context_->isPlanKilled()) { return nebula::cpp2::ErrorCode::E_PLAN_IS_KILLED; } @@ -93,6 +99,7 @@ class IndexEdgeNode final : public RelNode { IndexScanNode* indexScanNode_; const std::vector>& schemas_; const std::string& schemaName_; + int64_t limit_; std::vector data_; }; diff --git a/src/storage/exec/IndexScanNode.h b/src/storage/exec/IndexScanNode.h index 6ecce4e5d31..3576b73a57b 100644 --- a/src/storage/exec/IndexScanNode.h +++ b/src/storage/exec/IndexScanNode.h @@ -20,8 +20,9 @@ class IndexScanNode : public RelNode { IndexScanNode(RuntimeContext* context, IndexID indexId, - std::vector columnHints) - : context_(context), indexId_(indexId), columnHints_(std::move(columnHints)) { + std::vector columnHints, + int64_t limit) + : context_(context), indexId_(indexId), columnHints_(std::move(columnHints)), limit_(limit) { /** * columnHints's elements are {scanType = PREFIX|RANGE; beginStr; endStr}, * {scanType = PREFIX|RANGE; beginStr; @@ -71,7 +72,11 @@ class IndexScanNode : public RelNode { auto* sh = context_->isEdge() ? context_->edgeSchema_ : context_->tagSchema_; auto ttlProp = CommonUtils::ttlProps(sh); data_.clear(); + int64_t count = 0; while (!!iter_ && iter_->valid()) { + if (limit_ > -1 && count++ == limit_) { + break; + } if (context_->isPlanKilled()) { return {}; } @@ -172,6 +177,7 @@ class IndexScanNode : public RelNode { std::unique_ptr iter_; std::pair scanPair_; std::vector columnHints_; + int64_t limit_; std::vector data_; }; diff --git a/src/storage/exec/IndexVertexNode.h b/src/storage/exec/IndexVertexNode.h index 9ea3f43dcbc..c6a4ab17719 100644 --- a/src/storage/exec/IndexVertexNode.h +++ b/src/storage/exec/IndexVertexNode.h @@ -21,11 +21,13 @@ class IndexVertexNode final : public RelNode { IndexVertexNode(RuntimeContext* context, IndexScanNode* indexScanNode, const std::vector>& schemas, - const std::string& schemaName) + const std::string& schemaName, + int64_t limit) : context_(context), indexScanNode_(indexScanNode), schemas_(schemas), - schemaName_(schemaName) { + schemaName_(schemaName), + limit_(limit) { RelNode::name_ = "IndexVertexNode"; } @@ -40,7 +42,11 @@ class IndexVertexNode final : public RelNode { data_.clear(); std::vector vids; auto* iter = static_cast(indexScanNode_->iterator()); + int64_t count = 0; while (iter && iter->valid()) { + if (limit_ > -1 && count++ == limit_) { + break; + } if (context_->isPlanKilled()) { return nebula::cpp2::ErrorCode::E_PLAN_IS_KILLED; } @@ -84,6 +90,7 @@ class IndexVertexNode final : public RelNode { IndexScanNode* indexScanNode_; const std::vector>& schemas_; const std::string& schemaName_; + int64_t limit_; std::vector data_; }; diff --git a/src/storage/index/LookupBaseProcessor-inl.h b/src/storage/index/LookupBaseProcessor-inl.h index 19c39aa43d0..a01fb095497 100644 --- a/src/storage/index/LookupBaseProcessor-inl.h +++ b/src/storage/index/LookupBaseProcessor-inl.h @@ -21,10 +21,11 @@ nebula::cpp2::ErrorCode LookupBaseProcessor::requestCheck( this->planContext_ = std::make_unique( this->env_, spaceId_, this->spaceVidLen_, this->isIntId_, req.common_ref()); const auto& indices = req.get_indices(); - this->planContext_->isEdge_ = indices.get_is_edge(); + const auto& schemaId = indices.get_schema_id(); + this->planContext_->isEdge_ = schemaId.getType() == nebula::cpp2::SchemaID::Type::edge_type; this->context_ = std::make_unique(this->planContext_.get()); if (context_->isEdge()) { - context_->edgeType_ = indices.get_tag_or_edge_id(); + context_->edgeType_ = schemaId.get_edge_type(); auto edgeName = this->env_->schemaMan_->toEdgeName(spaceId_, context_->edgeType_); if (!edgeName.ok()) { return nebula::cpp2::ErrorCode::E_EDGE_NOT_FOUND; @@ -40,7 +41,7 @@ nebula::cpp2::ErrorCode LookupBaseProcessor::requestCheck( schemas_ = std::move(allEdges).value()[context_->edgeType_]; context_->edgeSchema_ = schemas_.back().get(); } else { - context_->tagId_ = indices.get_tag_or_edge_id(); + context_->tagId_ = schemaId.get_tag_id(); auto tagName = this->env_->schemaMan_->toTagName(spaceId_, context_->tagId_); if (!tagName.ok()) { return nebula::cpp2::ErrorCode::E_TAG_NOT_FOUND; @@ -75,6 +76,15 @@ nebula::cpp2::ErrorCode LookupBaseProcessor::requestCheck( } } + // limit + if (req.limit_ref().has_value()) { + if (*req.limit_ref() < 0) { + LOG(ERROR) << "Incorrect parameter : LIMIT = " << *req.limit_ref(); + return nebula::cpp2::ErrorCode::E_INVALID_PARM; + } + limit_ = *req.limit_ref(); + } + return nebula::cpp2::ErrorCode::SUCCEEDED; } @@ -275,8 +285,8 @@ std::unique_ptr> LookupBaseProcessor::buildP const std::vector& fields) { auto indexId = ctx.get_index_id(); auto colHints = ctx.get_column_hints(); - auto indexScan = - std::make_unique>(context_.get(), indexId, std::move(colHints)); + auto indexScan = std::make_unique>( + context_.get(), indexId, std::move(colHints), limit_); auto output = std::make_unique>( result, context_.get(), indexScan.get(), hasNullableCol, fields); @@ -309,11 +319,11 @@ std::unique_ptr> LookupBaseProcessor::buildP auto indexId = ctx.get_index_id(); auto colHints = ctx.get_column_hints(); - auto indexScan = - std::make_unique>(context_.get(), indexId, std::move(colHints)); + auto indexScan = std::make_unique>( + context_.get(), indexId, std::move(colHints), limit_); if (context_->isEdge()) { auto edge = std::make_unique>( - context_.get(), indexScan.get(), schemas_, context_->edgeName_); + context_.get(), indexScan.get(), schemas_, context_->edgeName_, limit_); edge->addDependency(indexScan.get()); auto output = std::make_unique>(result, context_.get(), edge.get()); output->addDependency(edge.get()); @@ -322,7 +332,7 @@ std::unique_ptr> LookupBaseProcessor::buildP return output; } else { auto vertex = std::make_unique>( - context_.get(), indexScan.get(), schemas_, context_->tagName_); + context_.get(), indexScan.get(), schemas_, context_->tagName_, limit_); vertex->addDependency(indexScan.get()); auto output = std::make_unique>(result, context_.get(), vertex.get()); output->addDependency(vertex.get()); @@ -360,8 +370,8 @@ std::unique_ptr> LookupBaseProcessor::buildP auto indexId = ctx.get_index_id(); auto colHints = ctx.get_column_hints(); - auto indexScan = - std::make_unique>(context_.get(), indexId, std::move(colHints)); + auto indexScan = std::make_unique>( + context_.get(), indexId, std::move(colHints), limit_); auto filter = std::make_unique>( context_.get(), indexScan.get(), exprCtx, exp, context_->isEdge()); @@ -411,11 +421,11 @@ LookupBaseProcessor::buildPlanWithDataAndFilter(nebula::DataSet* resu auto indexId = ctx.get_index_id(); auto colHints = ctx.get_column_hints(); - auto indexScan = - std::make_unique>(context_.get(), indexId, std::move(colHints)); + auto indexScan = std::make_unique>( + context_.get(), indexId, std::move(colHints), limit_); if (context_->isEdge()) { auto edge = std::make_unique>( - context_.get(), indexScan.get(), schemas_, context_->edgeName_); + context_.get(), indexScan.get(), schemas_, context_->edgeName_, limit_); edge->addDependency(indexScan.get()); auto filter = std::make_unique>(context_.get(), edge.get(), exprCtx, exp); @@ -429,7 +439,7 @@ LookupBaseProcessor::buildPlanWithDataAndFilter(nebula::DataSet* resu return output; } else { auto vertex = std::make_unique>( - context_.get(), indexScan.get(), schemas_, context_->tagName_); + context_.get(), indexScan.get(), schemas_, context_->tagName_, limit_); vertex->addDependency(indexScan.get()); auto filter = std::make_unique>(context_.get(), vertex.get(), exprCtx, exp); diff --git a/src/storage/index/LookupBaseProcessor.h b/src/storage/index/LookupBaseProcessor.h index 0a3e35ef6b7..1815e8a72fa 100644 --- a/src/storage/index/LookupBaseProcessor.h +++ b/src/storage/index/LookupBaseProcessor.h @@ -83,6 +83,7 @@ class LookupBaseProcessor : public BaseProcessor { // Save schemas when column is out of index, need to read from data std::vector> schemas_; std::vector deDupColPos_; + int64_t limit_ = -1; }; } // namespace storage diff --git a/src/storage/test/CMakeLists.txt b/src/storage/test/CMakeLists.txt index 437185d4692..1ee64ef9607 100644 --- a/src/storage/test/CMakeLists.txt +++ b/src/storage/test/CMakeLists.txt @@ -636,7 +636,26 @@ nebula_add_test( ${PROXYGEN_LIBRARIES} wangle gtest +) +nebula_add_test( + NAME + index_scan_limit_test + SOURCES + IndexScanLimitTest.cpp + OBJECTS + $ + $ + $ + $ + $ + ${storage_test_deps} + LIBRARIES + ${ROCKSDB_LIBRARIES} + ${THRIFT_LIBRARIES} + ${PROXYGEN_LIBRARIES} + wangle + gtest ) nebula_add_executable( diff --git a/src/storage/test/DeleteTagsTest.cpp b/src/storage/test/DeleteTagsTest.cpp index e6544c15016..abd9e7889bd 100644 --- a/src/storage/test/DeleteTagsTest.cpp +++ b/src/storage/test/DeleteTagsTest.cpp @@ -58,8 +58,9 @@ cpp2::LookupIndexRequest buildLookupRequest(int32_t totalParts, std::string play cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(1); - indices.set_tag_or_edge_id(1); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(1); + indices.set_schema_id(schemaId); std::vector parts; for (PartitionID partId = 1; partId <= totalParts; partId++) { parts.emplace_back(partId); diff --git a/src/storage/test/IndexScanLimitTest.cpp b/src/storage/test/IndexScanLimitTest.cpp new file mode 100644 index 00000000000..b71e3578898 --- /dev/null +++ b/src/storage/test/IndexScanLimitTest.cpp @@ -0,0 +1,342 @@ +/* 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. + */ + +#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 { +class IndexScanLimitTest : 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, true /*stats*/); + 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", meta::cpp2::PropertyType::INT64); + schema->addField("col2", meta::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.set_type(meta::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; + } + + // Edge and vertex have the same schema of structure, so it's good to only generate it once. + RowWriterV2 writer(tag.get()); + auto r1 = writer.setValue(0, 888); + if (r1 != WriteResult::SUCCEEDED) { + LOG(ERROR) << "Invalid prop col1"; + return false; + } + auto r2 = writer.setValue(1, "row"); + if (r2 != WriteResult::SUCCEEDED) { + LOG(ERROR) << "Invalid prop col2"; + return false; + } + auto ret = writer.finish(); + if (ret != WriteResult::SUCCEEDED) { + LOG(ERROR) << "Failed to write data"; + return false; + } + auto val = std::move(writer).moveEncodedStr(); + + for (auto pId : parts) { + std::vector data; + for (int64_t vid = pId * 1000; vid < (pId + 1) * 1000; vid++) { + auto vertex = folly::to(vid); + auto edgeKey = NebulaKeyUtils::edgeKey(8, pId, vertex, edgeType, 0, vertex); + auto vertexKey = NebulaKeyUtils::vertexKey(8, pId, vertex, tagId); + data.emplace_back(std::move(edgeKey), val); + data.emplace_back(std::move(vertexKey), val); + if (indexMan_ != nullptr) { + if (indexMan_->getTagIndex(spaceId, tagIndex).ok()) { + auto vertexIndexKey = IndexKeyUtils::vertexIndexKey( + vertexLen, pId, tagIndex, vertex, IndexKeyUtils::encodeValues({888}, genCols())); + data.emplace_back(std::move(vertexIndexKey), ""); + } + if (indexMan_->getEdgeIndex(spaceId, edgeIndex).ok()) { + auto edgeIndexKey = + IndexKeyUtils::edgeIndexKey(vertexLen, + pId, + edgeIndex, + vertex, + 0, + vertex, + IndexKeyUtils::encodeValues({888}, genCols())); + 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/IndexScanLimitTest.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}; +}; + +TEST_F(IndexScanLimitTest, LookupTagIndexLimit) { + cpp2::LookupIndexRequest req; + nebula::storage::cpp2::IndexSpec indices; + req.set_space_id(spaceId); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(tagId); + indices.set_schema_id(schemaId); + req.set_parts(parts); + std::vector returnCols; + returnCols.emplace_back(kVid); + req.set_return_columns(std::move(returnCols)); + cpp2::IndexQueryContext context1; + context1.set_index_id(tagIndex); + decltype(indices.contexts) contexts; + contexts.emplace_back(std::move(context1)); + indices.set_contexts(std::move(contexts)); + req.set_indices(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(6000, resp.get_data()->rows.size()); + } + + // limit == 0 + { + req.set_limit(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()); + } + + // limit 5 by each part + { + req.set_limit(5); + 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(5 * parts.size(), resp.get_data()->rows.size()); + } + + // limit 5 by each part through IndexScanNode->DataNode->FilterNode + { + req.set_limit(5); + cpp2::IndexColumnHint columnHint; + columnHint.set_begin_value(Value(888)); + columnHint.set_column_name("col1"); + columnHint.set_scan_type(cpp2::ScanType::PREFIX); + std::vector columnHints; + columnHints.emplace_back(std::move(columnHint)); + req.indices_ref().value().contexts_ref().value().begin()->set_column_hints( + 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(5 * parts.size(), resp.get_data()->rows.size()); + } +} + +TEST_F(IndexScanLimitTest, LookupEdgeIndexLimit) { + cpp2::LookupIndexRequest req; + nebula::storage::cpp2::IndexSpec indices; + req.set_space_id(spaceId); + nebula::cpp2::SchemaID schemaId; + schemaId.set_edge_type(edgeType); + indices.set_schema_id(schemaId); + req.set_parts(parts); + std::vector returnCols; + returnCols.emplace_back(kSrc); + req.set_return_columns(std::move(returnCols)); + cpp2::IndexQueryContext context1; + context1.set_index_id(edgeIndex); + decltype(indices.contexts) contexts; + contexts.emplace_back(std::move(context1)); + indices.set_contexts(std::move(contexts)); + req.set_indices(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(6000, resp.get_data()->rows.size()); + } + + // limit == 0 + { + req.set_limit(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()); + } + + // limit 5 by each part + { + req.set_limit(5); + 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(5 * parts.size(), resp.get_data()->rows.size()); + } + + // limit 5 by each part through IndexScanNode->DataNode->FilterNode + { + req.set_limit(5); + cpp2::IndexColumnHint columnHint; + columnHint.set_begin_value(Value(888)); + columnHint.set_column_name("col1"); + columnHint.set_scan_type(cpp2::ScanType::PREFIX); + std::vector columnHints; + columnHints.emplace_back(std::move(columnHint)); + req.indices_ref().value().contexts_ref().value().begin()->set_column_hints( + 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(5 * parts.size(), resp.get_data()->rows.size()); + } +} + +} // 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/src/storage/test/IndexWithTTLTest.cpp b/src/storage/test/IndexWithTTLTest.cpp index 6aa19a5982f..00f9d62d587 100644 --- a/src/storage/test/IndexWithTTLTest.cpp +++ b/src/storage/test/IndexWithTTLTest.cpp @@ -442,7 +442,7 @@ TEST(IndexWithTTLTest, RebuildTagIndexWithTTL) { // Wait for the task finished do { - usleep(500); + sleep(1); } while (!manager_->isFinished(context.jobId_, context.taskId_)); manager_->shutdown(); @@ -511,7 +511,7 @@ TEST(IndexWithTTLTest, RebuildEdgeIndexWithTTL) { // Wait for the task finished do { - usleep(500); + sleep(1); } while (!manager_->isFinished(context.jobId_, context.taskId_)); manager_->shutdown(); @@ -582,7 +582,7 @@ TEST(IndexWithTTLTest, RebuildTagIndexWithTTLExpired) { // Wait for the task finished do { - usleep(500); + sleep(1); } while (!manager_->isFinished(context.jobId_, context.taskId_)); manager_->shutdown(); @@ -653,7 +653,7 @@ TEST(IndexWithTTLTest, RebuildEdgeIndexWithTTLExpired) { // Wait for the task finished do { - usleep(500); + sleep(1); } while (!manager_->isFinished(context.jobId_, context.taskId_)); manager_->shutdown(); @@ -687,8 +687,9 @@ TEST(IndexWithTTLTest, LookupTagIndexWithTTL) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(1); - indices.set_tag_or_edge_id(2021001); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(2021001); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= 6; p++) { parts.emplace_back(p); @@ -730,8 +731,10 @@ TEST(IndexWithTTLTest, LookupEdgeIndexWithTTL) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(1); - indices.set_tag_or_edge_id(2021001); - indices.set_is_edge(true); + nebula::cpp2::SchemaID schemaId; + schemaId.set_edge_type(2021001); + indices.set_schema_id(schemaId); + std::vector parts; for (int32_t p = 1; p <= 6; p++) { parts.emplace_back(p); @@ -775,8 +778,9 @@ TEST(IndexWithTTLTest, LookupTagIndexWithTTLExpired) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(1); - indices.set_tag_or_edge_id(2021001); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(2021001); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= 6; p++) { parts.emplace_back(p); @@ -820,8 +824,9 @@ TEST(IndexWithTTLTest, LookupEdgeIndexWithTTLExpired) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(1); - indices.set_tag_or_edge_id(2021001); - indices.set_is_edge(true); + nebula::cpp2::SchemaID schemaId; + schemaId.set_edge_type(2021001); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= 6; p++) { parts.emplace_back(p); diff --git a/src/storage/test/KillQueryTest.cpp b/src/storage/test/KillQueryTest.cpp index 8bfa64ac119..11ccacc746c 100644 --- a/src/storage/test/KillQueryTest.cpp +++ b/src/storage/test/KillQueryTest.cpp @@ -102,8 +102,9 @@ TEST_F(KillQueryTest, TagIndex) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(1); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(1); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -161,8 +162,9 @@ TEST_F(KillQueryTest, EdgeIndex) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(102); - indices.set_is_edge(true); + nebula::cpp2::SchemaID schemaId; + schemaId.set_edge_type(102); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); diff --git a/src/storage/test/LookupIndexTest.cpp b/src/storage/test/LookupIndexTest.cpp index 4354be9bf91..c9d35dc892e 100644 --- a/src/storage/test/LookupIndexTest.cpp +++ b/src/storage/test/LookupIndexTest.cpp @@ -130,7 +130,9 @@ TEST_P(LookupIndexTest, LookupIndexTestV1) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(1); - indices.set_tag_or_edge_id(3); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(3); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -220,8 +222,9 @@ TEST_P(LookupIndexTest, SimpleTagIndexTest) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(1); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(1); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -287,8 +290,9 @@ TEST_P(LookupIndexTest, SimpleTagIndexTest) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(1); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(1); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -389,8 +393,9 @@ TEST_P(LookupIndexTest, SimpleEdgeIndexTest) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(102); - indices.set_is_edge(true); + nebula::cpp2::SchemaID schemaId; + schemaId.set_edge_type(102); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -473,8 +478,9 @@ TEST_P(LookupIndexTest, SimpleEdgeIndexTest) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(102); - indices.set_is_edge(true); + nebula::cpp2::SchemaID schemaId; + schemaId.set_edge_type(102); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -605,8 +611,9 @@ TEST_P(LookupIndexTest, TagIndexFilterTest) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(1); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(1); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -680,8 +687,9 @@ TEST_P(LookupIndexTest, TagIndexFilterTest) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(1); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(1); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -762,8 +770,9 @@ TEST_P(LookupIndexTest, EdgeIndexFilterTest) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(102); - indices.set_is_edge(true); + nebula::cpp2::SchemaID schemaId; + schemaId.set_edge_type(102); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -855,8 +864,9 @@ TEST_P(LookupIndexTest, EdgeIndexFilterTest) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(102); - indices.set_is_edge(true); + nebula::cpp2::SchemaID schemaId; + schemaId.set_edge_type(102); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -941,8 +951,9 @@ TEST_P(LookupIndexTest, TagIndexWithDataTest) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(1); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(1); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -1027,8 +1038,9 @@ TEST_P(LookupIndexTest, EdgeIndexWithDataTest) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(102); - indices.set_is_edge(true); + nebula::cpp2::SchemaID schemaId; + schemaId.set_edge_type(102); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -1125,8 +1137,9 @@ TEST_P(LookupIndexTest, TagWithPropStatsVerticesIndexTest) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(1); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(1); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -1197,8 +1210,9 @@ TEST_P(LookupIndexTest, TagWithoutPropStatsVerticesIndexTest) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(1); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(1); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -1270,8 +1284,9 @@ TEST_P(LookupIndexTest, EdgeWithPropStatsVerticesIndexTest) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(101); - indices.set_is_edge(true); + nebula::cpp2::SchemaID schemaId; + schemaId.set_edge_type(101); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -1352,8 +1367,9 @@ TEST_P(LookupIndexTest, EdgeWithoutPropStatsVerticesIndexTest) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(101); - indices.set_is_edge(true); + nebula::cpp2::SchemaID schemaId; + schemaId.set_edge_type(101); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -1580,8 +1596,9 @@ TEST_P(LookupIndexTest, NullableInIndexAndFilterTest) { context.set_column_hints({columnHint}); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -1606,8 +1623,9 @@ TEST_P(LookupIndexTest, NullableInIndexAndFilterTest) { context.set_column_hints({columnHint}); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -1644,8 +1662,9 @@ TEST_P(LookupIndexTest, NullableInIndexAndFilterTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -1672,8 +1691,9 @@ TEST_P(LookupIndexTest, NullableInIndexAndFilterTest) { context.set_column_hints({columnHint}); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -1700,8 +1720,9 @@ TEST_P(LookupIndexTest, NullableInIndexAndFilterTest) { context.set_column_hints({columnHint}); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -1730,8 +1751,9 @@ TEST_P(LookupIndexTest, NullableInIndexAndFilterTest) { context.set_column_hints({columnHint}); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -1776,8 +1798,9 @@ TEST_P(LookupIndexTest, NullableInIndexAndFilterTest) { context.set_column_hints(columnHints); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -1821,8 +1844,9 @@ TEST_P(LookupIndexTest, NullableInIndexAndFilterTest) { context.set_column_hints(columnHints); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -1869,9 +1893,10 @@ TEST_P(LookupIndexTest, NullableInIndexAndFilterTest) { context.set_index_id(222); context.set_column_hints(columnHints); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2084,8 +2109,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2111,8 +2137,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2138,8 +2165,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2165,8 +2193,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2192,8 +2221,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2219,8 +2249,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2250,8 +2281,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2281,8 +2313,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2310,8 +2343,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2339,8 +2373,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2372,8 +2407,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2402,8 +2438,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2428,8 +2465,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2457,8 +2495,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2489,8 +2528,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2523,8 +2563,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2553,8 +2594,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2585,8 +2627,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2619,8 +2662,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2646,8 +2690,9 @@ TEST_P(LookupIndexTest, NullablePropertyTest) { context.set_column_hints(std::move(columnHints)); cpp2::IndexSpec indices; - indices.set_tag_or_edge_id(111); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(111); + indices.set_schema_id(schemaId); indices.set_contexts({context}); req.set_indices(std::move(indices)); @@ -2698,8 +2743,9 @@ TEST_P(LookupIndexTest, DeDupTagIndexTest) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(1); - indices.set_is_edge(false); + nebula::cpp2::SchemaID schemaId; + schemaId.set_tag_id(1); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p); @@ -2795,8 +2841,9 @@ TEST_P(LookupIndexTest, DedupEdgeIndexTest) { cpp2::LookupIndexRequest req; nebula::storage::cpp2::IndexSpec indices; req.set_space_id(spaceId); - indices.set_tag_or_edge_id(102); - indices.set_is_edge(true); + nebula::cpp2::SchemaID schemaId; + schemaId.set_edge_type(102); + indices.set_schema_id(schemaId); std::vector parts; for (int32_t p = 1; p <= totalParts; p++) { parts.emplace_back(p);