hanahmily commented on code in PR #955:
URL: 
https://github.com/apache/skywalking-banyandb/pull/955#discussion_r2715192113


##########
banyand/property/db.go:
##########
@@ -216,34 +248,87 @@ func (db *database) query(ctx context.Context, req 
*propertyv1.QueryRequest) ([]
        return result, nil
 }
 
-func (db *database) loadShard(ctx context.Context, id common.ShardID) (*shard, 
error) {
+func (db *database) collectGroupShards(groupsMap *map[string]*groupShards, 
requestedGroups map[string]bool) []*shard {
+       var shards []*shard
+       for groupName, gs := range *groupsMap {
+               if len(requestedGroups) > 0 {
+                       if _, ok := requestedGroups[groupName]; !ok {
+                               continue
+                       }
+               }
+               sLst := gs.shards.Load()
+               if sLst == nil {
+                       continue
+               }
+               shards = append(shards, *sLst...)
+       }
+       return shards
+}
+
+func (db *database) loadShard(ctx context.Context, group string, id 
common.ShardID) (*shard, error) {
        if db.closed.Load() {
                return nil, errors.New("database is closed")
        }
-       if s, ok := db.getShard(id); ok {
+       if s, ok := db.getShard(group, id); ok {
                return s, nil
        }
        db.mu.Lock()
        defer db.mu.Unlock()
-       if s, ok := db.getShard(id); ok {
+       if s, ok := db.getShard(group, id); ok {
                return s, nil
        }
-       sd, err := db.newShard(context.WithValue(ctx, logger.ContextKey, 
db.logger), id, int64(db.flushInterval.Seconds()),
+
+       gs := db.getOrCreateGroupShards(group)
+       sd, err := db.newShard(context.WithValue(ctx, logger.ContextKey, 
db.logger),
+               group, id, int64(db.flushInterval.Seconds()),
                int64(db.expireDelete.Seconds()), db.repairBaseDir, 
db.repairTreeSlotCount)
        if err != nil {
                return nil, err
        }
-       sLst := db.sLst.Load()
+
+       gs.mu.Lock()
+       sLst := gs.shards.Load()
        if sLst == nil {
                sLst = &[]*shard{}
        }
-       *sLst = append(*sLst, sd)
-       db.sLst.Store(sLst)
+       newList := append(*sLst, sd)
+       gs.shards.Store(&newList)
+       gs.mu.Unlock()
        return sd, nil
 }
 
-func (db *database) getShard(id common.ShardID) (*shard, bool) {
-       sLst := db.sLst.Load()
+func (db *database) getOrCreateGroupShards(group string) *groupShards {
+       groupsMap := db.groups.Load()
+       if groupsMap != nil {
+               if gs, ok := (*groupsMap)[group]; ok {
+                       return gs
+               }
+       }
+       gs := &groupShards{
+               group:    group,
+               location: filepath.Join(db.location, group),
+       }
+       if groupsMap == nil {
+               newMap := make(map[string]*groupShards)
+               newMap[group] = gs
+               db.groups.Store(&newMap)
+       } else {
+               (*groupsMap)[group] = gs
+               db.groups.Store(groupsMap)

Review Comment:
   Use sync.Map to fix it.



##########
banyand/property/db.go:
##########
@@ -216,34 +248,87 @@ func (db *database) query(ctx context.Context, req 
*propertyv1.QueryRequest) ([]
        return result, nil
 }
 
-func (db *database) loadShard(ctx context.Context, id common.ShardID) (*shard, 
error) {
+func (db *database) collectGroupShards(groupsMap *map[string]*groupShards, 
requestedGroups map[string]bool) []*shard {
+       var shards []*shard
+       for groupName, gs := range *groupsMap {
+               if len(requestedGroups) > 0 {
+                       if _, ok := requestedGroups[groupName]; !ok {
+                               continue
+                       }
+               }
+               sLst := gs.shards.Load()
+               if sLst == nil {
+                       continue
+               }
+               shards = append(shards, *sLst...)
+       }
+       return shards
+}
+
+func (db *database) loadShard(ctx context.Context, group string, id 
common.ShardID) (*shard, error) {
        if db.closed.Load() {
                return nil, errors.New("database is closed")
        }
-       if s, ok := db.getShard(id); ok {
+       if s, ok := db.getShard(group, id); ok {
                return s, nil
        }
        db.mu.Lock()
        defer db.mu.Unlock()
-       if s, ok := db.getShard(id); ok {
+       if s, ok := db.getShard(group, id); ok {
                return s, nil
        }
-       sd, err := db.newShard(context.WithValue(ctx, logger.ContextKey, 
db.logger), id, int64(db.flushInterval.Seconds()),
+
+       gs := db.getOrCreateGroupShards(group)
+       sd, err := db.newShard(context.WithValue(ctx, logger.ContextKey, 
db.logger),
+               group, id, int64(db.flushInterval.Seconds()),
                int64(db.expireDelete.Seconds()), db.repairBaseDir, 
db.repairTreeSlotCount)
        if err != nil {
                return nil, err
        }
-       sLst := db.sLst.Load()
+
+       gs.mu.Lock()
+       sLst := gs.shards.Load()
        if sLst == nil {
                sLst = &[]*shard{}
        }
-       *sLst = append(*sLst, sd)
-       db.sLst.Store(sLst)
+       newList := append(*sLst, sd)
+       gs.shards.Store(&newList)

Review Comment:
   fix it.



##########
banyand/property/db.go:
##########
@@ -52,14 +52,21 @@ var (
        propertyScope = observability.RootScope.SubScope("property")
 )
 
+type groupShards struct {
+       shards   atomic.Pointer[[]*shard]
+       group    string
+       location string
+       mu       sync.RWMutex
+}
+
 type database struct {
        metadata            metadata.Repo
        omr                 observability.MetricsRegistry
        lfs                 fs.FileSystem
        lock                fs.File
        logger              *logger.Logger
        repairScheduler     *repairScheduler
-       sLst                atomic.Pointer[[]*shard]
+       groups              atomic.Pointer[map[string]*groupShards]

Review Comment:
   An atomic pointer cannot guarantee the safe operation on the map. Use 
sync.Map instead. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to