You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by GitBox <gi...@apache.org> on 2021/09/07 11:30:50 UTC

[GitHub] [skywalking-banyandb] hanahmily opened a new pull request #45: Introduce stream and tsdb module

hanahmily opened a new pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45


   The changes intend to improve the index subsystem by 
   
    * Introduce the series index to improve the space utilization
    * Keep the block attached index to solve the high cardinality of the entity.
   
   At the same time, some notions and modules are introduced to replace the old ones:
   
    * `trace`(`log`) + `storage` -> `stream` + `tsdb`
    * Move index into `pkg`.
    * Update API to `v2`
   
   @lujiajing1126 Please using the new modules and APIs to refactor the query module.
   
   Once the transferring is done, we will remove old modules that are `trace`, `storage`, and `index`.
   


-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-banyandb] wu-sheng merged pull request #45: Introduce stream and tsdb module

Posted by GitBox <gi...@apache.org>.
wu-sheng merged pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45


   


-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-banyandb] hanahmily commented on a change in pull request #45: Introduce stream and tsdb module

Posted by GitBox <gi...@apache.org>.
hanahmily commented on a change in pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45#discussion_r704331227



##########
File path: banyand/stream/index.go
##########
@@ -0,0 +1,121 @@
+// 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.
+
+package stream
+
+import (
+	"io"
+
+	"go.uber.org/multierr"
+
+	"github.com/apache/skywalking-banyandb/api/common"
+	databasev2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v2"
+	streamv2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/stream/v2"
+	"github.com/apache/skywalking-banyandb/banyand/tsdb"
+	"github.com/apache/skywalking-banyandb/pkg/index"
+	"github.com/apache/skywalking-banyandb/pkg/partition"
+)
+
+type indexMessage struct {
+	localWriter tsdb.Writer
+	blockCloser io.Closer
+	value       *streamv2.ElementValue
+}
+
+func (s *stream) bootIndexGenerator() {
+	go func() {
+		for {
+			m, more := <-s.indexCh
+			if !more {
+				return
+			}
+			var err error
+			for _, ruleIndex := range s.indexRuleIndex {
+				rule := ruleIndex.rule
+				switch rule.GetLocation() {
+				case databasev2.IndexRule_LOCATION_SERIES:
+					err = multierr.Append(err, writeLocalIndex(m.localWriter, ruleIndex, m.value))
+				case databasev2.IndexRule_LOCATION_GLOBAL:
+					err = multierr.Append(err, s.writeGlobalIndex(ruleIndex, m.localWriter.ItemID(), m.value))
+				}
+			}
+			err = multierr.Append(err, m.blockCloser.Close())
+			if err != nil {
+				s.l.Error().Err(err).Msg("encounter some errors when generating indices")
+			}
+		}
+	}()
+}
+
+//TODO: should listen to pipeline in a distributed cluster
+func (s *stream) writeGlobalIndex(ruleIndex indexRule, ref tsdb.GlobalItemID, value *streamv2.ElementValue) error {
+	val, err := getIndexValue(ruleIndex, value)
+	if err != nil {
+		return err
+	}
+	indexShardID, err := partition.ShardID(val, s.schema.ShardNum)

Review comment:
       The query result of a global index might refer to several locations across several shards. That's why we call it the "global" index.
   
   




-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-banyandb] hanahmily commented on a change in pull request #45: Introduce stream and tsdb module

Posted by GitBox <gi...@apache.org>.
hanahmily commented on a change in pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45#discussion_r703986681



##########
File path: banyand/metadata/schema/data/index_rule_binding.json
##########
@@ -0,0 +1,25 @@
+{

Review comment:
       `json` is supported by IDE widely and properly. That means we can edit these metadata more easily than `prototext`.
   
   BTW, once `metadata` management is ready, we will remove these files finally. 




-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-banyandb] lujiajing1126 commented on pull request #45: Introduce stream and tsdb module

Posted by GitBox <gi...@apache.org>.
lujiajing1126 commented on pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45#issuecomment-915967171


   > @lujiajing1126 Do you have some thoughts about these changes? I want to merge it into the main branch since it's changed a lot.
   
   I have no problem for this PR.
   
   I've roughly reviewed these changes this Tuesday and Wednesday. I will read the tsdb module these days.
   
   I will setup a meeting next Monday to discuss some more details.


-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-banyandb] lujiajing1126 commented on a change in pull request #45: Introduce stream and tsdb module

Posted by GitBox <gi...@apache.org>.
lujiajing1126 commented on a change in pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45#discussion_r704232470



##########
File path: banyand/metadata/schema/data/index_rule_binding.json
##########
@@ -0,0 +1,25 @@
+{

Review comment:
       Make sense to me




-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-banyandb] lujiajing1126 commented on a change in pull request #45: Introduce stream and tsdb module

Posted by GitBox <gi...@apache.org>.
lujiajing1126 commented on a change in pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45#discussion_r704173979



##########
File path: banyand/kv/kv.go
##########
@@ -206,9 +229,8 @@ func OpenIndexStore(shardID int, path string, options ...IndexOptions) (IndexSto
 	for _, opt := range options {
 		opt(bdb)
 	}
-	bdb.dbOpts = bdb.dbOpts.WithMaxLevels(1)
-	// Put all values into LSM
-	bdb.dbOpts = bdb.dbOpts.WithVLogPercentile(1.0)
+	bdb.dbOpts = bdb.dbOpts.WithMaxLevels(2)

Review comment:
       why we have to modify the default level?

##########
File path: banyand/stream/stream_query_test.go
##########
@@ -0,0 +1,532 @@
+// 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.
+
+package stream
+
+import (
+	"bytes"
+	"embed"
+	_ "embed"
+	"encoding/base64"
+	"encoding/json"
+	"fmt"
+	"io"
+	"sort"
+	"strconv"
+	"testing"
+	"time"
+
+	"github.com/golang/protobuf/jsonpb"
+	"github.com/pkg/errors"
+	"github.com/stretchr/testify/assert"
+	"google.golang.org/protobuf/types/known/timestamppb"
+
+	"github.com/apache/skywalking-banyandb/api/common"
+	commonv2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v2"
+	databasev2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v2"
+	modelv2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v2"
+	streamv2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/stream/v2"
+	"github.com/apache/skywalking-banyandb/banyand/tsdb"
+	"github.com/apache/skywalking-banyandb/pkg/convert"
+	"github.com/apache/skywalking-banyandb/pkg/index"
+	"github.com/apache/skywalking-banyandb/pkg/partition"
+)
+
+type shardStruct struct {
+	id       common.ShardID
+	location []string
+	elements []string
+}
+
+type shardsForTest []shardStruct
+
+func Test_Stream_SelectShard(t *testing.T) {
+	tester := assert.New(t)
+	s, deferFunc := setup(tester)
+	defer deferFunc()
+	_ = setupQueryData(tester, "multiple_shards.json", s)
+	tests := []struct {
+		name         string
+		entity       tsdb.Entity
+		wantShardNum int
+		wantErr      bool
+	}{
+		{
+			name:         "all shards",
+			wantShardNum: 2,
+		},
+		{
+			name:         "select a shard",
+			entity:       tsdb.Entity{tsdb.Entry("webapp_id"), tsdb.Entry("10.0.0.1_id"), convert.Int64ToBytes(0)},
+			wantShardNum: 1,
+		},
+		{
+			name:         "select shards",
+			entity:       tsdb.Entity{tsdb.Entry("webapp_id"), tsdb.AnyEntry, convert.Int64ToBytes(0)},
+			wantShardNum: 2,
+		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			shards, err := s.Shards(tt.entity)
+			if tt.wantErr {
+				tester.Error(err)
+				return
+			}
+			tester.NoError(err)
+			tester.Equal(tt.wantShardNum, len(shards))
+		})
+	}
+
+}
+
+func Test_Stream_Series(t *testing.T) {
+	tester := assert.New(t)
+	s, deferFunc := setup(tester)
+	defer deferFunc()
+	baseTime := setupQueryData(tester, "multiple_shards.json", s)
+	tests := []struct {
+		name    string
+		args    queryOpts
+		want    shardsForTest
+		wantErr bool
+	}{
+		{
+			name: "all",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.AnyEntry, tsdb.AnyEntry, tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+			},
+			want: shardsForTest{
+				{
+					id:       0,
+					location: []string{"series_12243341348514563931", "data_flow_0"},
+					elements: []string{"1"},
+				},
+				{
+					id:       0,
+					location: []string{"series_1671844747554927007", "data_flow_0"},
+					elements: []string{"2"},
+				},
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"5", "3"},
+				},
+				{
+					id:       1,
+					location: []string{"series_8429137420168685297", "data_flow_0"},
+					elements: []string{"4"},
+				},
+			},
+		},
+
+		{
+			name: "time range",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.AnyEntry, tsdb.AnyEntry, tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime.Add(1500*time.Millisecond), 1*time.Hour),
+			},
+			want: shardsForTest{
+				{
+					id:       0,
+					location: []string{"series_12243341348514563931", "data_flow_0"},
+				},
+				{
+					id:       0,
+					location: []string{"series_1671844747554927007", "data_flow_0"},
+				},
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"5"},
+				},
+				{
+					id:       1,
+					location: []string{"series_8429137420168685297", "data_flow_0"},
+					elements: []string{"4"},
+				},
+			},
+		},
+		{
+			name: "find series by service_id and instance_id",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.Entry("webapp_id"), tsdb.Entry("10.0.0.1_id"), tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+			},
+			want: shardsForTest{
+				{
+					id:       0,
+					location: []string{"series_12243341348514563931", "data_flow_0"},
+					elements: []string{"1"},
+				},
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"5", "3"},
+				},
+			},
+		},
+		{
+			name: "find a series",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.Entry("webapp_id"), tsdb.Entry("10.0.0.1_id"), convert.Uint64ToBytes(1)},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+			},
+			want: shardsForTest{
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"5", "3"},
+				},
+			},
+		},
+		{
+			name: "filter",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.AnyEntry, tsdb.AnyEntry, tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+				buildFn: func(builder tsdb.SeekerBuilder) {

Review comment:
       So the searchable tags (local indexes) can be directly processed by the storage layer now?




-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-banyandb] lujiajing1126 edited a comment on pull request #45: Introduce stream and tsdb module

Posted by GitBox <gi...@apache.org>.
lujiajing1126 edited a comment on pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45#issuecomment-915967171


   > @lujiajing1126 Do you have some thoughts about these changes? I want to merge it into the main branch since it's changed a lot.
   
   LGTM. I have no problem for this PR.
   
   I've roughly reviewed these changes this Tuesday and Wednesday. I will read the tsdb module these days.
   
   I will setup a meeting next Monday to discuss some more details.


-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-banyandb] hanahmily commented on a change in pull request #45: Introduce stream and tsdb module

Posted by GitBox <gi...@apache.org>.
hanahmily commented on a change in pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45#discussion_r703985587



##########
File path: api/proto/banyandb/stream/v2/write.proto
##########
@@ -0,0 +1,50 @@
+// 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.
+
+syntax = "proto3";
+
+option java_package = "org.apache.skywalking.banyandb.stream.v2";
+option go_package = "github.com/apache/skywalking-banyandb/api/proto/banyandb/stream/v2";
+
+package banyandb.stream.v2;
+
+import "google/protobuf/timestamp.proto";
+import "banyandb/common/v2/common.proto";
+import "banyandb/model/v2/common.proto";
+
+message ElementValue {
+  // element_id could be span_id of a Span or segment_id of a Segment in the context of stream
+  string element_id = 1;
+  // timestamp_nanoseconds is in the timeunit of nanoseconds. It represents
+  // 1) either the start time of a Span/Segment,
+  // 2) or the timestamp of a log
+  google.protobuf.Timestamp timestamp = 2;
+  message TagFamily {

Review comment:
       I prefer to use `TagFamilySpec` in `schema.proto`, in which `TagSpec` is using a similar convention.




-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-banyandb] lujiajing1126 commented on a change in pull request #45: Introduce stream and tsdb module

Posted by GitBox <gi...@apache.org>.
lujiajing1126 commented on a change in pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45#discussion_r703962416



##########
File path: api/proto/banyandb/stream/v2/write.proto
##########
@@ -0,0 +1,50 @@
+// 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.
+
+syntax = "proto3";
+
+option java_package = "org.apache.skywalking.banyandb.stream.v2";
+option go_package = "github.com/apache/skywalking-banyandb/api/proto/banyandb/stream/v2";
+
+package banyandb.stream.v2;
+
+import "google/protobuf/timestamp.proto";
+import "banyandb/common/v2/common.proto";
+import "banyandb/model/v2/common.proto";
+
+message ElementValue {
+  // element_id could be span_id of a Span or segment_id of a Segment in the context of stream
+  string element_id = 1;
+  // timestamp_nanoseconds is in the timeunit of nanoseconds. It represents
+  // 1) either the start time of a Span/Segment,
+  // 2) or the timestamp of a log
+  google.protobuf.Timestamp timestamp = 2;
+  message TagFamily {

Review comment:
       The name of `TagFamily` is duplicated here as we've already defined a `TagFamily` in `schema.proto`.
   
   Shall we use `TagValueFamily` here?

##########
File path: banyand/metadata/schema/data/index_rule_binding.json
##########
@@ -0,0 +1,25 @@
+{

Review comment:
       Now we are using `json` instead of `prototext` files?




-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-banyandb] lujiajing1126 commented on a change in pull request #45: Introduce stream and tsdb module

Posted by GitBox <gi...@apache.org>.
lujiajing1126 commented on a change in pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45#discussion_r704237289



##########
File path: banyand/stream/stream_query_test.go
##########
@@ -0,0 +1,532 @@
+// 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.
+
+package stream
+
+import (
+	"bytes"
+	"embed"
+	_ "embed"
+	"encoding/base64"
+	"encoding/json"
+	"fmt"
+	"io"
+	"sort"
+	"strconv"
+	"testing"
+	"time"
+
+	"github.com/golang/protobuf/jsonpb"
+	"github.com/pkg/errors"
+	"github.com/stretchr/testify/assert"
+	"google.golang.org/protobuf/types/known/timestamppb"
+
+	"github.com/apache/skywalking-banyandb/api/common"
+	commonv2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v2"
+	databasev2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v2"
+	modelv2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v2"
+	streamv2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/stream/v2"
+	"github.com/apache/skywalking-banyandb/banyand/tsdb"
+	"github.com/apache/skywalking-banyandb/pkg/convert"
+	"github.com/apache/skywalking-banyandb/pkg/index"
+	"github.com/apache/skywalking-banyandb/pkg/partition"
+)
+
+type shardStruct struct {
+	id       common.ShardID
+	location []string
+	elements []string
+}
+
+type shardsForTest []shardStruct
+
+func Test_Stream_SelectShard(t *testing.T) {
+	tester := assert.New(t)
+	s, deferFunc := setup(tester)
+	defer deferFunc()
+	_ = setupQueryData(tester, "multiple_shards.json", s)
+	tests := []struct {
+		name         string
+		entity       tsdb.Entity
+		wantShardNum int
+		wantErr      bool
+	}{
+		{
+			name:         "all shards",
+			wantShardNum: 2,
+		},
+		{
+			name:         "select a shard",
+			entity:       tsdb.Entity{tsdb.Entry("webapp_id"), tsdb.Entry("10.0.0.1_id"), convert.Int64ToBytes(0)},
+			wantShardNum: 1,
+		},
+		{
+			name:         "select shards",
+			entity:       tsdb.Entity{tsdb.Entry("webapp_id"), tsdb.AnyEntry, convert.Int64ToBytes(0)},
+			wantShardNum: 2,
+		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			shards, err := s.Shards(tt.entity)
+			if tt.wantErr {
+				tester.Error(err)
+				return
+			}
+			tester.NoError(err)
+			tester.Equal(tt.wantShardNum, len(shards))
+		})
+	}
+
+}
+
+func Test_Stream_Series(t *testing.T) {
+	tester := assert.New(t)
+	s, deferFunc := setup(tester)
+	defer deferFunc()
+	baseTime := setupQueryData(tester, "multiple_shards.json", s)
+	tests := []struct {
+		name    string
+		args    queryOpts
+		want    shardsForTest
+		wantErr bool
+	}{
+		{
+			name: "all",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.AnyEntry, tsdb.AnyEntry, tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+			},
+			want: shardsForTest{
+				{
+					id:       0,
+					location: []string{"series_12243341348514563931", "data_flow_0"},
+					elements: []string{"1"},
+				},
+				{
+					id:       0,
+					location: []string{"series_1671844747554927007", "data_flow_0"},
+					elements: []string{"2"},
+				},
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"5", "3"},
+				},
+				{
+					id:       1,
+					location: []string{"series_8429137420168685297", "data_flow_0"},
+					elements: []string{"4"},
+				},
+			},
+		},
+
+		{
+			name: "time range",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.AnyEntry, tsdb.AnyEntry, tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime.Add(1500*time.Millisecond), 1*time.Hour),
+			},
+			want: shardsForTest{
+				{
+					id:       0,
+					location: []string{"series_12243341348514563931", "data_flow_0"},
+				},
+				{
+					id:       0,
+					location: []string{"series_1671844747554927007", "data_flow_0"},
+				},
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"5"},
+				},
+				{
+					id:       1,
+					location: []string{"series_8429137420168685297", "data_flow_0"},
+					elements: []string{"4"},
+				},
+			},
+		},
+		{
+			name: "find series by service_id and instance_id",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.Entry("webapp_id"), tsdb.Entry("10.0.0.1_id"), tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+			},
+			want: shardsForTest{
+				{
+					id:       0,
+					location: []string{"series_12243341348514563931", "data_flow_0"},
+					elements: []string{"1"},
+				},
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"5", "3"},
+				},
+			},
+		},
+		{
+			name: "find a series",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.Entry("webapp_id"), tsdb.Entry("10.0.0.1_id"), convert.Uint64ToBytes(1)},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+			},
+			want: shardsForTest{
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"5", "3"},
+				},
+			},
+		},
+		{
+			name: "filter",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.AnyEntry, tsdb.AnyEntry, tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+				buildFn: func(builder tsdb.SeekerBuilder) {
+					builder.Filter(&databasev2.IndexRule{
+						Metadata: &commonv2.Metadata{
+							Name:  "endpoint_id",
+							Group: "default",
+						},
+						Tags:     []string{"endpoint_id"},
+						Type:     databasev2.IndexRule_TYPE_INVERTED,
+						Location: databasev2.IndexRule_LOCATION_SERIES,
+					}, tsdb.Condition{
+						"endpoint_id": []index.ConditionValue{
+							{
+								Op:     modelv2.Condition_BINARY_OP_EQ,
+								Values: [][]byte{[]byte("/home_id")},
+							},
+						},
+					})
+				},
+			},
+			want: shardsForTest{
+				{
+					id:       0,
+					location: []string{"series_12243341348514563931", "data_flow_0"},
+					elements: []string{"1"},
+				},
+				{
+					id:       0,
+					location: []string{"series_1671844747554927007", "data_flow_0"},
+				},
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"3"},
+				},
+				{
+					id:       1,
+					location: []string{"series_8429137420168685297", "data_flow_0"},
+				},
+			},
+		},
+		{
+			name: "order by duration",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.AnyEntry, tsdb.AnyEntry, tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+				buildFn: func(builder tsdb.SeekerBuilder) {
+					builder.OrderByIndex(&databasev2.IndexRule{
+						Metadata: &commonv2.Metadata{
+							Name:  "duration",
+							Group: "default",
+						},
+						Tags:     []string{"duration"},
+						Type:     databasev2.IndexRule_TYPE_TREE,
+						Location: databasev2.IndexRule_LOCATION_SERIES,
+					}, modelv2.QueryOrder_SORT_ASC)
+				},
+			},
+			want: shardsForTest{
+				{
+					id:       0,
+					location: []string{"series_12243341348514563931", "data_flow_0"},
+					elements: []string{"1"},
+				},
+				{
+					id:       0,
+					location: []string{"series_1671844747554927007", "data_flow_0"},
+					elements: []string{"2"},
+				},
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"3", "5"},
+				},
+				{
+					id:       1,
+					location: []string{"series_8429137420168685297", "data_flow_0"},
+					elements: []string{"4"},
+				},
+			},
+		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			got, err := queryData(tester, s, tt.args)
+			if tt.wantErr {
+				tester.Error(err)
+				return
+			}
+			tester.NoError(err)
+			sort.SliceStable(got, func(i, j int) bool {
+				a := got[i]
+				b := got[j]
+				if a.id > b.id {
+					return false
+				}
+				for i, al := range a.location {
+					bl := b.location[i]
+					if bytes.Compare([]byte(al), []byte(bl)) > 0 {
+						return false
+					}
+				}
+				return true
+			})
+			tester.Equal(tt.want, got)
+		})
+	}
+
+}
+
+func Test_Stream_Global_Index(t *testing.T) {
+	tester := assert.New(t)
+	s, deferFunc := setup(tester)
+	defer deferFunc()
+	_ = setupQueryData(tester, "global_index.json", s)
+	tests := []struct {
+		name                string
+		traceID             string
+		wantTraceSegmentNum int
+		wantErr             bool
+	}{
+		{
+			name:                "trace id is 1",
+			traceID:             "1",
+			wantTraceSegmentNum: 2,
+		},
+		{
+			name:                "trace id is 2",
+			traceID:             "2",
+			wantTraceSegmentNum: 3,
+		},
+		{
+			name:    "unknown trace id",
+			traceID: "foo",
+		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			shards, errShards := s.Shards(nil)
+			tester.NoError(errShards)
+			err := func() error {
+				for _, shard := range shards {
+					itemIDs, err := shard.Index().Seek(index.Field{

Review comment:
       global index means it is directly attached to a single shard, right?




-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-banyandb] hanahmily commented on a change in pull request #45: Introduce stream and tsdb module

Posted by GitBox <gi...@apache.org>.
hanahmily commented on a change in pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45#discussion_r704326774



##########
File path: banyand/stream/stream_query_test.go
##########
@@ -0,0 +1,532 @@
+// 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.
+
+package stream
+
+import (
+	"bytes"
+	"embed"
+	_ "embed"
+	"encoding/base64"
+	"encoding/json"
+	"fmt"
+	"io"
+	"sort"
+	"strconv"
+	"testing"
+	"time"
+
+	"github.com/golang/protobuf/jsonpb"
+	"github.com/pkg/errors"
+	"github.com/stretchr/testify/assert"
+	"google.golang.org/protobuf/types/known/timestamppb"
+
+	"github.com/apache/skywalking-banyandb/api/common"
+	commonv2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v2"
+	databasev2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v2"
+	modelv2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v2"
+	streamv2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/stream/v2"
+	"github.com/apache/skywalking-banyandb/banyand/tsdb"
+	"github.com/apache/skywalking-banyandb/pkg/convert"
+	"github.com/apache/skywalking-banyandb/pkg/index"
+	"github.com/apache/skywalking-banyandb/pkg/partition"
+)
+
+type shardStruct struct {
+	id       common.ShardID
+	location []string
+	elements []string
+}
+
+type shardsForTest []shardStruct
+
+func Test_Stream_SelectShard(t *testing.T) {
+	tester := assert.New(t)
+	s, deferFunc := setup(tester)
+	defer deferFunc()
+	_ = setupQueryData(tester, "multiple_shards.json", s)
+	tests := []struct {
+		name         string
+		entity       tsdb.Entity
+		wantShardNum int
+		wantErr      bool
+	}{
+		{
+			name:         "all shards",
+			wantShardNum: 2,
+		},
+		{
+			name:         "select a shard",
+			entity:       tsdb.Entity{tsdb.Entry("webapp_id"), tsdb.Entry("10.0.0.1_id"), convert.Int64ToBytes(0)},
+			wantShardNum: 1,
+		},
+		{
+			name:         "select shards",
+			entity:       tsdb.Entity{tsdb.Entry("webapp_id"), tsdb.AnyEntry, convert.Int64ToBytes(0)},
+			wantShardNum: 2,
+		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			shards, err := s.Shards(tt.entity)
+			if tt.wantErr {
+				tester.Error(err)
+				return
+			}
+			tester.NoError(err)
+			tester.Equal(tt.wantShardNum, len(shards))
+		})
+	}
+
+}
+
+func Test_Stream_Series(t *testing.T) {
+	tester := assert.New(t)
+	s, deferFunc := setup(tester)
+	defer deferFunc()
+	baseTime := setupQueryData(tester, "multiple_shards.json", s)
+	tests := []struct {
+		name    string
+		args    queryOpts
+		want    shardsForTest
+		wantErr bool
+	}{
+		{
+			name: "all",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.AnyEntry, tsdb.AnyEntry, tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+			},
+			want: shardsForTest{
+				{
+					id:       0,
+					location: []string{"series_12243341348514563931", "data_flow_0"},
+					elements: []string{"1"},
+				},
+				{
+					id:       0,
+					location: []string{"series_1671844747554927007", "data_flow_0"},
+					elements: []string{"2"},
+				},
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"5", "3"},
+				},
+				{
+					id:       1,
+					location: []string{"series_8429137420168685297", "data_flow_0"},
+					elements: []string{"4"},
+				},
+			},
+		},
+
+		{
+			name: "time range",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.AnyEntry, tsdb.AnyEntry, tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime.Add(1500*time.Millisecond), 1*time.Hour),
+			},
+			want: shardsForTest{
+				{
+					id:       0,
+					location: []string{"series_12243341348514563931", "data_flow_0"},
+				},
+				{
+					id:       0,
+					location: []string{"series_1671844747554927007", "data_flow_0"},
+				},
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"5"},
+				},
+				{
+					id:       1,
+					location: []string{"series_8429137420168685297", "data_flow_0"},
+					elements: []string{"4"},
+				},
+			},
+		},
+		{
+			name: "find series by service_id and instance_id",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.Entry("webapp_id"), tsdb.Entry("10.0.0.1_id"), tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+			},
+			want: shardsForTest{
+				{
+					id:       0,
+					location: []string{"series_12243341348514563931", "data_flow_0"},
+					elements: []string{"1"},
+				},
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"5", "3"},
+				},
+			},
+		},
+		{
+			name: "find a series",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.Entry("webapp_id"), tsdb.Entry("10.0.0.1_id"), convert.Uint64ToBytes(1)},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+			},
+			want: shardsForTest{
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"5", "3"},
+				},
+			},
+		},
+		{
+			name: "filter",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.AnyEntry, tsdb.AnyEntry, tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+				buildFn: func(builder tsdb.SeekerBuilder) {

Review comment:
       Exactly. 




-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-banyandb] hanahmily commented on a change in pull request #45: Introduce stream and tsdb module

Posted by GitBox <gi...@apache.org>.
hanahmily commented on a change in pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45#discussion_r704325953



##########
File path: banyand/kv/kv.go
##########
@@ -206,9 +229,8 @@ func OpenIndexStore(shardID int, path string, options ...IndexOptions) (IndexSto
 	for _, opt := range options {
 		opt(bdb)
 	}
-	bdb.dbOpts = bdb.dbOpts.WithMaxLevels(1)
-	// Put all values into LSM
-	bdb.dbOpts = bdb.dbOpts.WithVLogPercentile(1.0)
+	bdb.dbOpts = bdb.dbOpts.WithMaxLevels(2)

Review comment:
       Just for debugging, it's a challenge to find a single key from default 7 levels. Maybe I should parameter it.




-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-banyandb] lujiajing1126 commented on a change in pull request #45: Introduce stream and tsdb module

Posted by GitBox <gi...@apache.org>.
lujiajing1126 commented on a change in pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45#discussion_r704262493



##########
File path: banyand/stream/index.go
##########
@@ -0,0 +1,121 @@
+// 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.
+
+package stream
+
+import (
+	"io"
+
+	"go.uber.org/multierr"
+
+	"github.com/apache/skywalking-banyandb/api/common"
+	databasev2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v2"
+	streamv2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/stream/v2"
+	"github.com/apache/skywalking-banyandb/banyand/tsdb"
+	"github.com/apache/skywalking-banyandb/pkg/index"
+	"github.com/apache/skywalking-banyandb/pkg/partition"
+)
+
+type indexMessage struct {
+	localWriter tsdb.Writer
+	blockCloser io.Closer
+	value       *streamv2.ElementValue
+}
+
+func (s *stream) bootIndexGenerator() {
+	go func() {
+		for {
+			m, more := <-s.indexCh
+			if !more {
+				return
+			}
+			var err error
+			for _, ruleIndex := range s.indexRuleIndex {
+				rule := ruleIndex.rule
+				switch rule.GetLocation() {
+				case databasev2.IndexRule_LOCATION_SERIES:
+					err = multierr.Append(err, writeLocalIndex(m.localWriter, ruleIndex, m.value))
+				case databasev2.IndexRule_LOCATION_GLOBAL:
+					err = multierr.Append(err, s.writeGlobalIndex(ruleIndex, m.localWriter.ItemID(), m.value))
+				}
+			}
+			err = multierr.Append(err, m.blockCloser.Close())
+			if err != nil {
+				s.l.Error().Err(err).Msg("encounter some errors when generating indices")
+			}
+		}
+	}()
+}
+
+//TODO: should listen to pipeline in a distributed cluster
+func (s *stream) writeGlobalIndex(ruleIndex indexRule, ref tsdb.GlobalItemID, value *streamv2.ElementValue) error {
+	val, err := getIndexValue(ruleIndex, value)
+	if err != nil {
+		return err
+	}
+	indexShardID, err := partition.ShardID(val, s.schema.ShardNum)

Review comment:
       The global index may not in the same shard with the actual data?




-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-banyandb] hanahmily commented on pull request #45: Introduce stream and tsdb module

Posted by GitBox <gi...@apache.org>.
hanahmily commented on pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45#issuecomment-915886166


   @lujiajing1126 Do you have some thoughts about these changes? I want to merge it into the main branch since it's changed a lot.


-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-banyandb] hanahmily commented on a change in pull request #45: Introduce stream and tsdb module

Posted by GitBox <gi...@apache.org>.
hanahmily commented on a change in pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45#discussion_r703432158



##########
File path: banyand/stream/stream_query_test.go
##########
@@ -0,0 +1,532 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor

Review comment:
       @lujiajing1126 The test cases could help you understand how to use the new `stream` module to query data. They show us three scenarios:
   
    * Query data by entities which are composited by `service_id`, `service_instance_id`, and `state`
    * Query data by local indices. For example, `duration` and `endpoint_id`. You can sort data by the local indices(`duration`) or the time.
    * Query data by the global index. The `trace_id` is the only case for skywalking trace.




-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-banyandb] hanahmily commented on a change in pull request #45: Introduce stream and tsdb module

Posted by GitBox <gi...@apache.org>.
hanahmily commented on a change in pull request #45:
URL: https://github.com/apache/skywalking-banyandb/pull/45#discussion_r704329426



##########
File path: banyand/stream/stream_query_test.go
##########
@@ -0,0 +1,532 @@
+// 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.
+
+package stream
+
+import (
+	"bytes"
+	"embed"
+	_ "embed"
+	"encoding/base64"
+	"encoding/json"
+	"fmt"
+	"io"
+	"sort"
+	"strconv"
+	"testing"
+	"time"
+
+	"github.com/golang/protobuf/jsonpb"
+	"github.com/pkg/errors"
+	"github.com/stretchr/testify/assert"
+	"google.golang.org/protobuf/types/known/timestamppb"
+
+	"github.com/apache/skywalking-banyandb/api/common"
+	commonv2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v2"
+	databasev2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v2"
+	modelv2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v2"
+	streamv2 "github.com/apache/skywalking-banyandb/api/proto/banyandb/stream/v2"
+	"github.com/apache/skywalking-banyandb/banyand/tsdb"
+	"github.com/apache/skywalking-banyandb/pkg/convert"
+	"github.com/apache/skywalking-banyandb/pkg/index"
+	"github.com/apache/skywalking-banyandb/pkg/partition"
+)
+
+type shardStruct struct {
+	id       common.ShardID
+	location []string
+	elements []string
+}
+
+type shardsForTest []shardStruct
+
+func Test_Stream_SelectShard(t *testing.T) {
+	tester := assert.New(t)
+	s, deferFunc := setup(tester)
+	defer deferFunc()
+	_ = setupQueryData(tester, "multiple_shards.json", s)
+	tests := []struct {
+		name         string
+		entity       tsdb.Entity
+		wantShardNum int
+		wantErr      bool
+	}{
+		{
+			name:         "all shards",
+			wantShardNum: 2,
+		},
+		{
+			name:         "select a shard",
+			entity:       tsdb.Entity{tsdb.Entry("webapp_id"), tsdb.Entry("10.0.0.1_id"), convert.Int64ToBytes(0)},
+			wantShardNum: 1,
+		},
+		{
+			name:         "select shards",
+			entity:       tsdb.Entity{tsdb.Entry("webapp_id"), tsdb.AnyEntry, convert.Int64ToBytes(0)},
+			wantShardNum: 2,
+		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			shards, err := s.Shards(tt.entity)
+			if tt.wantErr {
+				tester.Error(err)
+				return
+			}
+			tester.NoError(err)
+			tester.Equal(tt.wantShardNum, len(shards))
+		})
+	}
+
+}
+
+func Test_Stream_Series(t *testing.T) {
+	tester := assert.New(t)
+	s, deferFunc := setup(tester)
+	defer deferFunc()
+	baseTime := setupQueryData(tester, "multiple_shards.json", s)
+	tests := []struct {
+		name    string
+		args    queryOpts
+		want    shardsForTest
+		wantErr bool
+	}{
+		{
+			name: "all",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.AnyEntry, tsdb.AnyEntry, tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+			},
+			want: shardsForTest{
+				{
+					id:       0,
+					location: []string{"series_12243341348514563931", "data_flow_0"},
+					elements: []string{"1"},
+				},
+				{
+					id:       0,
+					location: []string{"series_1671844747554927007", "data_flow_0"},
+					elements: []string{"2"},
+				},
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"5", "3"},
+				},
+				{
+					id:       1,
+					location: []string{"series_8429137420168685297", "data_flow_0"},
+					elements: []string{"4"},
+				},
+			},
+		},
+
+		{
+			name: "time range",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.AnyEntry, tsdb.AnyEntry, tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime.Add(1500*time.Millisecond), 1*time.Hour),
+			},
+			want: shardsForTest{
+				{
+					id:       0,
+					location: []string{"series_12243341348514563931", "data_flow_0"},
+				},
+				{
+					id:       0,
+					location: []string{"series_1671844747554927007", "data_flow_0"},
+				},
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"5"},
+				},
+				{
+					id:       1,
+					location: []string{"series_8429137420168685297", "data_flow_0"},
+					elements: []string{"4"},
+				},
+			},
+		},
+		{
+			name: "find series by service_id and instance_id",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.Entry("webapp_id"), tsdb.Entry("10.0.0.1_id"), tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+			},
+			want: shardsForTest{
+				{
+					id:       0,
+					location: []string{"series_12243341348514563931", "data_flow_0"},
+					elements: []string{"1"},
+				},
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"5", "3"},
+				},
+			},
+		},
+		{
+			name: "find a series",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.Entry("webapp_id"), tsdb.Entry("10.0.0.1_id"), convert.Uint64ToBytes(1)},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+			},
+			want: shardsForTest{
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"5", "3"},
+				},
+			},
+		},
+		{
+			name: "filter",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.AnyEntry, tsdb.AnyEntry, tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+				buildFn: func(builder tsdb.SeekerBuilder) {
+					builder.Filter(&databasev2.IndexRule{
+						Metadata: &commonv2.Metadata{
+							Name:  "endpoint_id",
+							Group: "default",
+						},
+						Tags:     []string{"endpoint_id"},
+						Type:     databasev2.IndexRule_TYPE_INVERTED,
+						Location: databasev2.IndexRule_LOCATION_SERIES,
+					}, tsdb.Condition{
+						"endpoint_id": []index.ConditionValue{
+							{
+								Op:     modelv2.Condition_BINARY_OP_EQ,
+								Values: [][]byte{[]byte("/home_id")},
+							},
+						},
+					})
+				},
+			},
+			want: shardsForTest{
+				{
+					id:       0,
+					location: []string{"series_12243341348514563931", "data_flow_0"},
+					elements: []string{"1"},
+				},
+				{
+					id:       0,
+					location: []string{"series_1671844747554927007", "data_flow_0"},
+				},
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"3"},
+				},
+				{
+					id:       1,
+					location: []string{"series_8429137420168685297", "data_flow_0"},
+				},
+			},
+		},
+		{
+			name: "order by duration",
+			args: queryOpts{
+				entity:    tsdb.Entity{tsdb.AnyEntry, tsdb.AnyEntry, tsdb.AnyEntry},
+				timeRange: tsdb.NewTimeRangeDuration(baseTime, 1*time.Hour),
+				buildFn: func(builder tsdb.SeekerBuilder) {
+					builder.OrderByIndex(&databasev2.IndexRule{
+						Metadata: &commonv2.Metadata{
+							Name:  "duration",
+							Group: "default",
+						},
+						Tags:     []string{"duration"},
+						Type:     databasev2.IndexRule_TYPE_TREE,
+						Location: databasev2.IndexRule_LOCATION_SERIES,
+					}, modelv2.QueryOrder_SORT_ASC)
+				},
+			},
+			want: shardsForTest{
+				{
+					id:       0,
+					location: []string{"series_12243341348514563931", "data_flow_0"},
+					elements: []string{"1"},
+				},
+				{
+					id:       0,
+					location: []string{"series_1671844747554927007", "data_flow_0"},
+					elements: []string{"2"},
+				},
+				{
+					id:       1,
+					location: []string{"series_2374367181827824198", "data_flow_0"},
+					elements: []string{"3", "5"},
+				},
+				{
+					id:       1,
+					location: []string{"series_8429137420168685297", "data_flow_0"},
+					elements: []string{"4"},
+				},
+			},
+		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			got, err := queryData(tester, s, tt.args)
+			if tt.wantErr {
+				tester.Error(err)
+				return
+			}
+			tester.NoError(err)
+			sort.SliceStable(got, func(i, j int) bool {
+				a := got[i]
+				b := got[j]
+				if a.id > b.id {
+					return false
+				}
+				for i, al := range a.location {
+					bl := b.location[i]
+					if bytes.Compare([]byte(al), []byte(bl)) > 0 {
+						return false
+					}
+				}
+				return true
+			})
+			tester.Equal(tt.want, got)
+		})
+	}
+
+}
+
+func Test_Stream_Global_Index(t *testing.T) {
+	tester := assert.New(t)
+	s, deferFunc := setup(tester)
+	defer deferFunc()
+	_ = setupQueryData(tester, "global_index.json", s)
+	tests := []struct {
+		name                string
+		traceID             string
+		wantTraceSegmentNum int
+		wantErr             bool
+	}{
+		{
+			name:                "trace id is 1",
+			traceID:             "1",
+			wantTraceSegmentNum: 2,
+		},
+		{
+			name:                "trace id is 2",
+			traceID:             "2",
+			wantTraceSegmentNum: 3,
+		},
+		{
+			name:    "unknown trace id",
+			traceID: "foo",
+		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			shards, errShards := s.Shards(nil)
+			tester.NoError(errShards)
+			err := func() error {
+				for _, shard := range shards {
+					itemIDs, err := shard.Index().Seek(index.Field{

Review comment:
       it belongs to a segment in a shard. It means the global indices could be updated from time to time. 
   
   Segment and block are the internal objects, query module could query a segment-attracted index, the global index, from a shard.




-- 
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: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org