This is an automated email from the ASF dual-hosted git repository.
wusheng pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/skywalking-banyandb.git
The following commit(s) were added to refs/heads/main by this push:
new 5c951cf3 Support sort in the property query (#861)
5c951cf3 is described below
commit 5c951cf3060c434b46730145fce798fb4cf2f72c
Author: mrproliu <[email protected]>
AuthorDate: Fri Dec 5 00:09:42 2025 +0800
Support sort in the property query (#861)
---
CHANGES.md | 2 +
api/proto/banyandb/property/v1/rpc.proto | 12 +
banyand/liaison/grpc/property.go | 304 ++++++++++++++++++---
banyand/property/db.go | 90 +++++-
banyand/property/listener.go | 1 +
banyand/property/repair_gossip.go | 2 +-
banyand/property/repair_gossip_test.go | 2 +-
banyand/property/shard.go | 85 ++++--
banyand/property/shard_test.go | 2 +-
docs/api-reference.md | 19 ++
docs/interacting/bydbql.md | 9 +-
pkg/bydbql/transformer.go | 13 +
test/cases/property/data/data.go | 8 +-
test/cases/property/data/input/order_by_asc.ql | 19 ++
test/cases/property/data/input/order_by_asc.yaml | 26 ++
test/cases/property/data/input/order_by_desc.ql | 19 ++
test/cases/property/data/input/order_by_desc.yaml | 26 ++
.../property/data/input/order_by_with_limit.ql | 19 ++
.../property/data/input/order_by_with_limit.yaml | 27 ++
.../data/input/order_without_projection.ql | 19 ++
.../data/input/order_without_projection.yaml | 25 ++
test/cases/property/data/input/query_with_order.ql | 21 ++
.../property/data/input/query_with_order.yaml | 33 +++
test/cases/property/data/want/order_by_asc.yaml | 52 ++++
test/cases/property/data/want/order_by_desc.yaml | 52 ++++
.../property/data/want/order_by_with_limit.yaml | 35 +++
.../data/want/order_without_projection.yaml | 44 +++
.../cases/property/data/want/query_with_order.yaml | 35 +++
test/cases/property/property.go | 5 +
29 files changed, 939 insertions(+), 67 deletions(-)
diff --git a/CHANGES.md b/CHANGES.md
index c614eeaf..eb033da5 100644
--- a/CHANGES.md
+++ b/CHANGES.md
@@ -9,6 +9,8 @@ Release Notes.
- Remove Bloom filter for dictionary-encoded tags.
- Implement BanyanDB MCP.
- Remove check requiring tags in criteria to be present in projection
+- Add sorted query support for the Property.
+- Update bydbQL to add sorted query support for the Property.
### Bug Fixes
diff --git a/api/proto/banyandb/property/v1/rpc.proto
b/api/proto/banyandb/property/v1/rpc.proto
index 408eeca3..ecfd6a9c 100644
--- a/api/proto/banyandb/property/v1/rpc.proto
+++ b/api/proto/banyandb/property/v1/rpc.proto
@@ -63,6 +63,13 @@ message DeleteResponse {
bool deleted = 1;
}
+message QueryOrder {
+ // tag_name is the name of the tag to be ordered.
+ string tag_name = 1;
+ // order_by is given to specify the sort for a tag.
+ model.v1.Sort sort = 12;
+}
+
// QueryRequest is the request contract for query.
message QueryRequest {
// groups indicate where the data points are stored.
@@ -78,6 +85,8 @@ message QueryRequest {
uint32 limit = 6;
// trace is used to enable trace for the query
bool trace = 7;
+ // order_by is given to specify the sort for a tag.
+ QueryOrder order_by = 8;
}
// QueryResponse is the response for a query to the Query module.
@@ -126,6 +135,9 @@ message InternalQueryResponse {
// it's mapping to the sources in the same order
// if the value is 0, it means the property is not deleted
repeated int64 deletes = 3;
+ // sorted_values contains pre-extracted sort values from shard searches,
+ // mapping to sources in the same order for optimized sorting at liaison
layer
+ repeated bytes sorted_values = 4;
}
message InternalRepairRequest {
diff --git a/banyand/liaison/grpc/property.go b/banyand/liaison/grpc/property.go
index 9f68570c..b759c4b4 100644
--- a/banyand/liaison/grpc/property.go
+++ b/banyand/liaison/grpc/property.go
@@ -18,9 +18,11 @@
package grpc
import (
+ "bytes"
"context"
"fmt"
"math"
+ "sort"
"sync"
"time"
@@ -42,6 +44,7 @@ import (
"github.com/apache/skywalking-banyandb/pkg/accesslog"
"github.com/apache/skywalking-banyandb/pkg/bus"
"github.com/apache/skywalking-banyandb/pkg/convert"
+ sortpkg "github.com/apache/skywalking-banyandb/pkg/iter/sort"
"github.com/apache/skywalking-banyandb/pkg/logger"
"github.com/apache/skywalking-banyandb/pkg/partition"
pbv1 "github.com/apache/skywalking-banyandb/pkg/pb/v1"
@@ -439,39 +442,33 @@ func (ps *propertyServer) Query(ctx context.Context, req
*propertyv1.QueryReques
if err != nil {
return nil, err
}
- res := make(map[string]*propertyWithCount)
- for n, nodeWithProperties := range nodeProperties {
- for _, propertyMetadata := range nodeWithProperties {
- entity :=
propertypkg.GetEntity(propertyMetadata.Property)
- cur, ok := res[entity]
- if !ok {
- res[entity] =
newPropertyWithCounts(propertyMetadata, n)
- continue
- }
- switch {
- case cur.Metadata.ModRevision <
propertyMetadata.Metadata.ModRevision: // newer revision
- res[entity] =
newPropertyWithCounts(propertyMetadata, n)
- case cur.Metadata.ModRevision ==
propertyMetadata.Metadata.ModRevision: // same revision
- cur.addExistNode(n)
- }
- }
- }
- if len(res) == 0 {
+ if len(nodeProperties) == 0 {
return &propertyv1.QueryResponse{Properties: nil, Trace:
trace}, nil
}
- properties := make([]*propertyv1.Property, 0, len(res))
- for entity, p := range res {
- if err := ps.repairPropertyIfNeed(entity, p, groups); err !=
nil {
+
+ var properties []*propertyWithCount
+
+ // Choose processing path based on whether ordering is requested
+ if req.OrderBy != nil && req.OrderBy.TagName != "" {
+ // Sorted query: Use unified k-way merge with binary search
insertion
+ properties = ps.sortedQueryWithDedup(nodeProperties, req)
+ } else {
+ // No ordering: Simple dedup without sorting
+ properties = ps.simpleDedupWithoutSort(nodeProperties)
+ }
+
+ result := make([]*propertyv1.Property, 0, len(properties))
+ for _, property := range properties {
+ if err := ps.repairPropertyIfNeed(property.entity, property,
groups); err != nil {
ps.log.Warn().Msgf("failed to repair properties when
query: %v", err)
}
-
// ignore deleted property in the query
- if p.deletedTime > 0 {
+ if property.deletedTime > 0 {
continue
}
if len(req.TagProjection) > 0 {
var tags []*modelv1.Tag
- for _, tag := range p.Tags {
+ for _, tag := range property.Tags {
for _, tp := range req.TagProjection {
if tp == tag.Key {
tags = append(tags, tag)
@@ -479,14 +476,211 @@ func (ps *propertyServer) Query(ctx context.Context, req
*propertyv1.QueryReques
}
}
}
- p.Tags = tags
+ property.Tags = tags
+ }
+ result = append(result, property.Property)
+ }
+
+ // Apply limit
+ if len(result) > int(req.Limit) {
+ result = result[:req.Limit]
+ }
+ return &propertyv1.QueryResponse{Properties: result, Trace: trace}, nil
+}
+
+// sortedQueryWithDedup implements unified sorted query handling for both
single and multi-node scenarios.
+// It uses k-way merge with binary search insertion to maintain sorted order
efficiently.
+func (ps *propertyServer) sortedQueryWithDedup(
+ nodeProperties map[string][]*propertyWithMetadata,
+ req *propertyv1.QueryRequest,
+) []*propertyWithCount {
+ // Initialize k-way merge iterators
+ iters := make([]sortpkg.Iterator[*propertyWithMetadata], 0,
len(nodeProperties))
+ for _, props := range nodeProperties {
+ if len(props) > 0 {
+ iters = append(iters,
newPropertyWithMetadataIterator(props))
+ }
+ }
+
+ if len(iters) == 0 {
+ return nil
+ }
+
+ // Create heap-based merge iterator
+ isDesc := req.OrderBy.Sort == modelv1.Sort_SORT_DESC
+ mergeIter := sortpkg.NewItemIter(iters, isDesc)
+ defer mergeIter.Close()
+
+ // Initialize deduplication state
+ // seenIDs tracks entity -> propertyWithCount for deduplication
+ seenIDs := make(map[string]*propertyWithCount)
+ // resultBuffer maintains sorted list of unique properties
+ resultBuffer := make([]*propertyWithCount, 0, req.Limit)
+
+ // Process items from k-way merge
+ for mergeIter.Next() {
+ p := mergeIter.Val()
+ entity := propertypkg.GetEntity(p.Property)
+
+ // Check if we've seen this entity before
+ if existingCount, seen := seenIDs[entity]; seen {
+ // Same modRevision - accumulate node
+ if p.Metadata.ModRevision ==
existingCount.Metadata.ModRevision {
+ existingCount.addExistNode(p.node)
+ continue
+ }
+
+ // Older modRevision - skip
+ if p.Metadata.ModRevision <
existingCount.Metadata.ModRevision {
+ continue
+ }
+
+ // Newer modRevision - replace old entry
+ // Find and remove old entry from resultBuffer using
binary search
+ oldIndex := ps.findPropertyInBuffer(resultBuffer,
existingCount, isDesc)
+ if oldIndex >= 0 && oldIndex < len(resultBuffer) {
+ // Remove old entry
+ resultBuffer = append(resultBuffer[:oldIndex],
resultBuffer[oldIndex+1:]...)
+ }
+
+ // Create new propertyWithCount for the newer revision
+ newCount := newPropertyWithCounts(p, entity, p.node)
+ seenIDs[entity] = newCount
+
+ // Insert new entry in sorted position using binary
search
+ insertPos := ps.findInsertPosition(resultBuffer,
newCount, isDesc)
+ resultBuffer = ps.insertAtPosition(resultBuffer,
newCount, insertPos)
+
+ continue
+ }
+
+ // New entity - first time seeing this entity
+ // Create propertyWithCount and add to seenIDs
+ propCount := newPropertyWithCounts(p, entity, p.node)
+ seenIDs[entity] = propCount
+
+ // Insert into resultBuffer at correct sorted position
+ resultBuffer = ps.sortInsertToBuffer(resultBuffer, propCount,
isDesc)
+ }
+
+ return resultBuffer
+}
+
+// sortInsertToBuffer sort insert the property to the buffer.
+func (ps *propertyServer) sortInsertToBuffer(buffer []*propertyWithCount,
+ newProp *propertyWithCount,
+ isDesc bool,
+) []*propertyWithCount {
+ insertPos := ps.findInsertPosition(buffer, newProp, isDesc)
+ return ps.insertAtPosition(buffer, newProp, insertPos)
+}
+
+// findInsertPosition finds the correct position to insert a new property
+// in the sorted resultBuffer using binary search.
+func (ps *propertyServer) findInsertPosition(
+ buffer []*propertyWithCount,
+ newProp *propertyWithCount,
+ isDesc bool,
+) int {
+ return sort.Search(len(buffer), func(i int) bool {
+ cmp := bytes.Compare(buffer[i].sortedValue, newProp.sortedValue)
+ if isDesc {
+ return cmp <= 0 // For descending: insert before items
<= new value
+ }
+ return cmp >= 0 // For ascending: insert before items >= new
value
+ })
+}
+
+// insertAtPosition inserts a property at the specified position in the buffer.
+func (ps *propertyServer) insertAtPosition(
+ buffer []*propertyWithCount,
+ prop *propertyWithCount,
+ pos int,
+) []*propertyWithCount {
+ // Grow buffer by one
+ buffer = append(buffer, nil)
+ // Shift elements to the right
+ copy(buffer[pos+1:], buffer[pos:])
+ // Insert new element
+ buffer[pos] = prop
+ return buffer
+}
+
+// findPropertyInBuffer finds the index of a specific property in the buffer.
+// Returns -1 if not found.
+func (ps *propertyServer) findPropertyInBuffer(
+ buffer []*propertyWithCount,
+ target *propertyWithCount,
+ isDesc bool,
+) int {
+ // Use binary search to find approximate position
+ searchPos := sort.Search(len(buffer), func(i int) bool {
+ cmp := bytes.Compare(buffer[i].sortedValue, target.sortedValue)
+ if isDesc {
+ return cmp <= 0
+ }
+ return cmp >= 0
+ })
+
+ // Linear search around the found position to handle duplicate
sortedValues
+ // (multiple properties might have same sortedValue)
+ for i := searchPos; i < len(buffer); i++ {
+ if bytes.Equal(buffer[i].sortedValue, target.sortedValue) {
+ // Check if it's the exact same property by comparing
entity
+ if propertypkg.GetEntity(buffer[i].Property) ==
propertypkg.GetEntity(target.Property) {
+ return i
+ }
+ } else {
+ // Moved past matching sortedValues
+ break
}
- properties = append(properties, p.Property)
- if len(properties) >= int(req.Limit) {
+ }
+
+ // Also check backwards from searchPos (in case binary search landed
after our target)
+ for i := searchPos - 1; i >= 0; i-- {
+ if bytes.Equal(buffer[i].sortedValue, target.sortedValue) {
+ if propertypkg.GetEntity(buffer[i].Property) ==
propertypkg.GetEntity(target.Property) {
+ return i
+ }
+ } else {
break
}
}
- return &propertyv1.QueryResponse{Properties: properties, Trace: trace},
nil
+
+ return -1 // Not found
+}
+
+// simpleDedupWithoutSort handles queries without ordering.
+func (ps *propertyServer) simpleDedupWithoutSort(
+ nodeProperties map[string][]*propertyWithMetadata,
+) []*propertyWithCount {
+ // Collect and deduplicate
+ seenIDs := make(map[string]*propertyWithCount)
+
+ for n, props := range nodeProperties {
+ for _, p := range props {
+ entity := propertypkg.GetEntity(p.Property)
+
+ if existing, seen := seenIDs[entity]; seen {
+ switch {
+ case existing.Metadata.ModRevision <
p.Metadata.ModRevision:
+ seenIDs[entity] =
newPropertyWithCounts(p, entity, n)
+ case existing.Metadata.ModRevision ==
p.Metadata.ModRevision:
+ existing.addExistNode(n)
+ }
+ } else {
+ seenIDs[entity] = newPropertyWithCounts(p,
entity, n)
+ }
+ }
+ }
+
+ // Process and collect results
+ result := make([]*propertyWithCount, 0, len(seenIDs))
+ for _, p := range seenIDs {
+ result = append(result, p)
+ }
+
+ return result
}
func (ps *propertyServer) repairPropertyIfNeed(entity string, p
*propertyWithCount, groups map[string]*commonv1.Group) error {
@@ -576,16 +770,24 @@ func (ps *propertyServer) queryProperties(
for i, s := range v.Sources {
var p propertyv1.Property
var deleteTime int64
- err = protojson.Unmarshal(s, &p)
- if err != nil {
- return nil, groups, trace, err
+ unmarshalErr := protojson.Unmarshal(s,
&p)
+ if unmarshalErr != nil {
+ return nil, groups, trace,
unmarshalErr
}
if i < len(v.Deletes) {
deleteTime = v.Deletes[i]
}
+
+ var sortedValue []byte
+ if i < len(v.SortedValues) {
+ sortedValue = v.SortedValues[i]
+ }
+
property := &propertyWithMetadata{
Property: &p,
+ sortedValue: sortedValue,
deletedTime: deleteTime,
+ node: m.Node(),
}
nodeWithProperties =
append(nodeWithProperties, property)
}
@@ -629,17 +831,55 @@ func (ps *propertyServer) startRepairQueue(stop chan
struct{}) {
type propertyWithMetadata struct {
*propertyv1.Property
+ node string
+ sortedValue []byte
deletedTime int64
}
+// SortedField implements sortpkg.Comparable interface for k-way merge sorting.
+func (p *propertyWithMetadata) SortedField() []byte {
+ return p.sortedValue
+}
+
+// propertyWithMetadataIterator wraps a slice to implement sortpkg.Iterator
interface.
+type propertyWithMetadataIterator struct {
+ data []*propertyWithMetadata
+ index int
+}
+
+func newPropertyWithMetadataIterator(data []*propertyWithMetadata)
*propertyWithMetadataIterator {
+ return &propertyWithMetadataIterator{
+ data: data,
+ index: -1,
+ }
+}
+
+func (it *propertyWithMetadataIterator) Next() bool {
+ it.index++
+ return it.index < len(it.data)
+}
+
+func (it *propertyWithMetadataIterator) Val() *propertyWithMetadata {
+ if it.index < 0 || it.index >= len(it.data) {
+ return nil
+ }
+ return it.data[it.index]
+}
+
+func (it *propertyWithMetadataIterator) Close() error {
+ return nil
+}
+
type propertyWithCount struct {
*propertyWithMetadata
existNodes map[string]bool
+ entity string
}
-func newPropertyWithCounts(p *propertyWithMetadata, existNode string)
*propertyWithCount {
+func newPropertyWithCounts(p *propertyWithMetadata, entity, existNode string)
*propertyWithCount {
res := &propertyWithCount{
propertyWithMetadata: p,
+ entity: entity,
}
res.addExistNode(existNode)
return res
diff --git a/banyand/property/db.go b/banyand/property/db.go
index ea5475ab..93232345 100644
--- a/banyand/property/db.go
+++ b/banyand/property/db.go
@@ -30,6 +30,7 @@ import (
"go.uber.org/multierr"
"github.com/apache/skywalking-banyandb/api/common"
+ modelv1
"github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v1"
propertyv1
"github.com/apache/skywalking-banyandb/api/proto/banyandb/property/v1"
"github.com/apache/skywalking-banyandb/banyand/internal/storage"
"github.com/apache/skywalking-banyandb/banyand/metadata"
@@ -37,6 +38,7 @@ import (
"github.com/apache/skywalking-banyandb/banyand/property/gossip"
"github.com/apache/skywalking-banyandb/pkg/fs"
"github.com/apache/skywalking-banyandb/pkg/index/inverted"
+ "github.com/apache/skywalking-banyandb/pkg/iter/sort"
"github.com/apache/skywalking-banyandb/pkg/logger"
)
@@ -169,15 +171,48 @@ func (db *database) query(ctx context.Context, req
*propertyv1.QueryRequest) ([]
if sLst == nil {
return nil, nil
}
- var res []*queryProperty
+
+ if req.OrderBy == nil {
+ var res []*queryProperty
+ for _, s := range *sLst {
+ r, searchErr := s.search(ctx, iq, nil, int(req.Limit))
+ if searchErr != nil {
+ return nil, searchErr
+ }
+ res = append(res, r...)
+ }
+ return res, nil
+ }
+
+ iters := make([]sort.Iterator[*queryProperty], 0, len(*sLst))
for _, s := range *sLst {
- r, err := s.search(ctx, iq, int(req.Limit))
- if err != nil {
- return nil, err
+ // Each shard returns pre-sorted results (via SeriesSort)
+ r, searchErr := s.search(ctx, iq, req.OrderBy, int(req.Limit))
+ if searchErr != nil {
+ return nil, searchErr
+ }
+ if len(r) > 0 {
+ // Wrap result slice as iterator and add to merge
+ iters = append(iters, newQueryPropertyIterator(r))
}
- res = append(res, r...)
}
- return res, nil
+
+ if len(iters) == 0 {
+ return nil, nil
+ }
+
+ // K-way merge
+ isDesc := req.OrderBy.Sort == modelv1.Sort_SORT_DESC
+ mergeIter := sort.NewItemIter(iters, isDesc)
+ defer mergeIter.Close()
+
+ // Collect merged results up to limit
+ result := make([]*queryProperty, 0, req.Limit)
+ for mergeIter.Next() {
+ result = append(result, mergeIter.Val())
+ }
+
+ return result, nil
}
func (db *database) loadShard(ctx context.Context, id common.ShardID) (*shard,
error) {
@@ -276,8 +311,43 @@ func walkDir(root, prefix string, wf walkFn) error {
}
type queryProperty struct {
- id []byte
- source []byte
- timestamp int64
- deleteTime int64
+ id []byte
+ source []byte
+ sortedValue []byte
+ timestamp int64
+ deleteTime int64
+}
+
+// SortedField implements sort.Comparable interface for k-way merge sorting.
+func (q *queryProperty) SortedField() []byte {
+ return q.sortedValue
+}
+
+// queryPropertyIterator wraps a slice of queryProperty to implement
sort.Iterator interface.
+type queryPropertyIterator struct {
+ data []*queryProperty
+ index int
+}
+
+func newQueryPropertyIterator(data []*queryProperty) *queryPropertyIterator {
+ return &queryPropertyIterator{
+ data: data,
+ index: -1,
+ }
+}
+
+func (it *queryPropertyIterator) Next() bool {
+ it.index++
+ return it.index < len(it.data)
+}
+
+func (it *queryPropertyIterator) Val() *queryProperty {
+ if it.index < 0 || it.index >= len(it.data) {
+ return nil
+ }
+ return it.data[it.index]
+}
+
+func (it *queryPropertyIterator) Close() error {
+ return nil
}
diff --git a/banyand/property/listener.go b/banyand/property/listener.go
index 50e6dfe8..32b93a13 100644
--- a/banyand/property/listener.go
+++ b/banyand/property/listener.go
@@ -197,6 +197,7 @@ func (h *queryListener) Rev(ctx context.Context, message
bus.Message) (resp bus.
for _, p := range properties {
qResp.Sources = append(qResp.Sources, p.source)
qResp.Deletes = append(qResp.Deletes, p.deleteTime)
+ qResp.SortedValues = append(qResp.SortedValues, p.sortedValue)
}
if tracer != nil {
qResp.Trace = tracer.ToProto()
diff --git a/banyand/property/repair_gossip.go
b/banyand/property/repair_gossip.go
index 5babb981..e48d0f56 100644
--- a/banyand/property/repair_gossip.go
+++ b/banyand/property/repair_gossip.go
@@ -166,7 +166,7 @@ func (b *repairGossipBase) queryProperty(ctx
context.Context, syncShard *shard,
if err != nil {
return nil, nil, fmt.Errorf("failed to build query from leaf
node entity %s: %w", leafNodeEntity, err)
}
- queriedProperties, err := syncShard.search(ctx, searchQuery,
gossipShardQueryDatabaseSize)
+ queriedProperties, err := syncShard.search(ctx, searchQuery, nil,
gossipShardQueryDatabaseSize)
if err != nil {
return nil, nil, fmt.Errorf("failed to search properties for
leaf node entity %s: %w", leafNodeEntity, err)
}
diff --git a/banyand/property/repair_gossip_test.go
b/banyand/property/repair_gossip_test.go
index ccab8075..d9923a50 100644
--- a/banyand/property/repair_gossip_test.go
+++ b/banyand/property/repair_gossip_test.go
@@ -507,7 +507,7 @@ func queryPropertyWithVerify(db *database, p property) {
gomega.Expect(err).NotTo(gomega.HaveOccurred())
gomega.Eventually(func() *property {
- dataList, err := s.search(context.Background(), query, 10)
+ dataList, err := s.search(context.Background(), query, nil, 10)
if err != nil {
return nil
}
diff --git a/banyand/property/shard.go b/banyand/property/shard.go
index f8ef9fe3..3568ffd8 100644
--- a/banyand/property/shard.go
+++ b/banyand/property/shard.go
@@ -18,6 +18,7 @@
package property
import (
+ "bytes"
"context"
"fmt"
"path"
@@ -28,9 +29,12 @@ import (
"github.com/RoaringBitmap/roaring"
segment "github.com/blugelabs/bluge_segment_api"
+ "go.uber.org/multierr"
"google.golang.org/protobuf/encoding/protojson"
"github.com/apache/skywalking-banyandb/api/common"
+ commonv1
"github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1"
+ databasev1
"github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1"
propertyv1
"github.com/apache/skywalking-banyandb/api/proto/banyandb/property/v1"
"github.com/apache/skywalking-banyandb/pkg/convert"
"github.com/apache/skywalking-banyandb/pkg/index"
@@ -152,7 +156,7 @@ func (s *shard) buildUpdateDocument(id []byte, property
*propertyv1.Property, de
}
tagField := index.NewBytesField(index.FieldKey{IndexRuleID:
uint32(convert.HashStr(t.Key))}, tv)
tagField.Index = true
- tagField.NoSort = true
+ tagField.NoSort = false
doc.Fields = append(doc.Fields, tagField)
}
@@ -202,7 +206,7 @@ func (s *shard) buildDeleteFromTimeDocuments(ctx
context.Context, docID [][]byte
if err != nil {
return nil, fmt.Errorf("build property query failure: %w", err)
}
- exisingDocList, err := s.search(ctx, iq, len(docID))
+ exisingDocList, err := s.search(ctx, iq, nil, len(docID))
if err != nil {
return nil, fmt.Errorf("search existing documents failure: %w",
err)
}
@@ -250,13 +254,16 @@ func (s *shard) updateDocuments(docs index.Documents)
error {
return nil
}
-func (s *shard) search(ctx context.Context, indexQuery index.Query, limit int,
+func (s *shard) search(ctx context.Context, q index.Query, orderBy
*propertyv1.QueryOrder, limit int,
) (data []*queryProperty, err error) {
tracer := query.GetTracer(ctx)
if tracer != nil {
span, _ := tracer.StartSpan(ctx, "property.search")
- span.Tagf("query", "%s", indexQuery.String())
+ span.Tagf("query", "%s", q.String())
span.Tagf("shard", "%d", s.id)
+ if orderBy != nil {
+ span.Tagf("order", "%s(%s)", orderBy.TagName,
orderBy.Sort)
+ }
defer func() {
if data != nil {
span.Tagf("matched", "%d", len(data))
@@ -267,25 +274,67 @@ func (s *shard) search(ctx context.Context, indexQuery
index.Query, limit int,
span.Stop()
}()
}
- ss, err := s.store.Search(ctx, projection, indexQuery, limit)
+ if orderBy == nil {
+ ss, searchErr := s.store.Search(ctx, projection, q, limit)
+ if searchErr != nil {
+ return nil, searchErr
+ }
+
+ if len(ss) == 0 {
+ return nil, nil
+ }
+ data = make([]*queryProperty, 0, len(ss))
+ for _, s := range ss {
+ bytes := s.Fields[sourceField]
+ var deleteTime int64
+ if s.Fields[deleteField] != nil {
+ deleteTime =
convert.BytesToInt64(s.Fields[deleteField])
+ }
+ data = append(data, &queryProperty{
+ id: s.Key.EntityValues,
+ timestamp: s.Timestamp,
+ source: bytes,
+ deleteTime: deleteTime,
+ })
+ }
+ return data, nil
+ }
+ order := &index.OrderBy{
+ Index: &databasev1.IndexRule{
+ Metadata: &commonv1.Metadata{
+ Id: uint32(convert.HashStr(orderBy.TagName)),
+ },
+ },
+ Sort: orderBy.Sort,
+ Type: index.OrderByTypeIndex,
+ }
+ iter, err := s.store.SeriesSort(ctx, q, order, limit, projection)
if err != nil {
return nil, err
}
- if len(ss) == 0 {
- return nil, nil
- }
- data = make([]*queryProperty, 0, len(ss))
- for _, s := range ss {
- bytes := s.Fields[sourceField]
+ defer func() {
+ err = multierr.Append(err, iter.Close())
+ }()
+ data = make([]*queryProperty, 0, limit)
+ for iter.Next() {
+ val := iter.Val()
+
var deleteTime int64
- if s.Fields[deleteField] != nil {
- deleteTime = convert.BytesToInt64(s.Fields[deleteField])
+ if val.Values[deleteField] != nil {
+ deleteTime =
convert.BytesToInt64(val.Values[deleteField])
+ }
+
+ var sortedValue []byte
+ if len(val.SortedValue) > 0 {
+ sortedValue = bytes.Clone(val.SortedValue)
}
+
data = append(data, &queryProperty{
- id: s.Key.EntityValues,
- timestamp: s.Timestamp,
- source: bytes,
- deleteTime: deleteTime,
+ id: val.EntityValues,
+ timestamp: val.Timestamp,
+ source: val.Values[sourceField],
+ sortedValue: sortedValue,
+ deleteTime: deleteTime,
})
}
return data, nil
@@ -300,7 +349,7 @@ func (s *shard) repair(ctx context.Context, id []byte,
property *propertyv1.Prop
if err != nil {
return false, nil, fmt.Errorf("build property query failure:
%w", err)
}
- olderProperties, err := s.search(ctx, iq, 100)
+ olderProperties, err := s.search(ctx, iq, nil, 100)
if err != nil {
return false, nil, fmt.Errorf("query older properties failed:
%w", err)
}
diff --git a/banyand/property/shard_test.go b/banyand/property/shard_test.go
index 0a331cb4..1ec4f7ea 100644
--- a/banyand/property/shard_test.go
+++ b/banyand/property/shard_test.go
@@ -374,7 +374,7 @@ func queryShard(ctx context.Context, t *testing.T, s
*shard, id string) []*query
if err != nil {
t.Fatal(err)
}
- resp, err := s.search(ctx, iq, 100)
+ resp, err := s.search(ctx, iq, nil, 100)
if err != nil {
t.Fatal(err)
}
diff --git a/docs/api-reference.md b/docs/api-reference.md
index b8482a3b..b4bc6402 100644
--- a/docs/api-reference.md
+++ b/docs/api-reference.md
@@ -76,6 +76,7 @@
- [InternalRepairRequest](#banyandb-property-v1-InternalRepairRequest)
- [InternalRepairResponse](#banyandb-property-v1-InternalRepairResponse)
- [InternalUpdateRequest](#banyandb-property-v1-InternalUpdateRequest)
+ - [QueryOrder](#banyandb-property-v1-QueryOrder)
- [QueryRequest](#banyandb-property-v1-QueryRequest)
- [QueryResponse](#banyandb-property-v1-QueryResponse)
@@ -1344,6 +1345,7 @@ Property stores the user defined data
| sources | [bytes](#bytes) | repeated | |
| trace | [banyandb.common.v1.Trace](#banyandb-common-v1-Trace) | | |
| deletes | [int64](#int64) | repeated | deletes indicates the property is
deleted timestamps, it's mapping to the sources in the same order if the
value is 0, it means the property is not deleted |
+| sorted_values | [bytes](#bytes) | repeated | sorted_values contains
pre-extracted sort values from shard searches, mapping to sources in the same
order for optimized sorting at liaison layer |
@@ -1395,6 +1397,22 @@ Property stores the user defined data
+<a name="banyandb-property-v1-QueryOrder"></a>
+
+### QueryOrder
+
+
+
+| Field | Type | Label | Description |
+| ----- | ---- | ----- | ----------- |
+| tag_name | [string](#string) | | tag_name is the name of the tag to be
ordered. |
+| sort | [banyandb.model.v1.Sort](#banyandb-model-v1-Sort) | | order_by is
given to specify the sort for a tag. |
+
+
+
+
+
+
<a name="banyandb-property-v1-QueryRequest"></a>
### QueryRequest
@@ -1410,6 +1428,7 @@ QueryRequest is the request contract for query.
| tag_projection | [string](#string) | repeated | tag_projection can be used
to select tags of the data points in the response |
| limit | [uint32](#uint32) | | |
| trace | [bool](#bool) | | trace is used to enable trace for the query |
+| order_by | [QueryOrder](#banyandb-property-v1-QueryOrder) | | order_by is
given to specify the sort for a tag. |
diff --git a/docs/interacting/bydbql.md b/docs/interacting/bydbql.md
index 8a035cdd..9357b08b 100644
--- a/docs/interacting/bydbql.md
+++ b/docs/interacting/bydbql.md
@@ -732,7 +732,7 @@ BydbQL for properties is designed for simple key-value
lookups and metadata filt
### 7.1. Grammar
```
-property_query ::= SELECT projection from_property_clause [WHERE
criteria] [LIMIT integer] [WITH QUERY_TRACE]
+property_query ::= SELECT projection from_property_clause [WHERE
criteria] [LIMIT integer] [ORDER BY value ["ASC"|"DESC"]] [WITH QUERY_TRACE]
from_property_clause ::= "FROM PROPERTY" identifier "IN" ["("] group_list [")"]
projection ::= "*" | column_list
column_list ::= identifier ("," identifier)*
@@ -767,6 +767,13 @@ FROM PROPERTY server_metadata IN datacenter-1
WHERE datacenter = 'dc-101' AND in_service = 'true'
LIMIT 50;
+-- Find properties by filtering on their tags and order by
+SELECT ip, owner
+FROM PROPERTY server_metadata IN datacenter-1
+WHERE datacenter = 'dc-101' AND in_service = 'true'
+LIMIT 50
+ORDER BY ip;
+
-- Retrieve a specific property by its unique ID
SELECT *
FROM PROPERTY server_metadata IN datacenter-1
diff --git a/pkg/bydbql/transformer.go b/pkg/bydbql/transformer.go
index 3e4fab1b..ac6d6bc4 100644
--- a/pkg/bydbql/transformer.go
+++ b/pkg/bydbql/transformer.go
@@ -498,6 +498,18 @@ func (t *Transformer) transformPropertyQuery(ctx
context.Context, grammar *Gramm
limit = uint32(statement.Limit.Value)
}
+ // handle ORDER BY
+ var orderBy *propertyv1.QueryOrder
+ if statement.OrderBy != nil {
+ modelQueryOrder := t.convertSelectOrderBy(statement.OrderBy)
+ if modelQueryOrder != nil {
+ orderBy = &propertyv1.QueryOrder{
+ TagName: modelQueryOrder.IndexRuleName,
+ Sort: modelQueryOrder.Sort,
+ }
+ }
+ }
+
return &TransformResult{
Type: QueryTypeProperty,
Original: grammar,
@@ -508,6 +520,7 @@ func (t *Transformer) transformPropertyQuery(ctx
context.Context, grammar *Gramm
Criteria: criteria,
TagProjection: tagProjection,
Limit: limit,
+ OrderBy: orderBy,
Trace: statement.WithQueryTrace != nil,
},
}, nil
diff --git a/test/cases/property/data/data.go b/test/cases/property/data/data.go
index f6c96bf2..1844e656 100644
--- a/test/cases/property/data/data.go
+++ b/test/cases/property/data/data.go
@@ -90,9 +90,11 @@ func verifyWithContext(ctx context.Context, innerGm
gm.Gomega, sharedContext hel
want := &propertyv1.QueryResponse{}
helpers.UnmarshalYAML(w, want)
innerGm.Expect(resp.GetProperties()).To(gm.HaveLen(len(want.GetProperties())),
query.String())
- slices.SortFunc(want.GetProperties(), func(a, b *propertyv1.Property)
int {
- return strings.Compare(a.Id, b.Id)
- })
+ if query.OrderBy == nil {
+ slices.SortFunc(want.GetProperties(), func(a, b
*propertyv1.Property) int {
+ return strings.Compare(a.Id, b.Id)
+ })
+ }
success := innerGm.Expect(cmp.Equal(resp, want,
protocmp.IgnoreUnknown(),
protocmp.IgnoreFields(&propertyv1.Property{}, "updated_at"),
diff --git a/test/cases/property/data/input/order_by_asc.ql
b/test/cases/property/data/input/order_by_asc.ql
new file mode 100644
index 00000000..fba11e25
--- /dev/null
+++ b/test/cases/property/data/input/order_by_asc.ql
@@ -0,0 +1,19 @@
+# Licensed to Apache Software Foundation (ASF) under one or more contributor
+# license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright
+# ownership. Apache Software Foundation (ASF) licenses this file to you under
+# the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+
+SELECT menu_name, configuration, update_time FROM PROPERTY ui_menu IN sw ORDER
BY update_time ASC
diff --git a/test/cases/property/data/input/order_by_asc.yaml
b/test/cases/property/data/input/order_by_asc.yaml
new file mode 100644
index 00000000..188bb786
--- /dev/null
+++ b/test/cases/property/data/input/order_by_asc.yaml
@@ -0,0 +1,26 @@
+# Licensed to Apache Software Foundation (ASF) under one or more contributor
+# license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright
+# ownership. Apache Software Foundation (ASF) licenses this file to you under
+# the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+name: "ui_menu"
+groups: ["sw"]
+tag_projection:
+ - menu_name
+ - configuration
+ - update_time
+order_by:
+ tag_name: "update_time"
+ sort: SORT_ASC
diff --git a/test/cases/property/data/input/order_by_desc.ql
b/test/cases/property/data/input/order_by_desc.ql
new file mode 100644
index 00000000..7e9e0213
--- /dev/null
+++ b/test/cases/property/data/input/order_by_desc.ql
@@ -0,0 +1,19 @@
+# Licensed to Apache Software Foundation (ASF) under one or more contributor
+# license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright
+# ownership. Apache Software Foundation (ASF) licenses this file to you under
+# the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+
+SELECT menu_name, configuration, update_time FROM PROPERTY ui_menu IN sw ORDER
BY update_time DESC
diff --git a/test/cases/property/data/input/order_by_desc.yaml
b/test/cases/property/data/input/order_by_desc.yaml
new file mode 100644
index 00000000..1cc28a1c
--- /dev/null
+++ b/test/cases/property/data/input/order_by_desc.yaml
@@ -0,0 +1,26 @@
+# Licensed to Apache Software Foundation (ASF) under one or more contributor
+# license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright
+# ownership. Apache Software Foundation (ASF) licenses this file to you under
+# the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+name: "ui_menu"
+groups: ["sw"]
+tag_projection:
+ - menu_name
+ - configuration
+ - update_time
+order_by:
+ tag_name: "update_time"
+ sort: SORT_DESC
diff --git a/test/cases/property/data/input/order_by_with_limit.ql
b/test/cases/property/data/input/order_by_with_limit.ql
new file mode 100644
index 00000000..86746d3a
--- /dev/null
+++ b/test/cases/property/data/input/order_by_with_limit.ql
@@ -0,0 +1,19 @@
+# Licensed to Apache Software Foundation (ASF) under one or more contributor
+# license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright
+# ownership. Apache Software Foundation (ASF) licenses this file to you under
+# the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+
+SELECT menu_name, configuration, update_time FROM PROPERTY ui_menu IN sw ORDER
BY update_time DESC LIMIT 1
diff --git a/test/cases/property/data/input/order_by_with_limit.yaml
b/test/cases/property/data/input/order_by_with_limit.yaml
new file mode 100644
index 00000000..e197257e
--- /dev/null
+++ b/test/cases/property/data/input/order_by_with_limit.yaml
@@ -0,0 +1,27 @@
+# Licensed to Apache Software Foundation (ASF) under one or more contributor
+# license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright
+# ownership. Apache Software Foundation (ASF) licenses this file to you under
+# the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+name: "ui_menu"
+groups: ["sw"]
+tag_projection:
+ - menu_name
+ - configuration
+ - update_time
+limit: 1
+order_by:
+ tag_name: "update_time"
+ sort: SORT_DESC
diff --git a/test/cases/property/data/input/order_without_projection.ql
b/test/cases/property/data/input/order_without_projection.ql
new file mode 100644
index 00000000..b8dfb562
--- /dev/null
+++ b/test/cases/property/data/input/order_without_projection.ql
@@ -0,0 +1,19 @@
+# Licensed to Apache Software Foundation (ASF) under one or more contributor
+# license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright
+# ownership. Apache Software Foundation (ASF) licenses this file to you under
+# the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+
+SELECT menu_name, configuration FROM PROPERTY ui_menu IN sw ORDER BY
update_time ASC
diff --git a/test/cases/property/data/input/order_without_projection.yaml
b/test/cases/property/data/input/order_without_projection.yaml
new file mode 100644
index 00000000..fc659a88
--- /dev/null
+++ b/test/cases/property/data/input/order_without_projection.yaml
@@ -0,0 +1,25 @@
+# Licensed to Apache Software Foundation (ASF) under one or more contributor
+# license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright
+# ownership. Apache Software Foundation (ASF) licenses this file to you under
+# the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+name: "ui_menu"
+groups: ["sw"]
+tag_projection:
+ - menu_name
+ - configuration
+order_by:
+ tag_name: "update_time"
+ sort: SORT_ASC
diff --git a/test/cases/property/data/input/query_with_order.ql
b/test/cases/property/data/input/query_with_order.ql
new file mode 100644
index 00000000..f384ba66
--- /dev/null
+++ b/test/cases/property/data/input/query_with_order.ql
@@ -0,0 +1,21 @@
+# Licensed to Apache Software Foundation (ASF) under one or more contributor
+# license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright
+# ownership. Apache Software Foundation (ASF) licenses this file to you under
+# the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+
+SELECT menu_name, configuration, update_time FROM PROPERTY ui_menu IN sw
+WHERE menu_name = 'test2'
+ORDER BY update_time ASC
diff --git a/test/cases/property/data/input/query_with_order.yaml
b/test/cases/property/data/input/query_with_order.yaml
new file mode 100644
index 00000000..9140793a
--- /dev/null
+++ b/test/cases/property/data/input/query_with_order.yaml
@@ -0,0 +1,33 @@
+# Licensed to Apache Software Foundation (ASF) under one or more contributor
+# license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright
+# ownership. Apache Software Foundation (ASF) licenses this file to you under
+# the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+name: "ui_menu"
+groups: ["sw"]
+tag_projection:
+ - menu_name
+ - configuration
+ - update_time
+criteria:
+ condition:
+ name: "menu_name"
+ op: "BINARY_OP_EQ"
+ value:
+ str:
+ value: "test2"
+order_by:
+ tag_name: "update_time"
+ sort: SORT_ASC
diff --git a/test/cases/property/data/want/order_by_asc.yaml
b/test/cases/property/data/want/order_by_asc.yaml
new file mode 100644
index 00000000..1b43f19e
--- /dev/null
+++ b/test/cases/property/data/want/order_by_asc.yaml
@@ -0,0 +1,52 @@
+# Licensed to Apache Software Foundation (ASF) under one or more contributor
+# license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright
+# ownership. Apache Software Foundation (ASF) licenses this file to you under
+# the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+properties:
+ - id: "1"
+ metadata:
+ group: sw
+ name: ui_menu
+ tags:
+ - key: menu_name
+ value:
+ str:
+ value: test1
+ - key: configuration
+ value:
+ str:
+ value: test1
+ - key: update_time
+ value:
+ int:
+ value: "1760606198934"
+ - id: "2"
+ metadata:
+ group: sw
+ name: ui_menu
+ tags:
+ - key: menu_name
+ value:
+ str:
+ value: test2
+ - key: configuration
+ value:
+ str:
+ value: test2
+ - key: update_time
+ value:
+ int:
+ value: "1760606208842"
diff --git a/test/cases/property/data/want/order_by_desc.yaml
b/test/cases/property/data/want/order_by_desc.yaml
new file mode 100644
index 00000000..060a5e1e
--- /dev/null
+++ b/test/cases/property/data/want/order_by_desc.yaml
@@ -0,0 +1,52 @@
+# Licensed to Apache Software Foundation (ASF) under one or more contributor
+# license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright
+# ownership. Apache Software Foundation (ASF) licenses this file to you under
+# the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+properties:
+ - id: "2"
+ metadata:
+ group: sw
+ name: ui_menu
+ tags:
+ - key: menu_name
+ value:
+ str:
+ value: test2
+ - key: configuration
+ value:
+ str:
+ value: test2
+ - key: update_time
+ value:
+ int:
+ value: "1760606208842"
+ - id: "1"
+ metadata:
+ group: sw
+ name: ui_menu
+ tags:
+ - key: menu_name
+ value:
+ str:
+ value: test1
+ - key: configuration
+ value:
+ str:
+ value: test1
+ - key: update_time
+ value:
+ int:
+ value: "1760606198934"
diff --git a/test/cases/property/data/want/order_by_with_limit.yaml
b/test/cases/property/data/want/order_by_with_limit.yaml
new file mode 100644
index 00000000..ae4577a7
--- /dev/null
+++ b/test/cases/property/data/want/order_by_with_limit.yaml
@@ -0,0 +1,35 @@
+# Licensed to Apache Software Foundation (ASF) under one or more contributor
+# license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright
+# ownership. Apache Software Foundation (ASF) licenses this file to you under
+# the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+properties:
+ - id: "2"
+ metadata:
+ group: sw
+ name: ui_menu
+ tags:
+ - key: menu_name
+ value:
+ str:
+ value: test2
+ - key: configuration
+ value:
+ str:
+ value: test2
+ - key: update_time
+ value:
+ int:
+ value: "1760606208842"
diff --git a/test/cases/property/data/want/order_without_projection.yaml
b/test/cases/property/data/want/order_without_projection.yaml
new file mode 100644
index 00000000..e1d29ee6
--- /dev/null
+++ b/test/cases/property/data/want/order_without_projection.yaml
@@ -0,0 +1,44 @@
+# Licensed to Apache Software Foundation (ASF) under one or more contributor
+# license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright
+# ownership. Apache Software Foundation (ASF) licenses this file to you under
+# the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+properties:
+ - id: "1"
+ metadata:
+ group: sw
+ name: ui_menu
+ tags:
+ - key: menu_name
+ value:
+ str:
+ value: test1
+ - key: configuration
+ value:
+ str:
+ value: test1
+ - id: "2"
+ metadata:
+ group: sw
+ name: ui_menu
+ tags:
+ - key: menu_name
+ value:
+ str:
+ value: test2
+ - key: configuration
+ value:
+ str:
+ value: test2
diff --git a/test/cases/property/data/want/query_with_order.yaml
b/test/cases/property/data/want/query_with_order.yaml
new file mode 100644
index 00000000..ae4577a7
--- /dev/null
+++ b/test/cases/property/data/want/query_with_order.yaml
@@ -0,0 +1,35 @@
+# Licensed to Apache Software Foundation (ASF) under one or more contributor
+# license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright
+# ownership. Apache Software Foundation (ASF) licenses this file to you under
+# the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+properties:
+ - id: "2"
+ metadata:
+ group: sw
+ name: ui_menu
+ tags:
+ - key: menu_name
+ value:
+ str:
+ value: test2
+ - key: configuration
+ value:
+ str:
+ value: test2
+ - key: update_time
+ value:
+ int:
+ value: "1760606208842"
diff --git a/test/cases/property/property.go b/test/cases/property/property.go
index 41c72bbb..bb131155 100644
--- a/test/cases/property/property.go
+++ b/test/cases/property/property.go
@@ -42,4 +42,9 @@ var _ = g.DescribeTable("Scanning Properties", verify,
g.Entry("limit", helpers.Args{Input: "limit"}),
g.Entry("query by criteria", helpers.Args{Input: "query_by_criteria"}),
g.Entry("query by ids", helpers.Args{Input: "query_by_ids"}),
+ g.Entry("order by asc", helpers.Args{Input: "order_by_asc"}),
+ g.Entry("order by desc", helpers.Args{Input: "order_by_desc"}),
+ g.Entry("order by with limit", helpers.Args{Input:
"order_by_with_limit"}),
+ g.Entry("order without projection", helpers.Args{Input:
"order_without_projection"}),
+ g.Entry("query with order", helpers.Args{Input: "query_with_order"}),
)