mirror of
https://github.com/grafana/grafana.git
synced 2025-02-25 18:55:37 -06:00
Live: using organisation IDs throughout stack (#33714)
This commit is contained in:
parent
17305ccda3
commit
95a356a840
@ -271,7 +271,7 @@ func (hs *HTTPServer) deleteDashboard(c *models.ReqContext) response.Response {
|
||||
}
|
||||
|
||||
if hs.Live != nil {
|
||||
err = hs.Live.GrafanaScope.Dashboards.DashboardDeleted(c.ToUserDisplayDTO(), dash.Uid)
|
||||
err := hs.Live.GrafanaScope.Dashboards.DashboardDeleted(c.OrgId, c.ToUserDisplayDTO(), dash.Uid)
|
||||
if err != nil {
|
||||
hs.log.Error("Failed to broadcast delete info", "dashboard", dash.Uid, "error", err)
|
||||
}
|
||||
@ -346,10 +346,10 @@ func (hs *HTTPServer) PostDashboard(c *models.ReqContext, cmd models.SaveDashboa
|
||||
// This will broadcast all save requets only if a `gitops` observer exists.
|
||||
// gitops is useful when trying to save dashboards in an environment where the user can not save
|
||||
channel := hs.Live.GrafanaScope.Dashboards
|
||||
liveerr := channel.DashboardSaved(c.SignedInUser.ToUserDisplayDTO(), cmd.Message, dashboard, err)
|
||||
liveerr := channel.DashboardSaved(c.SignedInUser.OrgId, c.SignedInUser.ToUserDisplayDTO(), cmd.Message, dashboard, err)
|
||||
|
||||
// When an error exists, but the value broadcast to a gitops listener return 202
|
||||
if liveerr == nil && err != nil && channel.HasGitOpsObserver() {
|
||||
if liveerr == nil && err != nil && channel.HasGitOpsObserver(c.SignedInUser.OrgId) {
|
||||
return response.JSON(202, util.DynMap{
|
||||
"status": "pending",
|
||||
"message": "changes were broadcast to the gitops listener",
|
||||
|
@ -9,10 +9,10 @@ import (
|
||||
)
|
||||
|
||||
// ChannelPublisher writes data into a channel. Note that permissions are not checked.
|
||||
type ChannelPublisher func(channel string, data []byte) error
|
||||
type ChannelPublisher func(orgID int64, channel string, data []byte) error
|
||||
|
||||
// ChannelClientCount will return the number of clients for a channel
|
||||
type ChannelClientCount func(channel string) (int, error)
|
||||
type ChannelClientCount func(orgID int64, channel string) (int, error)
|
||||
|
||||
// SubscribeEvent contains subscription data.
|
||||
type SubscribeEvent struct {
|
||||
@ -70,14 +70,14 @@ type DashboardActivityChannel interface {
|
||||
// gitops workflow that knows if the value was saved to the local database or not
|
||||
// in many cases all direct save requests will fail, but the request should be forwarded
|
||||
// to any gitops observers
|
||||
DashboardSaved(user *UserDisplayDTO, message string, dashboard *Dashboard, err error) error
|
||||
DashboardSaved(orgID int64, user *UserDisplayDTO, message string, dashboard *Dashboard, err error) error
|
||||
|
||||
// Called when a dashboard is deleted
|
||||
DashboardDeleted(user *UserDisplayDTO, uid string) error
|
||||
DashboardDeleted(orgID int64, user *UserDisplayDTO, uid string) error
|
||||
|
||||
// Experimental! Indicate is GitOps is active. This really means
|
||||
// someone is subscribed to the `grafana/dashboards/gitops` channel
|
||||
HasGitOpsObserver() bool
|
||||
HasGitOpsObserver(orgID int64) bool
|
||||
}
|
||||
|
||||
type LiveMessage struct {
|
||||
|
@ -40,13 +40,13 @@ func (s *Demultiplexer) OnSubscribe(_ context.Context, _ *models.SignedInUser, _
|
||||
return models.SubscribeReply{}, backend.SubscribeStreamStatusPermissionDenied, nil
|
||||
}
|
||||
|
||||
func (s *Demultiplexer) OnPublish(ctx context.Context, _ *models.SignedInUser, evt models.PublishEvent) (models.PublishReply, backend.PublishStreamStatus, error) {
|
||||
func (s *Demultiplexer) OnPublish(ctx context.Context, u *models.SignedInUser, evt models.PublishEvent) (models.PublishReply, backend.PublishStreamStatus, error) {
|
||||
urlValues, ok := livecontext.GetContextValues(ctx)
|
||||
if !ok {
|
||||
return models.PublishReply{}, 0, errors.New("error extracting context url values")
|
||||
}
|
||||
|
||||
stream, err := s.managedStreamRunner.GetOrCreateStream(s.streamID)
|
||||
stream, err := s.managedStreamRunner.GetOrCreateStream(u.OrgId, s.streamID)
|
||||
if err != nil {
|
||||
logger.Error("Error getting stream", "error", err, "streamId", s.streamID)
|
||||
return models.PublishReply{}, 0, err
|
||||
@ -69,7 +69,7 @@ func (s *Demultiplexer) OnPublish(ctx context.Context, _ *models.SignedInUser, e
|
||||
return models.PublishReply{}, 0, err
|
||||
}
|
||||
for _, mf := range metricFrames {
|
||||
err := stream.Push(mf.Key(), mf.Frame(), unstableSchema)
|
||||
err := stream.Push(u.OrgId, mf.Key(), mf.Frame(), unstableSchema)
|
||||
if err != nil {
|
||||
return models.PublishReply{}, 0, err
|
||||
}
|
||||
|
@ -139,7 +139,7 @@ func (h *DashboardHandler) OnPublish(ctx context.Context, user *models.SignedInU
|
||||
}
|
||||
|
||||
// DashboardSaved should broadcast to the appropriate stream
|
||||
func (h *DashboardHandler) publish(event dashboardEvent) error {
|
||||
func (h *DashboardHandler) publish(orgID int64, event dashboardEvent) error {
|
||||
msg, err := json.Marshal(event)
|
||||
if err != nil {
|
||||
return err
|
||||
@ -147,19 +147,19 @@ func (h *DashboardHandler) publish(event dashboardEvent) error {
|
||||
|
||||
// Only broadcast non-error events
|
||||
if event.Error == "" {
|
||||
err = h.Publisher("grafana/dashboard/uid/"+event.UID, msg)
|
||||
err = h.Publisher(orgID, "grafana/dashboard/uid/"+event.UID, msg)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// Send everything to the gitops channel
|
||||
return h.Publisher(GITOPS_CHANNEL, msg)
|
||||
return h.Publisher(orgID, GITOPS_CHANNEL, msg)
|
||||
}
|
||||
|
||||
// DashboardSaved will broadcast to all connected dashboards
|
||||
func (h *DashboardHandler) DashboardSaved(user *models.UserDisplayDTO, message string, dashboard *models.Dashboard, err error) error {
|
||||
if err != nil && !h.HasGitOpsObserver() {
|
||||
func (h *DashboardHandler) DashboardSaved(orgID int64, user *models.UserDisplayDTO, message string, dashboard *models.Dashboard, err error) error {
|
||||
if err != nil && !h.HasGitOpsObserver(orgID) {
|
||||
return nil // only broadcast if it was OK
|
||||
}
|
||||
|
||||
@ -175,12 +175,12 @@ func (h *DashboardHandler) DashboardSaved(user *models.UserDisplayDTO, message s
|
||||
msg.Error = err.Error()
|
||||
}
|
||||
|
||||
return h.publish(msg)
|
||||
return h.publish(orgID, msg)
|
||||
}
|
||||
|
||||
// DashboardDeleted will broadcast to all connected dashboards
|
||||
func (h *DashboardHandler) DashboardDeleted(user *models.UserDisplayDTO, uid string) error {
|
||||
return h.publish(dashboardEvent{
|
||||
func (h *DashboardHandler) DashboardDeleted(orgID int64, user *models.UserDisplayDTO, uid string) error {
|
||||
return h.publish(orgID, dashboardEvent{
|
||||
UID: uid,
|
||||
Action: ACTION_DELETED,
|
||||
User: user,
|
||||
@ -188,8 +188,8 @@ func (h *DashboardHandler) DashboardDeleted(user *models.UserDisplayDTO, uid str
|
||||
}
|
||||
|
||||
// HasGitOpsObserver will return true if anyone is listening to the `gitops` channel
|
||||
func (h *DashboardHandler) HasGitOpsObserver() bool {
|
||||
count, err := h.ClientCount(GITOPS_CHANNEL)
|
||||
func (h *DashboardHandler) HasGitOpsObserver(orgID int64) bool {
|
||||
count, err := h.ClientCount(orgID, GITOPS_CHANNEL)
|
||||
if err != nil {
|
||||
logger.Error("error getting client count", "error", err)
|
||||
return false
|
||||
|
@ -82,7 +82,7 @@ func (r *PluginPathRunner) OnSubscribe(ctx context.Context, user *models.SignedI
|
||||
}
|
||||
|
||||
if resp.UseRunStream {
|
||||
submitResult, err := r.runStreamManager.SubmitStream(ctx, e.Channel, r.path, pCtx, r.handler)
|
||||
submitResult, err := r.runStreamManager.SubmitStream(ctx, user.OrgId, e.Channel, r.path, pCtx, r.handler)
|
||||
if err != nil {
|
||||
logger.Error("Error submitting stream to manager", "error", err, "path", r.path)
|
||||
return models.SubscribeReply{}, 0, centrifuge.ErrorInternal
|
||||
|
@ -24,6 +24,7 @@ import (
|
||||
"github.com/grafana/grafana/pkg/services/live/features"
|
||||
"github.com/grafana/grafana/pkg/services/live/livecontext"
|
||||
"github.com/grafana/grafana/pkg/services/live/managedstream"
|
||||
"github.com/grafana/grafana/pkg/services/live/orgchannel"
|
||||
"github.com/grafana/grafana/pkg/services/live/pushws"
|
||||
"github.com/grafana/grafana/pkg/services/live/runstream"
|
||||
"github.com/grafana/grafana/pkg/services/sqlstore"
|
||||
@ -285,18 +286,32 @@ func runConcurrentlyIfNeeded(ctx context.Context, semaphore chan struct{}, fn fu
|
||||
|
||||
func (g *GrafanaLive) handleOnSubscribe(client *centrifuge.Client, e centrifuge.SubscribeEvent) (centrifuge.SubscribeReply, error) {
|
||||
logger.Debug("Client wants to subscribe", "user", client.UserID(), "client", client.ID(), "channel", e.Channel)
|
||||
|
||||
user, ok := livecontext.GetContextSignedUser(client.Context())
|
||||
if !ok {
|
||||
logger.Error("No user found in context", "user", client.UserID(), "client", client.ID(), "channel", e.Channel)
|
||||
return centrifuge.SubscribeReply{}, centrifuge.ErrorInternal
|
||||
}
|
||||
handler, addr, err := g.GetChannelHandler(user, e.Channel)
|
||||
|
||||
// See a detailed comment for StripOrgID about orgID management in Live.
|
||||
orgID, channel, err := orgchannel.StripOrgID(e.Channel)
|
||||
if err != nil {
|
||||
logger.Error("Error parsing channel", "user", client.UserID(), "client", client.ID(), "channel", e.Channel, "error", err)
|
||||
return centrifuge.SubscribeReply{}, centrifuge.ErrorInternal
|
||||
}
|
||||
|
||||
if user.OrgId != orgID {
|
||||
logger.Info("Error subscribing: wrong orgId", "user", client.UserID(), "client", client.ID(), "channel", e.Channel)
|
||||
return centrifuge.SubscribeReply{}, centrifuge.ErrorPermissionDenied
|
||||
}
|
||||
|
||||
handler, addr, err := g.GetChannelHandler(user, channel)
|
||||
if err != nil {
|
||||
logger.Error("Error getting channel handler", "user", client.UserID(), "client", client.ID(), "channel", e.Channel, "error", err)
|
||||
return centrifuge.SubscribeReply{}, centrifuge.ErrorInternal
|
||||
}
|
||||
reply, status, err := handler.OnSubscribe(client.Context(), user, models.SubscribeEvent{
|
||||
Channel: e.Channel,
|
||||
Channel: channel,
|
||||
Path: addr.Path,
|
||||
})
|
||||
if err != nil {
|
||||
@ -322,18 +337,32 @@ func (g *GrafanaLive) handleOnSubscribe(client *centrifuge.Client, e centrifuge.
|
||||
|
||||
func (g *GrafanaLive) handleOnPublish(client *centrifuge.Client, e centrifuge.PublishEvent) (centrifuge.PublishReply, error) {
|
||||
logger.Debug("Client wants to publish", "user", client.UserID(), "client", client.ID(), "channel", e.Channel)
|
||||
|
||||
user, ok := livecontext.GetContextSignedUser(client.Context())
|
||||
if !ok {
|
||||
logger.Error("No user found in context", "user", client.UserID(), "client", client.ID(), "channel", e.Channel)
|
||||
return centrifuge.PublishReply{}, centrifuge.ErrorInternal
|
||||
}
|
||||
handler, addr, err := g.GetChannelHandler(user, e.Channel)
|
||||
|
||||
// See a detailed comment for StripOrgID about orgID management in Live.
|
||||
orgID, channel, err := orgchannel.StripOrgID(e.Channel)
|
||||
if err != nil {
|
||||
logger.Error("Error parsing channel", "user", client.UserID(), "client", client.ID(), "channel", e.Channel, "error", err)
|
||||
return centrifuge.PublishReply{}, centrifuge.ErrorInternal
|
||||
}
|
||||
|
||||
if user.OrgId != orgID {
|
||||
logger.Info("Error subscribing: wrong orgId", "user", client.UserID(), "client", client.ID(), "channel", e.Channel)
|
||||
return centrifuge.PublishReply{}, centrifuge.ErrorPermissionDenied
|
||||
}
|
||||
|
||||
handler, addr, err := g.GetChannelHandler(user, channel)
|
||||
if err != nil {
|
||||
logger.Error("Error getting channel handler", "user", client.UserID(), "client", client.ID(), "channel", e.Channel, "error", err)
|
||||
return centrifuge.PublishReply{}, centrifuge.ErrorInternal
|
||||
}
|
||||
reply, status, err := handler.OnPublish(client.Context(), user, models.PublishEvent{
|
||||
Channel: e.Channel,
|
||||
Channel: channel,
|
||||
Path: addr.Path,
|
||||
Data: e.Data,
|
||||
})
|
||||
@ -478,8 +507,8 @@ func (g *GrafanaLive) handlePluginScope(_ *models.SignedInUser, namespace string
|
||||
), nil
|
||||
}
|
||||
|
||||
func (g *GrafanaLive) handleStreamScope(_ *models.SignedInUser, namespace string) (models.ChannelHandlerFactory, error) {
|
||||
return g.ManagedStreamRunner.GetOrCreateStream(namespace)
|
||||
func (g *GrafanaLive) handleStreamScope(u *models.SignedInUser, namespace string) (models.ChannelHandlerFactory, error) {
|
||||
return g.ManagedStreamRunner.GetOrCreateStream(u.OrgId, namespace)
|
||||
}
|
||||
|
||||
func (g *GrafanaLive) handlePushScope(_ *models.SignedInUser, namespace string) (models.ChannelHandlerFactory, error) {
|
||||
@ -512,14 +541,14 @@ func (g *GrafanaLive) handleDatasourceScope(user *models.SignedInUser, namespace
|
||||
}
|
||||
|
||||
// Publish sends the data to the channel without checking permissions etc
|
||||
func (g *GrafanaLive) Publish(channel string, data []byte) error {
|
||||
_, err := g.node.Publish(channel, data)
|
||||
func (g *GrafanaLive) Publish(orgID int64, channel string, data []byte) error {
|
||||
_, err := g.node.Publish(orgchannel.PrependOrgID(orgID, channel), data)
|
||||
return err
|
||||
}
|
||||
|
||||
// ClientCount returns the number of clients
|
||||
func (g *GrafanaLive) ClientCount(channel string) (int, error) {
|
||||
p, err := g.node.Presence(channel)
|
||||
func (g *GrafanaLive) ClientCount(orgID int64, channel string) (int, error) {
|
||||
p, err := g.node.Presence(orgchannel.PrependOrgID(orgID, channel))
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
@ -561,11 +590,11 @@ func (g *GrafanaLive) HandleHTTPPublish(ctx *models.ReqContext, cmd dtos.LivePub
|
||||
}
|
||||
|
||||
// HandleListHTTP returns metadata so the UI can build a nice form
|
||||
func (g *GrafanaLive) HandleListHTTP(_ *models.ReqContext) response.Response {
|
||||
func (g *GrafanaLive) HandleListHTTP(c *models.ReqContext) response.Response {
|
||||
info := util.DynMap{}
|
||||
channels := make([]util.DynMap, 0)
|
||||
for k, v := range g.ManagedStreamRunner.Streams() {
|
||||
channels = append(channels, v.ListChannels("stream/"+k+"/")...)
|
||||
for k, v := range g.ManagedStreamRunner.Streams(c.SignedInUser.OrgId) {
|
||||
channels = append(channels, v.ListChannels(c.SignedInUser.OrgId, "stream/"+k+"/")...)
|
||||
}
|
||||
|
||||
// Hardcode sample streams
|
||||
@ -595,7 +624,7 @@ func (g *GrafanaLive) HandleInfoHTTP(ctx *models.ReqContext) response.Response {
|
||||
path := ctx.Params("*")
|
||||
if path == "grafana/dashboards/gitops" {
|
||||
return response.JSON(200, util.DynMap{
|
||||
"active": g.GrafanaScope.Dashboards.HasGitOpsObserver(),
|
||||
"active": g.GrafanaScope.Dashboards.HasGitOpsObserver(ctx.SignedInUser.OrgId),
|
||||
})
|
||||
}
|
||||
return response.JSONStreaming(404, util.DynMap{
|
||||
|
@ -21,7 +21,7 @@ var (
|
||||
// Runner keeps ManagedStream per streamID.
|
||||
type Runner struct {
|
||||
mu sync.RWMutex
|
||||
streams map[string]*ManagedStream
|
||||
streams map[int64]map[string]*ManagedStream
|
||||
publisher models.ChannelPublisher
|
||||
}
|
||||
|
||||
@ -29,16 +29,19 @@ type Runner struct {
|
||||
func NewRunner(publisher models.ChannelPublisher) *Runner {
|
||||
return &Runner{
|
||||
publisher: publisher,
|
||||
streams: map[string]*ManagedStream{},
|
||||
streams: map[int64]map[string]*ManagedStream{},
|
||||
}
|
||||
}
|
||||
|
||||
// Streams returns a map of active managed streams (per streamID).
|
||||
func (r *Runner) Streams() map[string]*ManagedStream {
|
||||
func (r *Runner) Streams(orgID int64) map[string]*ManagedStream {
|
||||
r.mu.RLock()
|
||||
defer r.mu.RUnlock()
|
||||
streams := make(map[string]*ManagedStream, len(r.streams))
|
||||
for k, v := range r.streams {
|
||||
if _, ok := r.streams[orgID]; !ok {
|
||||
return map[string]*ManagedStream{}
|
||||
}
|
||||
streams := make(map[string]*ManagedStream, len(r.streams[orgID]))
|
||||
for k, v := range r.streams[orgID] {
|
||||
streams[k] = v
|
||||
}
|
||||
return streams
|
||||
@ -46,13 +49,17 @@ func (r *Runner) Streams() map[string]*ManagedStream {
|
||||
|
||||
// GetOrCreateStream -- for now this will create new manager for each key.
|
||||
// Eventually, the stream behavior will need to be configured explicitly
|
||||
func (r *Runner) GetOrCreateStream(streamID string) (*ManagedStream, error) {
|
||||
func (r *Runner) GetOrCreateStream(orgID int64, streamID string) (*ManagedStream, error) {
|
||||
r.mu.Lock()
|
||||
defer r.mu.Unlock()
|
||||
s, ok := r.streams[streamID]
|
||||
_, ok := r.streams[orgID]
|
||||
if !ok {
|
||||
r.streams[orgID] = map[string]*ManagedStream{}
|
||||
}
|
||||
s, ok := r.streams[orgID][streamID]
|
||||
if !ok {
|
||||
s = NewManagedStream(streamID, r.publisher)
|
||||
r.streams[streamID] = s
|
||||
r.streams[orgID][streamID] = s
|
||||
}
|
||||
return s, nil
|
||||
}
|
||||
@ -62,7 +69,7 @@ type ManagedStream struct {
|
||||
mu sync.RWMutex
|
||||
id string
|
||||
start time.Time
|
||||
last map[string]json.RawMessage
|
||||
last map[int64]map[string]json.RawMessage
|
||||
publisher models.ChannelPublisher
|
||||
}
|
||||
|
||||
@ -71,18 +78,22 @@ func NewManagedStream(id string, publisher models.ChannelPublisher) *ManagedStre
|
||||
return &ManagedStream{
|
||||
id: id,
|
||||
start: time.Now(),
|
||||
last: map[string]json.RawMessage{},
|
||||
last: map[int64]map[string]json.RawMessage{},
|
||||
publisher: publisher,
|
||||
}
|
||||
}
|
||||
|
||||
// ListChannels returns info for the UI about this stream.
|
||||
func (s *ManagedStream) ListChannels(prefix string) []util.DynMap {
|
||||
func (s *ManagedStream) ListChannels(orgID int64, prefix string) []util.DynMap {
|
||||
s.mu.RLock()
|
||||
defer s.mu.RUnlock()
|
||||
|
||||
info := make([]util.DynMap, 0, len(s.last))
|
||||
for k, v := range s.last {
|
||||
if _, ok := s.last[orgID]; !ok {
|
||||
return []util.DynMap{}
|
||||
}
|
||||
|
||||
info := make([]util.DynMap, 0, len(s.last[orgID]))
|
||||
for k, v := range s.last[orgID] {
|
||||
ch := util.DynMap{}
|
||||
ch["channel"] = prefix + k
|
||||
ch["data"] = v
|
||||
@ -93,7 +104,7 @@ func (s *ManagedStream) ListChannels(prefix string) []util.DynMap {
|
||||
|
||||
// Push sends frame to the stream and saves it for later retrieval by subscribers.
|
||||
// unstableSchema flag can be set to disable schema caching for a path.
|
||||
func (s *ManagedStream) Push(path string, frame *data.Frame, unstableSchema bool) error {
|
||||
func (s *ManagedStream) Push(orgID int64, path string, frame *data.Frame, unstableSchema bool) error {
|
||||
// Keep schema + data for last packet.
|
||||
frameJSON, err := data.FrameToJSON(frame, true, true)
|
||||
if err != nil {
|
||||
@ -105,8 +116,11 @@ func (s *ManagedStream) Push(path string, frame *data.Frame, unstableSchema bool
|
||||
// If schema is stable we can safely cache it, and only send values if
|
||||
// stream already has schema cached.
|
||||
s.mu.Lock()
|
||||
_, exists := s.last[path]
|
||||
s.last[path] = frameJSON
|
||||
if _, ok := s.last[orgID]; !ok {
|
||||
s.last[orgID] = map[string]json.RawMessage{}
|
||||
}
|
||||
_, exists := s.last[orgID][path]
|
||||
s.last[orgID][path] = frameJSON
|
||||
s.mu.Unlock()
|
||||
|
||||
// When the packet already exits, only send the data.
|
||||
@ -125,20 +139,26 @@ func (s *ManagedStream) Push(path string, frame *data.Frame, unstableSchema bool
|
||||
// And we don't want to cache schema for unstable case. But we still need to
|
||||
// set path to a map to make stream visible in UI stream select widget.
|
||||
s.mu.Lock()
|
||||
s.last[path] = nil
|
||||
if _, ok := s.last[orgID]; ok {
|
||||
s.last[orgID][path] = nil
|
||||
}
|
||||
s.mu.Unlock()
|
||||
}
|
||||
// The channel this will be posted into.
|
||||
channel := live.Channel{Scope: live.ScopeStream, Namespace: s.id, Path: path}.String()
|
||||
logger.Debug("Publish data to channel", "channel", channel, "dataLength", len(frameJSON))
|
||||
return s.publisher(channel, frameJSON)
|
||||
return s.publisher(orgID, channel, frameJSON)
|
||||
}
|
||||
|
||||
// getLastPacket retrieves schema for a channel.
|
||||
func (s *ManagedStream) getLastPacket(path string) (json.RawMessage, bool) {
|
||||
func (s *ManagedStream) getLastPacket(orgId int64, path string) (json.RawMessage, bool) {
|
||||
s.mu.RLock()
|
||||
defer s.mu.RUnlock()
|
||||
schema, ok := s.last[path]
|
||||
_, ok := s.last[orgId]
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
schema, ok := s.last[orgId][path]
|
||||
return schema, ok && schema != nil
|
||||
}
|
||||
|
||||
@ -146,23 +166,23 @@ func (s *ManagedStream) GetHandlerForPath(_ string) (models.ChannelHandler, erro
|
||||
return s, nil
|
||||
}
|
||||
|
||||
func (s *ManagedStream) OnSubscribe(_ context.Context, _ *models.SignedInUser, e models.SubscribeEvent) (models.SubscribeReply, backend.SubscribeStreamStatus, error) {
|
||||
func (s *ManagedStream) OnSubscribe(_ context.Context, u *models.SignedInUser, e models.SubscribeEvent) (models.SubscribeReply, backend.SubscribeStreamStatus, error) {
|
||||
reply := models.SubscribeReply{}
|
||||
packet, ok := s.getLastPacket(e.Path)
|
||||
packet, ok := s.getLastPacket(u.OrgId, e.Path)
|
||||
if ok {
|
||||
reply.Data = packet
|
||||
}
|
||||
return reply, backend.SubscribeStreamStatusOK, nil
|
||||
}
|
||||
|
||||
func (s *ManagedStream) OnPublish(_ context.Context, _ *models.SignedInUser, evt models.PublishEvent) (models.PublishReply, backend.PublishStreamStatus, error) {
|
||||
func (s *ManagedStream) OnPublish(_ context.Context, u *models.SignedInUser, evt models.PublishEvent) (models.PublishReply, backend.PublishStreamStatus, error) {
|
||||
var frame data.Frame
|
||||
err := json.Unmarshal(evt.Data, &frame)
|
||||
if err != nil {
|
||||
// Stream scope only deals with data frames.
|
||||
return models.PublishReply{}, 0, err
|
||||
}
|
||||
err = s.Push(evt.Path, &frame, true)
|
||||
err = s.Push(u.OrgId, evt.Path, &frame, true)
|
||||
if err != nil {
|
||||
// Stream scope only deals with data frames.
|
||||
return models.PublishReply{}, 0, err
|
||||
|
@ -7,35 +7,46 @@ import (
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
var noopPublisher = func(p string, b []byte) error {
|
||||
type testPublisher struct {
|
||||
orgID int64
|
||||
t *testing.T
|
||||
}
|
||||
|
||||
func (p *testPublisher) publish(orgID int64, _ string, _ []byte) error {
|
||||
require.Equal(p.t, p.orgID, orgID)
|
||||
return nil
|
||||
}
|
||||
|
||||
func TestNewManagedStream(t *testing.T) {
|
||||
c := NewManagedStream("a", noopPublisher)
|
||||
publisher := &testPublisher{orgID: 1, t: t}
|
||||
c := NewManagedStream("a", publisher.publish)
|
||||
require.NotNil(t, c)
|
||||
}
|
||||
|
||||
func TestManagedStream_GetLastPacket_UnstableSchema(t *testing.T) {
|
||||
c := NewManagedStream("a", noopPublisher)
|
||||
_, ok := c.getLastPacket("test")
|
||||
var orgID int64 = 1
|
||||
publisher := &testPublisher{orgID: orgID, t: t}
|
||||
c := NewManagedStream("a", publisher.publish)
|
||||
_, ok := c.getLastPacket(orgID, "test")
|
||||
require.False(t, ok)
|
||||
err := c.Push("test", data.NewFrame("hello"), true)
|
||||
err := c.Push(orgID, "test", data.NewFrame("hello"), true)
|
||||
require.NoError(t, err)
|
||||
|
||||
_, ok = c.getLastPacket("test")
|
||||
_, ok = c.getLastPacket(orgID, "test")
|
||||
require.NoError(t, err)
|
||||
require.False(t, ok)
|
||||
}
|
||||
|
||||
func TestManagedStream_GetLastPacket(t *testing.T) {
|
||||
c := NewManagedStream("a", noopPublisher)
|
||||
_, ok := c.getLastPacket("test")
|
||||
var orgID int64 = 1
|
||||
publisher := &testPublisher{orgID: orgID, t: t}
|
||||
c := NewManagedStream("a", publisher.publish)
|
||||
_, ok := c.getLastPacket(orgID, "test")
|
||||
require.False(t, ok)
|
||||
err := c.Push("test", data.NewFrame("hello"), false)
|
||||
err := c.Push(orgID, "test", data.NewFrame("hello"), false)
|
||||
require.NoError(t, err)
|
||||
|
||||
s, ok := c.getLastPacket("test")
|
||||
s, ok := c.getLastPacket(orgID, "test")
|
||||
require.NoError(t, err)
|
||||
require.True(t, ok)
|
||||
require.Equal(t, `{"schema":{"name":"hello","fields":[]},"data":{"values":[]}}`, string(s))
|
||||
|
30
pkg/services/live/orgchannel/orgchannel.go
Normal file
30
pkg/services/live/orgchannel/orgchannel.go
Normal file
@ -0,0 +1,30 @@
|
||||
package orgchannel
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strconv"
|
||||
"strings"
|
||||
)
|
||||
|
||||
// PrependOrgID prepends orgID to a channel.
|
||||
func PrependOrgID(orgID int64, channel string) string {
|
||||
return strconv.FormatInt(orgID, 10) + "/" + channel
|
||||
}
|
||||
|
||||
// StripOrgID strips organization ID from channel ID.
|
||||
// The reason why we strip orgID is because we need to maintain multi-tenancy.
|
||||
// Each organization can have the same channels which should not overlap. Due
|
||||
// to this every channel in Centrifuge has orgID prefix. Internally in Grafana
|
||||
// we strip this prefix since orgID is part of user identity and channel handlers
|
||||
// supposed to have the same business logic for all organizations.
|
||||
func StripOrgID(channel string) (int64, string, error) {
|
||||
parts := strings.SplitN(channel, "/", 2)
|
||||
if len(parts) != 2 {
|
||||
return 0, "", fmt.Errorf("malformed channel: %s", channel)
|
||||
}
|
||||
orgID, err := strconv.ParseInt(parts[0], 10, 64)
|
||||
if err != nil {
|
||||
return 0, "", fmt.Errorf("invalid orgID part: %s", parts[0])
|
||||
}
|
||||
return orgID, parts[1], nil
|
||||
}
|
29
pkg/services/live/orgchannel/orgchannel_test.go
Normal file
29
pkg/services/live/orgchannel/orgchannel_test.go
Normal file
@ -0,0 +1,29 @@
|
||||
package orgchannel
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func TestStripOrgID(t *testing.T) {
|
||||
channelID := "stream"
|
||||
_, _, err := StripOrgID(channelID)
|
||||
require.Error(t, err)
|
||||
|
||||
channelID = "plugin/testdata/random-20Hz-stream"
|
||||
_, _, err = StripOrgID(channelID)
|
||||
require.Error(t, err)
|
||||
|
||||
channelID = "1/plugin/testdata/random-20Hz-stream"
|
||||
orgID, channel, err := StripOrgID(channelID)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, int64(1), orgID)
|
||||
require.Equal(t, "plugin/testdata/random-20Hz-stream", channel)
|
||||
}
|
||||
|
||||
func TestPrependOrgID(t *testing.T) {
|
||||
channel := "plugin/testdata/random-20Hz-stream"
|
||||
channelID := PrependOrgID(2, channel)
|
||||
require.Equal(t, "2/plugin/testdata/random-20Hz-stream", channelID)
|
||||
}
|
@ -47,7 +47,7 @@ func (g *Gateway) Run(ctx context.Context) error {
|
||||
func (g *Gateway) Handle(ctx *models.ReqContext) {
|
||||
streamID := ctx.Params(":streamId")
|
||||
|
||||
stream, err := g.GrafanaLive.ManagedStreamRunner.GetOrCreateStream(streamID)
|
||||
stream, err := g.GrafanaLive.ManagedStreamRunner.GetOrCreateStream(ctx.SignedInUser.OrgId, streamID)
|
||||
if err != nil {
|
||||
logger.Error("Error getting stream", "error", err)
|
||||
ctx.Resp.WriteHeader(http.StatusInternalServerError)
|
||||
@ -88,7 +88,7 @@ func (g *Gateway) Handle(ctx *models.ReqContext) {
|
||||
// interval = "1s" vs flush_interval = "5s"
|
||||
|
||||
for _, mf := range metricFrames {
|
||||
err := stream.Push(mf.Key(), mf.Frame(), unstableSchema)
|
||||
err := stream.Push(ctx.SignedInUser.OrgId, mf.Key(), mf.Frame(), unstableSchema)
|
||||
if err != nil {
|
||||
ctx.Resp.WriteHeader(http.StatusInternalServerError)
|
||||
return
|
||||
|
@ -152,13 +152,20 @@ func (s *Handler) ServeHTTP(rw http.ResponseWriter, r *http.Request) {
|
||||
}
|
||||
}()
|
||||
|
||||
user, ok := livecontext.GetContextSignedUser(r.Context())
|
||||
if !ok {
|
||||
logger.Error("No user found in context")
|
||||
rw.WriteHeader(http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
|
||||
for {
|
||||
_, body, err := conn.ReadMessage()
|
||||
if err != nil {
|
||||
break
|
||||
}
|
||||
|
||||
stream, err := s.managedStreamRunner.GetOrCreateStream(streamID)
|
||||
stream, err := s.managedStreamRunner.GetOrCreateStream(user.OrgId, streamID)
|
||||
if err != nil {
|
||||
logger.Error("Error getting stream", "error", err)
|
||||
continue
|
||||
@ -184,7 +191,7 @@ func (s *Handler) ServeHTTP(rw http.ResponseWriter, r *http.Request) {
|
||||
}
|
||||
|
||||
for _, mf := range metricFrames {
|
||||
err := stream.Push(mf.Key(), mf.Frame(), unstableSchema)
|
||||
err := stream.Push(user.OrgId, mf.Key(), mf.Frame(), unstableSchema)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
@ -8,6 +8,7 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/grafana/grafana/pkg/infra/log"
|
||||
"github.com/grafana/grafana/pkg/services/live/orgchannel"
|
||||
|
||||
"github.com/grafana/grafana-plugin-sdk-go/backend"
|
||||
)
|
||||
@ -253,11 +254,11 @@ type submitResponse struct {
|
||||
|
||||
// SubmitStream submits stream handler in Manager to manage.
|
||||
// The stream will be opened and kept till channel has active subscribers.
|
||||
func (s *Manager) SubmitStream(ctx context.Context, channel string, path string, pCtx backend.PluginContext, streamRunner StreamRunner) (*submitResult, error) {
|
||||
func (s *Manager) SubmitStream(ctx context.Context, orgID int64, channel string, path string, pCtx backend.PluginContext, streamRunner StreamRunner) (*submitResult, error) {
|
||||
req := submitRequest{
|
||||
responseCh: make(chan submitResponse, 1),
|
||||
streamRequest: streamRequest{
|
||||
Channel: channel,
|
||||
Channel: orgchannel.PrependOrgID(orgID, channel),
|
||||
Path: path,
|
||||
PluginContext: pCtx,
|
||||
StreamRunner: streamRunner,
|
||||
|
@ -59,7 +59,7 @@ func TestStreamManager_SubmitStream_Send(t *testing.T) {
|
||||
startedCh := make(chan struct{})
|
||||
doneCh := make(chan struct{})
|
||||
|
||||
mockPacketSender.EXPECT().Send("test", gomock.Any()).Times(1)
|
||||
mockPacketSender.EXPECT().Send("1/test", gomock.Any()).Times(1)
|
||||
|
||||
mockStreamRunner := NewMockStreamRunner(mockCtrl)
|
||||
mockStreamRunner.EXPECT().RunStream(
|
||||
@ -76,12 +76,12 @@ func TestStreamManager_SubmitStream_Send(t *testing.T) {
|
||||
return ctx.Err()
|
||||
}).Times(1)
|
||||
|
||||
result, err := manager.SubmitStream(context.Background(), "test", "test", backend.PluginContext{}, mockStreamRunner)
|
||||
result, err := manager.SubmitStream(context.Background(), 1, "test", "test", backend.PluginContext{}, mockStreamRunner)
|
||||
require.NoError(t, err)
|
||||
require.False(t, result.StreamExists)
|
||||
|
||||
// try submit the same.
|
||||
result, err = manager.SubmitStream(context.Background(), "test", "test", backend.PluginContext{}, mockStreamRunner)
|
||||
result, err = manager.SubmitStream(context.Background(), 1, "test", "test", backend.PluginContext{}, mockStreamRunner)
|
||||
require.NoError(t, err)
|
||||
require.True(t, result.StreamExists)
|
||||
|
||||
@ -91,6 +91,76 @@ func TestStreamManager_SubmitStream_Send(t *testing.T) {
|
||||
waitWithTimeout(t, doneCh, time.Second)
|
||||
}
|
||||
|
||||
func TestStreamManager_SubmitStream_DifferentOrgID(t *testing.T) {
|
||||
mockCtrl := gomock.NewController(t)
|
||||
defer mockCtrl.Finish()
|
||||
|
||||
mockPacketSender := NewMockStreamPacketSender(mockCtrl)
|
||||
mockPresenceGetter := NewMockPresenceGetter(mockCtrl)
|
||||
|
||||
manager := NewManager(mockPacketSender, mockPresenceGetter)
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
go func() {
|
||||
_ = manager.Run(ctx)
|
||||
}()
|
||||
|
||||
startedCh1 := make(chan struct{})
|
||||
startedCh2 := make(chan struct{})
|
||||
doneCh1 := make(chan struct{})
|
||||
doneCh2 := make(chan struct{})
|
||||
|
||||
mockPacketSender.EXPECT().Send("1/test", gomock.Any()).Times(1)
|
||||
mockPacketSender.EXPECT().Send("2/test", gomock.Any()).Times(1)
|
||||
|
||||
mockStreamRunner1 := NewMockStreamRunner(mockCtrl)
|
||||
mockStreamRunner1.EXPECT().RunStream(
|
||||
gomock.Any(), gomock.Any(), gomock.Any(),
|
||||
).DoAndReturn(func(ctx context.Context, req *backend.RunStreamRequest, sender backend.StreamPacketSender) error {
|
||||
require.Equal(t, "test", req.Path)
|
||||
close(startedCh1)
|
||||
err := sender.Send(&backend.StreamPacket{
|
||||
Data: []byte("test"),
|
||||
})
|
||||
require.NoError(t, err)
|
||||
<-ctx.Done()
|
||||
close(doneCh1)
|
||||
return ctx.Err()
|
||||
}).Times(1)
|
||||
|
||||
mockStreamRunner2 := NewMockStreamRunner(mockCtrl)
|
||||
mockStreamRunner2.EXPECT().RunStream(
|
||||
gomock.Any(), gomock.Any(), gomock.Any(),
|
||||
).DoAndReturn(func(ctx context.Context, req *backend.RunStreamRequest, sender backend.StreamPacketSender) error {
|
||||
require.Equal(t, "test", req.Path)
|
||||
close(startedCh2)
|
||||
err := sender.Send(&backend.StreamPacket{
|
||||
Data: []byte("test"),
|
||||
})
|
||||
require.NoError(t, err)
|
||||
<-ctx.Done()
|
||||
close(doneCh2)
|
||||
return ctx.Err()
|
||||
}).Times(1)
|
||||
|
||||
result, err := manager.SubmitStream(context.Background(), 1, "test", "test", backend.PluginContext{}, mockStreamRunner1)
|
||||
require.NoError(t, err)
|
||||
require.False(t, result.StreamExists)
|
||||
|
||||
// try submit the same channel but different orgID.
|
||||
result, err = manager.SubmitStream(context.Background(), 2, "test", "test", backend.PluginContext{}, mockStreamRunner2)
|
||||
require.NoError(t, err)
|
||||
require.False(t, result.StreamExists)
|
||||
|
||||
waitWithTimeout(t, startedCh1, time.Second)
|
||||
waitWithTimeout(t, startedCh2, time.Second)
|
||||
require.Len(t, manager.streams, 2)
|
||||
cancel()
|
||||
waitWithTimeout(t, doneCh1, time.Second)
|
||||
waitWithTimeout(t, doneCh2, time.Second)
|
||||
}
|
||||
|
||||
func TestStreamManager_SubmitStream_CloseNoSubscribers(t *testing.T) {
|
||||
mockCtrl := gomock.NewController(t)
|
||||
defer mockCtrl.Finish()
|
||||
@ -114,7 +184,7 @@ func TestStreamManager_SubmitStream_CloseNoSubscribers(t *testing.T) {
|
||||
startedCh := make(chan struct{})
|
||||
doneCh := make(chan struct{})
|
||||
|
||||
mockPresenceGetter.EXPECT().GetNumSubscribers("test").Return(0, nil).Times(3)
|
||||
mockPresenceGetter.EXPECT().GetNumSubscribers("1/test").Return(0, nil).Times(3)
|
||||
|
||||
mockStreamRunner := NewMockStreamRunner(mockCtrl)
|
||||
mockStreamRunner.EXPECT().RunStream(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn(func(ctx context.Context, req *backend.RunStreamRequest, sender backend.StreamPacketSender) error {
|
||||
@ -124,7 +194,7 @@ func TestStreamManager_SubmitStream_CloseNoSubscribers(t *testing.T) {
|
||||
return ctx.Err()
|
||||
}).Times(1)
|
||||
|
||||
_, err := manager.SubmitStream(context.Background(), "test", "test", backend.PluginContext{}, mockStreamRunner)
|
||||
_, err := manager.SubmitStream(context.Background(), 1, "test", "test", backend.PluginContext{}, mockStreamRunner)
|
||||
require.NoError(t, err)
|
||||
|
||||
waitWithTimeout(t, startedCh, time.Second)
|
||||
@ -161,7 +231,7 @@ func TestStreamManager_SubmitStream_ErrorRestartsRunStream(t *testing.T) {
|
||||
return errors.New("boom")
|
||||
}).Times(numErrors + 1)
|
||||
|
||||
result, err := manager.SubmitStream(context.Background(), "test", "test", backend.PluginContext{}, mockStreamRunner)
|
||||
result, err := manager.SubmitStream(context.Background(), 1, "test", "test", backend.PluginContext{}, mockStreamRunner)
|
||||
require.NoError(t, err)
|
||||
require.False(t, result.StreamExists)
|
||||
|
||||
@ -190,7 +260,7 @@ func TestStreamManager_SubmitStream_NilErrorStopsRunStream(t *testing.T) {
|
||||
return nil
|
||||
}).Times(1)
|
||||
|
||||
result, err := manager.SubmitStream(context.Background(), "test", "test", backend.PluginContext{}, mockStreamRunner)
|
||||
result, err := manager.SubmitStream(context.Background(), 1, "test", "test", backend.PluginContext{}, mockStreamRunner)
|
||||
require.NoError(t, err)
|
||||
require.False(t, result.StreamExists)
|
||||
waitWithTimeout(t, result.CloseNotify, time.Second)
|
||||
|
@ -66,7 +66,7 @@ func (r *logQueryRunner) OnSubscribe(ctx context.Context, user *models.SignedInU
|
||||
|
||||
r.running[e.Channel] = true
|
||||
go func() {
|
||||
if err := r.publishResults(e.Channel); err != nil {
|
||||
if err := r.publishResults(user.OrgId, e.Channel); err != nil {
|
||||
plog.Error(err.Error())
|
||||
}
|
||||
}()
|
||||
@ -79,7 +79,7 @@ func (r *logQueryRunner) OnPublish(ctx context.Context, user *models.SignedInUse
|
||||
return models.PublishReply{}, backend.PublishStreamStatusPermissionDenied, nil
|
||||
}
|
||||
|
||||
func (r *logQueryRunner) publishResults(channelName string) error {
|
||||
func (r *logQueryRunner) publishResults(orgID int64, channelName string) error {
|
||||
defer func() {
|
||||
r.service.DeleteResponseChannel(channelName)
|
||||
r.runningMu.Lock()
|
||||
@ -98,7 +98,7 @@ func (r *logQueryRunner) publishResults(channelName string) error {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := r.publish(channelName, responseBytes); err != nil {
|
||||
if err := r.publish(orgID, channelName, responseBytes); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
@ -51,15 +51,18 @@ export class CentrifugeSrv implements GrafanaLiveSrv {
|
||||
readonly connectionState: BehaviorSubject<boolean>;
|
||||
readonly connectionBlocker: Promise<void>;
|
||||
readonly scopes: Record<LiveChannelScope, GrafanaLiveScope>;
|
||||
private orgId: number;
|
||||
|
||||
constructor() {
|
||||
// build live url replacing scheme in appUrl.
|
||||
const liveUrl = `${config.appUrl.replace('http', 'ws')}api/live/ws`;
|
||||
this.orgId = (window as any).grafanaBootData.user.orgId;
|
||||
this.centrifuge = new Centrifuge(liveUrl, {
|
||||
debug: true,
|
||||
});
|
||||
this.centrifuge.setConnectData({
|
||||
sessionId,
|
||||
orgId: this.orgId,
|
||||
});
|
||||
this.centrifuge.connect(); // do connection
|
||||
this.connectionState = new BehaviorSubject<boolean>(this.centrifuge.isConnected());
|
||||
@ -110,7 +113,7 @@ export class CentrifugeSrv implements GrafanaLiveSrv {
|
||||
* channel will be returned with an error state indicated in its status
|
||||
*/
|
||||
getChannel<TMessage>(addr: LiveChannelAddress): CentrifugeLiveChannel<TMessage> {
|
||||
const id = `${addr.scope}/${addr.namespace}/${addr.path}`;
|
||||
const id = `${this.orgId}/${addr.scope}/${addr.namespace}/${addr.path}`;
|
||||
let channel = this.open.get(id);
|
||||
if (channel != null) {
|
||||
return channel;
|
||||
@ -323,7 +326,7 @@ export class CentrifugeSrv implements GrafanaLiveSrv {
|
||||
*/
|
||||
publish(address: LiveChannelAddress, data: any): Promise<any> {
|
||||
return getBackendSrv().post(`api/live/publish`, {
|
||||
channel: toLiveChannelId(address),
|
||||
channel: toLiveChannelId(address), // orgId is from user
|
||||
data,
|
||||
});
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user