diff --git a/cmd/frostfs-cli/modules/tree/move.go b/cmd/frostfs-cli/modules/tree/move.go
index 84b2fb80e7..95516940c3 100644
--- a/cmd/frostfs-cli/modules/tree/move.go
+++ b/cmd/frostfs-cli/modules/tree/move.go
@@ -66,7 +66,7 @@ func move(cmd *cobra.Command, _ []string) {
 		Body: &tree.GetSubTreeRequest_Body{
 			ContainerId: rawCID,
 			TreeId:      tid,
-			RootId:      nid,
+			RootId:      []uint64{nid},
 			Depth:       1,
 			BearerToken: bt,
 		},
diff --git a/cmd/frostfs-cli/modules/tree/subtree.go b/cmd/frostfs-cli/modules/tree/subtree.go
index 64cb351ec0..e58a13fd66 100644
--- a/cmd/frostfs-cli/modules/tree/subtree.go
+++ b/cmd/frostfs-cli/modules/tree/subtree.go
@@ -68,7 +68,7 @@ func getSubTree(cmd *cobra.Command, _ []string) {
 		Body: &tree.GetSubTreeRequest_Body{
 			ContainerId: rawCID,
 			TreeId:      tid,
-			RootId:      rid,
+			RootId:      []uint64{rid},
 			Depth:       depth,
 			BearerToken: bt,
 		},
@@ -83,10 +83,15 @@ func getSubTree(cmd *cobra.Command, _ []string) {
 	for ; err == nil; subtreeResp, err = resp.Recv() {
 		b := subtreeResp.GetBody()
 
-		cmd.Printf("Node ID: %d\n", b.GetNodeId())
-
-		cmd.Println("\tParent ID: ", b.GetParentId())
-		cmd.Println("\tTimestamp: ", b.GetTimestamp())
+		if len(b.GetNodeId()) == 1 {
+			cmd.Printf("Node ID: %d\n", b.GetNodeId())
+			cmd.Println("\tParent ID: ", b.GetParentId())
+			cmd.Println("\tTimestamp: ", b.GetTimestamp())
+		} else {
+			cmd.Printf("Node IDs: %v\n", b.GetNodeId())
+			cmd.Println("\tParent IDs: ", b.GetParentId())
+			cmd.Println("\tTimestamps: ", b.GetTimestamp())
+		}
 
 		if meta := b.GetMeta(); len(meta) > 0 {
 			cmd.Println("\tMeta pairs: ")
diff --git a/pkg/local_object_storage/engine/tree.go b/pkg/local_object_storage/engine/tree.go
index 8bacdba767..39122628f2 100644
--- a/pkg/local_object_storage/engine/tree.go
+++ b/pkg/local_object_storage/engine/tree.go
@@ -210,18 +210,17 @@ func (e *StorageEngine) TreeGetChildren(ctx context.Context, cid cidSDK.ID, tree
 }
 
 // TreeSortedByFilename implements the pilorama.Forest interface.
-func (e *StorageEngine) TreeSortedByFilename(ctx context.Context, cid cidSDK.ID, treeID string, nodeID pilorama.Node, last *string, count int) ([]pilorama.NodeInfo, *string, error) {
+func (e *StorageEngine) TreeSortedByFilename(ctx context.Context, cid cidSDK.ID, treeID string, nodeID pilorama.MultiNode, last *string, count int) ([]pilorama.MultiNodeInfo, *string, error) {
 	ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.TreeSortedByFilename",
 		trace.WithAttributes(
 			attribute.String("container_id", cid.EncodeToString()),
 			attribute.String("tree_id", treeID),
-			attribute.String("node_id", strconv.FormatUint(nodeID, 10)),
 		),
 	)
 	defer span.End()
 
 	var err error
-	var nodes []pilorama.NodeInfo
+	var nodes []pilorama.MultiNodeInfo
 	var cursor *string
 	for _, sh := range e.sortShards(cid) {
 		nodes, cursor, err = sh.TreeSortedByFilename(ctx, cid, treeID, nodeID, last, count)
diff --git a/pkg/local_object_storage/pilorama/boltdb.go b/pkg/local_object_storage/pilorama/boltdb.go
index ee28df426a..48363ceacd 100644
--- a/pkg/local_object_storage/pilorama/boltdb.go
+++ b/pkg/local_object_storage/pilorama/boltdb.go
@@ -9,6 +9,7 @@ import (
 	"math/rand"
 	"os"
 	"path/filepath"
+	"slices"
 	"sort"
 	"strconv"
 	"sync"
@@ -990,23 +991,26 @@ func (t *boltForest) TreeGetMeta(ctx context.Context, cid cidSDK.ID, treeID stri
 	return m, parentID, metaerr.Wrap(err)
 }
 
-func (t *boltForest) hasFewChildren(b *bbolt.Bucket, nodeID Node, threshold int) bool {
+func (t *boltForest) hasFewChildren(b *bbolt.Bucket, nodeIDs MultiNode, threshold int) bool {
 	key := make([]byte, 9)
 	key[0] = 'c'
-	binary.LittleEndian.PutUint64(key[1:], nodeID)
 
 	count := 0
-	c := b.Cursor()
-	for k, _ := c.Seek(key); len(k) == childrenKeySize && binary.LittleEndian.Uint64(k[1:]) == nodeID; k, _ = c.Next() {
-		if count++; count > threshold {
-			return false
+	for _, nodeID := range nodeIDs {
+		binary.LittleEndian.PutUint64(key[1:], nodeID)
+
+		c := b.Cursor()
+		for k, _ := c.Seek(key); len(k) == childrenKeySize && binary.LittleEndian.Uint64(k[1:]) == nodeID; k, _ = c.Next() {
+			if count++; count > threshold {
+				return false
+			}
 		}
 	}
 	return true
 }
 
 // TreeSortedByFilename implements the Forest interface.
-func (t *boltForest) TreeSortedByFilename(ctx context.Context, cid cidSDK.ID, treeID string, nodeID Node, last *string, count int) ([]NodeInfo, *string, error) {
+func (t *boltForest) TreeSortedByFilename(ctx context.Context, cid cidSDK.ID, treeID string, nodeIDs MultiNode, last *string, count int) ([]MultiNodeInfo, *string, error) {
 	var (
 		startedAt = time.Now()
 		success   = false
@@ -1019,7 +1023,6 @@ func (t *boltForest) TreeSortedByFilename(ctx context.Context, cid cidSDK.ID, tr
 		trace.WithAttributes(
 			attribute.String("container_id", cid.EncodeToString()),
 			attribute.String("tree_id", treeID),
-			attribute.String("node_id", strconv.FormatUint(nodeID, 10)),
 		),
 	)
 	defer span.End()
@@ -1030,6 +1033,9 @@ func (t *boltForest) TreeSortedByFilename(ctx context.Context, cid cidSDK.ID, tr
 	if t.mode.NoMetabase() {
 		return nil, last, ErrDegradedMode
 	}
+	if len(nodeIDs) == 0 {
+		return nil, last, errors.New("empty node list")
+	}
 
 	h := newHeap(last, count)
 	key := make([]byte, 9)
@@ -1048,20 +1054,22 @@ func (t *boltForest) TreeSortedByFilename(ctx context.Context, cid cidSDK.ID, tr
 		// If the node is a leaf, we could scan all filenames in the tree.
 		// To prevent this we first count the number of children: if it is less than
 		// the number of nodes we need to return, fallback to TreeGetChildren() implementation.
-		if fewChildren = t.hasFewChildren(b, nodeID, count); fewChildren {
+		if fewChildren = t.hasFewChildren(b, nodeIDs, count); fewChildren {
 			var err error
-			result, err = t.getChildren(b, nodeID)
+			result, err = t.getChildren(b, nodeIDs)
 			return err
 		}
 
-		t.fillSortedChildren(b, nodeID, h)
+		t.fillSortedChildren(b, nodeIDs, h)
 
 		for info, ok := h.pop(); ok; info, ok = h.pop() {
-			childInfo, err := t.getChildInfo(b, key, info.id)
-			if err != nil {
-				return err
+			for _, id := range info.id {
+				childInfo, err := t.getChildInfo(b, key, id)
+				if err != nil {
+					return err
+				}
+				result = append(result, childInfo)
 			}
-			result = append(result, childInfo)
 		}
 		return nil
 	})
@@ -1074,11 +1082,15 @@ func (t *boltForest) TreeSortedByFilename(ctx context.Context, cid cidSDK.ID, tr
 	if fewChildren {
 		result = sortAndCut(result, last)
 	}
-	if len(result) != 0 {
-		s := string(result[len(result)-1].Meta.GetAttr(AttributeFilename))
+	res := mergeNodeInfos(result)
+	if len(res) > count {
+		res = res[:count]
+	}
+	if len(res) != 0 {
+		s := string(findAttr(res[len(res)-1].Meta, AttributeFilename))
 		last = &s
 	}
-	return result, last, metaerr.Wrap(err)
+	return res, last, metaerr.Wrap(err)
 }
 
 func sortAndCut(result []NodeInfo, last *string) []NodeInfo {
@@ -1109,37 +1121,56 @@ func (t *boltForest) getChildInfo(b *bbolt.Bucket, key []byte, childID Node) (No
 	return childInfo, nil
 }
 
-func (t *boltForest) fillSortedChildren(b *bbolt.Bucket, nodeID Node, h *fixedHeap) {
+func (t *boltForest) fillSortedChildren(b *bbolt.Bucket, nodeIDs MultiNode, h *fixedHeap) {
 	c := b.Cursor()
 	prefix := internalKeyPrefix(nil, AttributeFilename)
 
 	length := uint16(0)
 	count := 0
+
+	var nodes []uint64
+	var lastFilename *string
 	for k, _ := c.Seek(prefix); len(k) > 0 && k[0] == 'i'; k, _ = c.Next() {
 		if len(k) < len(prefix)+2+16 {
 			continue
 		}
 
 		parentID := binary.LittleEndian.Uint64(k[len(k)-16:])
-		if parentID != nodeID {
+		if !slices.Contains(nodeIDs, parentID) {
 			continue
 		}
 
 		actualLength := binary.LittleEndian.Uint16(k[len(prefix):])
 		childID := binary.LittleEndian.Uint64(k[len(k)-8:])
 		filename := string(k[len(prefix)+2 : len(k)-16])
-		processed := h.push(childID, filename)
-		if actualLength != length {
-			length = actualLength
-			count = 1
-		} else if processed {
-			if count++; count > h.count {
-				length = actualLength + 1
-				c.Seek(append(prefix, byte(length), byte(length>>8)))
-				c.Prev() // c.Next() will be performed by for loop
+
+		if lastFilename == nil {
+			lastFilename = &filename
+			nodes = append(nodes, childID)
+		} else if *lastFilename == filename {
+			nodes = append(nodes, childID)
+		} else {
+			processed := h.push(nodes, *lastFilename)
+			nodes = MultiNode{childID}
+			lastFilename = &filename
+			if actualLength != length {
+				length = actualLength
+				count = 1
+			} else if processed {
+				if count++; count > h.count {
+					lastFilename = nil
+					nodes = nil
+					length = actualLength + 1
+					c.Seek(append(prefix, byte(length), byte(length>>8)))
+					c.Prev() // c.Next() will be performed by for loop
+				}
 			}
 		}
 	}
+
+	if len(nodes) != 0 && lastFilename != nil {
+		h.push(nodes, *lastFilename)
+	}
 }
 
 // TreeGetChildren implements the Forest interface.
@@ -1179,28 +1210,30 @@ func (t *boltForest) TreeGetChildren(ctx context.Context, cid cidSDK.ID, treeID
 		b := treeRoot.Bucket(dataBucket)
 
 		var err error
-		result, err = t.getChildren(b, nodeID)
+		result, err = t.getChildren(b, []Node{nodeID})
 		return err
 	})
 	success = err == nil
 	return result, metaerr.Wrap(err)
 }
 
-func (t *boltForest) getChildren(b *bbolt.Bucket, nodeID Node) ([]NodeInfo, error) {
+func (t *boltForest) getChildren(b *bbolt.Bucket, nodeIDs MultiNode) ([]NodeInfo, error) {
 	var result []NodeInfo
 
 	key := make([]byte, 9)
-	key[0] = 'c'
-	binary.LittleEndian.PutUint64(key[1:], nodeID)
+	for _, nodeID := range nodeIDs {
+		key[0] = 'c'
+		binary.LittleEndian.PutUint64(key[1:], nodeID)
 
-	c := b.Cursor()
-	for k, _ := c.Seek(key); len(k) == childrenKeySize && binary.LittleEndian.Uint64(k[1:]) == nodeID; k, _ = c.Next() {
-		childID := binary.LittleEndian.Uint64(k[9:])
-		childInfo, err := t.getChildInfo(b, key, childID)
-		if err != nil {
-			return nil, err
+		c := b.Cursor()
+		for k, _ := c.Seek(key); len(k) == childrenKeySize && binary.LittleEndian.Uint64(k[1:]) == nodeID; k, _ = c.Next() {
+			childID := binary.LittleEndian.Uint64(k[9:])
+			childInfo, err := t.getChildInfo(b, key, childID)
+			if err != nil {
+				return nil, err
+			}
+			result = append(result, childInfo)
 		}
-		result = append(result, childInfo)
 	}
 	return result, nil
 }
diff --git a/pkg/local_object_storage/pilorama/forest.go b/pkg/local_object_storage/pilorama/forest.go
index b45a77b99d..78503bada6 100644
--- a/pkg/local_object_storage/pilorama/forest.go
+++ b/pkg/local_object_storage/pilorama/forest.go
@@ -156,7 +156,7 @@ func (f *memoryForest) TreeGetMeta(_ context.Context, cid cid.ID, treeID string,
 }
 
 // TreeSortedByFilename implements the Forest interface.
-func (f *memoryForest) TreeSortedByFilename(_ context.Context, cid cid.ID, treeID string, nodeID Node, start *string, count int) ([]NodeInfo, *string, error) {
+func (f *memoryForest) TreeSortedByFilename(_ context.Context, cid cid.ID, treeID string, nodeIDs MultiNode, start *string, count int) ([]MultiNodeInfo, *string, error) {
 	fullID := cid.String() + "/" + treeID
 	s, ok := f.treeMap[fullID]
 	if !ok {
@@ -166,40 +166,45 @@ func (f *memoryForest) TreeSortedByFilename(_ context.Context, cid cid.ID, treeI
 		return nil, start, nil
 	}
 
-	children := s.tree.getChildren(nodeID)
-	res := make([]NodeInfo, 0, len(children))
-	for _, childID := range children {
-		var found bool
-		for _, kv := range s.infoMap[childID].Meta.Items {
-			if kv.Key == AttributeFilename {
-				found = true
-				break
+	var res []NodeInfo
+
+	for _, nodeID := range nodeIDs {
+		children := s.tree.getChildren(nodeID)
+		for _, childID := range children {
+			var found bool
+			for _, kv := range s.infoMap[childID].Meta.Items {
+				if kv.Key == AttributeFilename {
+					found = true
+					break
+				}
 			}
+			if !found {
+				continue
+			}
+			res = append(res, NodeInfo{
+				ID:       childID,
+				Meta:     s.infoMap[childID].Meta,
+				ParentID: s.infoMap[childID].Parent,
+			})
 		}
-		if !found {
-			continue
-		}
-		res = append(res, NodeInfo{
-			ID:       childID,
-			Meta:     s.infoMap[childID].Meta,
-			ParentID: s.infoMap[childID].Parent,
-		})
 	}
 	if len(res) == 0 {
-		return res, start, nil
+		return nil, start, nil
 	}
 
 	sort.Slice(res, func(i, j int) bool {
 		return bytes.Compare(res[i].Meta.GetAttr(AttributeFilename), res[j].Meta.GetAttr(AttributeFilename)) == -1
 	})
-	for i := range res {
-		if start == nil || string(res[i].Meta.GetAttr(AttributeFilename)) > *start {
+
+	r := mergeNodeInfos(res)
+	for i := range r {
+		if start == nil || string(findAttr(r[i].Meta, AttributeFilename)) > *start {
 			finish := i + count
 			if len(res) < finish {
 				finish = len(res)
 			}
-			last := string(res[finish-1].Meta.GetAttr(AttributeFilename))
-			return res[i:finish], &last, nil
+			last := string(findAttr(r[finish-1].Meta, AttributeFilename))
+			return r[i:finish], &last, nil
 		}
 	}
 	last := string(res[len(res)-1].Meta.GetAttr(AttributeFilename))
diff --git a/pkg/local_object_storage/pilorama/forest_test.go b/pkg/local_object_storage/pilorama/forest_test.go
index 9da0177cca..001d095c80 100644
--- a/pkg/local_object_storage/pilorama/forest_test.go
+++ b/pkg/local_object_storage/pilorama/forest_test.go
@@ -215,7 +215,7 @@ func BenchmarkForestSortedIteration(b *testing.B) {
 
 		b.Run(providers[i].name+",root", func(b *testing.B) {
 			for i := 0; i < b.N; i++ {
-				res, _, err := f.TreeSortedByFilename(context.Background(), cnr, treeID, RootID, nil, 100)
+				res, _, err := f.TreeSortedByFilename(context.Background(), cnr, treeID, MultiNode{RootID}, nil, 100)
 				if err != nil || len(res) != 100 {
 					b.Fatalf("err %v, count %d", err, len(res))
 				}
@@ -223,7 +223,7 @@ func BenchmarkForestSortedIteration(b *testing.B) {
 		})
 		b.Run(providers[i].name+",leaf", func(b *testing.B) {
 			for i := 0; i < b.N; i++ {
-				res, _, err := f.TreeSortedByFilename(context.Background(), cnr, treeID, 1, nil, 100)
+				res, _, err := f.TreeSortedByFilename(context.Background(), cnr, treeID, MultiNode{1}, nil, 100)
 				if err != nil || len(res) != 0 {
 					b.FailNow()
 				}
@@ -266,9 +266,9 @@ func testForestTreeSortedIteration(t *testing.T, s ForestStorage) {
 		treeAdd(t, i+1, strconv.Itoa(i+1))
 	}
 
-	var result []NodeInfo
+	var result []MultiNodeInfo
 	treeAppend := func(t *testing.T, last *string, count int) *string {
-		res, cursor, err := s.TreeSortedByFilename(context.Background(), d.CID, treeID, RootID, last, count)
+		res, cursor, err := s.TreeSortedByFilename(context.Background(), d.CID, treeID, MultiNode{RootID}, last, count)
 		require.NoError(t, err)
 		result = append(result, res...)
 		return cursor
@@ -282,11 +282,11 @@ func testForestTreeSortedIteration(t *testing.T, s ForestStorage) {
 
 	require.Len(t, result, count)
 	for i := range result {
-		require.Equal(t, RootID+uint64(i+1), result[i].ID)
+		require.Equal(t, MultiNode{RootID + uint64(i+1)}, result[i].Children)
 		if i == 0 {
-			require.Equal(t, "", string(result[i].Meta.GetAttr(AttributeFilename)))
+			require.Equal(t, "", string(findAttr(result[i].Meta, AttributeFilename)))
 		} else {
-			require.Equal(t, strconv.Itoa(RootID+i+1), string(result[i].Meta.GetAttr(AttributeFilename)))
+			require.Equal(t, strconv.Itoa(RootID+i+1), string(findAttr(result[i].Meta, AttributeFilename)))
 		}
 	}
 }
@@ -318,12 +318,12 @@ func testForestTreeSortedByFilename(t *testing.T, s ForestStorage) {
 		require.NoError(t, err)
 	}
 
-	expectAttributes := func(t *testing.T, attr string, expected []string, res []NodeInfo) {
+	expectAttributes := func(t *testing.T, attr string, expected []string, res []MultiNodeInfo) {
 		require.Equal(t, len(expected), len(res))
 
 		actual := make([]string, len(res))
 		for i := range actual {
-			actual[i] = string(res[i].Meta.GetAttr(attr))
+			actual[i] = string(findAttr(res[i].Meta, attr))
 		}
 		require.Equal(t, expected, actual)
 	}
@@ -345,40 +345,40 @@ func testForestTreeSortedByFilename(t *testing.T, s ForestStorage) {
 		treeAddByPath(t, items[i])
 	}
 
-	getChildren := func(t *testing.T, id Node) []NodeInfo {
+	getChildren := func(t *testing.T, id MultiNode) []MultiNodeInfo {
 		res, _, err := s.TreeSortedByFilename(context.Background(), d.CID, treeID, id, nil, len(items))
 		require.NoError(t, err)
 		return res
 	}
 
-	res := getChildren(t, RootID)
+	res := getChildren(t, MultiNode{RootID})
 	expectAttributes(t, AttributeFilename, []string{"a", "b", "c"}, res)
 	expectAttributes(t, controlAttr, []string{"", "", "c"}, res)
 
 	{
-		ra := getChildren(t, res[0].ID)
+		ra := getChildren(t, res[0].Children)
 		expectAttributes(t, AttributeFilename, []string{"bbb"}, ra)
 		expectAttributes(t, controlAttr, []string{""}, ra)
 
-		rabbb := getChildren(t, ra[0].ID)
+		rabbb := getChildren(t, ra[0].Children)
 		expectAttributes(t, AttributeFilename, []string{"ccc", "xxx", "z"}, rabbb)
 		expectAttributes(t, controlAttr, []string{"a/bbb/ccc", "a/bbb/xxx", "a/bbb/z"}, rabbb)
 	}
 	{
-		rb := getChildren(t, res[1].ID)
+		rb := getChildren(t, res[1].Children)
 		expectAttributes(t, AttributeFilename, []string{"bbb", "xxx"}, rb)
 		expectAttributes(t, controlAttr, []string{"", ""}, rb)
 
-		rbbbb := getChildren(t, rb[0].ID)
+		rbbbb := getChildren(t, rb[0].Children)
 		expectAttributes(t, AttributeFilename, []string{"ccc"}, rbbbb)
 		expectAttributes(t, controlAttr, []string{"b/bbb/ccc"}, rbbbb)
 
-		rbxxx := getChildren(t, rb[1].ID)
+		rbxxx := getChildren(t, rb[1].Children)
 		expectAttributes(t, AttributeFilename, []string{"z"}, rbxxx)
 		expectAttributes(t, controlAttr, []string{"b/xxx/z"}, rbxxx)
 	}
 	{
-		rc := getChildren(t, res[2].ID)
+		rc := getChildren(t, res[2].Children)
 		require.Len(t, rc, 0)
 	}
 }
diff --git a/pkg/local_object_storage/pilorama/heap.go b/pkg/local_object_storage/pilorama/heap.go
index e8a21e221c..ec57b9e1fd 100644
--- a/pkg/local_object_storage/pilorama/heap.go
+++ b/pkg/local_object_storage/pilorama/heap.go
@@ -5,7 +5,7 @@ import (
 )
 
 type heapInfo struct {
-	id       Node
+	id       MultiNode
 	filename string
 }
 
@@ -44,7 +44,7 @@ func newHeap(start *string, count int) *fixedHeap {
 	}
 }
 
-func (h *fixedHeap) push(id Node, filename string) bool {
+func (h *fixedHeap) push(id MultiNode, filename string) bool {
 	if h.start != nil && filename <= *h.start {
 		return false
 	}
diff --git a/pkg/local_object_storage/pilorama/interface.go b/pkg/local_object_storage/pilorama/interface.go
index f7f5a85b1b..61a3849bf6 100644
--- a/pkg/local_object_storage/pilorama/interface.go
+++ b/pkg/local_object_storage/pilorama/interface.go
@@ -35,7 +35,7 @@ type Forest interface {
 	TreeGetChildren(ctx context.Context, cid cidSDK.ID, treeID string, nodeID Node) ([]NodeInfo, error)
 	// TreeSortedByFilename returns children of the node with the specified ID. The nodes are sorted by the filename attribute..
 	// Should return ErrTreeNotFound if the tree is not found, and empty result if the node is not in the tree.
-	TreeSortedByFilename(ctx context.Context, cid cidSDK.ID, treeID string, nodeID Node, last *string, count int) ([]NodeInfo, *string, error)
+	TreeSortedByFilename(ctx context.Context, cid cidSDK.ID, treeID string, nodeID MultiNode, last *string, count int) ([]MultiNodeInfo, *string, error)
 	// TreeGetOpLog returns first log operation stored at or above the height.
 	// In case no such operation is found, empty Move and nil error should be returned.
 	TreeGetOpLog(ctx context.Context, cid cidSDK.ID, treeID string, height uint64) (Move, error)
diff --git a/pkg/local_object_storage/pilorama/meta.go b/pkg/local_object_storage/pilorama/meta.go
index 49b7f64774..45e9c2f790 100644
--- a/pkg/local_object_storage/pilorama/meta.go
+++ b/pkg/local_object_storage/pilorama/meta.go
@@ -21,7 +21,11 @@ func (x Meta) Bytes() []byte {
 }
 
 func (x Meta) GetAttr(name string) []byte {
-	for _, kv := range x.Items {
+	return findAttr(x.Items, name)
+}
+
+func findAttr(ms []KeyValue, name string) []byte {
+	for _, kv := range ms {
 		if kv.Key == name {
 			return kv.Value
 		}
diff --git a/pkg/local_object_storage/pilorama/multinode.go b/pkg/local_object_storage/pilorama/multinode.go
new file mode 100644
index 0000000000..106ba6ae9c
--- /dev/null
+++ b/pkg/local_object_storage/pilorama/multinode.go
@@ -0,0 +1,49 @@
+package pilorama
+
+import "bytes"
+
+// MultiNode represents a group of internal nodes accessible by the same path, but having different id.
+type MultiNode []Node
+
+// MultiNodeInfo represents a group of internal nodes accessible by the same path, but having different id.
+type MultiNodeInfo struct {
+	Children   MultiNode
+	Parents    MultiNode
+	Timestamps []uint64
+	Meta       []KeyValue
+}
+
+func (r *MultiNodeInfo) Add(info NodeInfo) bool {
+	if !isInternal(info.Meta.Items) || !isInternal(r.Meta) ||
+		!bytes.Equal(r.Meta[0].Value, info.Meta.Items[0].Value) {
+		return false
+	}
+
+	r.Children = append(r.Children, info.ID)
+	r.Parents = append(r.Parents, info.ParentID)
+	r.Timestamps = append(r.Timestamps, info.Meta.Time)
+	return true
+}
+
+func (n NodeInfo) ToMultiNode() MultiNodeInfo {
+	return MultiNodeInfo{
+		Children:   MultiNode{n.ID},
+		Parents:    MultiNode{n.ParentID},
+		Timestamps: []uint64{n.Meta.Time},
+		Meta:       n.Meta.Items,
+	}
+}
+
+func isInternal(m []KeyValue) bool {
+	return len(m) == 1 && m[0].Key == AttributeFilename
+}
+
+func mergeNodeInfos(ns []NodeInfo) []MultiNodeInfo {
+	var r []MultiNodeInfo
+	for _, info := range ns {
+		if len(r) == 0 || !r[len(r)-1].Add(info) {
+			r = append(r, info.ToMultiNode())
+		}
+	}
+	return r
+}
diff --git a/pkg/local_object_storage/pilorama/split_test.go b/pkg/local_object_storage/pilorama/split_test.go
index 3bd581ce6e..54c2b90a65 100644
--- a/pkg/local_object_storage/pilorama/split_test.go
+++ b/pkg/local_object_storage/pilorama/split_test.go
@@ -95,4 +95,61 @@ func testDuplicateDirectory(t *testing.T, f Forest) {
 	require.Equal(t, []byte{4}, testGetByPath(t, "dir1/dir3/value3"))
 	require.Equal(t, []byte{8}, testGetByPath(t, "dir1/dir3/value4"))
 	require.Equal(t, []byte{10}, testGetByPath(t, "value0"))
+
+	testSortedByFilename := func(t *testing.T, root MultiNode, last *string, batchSize int) ([]MultiNodeInfo, *string) {
+		res, last, err := f.TreeSortedByFilename(context.Background(), d.CID, treeID, root, last, batchSize)
+		require.NoError(t, err)
+		return res, last
+	}
+
+	t.Run("test sorted listing, full children branch", func(t *testing.T) {
+		t.Run("big batch size", func(t *testing.T) {
+			res, _ := testSortedByFilename(t, MultiNode{RootID}, nil, 10)
+			require.Equal(t, 3, len(res))
+			require.Equal(t, MultiNode{1, 5}, res[0].Children)
+			require.Equal(t, MultiNode{9}, res[1].Children)
+			require.Equal(t, MultiNode{10}, res[2].Children)
+
+			t.Run("multi-root", func(t *testing.T) {
+				res, _ := testSortedByFilename(t, MultiNode{1, 5}, nil, 10)
+				require.Equal(t, 3, len(res))
+				require.Equal(t, MultiNode{3, 7}, res[0].Children)
+				require.Equal(t, MultiNode{2}, res[1].Children)
+				require.Equal(t, MultiNode{6}, res[2].Children)
+			})
+		})
+		t.Run("small batch size", func(t *testing.T) {
+			res, last := testSortedByFilename(t, MultiNode{RootID}, nil, 1)
+			require.Equal(t, 1, len(res))
+			require.Equal(t, MultiNode{1, 5}, res[0].Children)
+
+			res, last = testSortedByFilename(t, MultiNode{RootID}, last, 1)
+			require.Equal(t, 1, len(res))
+			require.Equal(t, MultiNode{9}, res[0].Children)
+
+			res, last = testSortedByFilename(t, MultiNode{RootID}, last, 1)
+			require.Equal(t, 1, len(res))
+			require.Equal(t, MultiNode{10}, res[0].Children)
+
+			res, _ = testSortedByFilename(t, MultiNode{RootID}, last, 1)
+			require.Equal(t, 0, len(res))
+
+			t.Run("multi-root", func(t *testing.T) {
+				res, last := testSortedByFilename(t, MultiNode{1, 5}, nil, 1)
+				require.Equal(t, 1, len(res))
+				require.Equal(t, MultiNode{3, 7}, res[0].Children)
+
+				res, last = testSortedByFilename(t, MultiNode{1, 5}, last, 1)
+				require.Equal(t, 1, len(res))
+				require.Equal(t, MultiNode{2}, res[0].Children)
+
+				res, last = testSortedByFilename(t, MultiNode{1, 5}, last, 1)
+				require.Equal(t, 1, len(res))
+				require.Equal(t, MultiNode{6}, res[0].Children)
+
+				res, _ = testSortedByFilename(t, MultiNode{RootID}, last, 1)
+				require.Equal(t, 0, len(res))
+			})
+		})
+	})
 }
diff --git a/pkg/local_object_storage/shard/tree.go b/pkg/local_object_storage/shard/tree.go
index e92a61e5b7..26dc8ec1eb 100644
--- a/pkg/local_object_storage/shard/tree.go
+++ b/pkg/local_object_storage/shard/tree.go
@@ -184,13 +184,12 @@ func (s *Shard) TreeGetChildren(ctx context.Context, cid cidSDK.ID, treeID strin
 }
 
 // TreeSortedByFilename implements the pilorama.Forest interface.
-func (s *Shard) TreeSortedByFilename(ctx context.Context, cid cidSDK.ID, treeID string, nodeID pilorama.Node, last *string, count int) ([]pilorama.NodeInfo, *string, error) {
+func (s *Shard) TreeSortedByFilename(ctx context.Context, cid cidSDK.ID, treeID string, nodeID pilorama.MultiNode, last *string, count int) ([]pilorama.MultiNodeInfo, *string, error) {
 	ctx, span := tracing.StartSpanFromContext(ctx, "Shard.TreeSortedByFilename",
 		trace.WithAttributes(
 			attribute.String("shard_id", s.ID().String()),
 			attribute.String("container_id", cid.EncodeToString()),
 			attribute.String("tree_id", treeID),
-			attribute.String("node_id", strconv.FormatUint(nodeID, 10)),
 		),
 	)
 	defer span.End()
diff --git a/pkg/services/tree/getsubtree_test.go b/pkg/services/tree/getsubtree_test.go
index 305c2bac94..677431889d 100644
--- a/pkg/services/tree/getsubtree_test.go
+++ b/pkg/services/tree/getsubtree_test.go
@@ -48,7 +48,7 @@ func TestGetSubTree(t *testing.T) {
 		acc := subTreeAcc{errIndex: errIndex}
 		err := getSubTree(context.Background(), &acc, d.CID, &GetSubTreeRequest_Body{
 			TreeId: treeID,
-			RootId: rootID,
+			RootId: []uint64{rootID},
 			Depth:  depth,
 		}, p)
 		if errIndex == -1 {
@@ -58,12 +58,12 @@ func TestGetSubTree(t *testing.T) {
 		}
 
 		// GetSubTree must return child only after is has returned the parent.
-		require.Equal(t, rootID, acc.seen[0].Body.NodeId)
+		require.Equal(t, rootID, acc.seen[0].Body.NodeId[0])
 	loop:
 		for i := 1; i < len(acc.seen); i++ {
 			parent := acc.seen[i].Body.ParentId
 			for j := 0; j < i; j++ {
-				if acc.seen[j].Body.NodeId == parent {
+				if acc.seen[j].Body.NodeId[0] == parent[0] {
 					continue loop
 				}
 			}
@@ -73,16 +73,16 @@ func TestGetSubTree(t *testing.T) {
 		// GetSubTree must return valid meta.
 		for i := range acc.seen {
 			b := acc.seen[i].Body
-			meta, node, err := p.TreeGetMeta(context.Background(), d.CID, treeID, b.NodeId)
+			meta, node, err := p.TreeGetMeta(context.Background(), d.CID, treeID, b.NodeId[0])
 			require.NoError(t, err)
-			require.Equal(t, node, b.ParentId)
-			require.Equal(t, meta.Time, b.Timestamp)
+			require.Equal(t, node, b.ParentId[0])
+			require.Equal(t, meta.Time, b.Timestamp[0])
 			require.Equal(t, metaToProto(meta.Items), b.Meta)
 		}
 
 		ordered := make([]uint64, len(acc.seen))
 		for i := range acc.seen {
-			ordered[i] = acc.seen[i].Body.NodeId
+			ordered[i] = acc.seen[i].Body.NodeId[0]
 		}
 		return ordered
 	}
@@ -184,7 +184,7 @@ func testGetSubTreeOrderAsc(t *testing.T, p pilorama.ForestStorage) {
 			}
 			found := false
 			for j := range tree {
-				if acc.seen[i].Body.NodeId == tree[j].id {
+				if acc.seen[i].Body.NodeId[0] == tree[j].id {
 					found = true
 					paths = append(paths, path.Join(tree[j].path...))
 				}
@@ -205,7 +205,7 @@ func testGetSubTreeOrderAsc(t *testing.T, p pilorama.ForestStorage) {
 		}, p)
 		require.NoError(t, err)
 		require.Len(t, acc.seen, 1)
-		require.Equal(t, uint64(0), acc.seen[0].Body.NodeId)
+		require.Equal(t, uint64(0), acc.seen[0].Body.NodeId[0])
 	})
 	t.Run("depth=2", func(t *testing.T) {
 		acc := subTreeAcc{errIndex: -1}
@@ -218,15 +218,16 @@ func testGetSubTreeOrderAsc(t *testing.T, p pilorama.ForestStorage) {
 		}, p)
 		require.NoError(t, err)
 		require.Len(t, acc.seen, 3)
-		require.Equal(t, uint64(0), acc.seen[0].Body.NodeId)
-		require.Equal(t, uint64(0), acc.seen[1].GetBody().GetParentId())
-		require.Equal(t, uint64(0), acc.seen[2].GetBody().GetParentId())
+		require.Equal(t, uint64(0), acc.seen[0].Body.NodeId[0])
+		require.Equal(t, uint64(0), acc.seen[1].GetBody().GetParentId()[0])
+		require.Equal(t, uint64(0), acc.seen[2].GetBody().GetParentId()[0])
 	})
 }
 
 var (
 	errSubTreeSend           = errors.New("send finished with error")
 	errSubTreeSendAfterError = errors.New("send was invoked after an error occurred")
+	errInvalidResponse       = errors.New("send got invalid response")
 )
 
 type subTreeAcc struct {
@@ -239,6 +240,16 @@ type subTreeAcc struct {
 var _ TreeService_GetSubTreeServer = &subTreeAcc{}
 
 func (s *subTreeAcc) Send(r *GetSubTreeResponse) error {
+	b := r.GetBody()
+	if len(b.GetNodeId()) > 1 {
+		return errInvalidResponse
+	}
+	if len(b.GetParentId()) > 1 {
+		return errInvalidResponse
+	}
+	if len(b.GetTimestamp()) > 1 {
+		return errInvalidResponse
+	}
 	s.seen = append(s.seen, r)
 	if s.errIndex >= 0 {
 		if len(s.seen) == s.errIndex+1 {
diff --git a/pkg/services/tree/service.go b/pkg/services/tree/service.go
index 85f5314660..2012f53d28 100644
--- a/pkg/services/tree/service.go
+++ b/pkg/services/tree/service.go
@@ -16,6 +16,8 @@ import (
 	netmapSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/netmap"
 	"github.com/panjf2000/ants/v2"
 	"go.uber.org/zap"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
 )
 
 // Service represents tree-service capable of working with multiple
@@ -440,29 +442,50 @@ func (s *Service) GetSubTree(req *GetSubTreeRequest, srv TreeService_GetSubTreeS
 	return getSubTree(srv.Context(), srv, cid, b, s.forest)
 }
 
+type stackItem struct {
+	values []pilorama.MultiNodeInfo
+	parent pilorama.MultiNode
+	last   *string
+}
+
 func getSortedSubTree(ctx context.Context, srv TreeService_GetSubTreeServer, cid cidSDK.ID, b *GetSubTreeRequest_Body, forest pilorama.Forest) error {
 	const batchSize = 1000
 
-	type stackItem struct {
-		values []pilorama.NodeInfo
-		parent pilorama.Node
-		last   *string
+	// For backward compatibility.
+	rootIDs := b.GetRootId()
+	if len(rootIDs) == 0 {
+		rootIDs = []uint64{0}
 	}
 
 	// Traverse the tree in a DFS manner. Because we need to support arbitrary depth,
 	// recursive implementation is not suitable here, so we maintain explicit stack.
-	m, p, err := forest.TreeGetMeta(ctx, cid, b.GetTreeId(), b.GetRootId())
-	if err != nil {
-		return err
+	var ms []pilorama.KeyValue
+	var ps []uint64
+	var ts []uint64
+	for _, rootID := range rootIDs {
+		m, p, err := forest.TreeGetMeta(ctx, cid, b.GetTreeId(), rootID)
+		if err != nil {
+			return err
+		}
+		if ms == nil {
+			ms = m.Items
+		} else {
+			if len(m.Items) != 1 {
+				return status.Error(codes.InvalidArgument, "multiple non-internal nodes provided")
+			}
+		}
+		ts = append(ts, m.Time)
+		ps = append(ps, p)
 	}
 
 	stack := []stackItem{{
-		values: []pilorama.NodeInfo{{
-			ID:       b.GetRootId(),
-			Meta:     m,
-			ParentID: p,
+		values: []pilorama.MultiNodeInfo{{
+			Children:   rootIDs,
+			Timestamps: ts,
+			Meta:       ms,
+			Parents:    ps,
 		}},
-		parent: p,
+		parent: ps,
 	}}
 
 	for {
@@ -486,30 +509,20 @@ func getSortedSubTree(ctx context.Context, srv TreeService_GetSubTreeServer, cid
 			}
 		}
 
-		node := stack[len(stack)-1].values[0]
-		stack[len(stack)-1].values = stack[len(stack)-1].values[1:]
-
-		err = srv.Send(&GetSubTreeResponse{
-			Body: &GetSubTreeResponse_Body{
-				NodeId:    node.ID,
-				ParentId:  node.ParentID,
-				Timestamp: node.Meta.Time,
-				Meta:      metaToProto(node.Meta.Items),
-			},
-		})
+		node, err := stackPopAndSend(stack, srv)
 		if err != nil {
 			return err
 		}
 
 		if b.GetDepth() == 0 || uint32(len(stack)) < b.GetDepth() {
-			children, last, err := forest.TreeSortedByFilename(ctx, cid, b.GetTreeId(), node.ID, nil, batchSize)
+			children, last, err := forest.TreeSortedByFilename(ctx, cid, b.GetTreeId(), node.Children, nil, batchSize)
 			if err != nil {
 				return err
 			}
 			if len(children) != 0 {
 				stack = append(stack, stackItem{
 					values: children,
-					parent: node.ID,
+					parent: node.Children,
 					last:   last,
 				})
 			}
@@ -518,19 +531,38 @@ func getSortedSubTree(ctx context.Context, srv TreeService_GetSubTreeServer, cid
 	return nil
 }
 
+func stackPopAndSend(stack []stackItem, srv TreeService_GetSubTreeServer) (pilorama.MultiNodeInfo, error) {
+	node := stack[len(stack)-1].values[0]
+	stack[len(stack)-1].values = stack[len(stack)-1].values[1:]
+
+	return node, srv.Send(&GetSubTreeResponse{
+		Body: &GetSubTreeResponse_Body{
+			NodeId:    node.Children,
+			ParentId:  node.Parents,
+			Timestamp: node.Timestamps,
+			Meta:      metaToProto(node.Meta),
+		},
+	})
+}
+
 func getSubTree(ctx context.Context, srv TreeService_GetSubTreeServer, cid cidSDK.ID, b *GetSubTreeRequest_Body, forest pilorama.Forest) error {
 	if b.GetOrderBy().GetDirection() == GetSubTreeRequest_Body_Order_Asc {
 		return getSortedSubTree(ctx, srv, cid, b, forest)
 	}
 
+	var rootID uint64
+	if len(b.GetRootId()) > 0 {
+		rootID = b.GetRootId()[0]
+	}
+
 	// Traverse the tree in a DFS manner. Because we need to support arbitrary depth,
 	// recursive implementation is not suitable here, so we maintain explicit stack.
-	m, p, err := forest.TreeGetMeta(ctx, cid, b.GetTreeId(), b.GetRootId())
+	m, p, err := forest.TreeGetMeta(ctx, cid, b.GetTreeId(), rootID)
 	if err != nil {
 		return err
 	}
 	stack := [][]pilorama.NodeInfo{{{
-		ID:       b.GetRootId(),
+		ID:       rootID,
 		Meta:     m,
 		ParentID: p,
 	}}}
@@ -548,9 +580,9 @@ func getSubTree(ctx context.Context, srv TreeService_GetSubTreeServer, cid cidSD
 
 		err = srv.Send(&GetSubTreeResponse{
 			Body: &GetSubTreeResponse_Body{
-				NodeId:    node.ID,
-				ParentId:  node.ParentID,
-				Timestamp: node.Meta.Time,
+				NodeId:    []uint64{node.ID},
+				ParentId:  []uint64{node.ParentID},
+				Timestamp: []uint64{node.Meta.Time},
 				Meta:      metaToProto(node.Meta.Items),
 			},
 		})
diff --git a/pkg/services/tree/service.pb.go b/pkg/services/tree/service.pb.go
index 7930361e17..f439e3f282 100644
--- a/pkg/services/tree/service.pb.go
+++ b/pkg/services/tree/service.pb.go
@@ -1979,8 +1979,8 @@ type GetSubTreeRequest_Body struct {
 	ContainerId []byte `protobuf:"bytes,1,opt,name=container_id,json=containerId,proto3" json:"container_id,omitempty"`
 	// The name of the tree.
 	TreeId string `protobuf:"bytes,2,opt,name=tree_id,json=treeId,proto3" json:"tree_id,omitempty"`
-	// ID of the root node of a subtree.
-	RootId uint64 `protobuf:"varint,3,opt,name=root_id,json=rootId,proto3" json:"root_id,omitempty"`
+	// IDs of the root nodes of a subtree forest.
+	RootId []uint64 `protobuf:"varint,3,rep,name=root_id,json=rootId,proto3" json:"root_id,omitempty"`
 	// Optional depth of the traversal. Zero means return only root.
 	// Maximum depth is 10.
 	Depth uint32 `protobuf:"varint,4,opt,name=depth,proto3" json:"depth,omitempty"`
@@ -2036,11 +2036,11 @@ func (x *GetSubTreeRequest_Body) GetTreeId() string {
 	return ""
 }
 
-func (x *GetSubTreeRequest_Body) GetRootId() uint64 {
+func (x *GetSubTreeRequest_Body) GetRootId() []uint64 {
 	if x != nil {
 		return x.RootId
 	}
-	return 0
+	return nil
 }
 
 func (x *GetSubTreeRequest_Body) GetDepth() uint32 {
@@ -2117,11 +2117,11 @@ type GetSubTreeResponse_Body struct {
 	unknownFields protoimpl.UnknownFields
 
 	// ID of the node.
-	NodeId uint64 `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"`
+	NodeId []uint64 `protobuf:"varint,1,rep,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"`
 	// ID of the parent.
-	ParentId uint64 `protobuf:"varint,2,opt,name=parent_id,json=parentId,proto3" json:"parent_id,omitempty"`
+	ParentId []uint64 `protobuf:"varint,2,rep,name=parent_id,json=parentId,proto3" json:"parent_id,omitempty"`
 	// Time node was first added to a tree.
-	Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
+	Timestamp []uint64 `protobuf:"varint,3,rep,name=timestamp,proto3" json:"timestamp,omitempty"`
 	// Node meta-information.
 	Meta []*KeyValue `protobuf:"bytes,4,rep,name=meta,proto3" json:"meta,omitempty"`
 }
@@ -2158,25 +2158,25 @@ func (*GetSubTreeResponse_Body) Descriptor() ([]byte, []int) {
 	return file_pkg_services_tree_service_proto_rawDescGZIP(), []int{11, 0}
 }
 
-func (x *GetSubTreeResponse_Body) GetNodeId() uint64 {
+func (x *GetSubTreeResponse_Body) GetNodeId() []uint64 {
 	if x != nil {
 		return x.NodeId
 	}
-	return 0
+	return nil
 }
 
-func (x *GetSubTreeResponse_Body) GetParentId() uint64 {
+func (x *GetSubTreeResponse_Body) GetParentId() []uint64 {
 	if x != nil {
 		return x.ParentId
 	}
-	return 0
+	return nil
 }
 
-func (x *GetSubTreeResponse_Body) GetTimestamp() uint64 {
+func (x *GetSubTreeResponse_Body) GetTimestamp() []uint64 {
 	if x != nil {
 		return x.Timestamp
 	}
-	return 0
+	return nil
 }
 
 func (x *GetSubTreeResponse_Body) GetMeta() []*KeyValue {
@@ -2742,173 +2742,175 @@ var file_pkg_services_tree_service_proto_rawDesc = []byte{
 	0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e,
 	0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x42, 0x79, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x73,
 	0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x6e, 0x6f, 0x64, 0x65,
-	0x73, 0x22, 0xbf, 0x03, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x53, 0x75, 0x62, 0x54, 0x72, 0x65, 0x65,
+	0x73, 0x22, 0xc3, 0x03, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x53, 0x75, 0x62, 0x54, 0x72, 0x65, 0x65,
 	0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18,
 	0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x47, 0x65, 0x74,
 	0x53, 0x75, 0x62, 0x54, 0x72, 0x65, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x42,
 	0x6f, 0x64, 0x79, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x12, 0x2d, 0x0a, 0x09, 0x73, 0x69, 0x67,
 	0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74,
 	0x72, 0x65, 0x65, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x09, 0x73,
-	0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x1a, 0xc8, 0x02, 0x0a, 0x04, 0x42, 0x6f, 0x64,
+	0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x1a, 0xcc, 0x02, 0x0a, 0x04, 0x42, 0x6f, 0x64,
 	0x79, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x5f, 0x69,
 	0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e,
 	0x65, 0x72, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x72, 0x65, 0x65, 0x5f, 0x69, 0x64, 0x18,
-	0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x72, 0x65, 0x65, 0x49, 0x64, 0x12, 0x17, 0x0a,
-	0x07, 0x72, 0x6f, 0x6f, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06,
-	0x72, 0x6f, 0x6f, 0x74, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x64, 0x65, 0x70, 0x74, 0x68, 0x18,
-	0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x64, 0x65, 0x70, 0x74, 0x68, 0x12, 0x21, 0x0a, 0x0c,
-	0x62, 0x65, 0x61, 0x72, 0x65, 0x72, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x05, 0x20, 0x01,
-	0x28, 0x0c, 0x52, 0x0b, 0x62, 0x65, 0x61, 0x72, 0x65, 0x72, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12,
-	0x3d, 0x0a, 0x08, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x5f, 0x62, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28,
-	0x0b, 0x32, 0x22, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x75, 0x62, 0x54,
-	0x72, 0x65, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x42, 0x6f, 0x64, 0x79, 0x2e,
-	0x4f, 0x72, 0x64, 0x65, 0x72, 0x52, 0x07, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x42, 0x79, 0x1a, 0x73,
-	0x0a, 0x05, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x12, 0x4a, 0x0a, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63,
-	0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x74, 0x72, 0x65,
-	0x65, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x75, 0x62, 0x54, 0x72, 0x65, 0x65, 0x52, 0x65, 0x71, 0x75,
-	0x65, 0x73, 0x74, 0x2e, 0x42, 0x6f, 0x64, 0x79, 0x2e, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x2e, 0x44,
-	0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74,
-	0x69, 0x6f, 0x6e, 0x22, 0x1e, 0x0a, 0x09, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
-	0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x73,
-	0x63, 0x10, 0x01, 0x22, 0xf6, 0x01, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x53, 0x75, 0x62, 0x54, 0x72,
-	0x65, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x31, 0x0a, 0x04, 0x62, 0x6f,
-	0x64, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e,
-	0x47, 0x65, 0x74, 0x53, 0x75, 0x62, 0x54, 0x72, 0x65, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
-	0x73, 0x65, 0x2e, 0x42, 0x6f, 0x64, 0x79, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x12, 0x2d, 0x0a,
-	0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b,
-	0x32, 0x0f, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72,
-	0x65, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x1a, 0x7e, 0x0a, 0x04,
-	0x42, 0x6f, 0x64, 0x79, 0x12, 0x17, 0x0a, 0x07, 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x69, 0x64, 0x18,
-	0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x64, 0x12, 0x1b, 0x0a,
-	0x09, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04,
-	0x52, 0x08, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69,
-	0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74,
-	0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x22, 0x0a, 0x04, 0x6d, 0x65, 0x74, 0x61,
-	0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x4b, 0x65,
-	0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x04, 0x6d, 0x65, 0x74, 0x61, 0x22, 0x9b, 0x01, 0x0a,
-	0x0f, 0x54, 0x72, 0x65, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
-	0x12, 0x2e, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a,
-	0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x54, 0x72, 0x65, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65,
-	0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x42, 0x6f, 0x64, 0x79, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79,
-	0x12, 0x2d, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20,
-	0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61,
-	0x74, 0x75, 0x72, 0x65, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x1a,
-	0x29, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x74, 0x61,
-	0x69, 0x6e, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x63,
-	0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x49, 0x64, 0x22, 0x8c, 0x01, 0x0a, 0x10, 0x54,
-	0x72, 0x65, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12,
-	0x2f, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e,
-	0x74, 0x72, 0x65, 0x65, 0x2e, 0x54, 0x72, 0x65, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73,
-	0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x42, 0x6f, 0x64, 0x79, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79,
-	0x12, 0x2d, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20,
-	0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61,
-	0x74, 0x75, 0x72, 0x65, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x1a,
-	0x18, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x69, 0x64, 0x73, 0x18, 0x01,
-	0x20, 0x03, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x73, 0x22, 0xdb, 0x01, 0x0a, 0x0c, 0x41, 0x70,
-	0x70, 0x6c, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2b, 0x0a, 0x04, 0x62, 0x6f,
-	0x64, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e,
-	0x41, 0x70, 0x70, 0x6c, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x42, 0x6f, 0x64,
-	0x79, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x12, 0x2d, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61,
-	0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x72, 0x65,
-	0x65, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x09, 0x73, 0x69, 0x67,
-	0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x1a, 0x6f, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x21,
-	0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01,
-	0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x49,
-	0x64, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x72, 0x65, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01,
-	0x28, 0x09, 0x52, 0x06, 0x74, 0x72, 0x65, 0x65, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x09, 0x6f, 0x70,
-	0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e,
-	0x74, 0x72, 0x65, 0x65, 0x2e, 0x4c, 0x6f, 0x67, 0x4d, 0x6f, 0x76, 0x65, 0x52, 0x09, 0x6f, 0x70,
-	0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x74, 0x0a, 0x0d, 0x41, 0x70, 0x70, 0x6c, 0x79,
-	0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79,
-	0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x41, 0x70,
-	0x70, 0x6c, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x42, 0x6f, 0x64, 0x79,
-	0x52, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x12, 0x2d, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74,
-	0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x72, 0x65, 0x65,
-	0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e,
-	0x61, 0x74, 0x75, 0x72, 0x65, 0x1a, 0x06, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79, 0x22, 0xe2, 0x01,
-	0x0a, 0x0f, 0x47, 0x65, 0x74, 0x4f, 0x70, 0x4c, 0x6f, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+	0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x72, 0x65, 0x65, 0x49, 0x64, 0x12, 0x1b, 0x0a,
+	0x07, 0x72, 0x6f, 0x6f, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x03, 0x28, 0x04, 0x42, 0x02,
+	0x10, 0x00, 0x52, 0x06, 0x72, 0x6f, 0x6f, 0x74, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x64, 0x65,
+	0x70, 0x74, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x64, 0x65, 0x70, 0x74, 0x68,
+	0x12, 0x21, 0x0a, 0x0c, 0x62, 0x65, 0x61, 0x72, 0x65, 0x72, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e,
+	0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x62, 0x65, 0x61, 0x72, 0x65, 0x72, 0x54, 0x6f,
+	0x6b, 0x65, 0x6e, 0x12, 0x3d, 0x0a, 0x08, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x5f, 0x62, 0x79, 0x18,
+	0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x47, 0x65, 0x74,
+	0x53, 0x75, 0x62, 0x54, 0x72, 0x65, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x42,
+	0x6f, 0x64, 0x79, 0x2e, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x52, 0x07, 0x6f, 0x72, 0x64, 0x65, 0x72,
+	0x42, 0x79, 0x1a, 0x73, 0x0a, 0x05, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x12, 0x4a, 0x0a, 0x09, 0x64,
+	0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2c,
+	0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x75, 0x62, 0x54, 0x72, 0x65, 0x65,
+	0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x42, 0x6f, 0x64, 0x79, 0x2e, 0x4f, 0x72, 0x64,
+	0x65, 0x72, 0x2e, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x64, 0x69,
+	0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x1e, 0x0a, 0x09, 0x44, 0x69, 0x72, 0x65, 0x63,
+	0x74, 0x69, 0x6f, 0x6e, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x07,
+	0x0a, 0x03, 0x41, 0x73, 0x63, 0x10, 0x01, 0x22, 0x83, 0x02, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x53,
+	0x75, 0x62, 0x54, 0x72, 0x65, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x31,
+	0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x74,
+	0x72, 0x65, 0x65, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x75, 0x62, 0x54, 0x72, 0x65, 0x65, 0x52, 0x65,
+	0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x42, 0x6f, 0x64, 0x79, 0x52, 0x04, 0x62, 0x6f, 0x64,
+	0x79, 0x12, 0x2d, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x53, 0x69, 0x67, 0x6e,
+	0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65,
+	0x1a, 0x8a, 0x01, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x1b, 0x0a, 0x07, 0x6e, 0x6f, 0x64,
+	0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x04, 0x42, 0x02, 0x10, 0x00, 0x52, 0x06,
+	0x6e, 0x6f, 0x64, 0x65, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74,
+	0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x04, 0x42, 0x02, 0x10, 0x00, 0x52, 0x08, 0x70,
+	0x61, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x20, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73,
+	0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x03, 0x28, 0x04, 0x42, 0x02, 0x10, 0x00, 0x52, 0x09,
+	0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x22, 0x0a, 0x04, 0x6d, 0x65, 0x74,
+	0x61, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x4b,
+	0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x04, 0x6d, 0x65, 0x74, 0x61, 0x22, 0x9b, 0x01,
+	0x0a, 0x0f, 0x54, 0x72, 0x65, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
 	0x74, 0x12, 0x2e, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32,
-	0x1a, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x47, 0x65, 0x74, 0x4f, 0x70, 0x4c, 0x6f, 0x67, 0x52,
+	0x1a, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x54, 0x72, 0x65, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x52,
 	0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x42, 0x6f, 0x64, 0x79, 0x52, 0x04, 0x62, 0x6f, 0x64,
 	0x79, 0x12, 0x2d, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02,
 	0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x53, 0x69, 0x67, 0x6e,
 	0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65,
-	0x1a, 0x70, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x74,
+	0x1a, 0x29, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x74,
 	0x61, 0x69, 0x6e, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b,
-	0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x74,
-	0x72, 0x65, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x72,
-	0x65, 0x65, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x68, 0x65, 0x69, 0x67, 0x68, 0x74, 0x18, 0x03,
-	0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x68, 0x65, 0x69, 0x67, 0x68, 0x74, 0x12, 0x14, 0x0a, 0x05,
-	0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x63, 0x6f, 0x75,
-	0x6e, 0x74, 0x22, 0xa7, 0x01, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x4f, 0x70, 0x4c, 0x6f, 0x67, 0x52,
-	0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18,
-	0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x47, 0x65, 0x74,
-	0x4f, 0x70, 0x4c, 0x6f, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x42, 0x6f,
+	0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x49, 0x64, 0x22, 0x8c, 0x01, 0x0a, 0x10,
+	0x54, 0x72, 0x65, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+	0x12, 0x2f, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b,
+	0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x54, 0x72, 0x65, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65,
+	0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x42, 0x6f, 0x64, 0x79, 0x52, 0x04, 0x62, 0x6f, 0x64,
+	0x79, 0x12, 0x2d, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x53, 0x69, 0x67, 0x6e,
+	0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65,
+	0x1a, 0x18, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x69, 0x64, 0x73, 0x18,
+	0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x73, 0x22, 0xdb, 0x01, 0x0a, 0x0c, 0x41,
+	0x70, 0x70, 0x6c, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2b, 0x0a, 0x04, 0x62,
+	0x6f, 0x64, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x74, 0x72, 0x65, 0x65,
+	0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x42, 0x6f,
 	0x64, 0x79, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x12, 0x2d, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e,
 	0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x72,
 	0x65, 0x65, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x09, 0x73, 0x69,
-	0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x1a, 0x33, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79, 0x12,
-	0x2b, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01,
-	0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x4c, 0x6f, 0x67, 0x4d, 0x6f, 0x76,
-	0x65, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x80, 0x01, 0x0a,
-	0x13, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70,
-	0x6f, 0x6e, 0x73, 0x65, 0x12, 0x32, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18, 0x01, 0x20, 0x01,
-	0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68,
-	0x63, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x42, 0x6f,
-	0x64, 0x79, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x12, 0x2d, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e,
-	0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x72,
-	0x65, 0x65, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x09, 0x73, 0x69,
-	0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x1a, 0x06, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79, 0x22,
-	0x7e, 0x0a, 0x12, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65,
-	0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x31, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18, 0x01, 0x20,
-	0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x48, 0x65, 0x61, 0x6c, 0x74,
-	0x68, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x42, 0x6f,
-	0x64, 0x79, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x12, 0x2d, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e,
-	0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x72,
-	0x65, 0x65, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x09, 0x73, 0x69,
-	0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x1a, 0x06, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79, 0x32,
-	0xd6, 0x04, 0x0a, 0x0b, 0x54, 0x72, 0x65, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12,
-	0x2a, 0x0a, 0x03, 0x41, 0x64, 0x64, 0x12, 0x10, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x41, 0x64,
-	0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e,
-	0x41, 0x64, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3c, 0x0a, 0x09, 0x41,
-	0x64, 0x64, 0x42, 0x79, 0x50, 0x61, 0x74, 0x68, 0x12, 0x16, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e,
-	0x41, 0x64, 0x64, 0x42, 0x79, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
-	0x1a, 0x17, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x41, 0x64, 0x64, 0x42, 0x79, 0x50, 0x61, 0x74,
-	0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x52, 0x65, 0x6d,
-	0x6f, 0x76, 0x65, 0x12, 0x13, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76,
-	0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e,
-	0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2d,
-	0x0a, 0x04, 0x4d, 0x6f, 0x76, 0x65, 0x12, 0x11, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x4d, 0x6f,
-	0x76, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x12, 0x2e, 0x74, 0x72, 0x65, 0x65,
-	0x2e, 0x4d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a,
-	0x0d, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x42, 0x79, 0x50, 0x61, 0x74, 0x68, 0x12, 0x1a,
-	0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x42, 0x79, 0x50,
-	0x61, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x74, 0x72, 0x65,
-	0x65, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x42, 0x79, 0x50, 0x61, 0x74, 0x68, 0x52,
-	0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x41, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x53, 0x75,
-	0x62, 0x54, 0x72, 0x65, 0x65, 0x12, 0x17, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x47, 0x65, 0x74,
-	0x53, 0x75, 0x62, 0x54, 0x72, 0x65, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x18,
-	0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x75, 0x62, 0x54, 0x72, 0x65, 0x65,
-	0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x30, 0x01, 0x12, 0x39, 0x0a, 0x08, 0x54, 0x72,
-	0x65, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x15, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x54, 0x72,
-	0x65, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e,
-	0x74, 0x72, 0x65, 0x65, 0x2e, 0x54, 0x72, 0x65, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73,
-	0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x30, 0x0a, 0x05, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x12, 0x12,
-	0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65,
-	0x73, 0x74, 0x1a, 0x13, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x52,
-	0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3b, 0x0a, 0x08, 0x47, 0x65, 0x74, 0x4f, 0x70,
-	0x4c, 0x6f, 0x67, 0x12, 0x15, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x47, 0x65, 0x74, 0x4f, 0x70,
-	0x4c, 0x6f, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x74, 0x72, 0x65,
-	0x65, 0x2e, 0x47, 0x65, 0x74, 0x4f, 0x70, 0x4c, 0x6f, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
-	0x73, 0x65, 0x30, 0x01, 0x12, 0x42, 0x0a, 0x0b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x63, 0x68,
-	0x65, 0x63, 0x6b, 0x12, 0x18, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x48, 0x65, 0x61, 0x6c, 0x74,
-	0x68, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e,
-	0x74, 0x72, 0x65, 0x65, 0x2e, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x63, 0x68, 0x65, 0x63, 0x6b,
-	0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x3e, 0x5a, 0x3c, 0x67, 0x69, 0x74, 0x2e,
-	0x66, 0x72, 0x6f, 0x73, 0x74, 0x66, 0x73, 0x2e, 0x69, 0x6e, 0x66, 0x6f, 0x2f, 0x54, 0x72, 0x75,
-	0x65, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x4c, 0x61, 0x62, 0x2f, 0x66, 0x72, 0x6f, 0x73, 0x74, 0x66,
-	0x73, 0x2d, 0x6e, 0x6f, 0x64, 0x65, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x73, 0x65, 0x72, 0x76, 0x69,
-	0x63, 0x65, 0x73, 0x2f, 0x74, 0x72, 0x65, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
+	0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x1a, 0x6f, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79, 0x12,
+	0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18,
+	0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72,
+	0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x72, 0x65, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20,
+	0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x72, 0x65, 0x65, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x09, 0x6f,
+	0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d,
+	0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x4c, 0x6f, 0x67, 0x4d, 0x6f, 0x76, 0x65, 0x52, 0x09, 0x6f,
+	0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x74, 0x0a, 0x0d, 0x41, 0x70, 0x70, 0x6c,
+	0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x04, 0x62, 0x6f, 0x64,
+	0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x41,
+	0x70, 0x70, 0x6c, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x42, 0x6f, 0x64,
+	0x79, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x12, 0x2d, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61,
+	0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x72, 0x65,
+	0x65, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x09, 0x73, 0x69, 0x67,
+	0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x1a, 0x06, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79, 0x22, 0xe2,
+	0x01, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x4f, 0x70, 0x4c, 0x6f, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65,
+	0x73, 0x74, 0x12, 0x2e, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,
+	0x32, 0x1a, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x47, 0x65, 0x74, 0x4f, 0x70, 0x4c, 0x6f, 0x67,
+	0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x42, 0x6f, 0x64, 0x79, 0x52, 0x04, 0x62, 0x6f,
+	0x64, 0x79, 0x12, 0x2d, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18,
+	0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x53, 0x69, 0x67,
+	0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72,
+	0x65, 0x1a, 0x70, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6e,
+	0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52,
+	0x0b, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07,
+	0x74, 0x72, 0x65, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74,
+	0x72, 0x65, 0x65, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x68, 0x65, 0x69, 0x67, 0x68, 0x74, 0x18,
+	0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x68, 0x65, 0x69, 0x67, 0x68, 0x74, 0x12, 0x14, 0x0a,
+	0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x63, 0x6f,
+	0x75, 0x6e, 0x74, 0x22, 0xa7, 0x01, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x4f, 0x70, 0x4c, 0x6f, 0x67,
+	0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x47, 0x65,
+	0x74, 0x4f, 0x70, 0x4c, 0x6f, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x42,
+	0x6f, 0x64, 0x79, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x12, 0x2d, 0x0a, 0x09, 0x73, 0x69, 0x67,
+	0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74,
+	0x72, 0x65, 0x65, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x09, 0x73,
+	0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x1a, 0x33, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79,
+	0x12, 0x2b, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20,
+	0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x4c, 0x6f, 0x67, 0x4d, 0x6f,
+	0x76, 0x65, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x80, 0x01,
+	0x0a, 0x13, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x73,
+	0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x32, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18, 0x01, 0x20,
+	0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x48, 0x65, 0x61, 0x6c, 0x74,
+	0x68, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x42,
+	0x6f, 0x64, 0x79, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x12, 0x2d, 0x0a, 0x09, 0x73, 0x69, 0x67,
+	0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74,
+	0x72, 0x65, 0x65, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x09, 0x73,
+	0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x1a, 0x06, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79,
+	0x22, 0x7e, 0x0a, 0x12, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x52,
+	0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x31, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18, 0x01,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x48, 0x65, 0x61, 0x6c,
+	0x74, 0x68, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x42,
+	0x6f, 0x64, 0x79, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x12, 0x2d, 0x0a, 0x09, 0x73, 0x69, 0x67,
+	0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74,
+	0x72, 0x65, 0x65, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x09, 0x73,
+	0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x1a, 0x06, 0x0a, 0x04, 0x42, 0x6f, 0x64, 0x79,
+	0x32, 0xd6, 0x04, 0x0a, 0x0b, 0x54, 0x72, 0x65, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65,
+	0x12, 0x2a, 0x0a, 0x03, 0x41, 0x64, 0x64, 0x12, 0x10, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x41,
+	0x64, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x74, 0x72, 0x65, 0x65,
+	0x2e, 0x41, 0x64, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3c, 0x0a, 0x09,
+	0x41, 0x64, 0x64, 0x42, 0x79, 0x50, 0x61, 0x74, 0x68, 0x12, 0x16, 0x2e, 0x74, 0x72, 0x65, 0x65,
+	0x2e, 0x41, 0x64, 0x64, 0x42, 0x79, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+	0x74, 0x1a, 0x17, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x41, 0x64, 0x64, 0x42, 0x79, 0x50, 0x61,
+	0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x52, 0x65,
+	0x6d, 0x6f, 0x76, 0x65, 0x12, 0x13, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x52, 0x65, 0x6d, 0x6f,
+	0x76, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x74, 0x72, 0x65, 0x65,
+	0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12,
+	0x2d, 0x0a, 0x04, 0x4d, 0x6f, 0x76, 0x65, 0x12, 0x11, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x4d,
+	0x6f, 0x76, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x12, 0x2e, 0x74, 0x72, 0x65,
+	0x65, 0x2e, 0x4d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48,
+	0x0a, 0x0d, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x42, 0x79, 0x50, 0x61, 0x74, 0x68, 0x12,
+	0x1a, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x42, 0x79,
+	0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x74, 0x72,
+	0x65, 0x65, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x42, 0x79, 0x50, 0x61, 0x74, 0x68,
+	0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x41, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x53,
+	0x75, 0x62, 0x54, 0x72, 0x65, 0x65, 0x12, 0x17, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x47, 0x65,
+	0x74, 0x53, 0x75, 0x62, 0x54, 0x72, 0x65, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
+	0x18, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x75, 0x62, 0x54, 0x72, 0x65,
+	0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x30, 0x01, 0x12, 0x39, 0x0a, 0x08, 0x54,
+	0x72, 0x65, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x15, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x54,
+	0x72, 0x65, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16,
+	0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x54, 0x72, 0x65, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65,
+	0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x30, 0x0a, 0x05, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x12,
+	0x12, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x52, 0x65, 0x71, 0x75,
+	0x65, 0x73, 0x74, 0x1a, 0x13, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79,
+	0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3b, 0x0a, 0x08, 0x47, 0x65, 0x74, 0x4f,
+	0x70, 0x4c, 0x6f, 0x67, 0x12, 0x15, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x47, 0x65, 0x74, 0x4f,
+	0x70, 0x4c, 0x6f, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x74, 0x72,
+	0x65, 0x65, 0x2e, 0x47, 0x65, 0x74, 0x4f, 0x70, 0x4c, 0x6f, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f,
+	0x6e, 0x73, 0x65, 0x30, 0x01, 0x12, 0x42, 0x0a, 0x0b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x63,
+	0x68, 0x65, 0x63, 0x6b, 0x12, 0x18, 0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x48, 0x65, 0x61, 0x6c,
+	0x74, 0x68, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19,
+	0x2e, 0x74, 0x72, 0x65, 0x65, 0x2e, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x63, 0x68, 0x65, 0x63,
+	0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x3e, 0x5a, 0x3c, 0x67, 0x69, 0x74,
+	0x2e, 0x66, 0x72, 0x6f, 0x73, 0x74, 0x66, 0x73, 0x2e, 0x69, 0x6e, 0x66, 0x6f, 0x2f, 0x54, 0x72,
+	0x75, 0x65, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x4c, 0x61, 0x62, 0x2f, 0x66, 0x72, 0x6f, 0x73, 0x74,
+	0x66, 0x73, 0x2d, 0x6e, 0x6f, 0x64, 0x65, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x73, 0x65, 0x72, 0x76,
+	0x69, 0x63, 0x65, 0x73, 0x2f, 0x74, 0x72, 0x65, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+	0x33,
 }
 
 var (
diff --git a/pkg/services/tree/service.proto b/pkg/services/tree/service.proto
index 582e748470..88bf0bca45 100644
--- a/pkg/services/tree/service.proto
+++ b/pkg/services/tree/service.proto
@@ -242,8 +242,8 @@ message GetSubTreeRequest {
     bytes container_id = 1;
     // The name of the tree.
     string tree_id = 2;
-    // ID of the root node of a subtree.
-    uint64 root_id = 3;
+    // IDs of the root nodes of a subtree forest.
+    repeated uint64 root_id = 3 [ packed = false ];
     // Optional depth of the traversal. Zero means return only root.
     // Maximum depth is 10.
     uint32 depth = 4;
@@ -262,11 +262,11 @@ message GetSubTreeRequest {
 message GetSubTreeResponse {
   message Body {
     // ID of the node.
-    uint64 node_id = 1;
+    repeated uint64 node_id = 1 [ packed = false ];
     // ID of the parent.
-    uint64 parent_id = 2;
+    repeated uint64 parent_id = 2 [ packed = false ];
     // Time node was first added to a tree.
-    uint64 timestamp = 3;
+    repeated uint64 timestamp = 3 [ packed = false ];
     // Node meta-information.
     repeated KeyValue meta = 4;
   }
diff --git a/pkg/services/tree/service_frostfs.pb.go b/pkg/services/tree/service_frostfs.pb.go
index 4b50af2aac..1a49c5c0c1 100644
--- a/pkg/services/tree/service_frostfs.pb.go
+++ b/pkg/services/tree/service_frostfs.pb.go
@@ -2,6 +2,11 @@
 
 package tree
 
+import (
+	binary "encoding/binary"
+	protowire "google.golang.org/protobuf/encoding/protowire"
+)
+
 import "git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/util/proto"
 
 // StableSize returns the size of x in protobuf format.
@@ -995,7 +1000,9 @@ func (x *GetSubTreeRequest_Body) StableSize() (size int) {
 	}
 	size += proto.BytesSize(1, x.ContainerId)
 	size += proto.StringSize(2, x.TreeId)
-	size += proto.UInt64Size(3, x.RootId)
+	for i := range x.RootId {
+		size += protowire.SizeGroup(protowire.Number(3), protowire.SizeVarint(x.RootId[i]))
+	}
 	size += proto.UInt32Size(4, x.Depth)
 	size += proto.BytesSize(5, x.BearerToken)
 	size += proto.NestedStructureSize(6, x.OrderBy)
@@ -1020,7 +1027,13 @@ func (x *GetSubTreeRequest_Body) StableMarshal(buf []byte) []byte {
 	var offset int
 	offset += proto.BytesMarshal(1, buf[offset:], x.ContainerId)
 	offset += proto.StringMarshal(2, buf[offset:], x.TreeId)
-	offset += proto.UInt64Marshal(3, buf[offset:], x.RootId)
+	for i := range x.RootId {
+		{
+			prefix := protowire.EncodeTag(protowire.Number(3), protowire.VarintType)
+			offset += binary.PutUvarint(buf[offset:], uint64(prefix))
+			offset += binary.PutUvarint(buf[offset:], x.RootId[i])
+		}
+	}
 	offset += proto.UInt32Marshal(4, buf[offset:], x.Depth)
 	offset += proto.BytesMarshal(5, buf[offset:], x.BearerToken)
 	offset += proto.NestedStructureMarshal(6, buf[offset:], x.OrderBy)
@@ -1089,9 +1102,15 @@ func (x *GetSubTreeResponse_Body) StableSize() (size int) {
 	if x == nil {
 		return 0
 	}
-	size += proto.UInt64Size(1, x.NodeId)
-	size += proto.UInt64Size(2, x.ParentId)
-	size += proto.UInt64Size(3, x.Timestamp)
+	for i := range x.NodeId {
+		size += protowire.SizeGroup(protowire.Number(1), protowire.SizeVarint(x.NodeId[i]))
+	}
+	for i := range x.ParentId {
+		size += protowire.SizeGroup(protowire.Number(2), protowire.SizeVarint(x.ParentId[i]))
+	}
+	for i := range x.Timestamp {
+		size += protowire.SizeGroup(protowire.Number(3), protowire.SizeVarint(x.Timestamp[i]))
+	}
 	for i := range x.Meta {
 		size += proto.NestedStructureSize(4, x.Meta[i])
 	}
@@ -1114,9 +1133,27 @@ func (x *GetSubTreeResponse_Body) StableMarshal(buf []byte) []byte {
 		buf = make([]byte, x.StableSize())
 	}
 	var offset int
-	offset += proto.UInt64Marshal(1, buf[offset:], x.NodeId)
-	offset += proto.UInt64Marshal(2, buf[offset:], x.ParentId)
-	offset += proto.UInt64Marshal(3, buf[offset:], x.Timestamp)
+	for i := range x.NodeId {
+		{
+			prefix := protowire.EncodeTag(protowire.Number(1), protowire.VarintType)
+			offset += binary.PutUvarint(buf[offset:], uint64(prefix))
+			offset += binary.PutUvarint(buf[offset:], x.NodeId[i])
+		}
+	}
+	for i := range x.ParentId {
+		{
+			prefix := protowire.EncodeTag(protowire.Number(2), protowire.VarintType)
+			offset += binary.PutUvarint(buf[offset:], uint64(prefix))
+			offset += binary.PutUvarint(buf[offset:], x.ParentId[i])
+		}
+	}
+	for i := range x.Timestamp {
+		{
+			prefix := protowire.EncodeTag(protowire.Number(3), protowire.VarintType)
+			offset += binary.PutUvarint(buf[offset:], uint64(prefix))
+			offset += binary.PutUvarint(buf[offset:], x.Timestamp[i])
+		}
+	}
 	for i := range x.Meta {
 		offset += proto.NestedStructureMarshal(4, buf[offset:], x.Meta[i])
 	}
diff --git a/pkg/services/tree/service_grpc.pb.go b/pkg/services/tree/service_grpc.pb.go
index 63f96e11a5..4c293a4c03 100644
--- a/pkg/services/tree/service_grpc.pb.go
+++ b/pkg/services/tree/service_grpc.pb.go
@@ -3,7 +3,7 @@
 
 // Code generated by protoc-gen-go-grpc. DO NOT EDIT.
 // versions:
-// - protoc-gen-go-grpc v1.3.0
+// - protoc-gen-go-grpc v1.4.0
 // - protoc             v4.25.0
 // source: pkg/services/tree/service.proto
 
@@ -18,8 +18,8 @@ import (
 
 // This is a compile-time assertion to ensure that this generated file
 // is compatible with the grpc package it is being compiled against.
-// Requires gRPC-Go v1.32.0 or later.
-const _ = grpc.SupportPackageIsVersion7
+// Requires gRPC-Go v1.62.0 or later.
+const _ = grpc.SupportPackageIsVersion8
 
 const (
 	TreeService_Add_FullMethodName           = "/tree.TreeService/Add"
@@ -70,8 +70,9 @@ func NewTreeServiceClient(cc grpc.ClientConnInterface) TreeServiceClient {
 }
 
 func (c *treeServiceClient) Add(ctx context.Context, in *AddRequest, opts ...grpc.CallOption) (*AddResponse, error) {
+	cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
 	out := new(AddResponse)
-	err := c.cc.Invoke(ctx, TreeService_Add_FullMethodName, in, out, opts...)
+	err := c.cc.Invoke(ctx, TreeService_Add_FullMethodName, in, out, cOpts...)
 	if err != nil {
 		return nil, err
 	}
@@ -79,8 +80,9 @@ func (c *treeServiceClient) Add(ctx context.Context, in *AddRequest, opts ...grp
 }
 
 func (c *treeServiceClient) AddByPath(ctx context.Context, in *AddByPathRequest, opts ...grpc.CallOption) (*AddByPathResponse, error) {
+	cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
 	out := new(AddByPathResponse)
-	err := c.cc.Invoke(ctx, TreeService_AddByPath_FullMethodName, in, out, opts...)
+	err := c.cc.Invoke(ctx, TreeService_AddByPath_FullMethodName, in, out, cOpts...)
 	if err != nil {
 		return nil, err
 	}
@@ -88,8 +90,9 @@ func (c *treeServiceClient) AddByPath(ctx context.Context, in *AddByPathRequest,
 }
 
 func (c *treeServiceClient) Remove(ctx context.Context, in *RemoveRequest, opts ...grpc.CallOption) (*RemoveResponse, error) {
+	cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
 	out := new(RemoveResponse)
-	err := c.cc.Invoke(ctx, TreeService_Remove_FullMethodName, in, out, opts...)
+	err := c.cc.Invoke(ctx, TreeService_Remove_FullMethodName, in, out, cOpts...)
 	if err != nil {
 		return nil, err
 	}
@@ -97,8 +100,9 @@ func (c *treeServiceClient) Remove(ctx context.Context, in *RemoveRequest, opts
 }
 
 func (c *treeServiceClient) Move(ctx context.Context, in *MoveRequest, opts ...grpc.CallOption) (*MoveResponse, error) {
+	cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
 	out := new(MoveResponse)
-	err := c.cc.Invoke(ctx, TreeService_Move_FullMethodName, in, out, opts...)
+	err := c.cc.Invoke(ctx, TreeService_Move_FullMethodName, in, out, cOpts...)
 	if err != nil {
 		return nil, err
 	}
@@ -106,8 +110,9 @@ func (c *treeServiceClient) Move(ctx context.Context, in *MoveRequest, opts ...g
 }
 
 func (c *treeServiceClient) GetNodeByPath(ctx context.Context, in *GetNodeByPathRequest, opts ...grpc.CallOption) (*GetNodeByPathResponse, error) {
+	cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
 	out := new(GetNodeByPathResponse)
-	err := c.cc.Invoke(ctx, TreeService_GetNodeByPath_FullMethodName, in, out, opts...)
+	err := c.cc.Invoke(ctx, TreeService_GetNodeByPath_FullMethodName, in, out, cOpts...)
 	if err != nil {
 		return nil, err
 	}
@@ -115,11 +120,12 @@ func (c *treeServiceClient) GetNodeByPath(ctx context.Context, in *GetNodeByPath
 }
 
 func (c *treeServiceClient) GetSubTree(ctx context.Context, in *GetSubTreeRequest, opts ...grpc.CallOption) (TreeService_GetSubTreeClient, error) {
-	stream, err := c.cc.NewStream(ctx, &TreeService_ServiceDesc.Streams[0], TreeService_GetSubTree_FullMethodName, opts...)
+	cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
+	stream, err := c.cc.NewStream(ctx, &TreeService_ServiceDesc.Streams[0], TreeService_GetSubTree_FullMethodName, cOpts...)
 	if err != nil {
 		return nil, err
 	}
-	x := &treeServiceGetSubTreeClient{stream}
+	x := &treeServiceGetSubTreeClient{ClientStream: stream}
 	if err := x.ClientStream.SendMsg(in); err != nil {
 		return nil, err
 	}
@@ -147,8 +153,9 @@ func (x *treeServiceGetSubTreeClient) Recv() (*GetSubTreeResponse, error) {
 }
 
 func (c *treeServiceClient) TreeList(ctx context.Context, in *TreeListRequest, opts ...grpc.CallOption) (*TreeListResponse, error) {
+	cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
 	out := new(TreeListResponse)
-	err := c.cc.Invoke(ctx, TreeService_TreeList_FullMethodName, in, out, opts...)
+	err := c.cc.Invoke(ctx, TreeService_TreeList_FullMethodName, in, out, cOpts...)
 	if err != nil {
 		return nil, err
 	}
@@ -156,8 +163,9 @@ func (c *treeServiceClient) TreeList(ctx context.Context, in *TreeListRequest, o
 }
 
 func (c *treeServiceClient) Apply(ctx context.Context, in *ApplyRequest, opts ...grpc.CallOption) (*ApplyResponse, error) {
+	cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
 	out := new(ApplyResponse)
-	err := c.cc.Invoke(ctx, TreeService_Apply_FullMethodName, in, out, opts...)
+	err := c.cc.Invoke(ctx, TreeService_Apply_FullMethodName, in, out, cOpts...)
 	if err != nil {
 		return nil, err
 	}
@@ -165,11 +173,12 @@ func (c *treeServiceClient) Apply(ctx context.Context, in *ApplyRequest, opts ..
 }
 
 func (c *treeServiceClient) GetOpLog(ctx context.Context, in *GetOpLogRequest, opts ...grpc.CallOption) (TreeService_GetOpLogClient, error) {
-	stream, err := c.cc.NewStream(ctx, &TreeService_ServiceDesc.Streams[1], TreeService_GetOpLog_FullMethodName, opts...)
+	cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
+	stream, err := c.cc.NewStream(ctx, &TreeService_ServiceDesc.Streams[1], TreeService_GetOpLog_FullMethodName, cOpts...)
 	if err != nil {
 		return nil, err
 	}
-	x := &treeServiceGetOpLogClient{stream}
+	x := &treeServiceGetOpLogClient{ClientStream: stream}
 	if err := x.ClientStream.SendMsg(in); err != nil {
 		return nil, err
 	}
@@ -197,8 +206,9 @@ func (x *treeServiceGetOpLogClient) Recv() (*GetOpLogResponse, error) {
 }
 
 func (c *treeServiceClient) Healthcheck(ctx context.Context, in *HealthcheckRequest, opts ...grpc.CallOption) (*HealthcheckResponse, error) {
+	cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
 	out := new(HealthcheckResponse)
-	err := c.cc.Invoke(ctx, TreeService_Healthcheck_FullMethodName, in, out, opts...)
+	err := c.cc.Invoke(ctx, TreeService_Healthcheck_FullMethodName, in, out, cOpts...)
 	if err != nil {
 		return nil, err
 	}
@@ -373,7 +383,7 @@ func _TreeService_GetSubTree_Handler(srv interface{}, stream grpc.ServerStream)
 	if err := stream.RecvMsg(m); err != nil {
 		return err
 	}
-	return srv.(TreeServiceServer).GetSubTree(m, &treeServiceGetSubTreeServer{stream})
+	return srv.(TreeServiceServer).GetSubTree(m, &treeServiceGetSubTreeServer{ServerStream: stream})
 }
 
 type TreeService_GetSubTreeServer interface {
@@ -430,7 +440,7 @@ func _TreeService_GetOpLog_Handler(srv interface{}, stream grpc.ServerStream) er
 	if err := stream.RecvMsg(m); err != nil {
 		return err
 	}
-	return srv.(TreeServiceServer).GetOpLog(m, &treeServiceGetOpLogServer{stream})
+	return srv.(TreeServiceServer).GetOpLog(m, &treeServiceGetOpLogServer{ServerStream: stream})
 }
 
 type TreeService_GetOpLogServer interface {