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&#39;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"}),
 )

Reply via email to