github.com/m3db/m3@v1.5.0/src/dbnode/network/server/tchannelthrift/cluster/service.go (about) 1 // Copyright (c) 2016 Uber Technologies, Inc. 2 // 3 // Permission is hereby granted, free of charge, to any person obtaining a copy 4 // of this software and associated documentation files (the "Software"), to deal 5 // in the Software without restriction, including without limitation the rights 6 // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 // copies of the Software, and to permit persons to whom the Software is 8 // furnished to do so, subject to the following conditions: 9 // 10 // The above copyright notice and this permission notice shall be included in 11 // all copies or substantial portions of the Software. 12 // 13 // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 // THE SOFTWARE. 20 21 package cluster 22 23 import ( 24 "errors" 25 "fmt" 26 "sync" 27 28 "github.com/m3db/m3/src/dbnode/client" 29 "github.com/m3db/m3/src/dbnode/encoding" 30 "github.com/m3db/m3/src/dbnode/generated/thrift/rpc" 31 "github.com/m3db/m3/src/dbnode/network/server/tchannelthrift" 32 "github.com/m3db/m3/src/dbnode/network/server/tchannelthrift/convert" 33 tterrors "github.com/m3db/m3/src/dbnode/network/server/tchannelthrift/errors" 34 "github.com/m3db/m3/src/dbnode/storage/index" 35 "github.com/m3db/m3/src/dbnode/topology" 36 "github.com/m3db/m3/src/x/checked" 37 xerrors "github.com/m3db/m3/src/x/errors" 38 "github.com/m3db/m3/src/x/ident" 39 xtime "github.com/m3db/m3/src/x/time" 40 41 "github.com/uber/tchannel-go/thrift" 42 ) 43 44 type service struct { 45 sync.RWMutex 46 47 client client.Client 48 active map[sessionOpts]client.Session 49 opts client.Options 50 idPool ident.Pool 51 health *rpc.HealthResult_ 52 } 53 54 // a composite key to lookup the type of session for the request. 55 type sessionOpts struct { 56 readConsistency topology.ReadConsistencyLevel 57 consistencyOverride bool 58 equalTimestampStrategy encoding.IterateEqualTimestampStrategy 59 } 60 61 // NewService creates a new cluster TChannel Thrift service 62 func NewService(c client.Client) rpc.TChanCluster { 63 s := &service{ 64 client: c, 65 active: make(map[sessionOpts]client.Session), 66 opts: c.Options(), 67 idPool: c.Options().IdentifierPool(), 68 health: &rpc.HealthResult_{Ok: true, Status: "up"}, 69 } 70 return s 71 } 72 73 func (s *service) session() (client.Session, error) { 74 return s.sessionForOpts(s.sessionOptsFromClusterQueryOpts(nil)) 75 } 76 77 func (s *service) sessionForOpts(opts sessionOpts) (client.Session, error) { 78 s.RLock() 79 session := s.active[opts] 80 s.RUnlock() 81 if session != nil { 82 return session, nil 83 } 84 85 s.Lock() 86 session = s.active[opts] 87 if session != nil { 88 s.Unlock() 89 return session, nil 90 } 91 clientOpts := s.client.Options() 92 iterOpts := clientOpts.IterationOptions() 93 iterOpts.IterateEqualTimestampStrategy = opts.equalTimestampStrategy 94 clientOpts = clientOpts. 95 SetReadConsistencyLevel(opts.readConsistency). 96 SetIterationOptions(iterOpts) 97 if opts.consistencyOverride { 98 // disable the runtime options so the request level consistency level is not overwritten. 99 clientOpts = clientOpts.SetRuntimeOptionsManager(nil) 100 } 101 session, err := s.client.NewSessionWithOptions(clientOpts) 102 if err != nil { 103 s.Unlock() 104 return nil, err 105 } 106 s.active[opts] = session 107 s.Unlock() 108 109 return session, nil 110 } 111 112 func (s *service) Close() error { 113 var multiErr xerrors.MultiError 114 s.Lock() 115 for _, sess := range s.active { 116 multiErr = multiErr.Add(sess.Close()) 117 } 118 s.Unlock() 119 return multiErr.FinalError() 120 } 121 122 func (s *service) Health(ctx thrift.Context) (*rpc.HealthResult_, error) { 123 s.RLock() 124 health := s.health 125 s.RUnlock() 126 return health, nil 127 } 128 129 func (s *service) Query(tctx thrift.Context, req *rpc.QueryRequest) (*rpc.QueryResult_, error) { 130 start, rangeStartErr := convert.ToTime(req.RangeStart, req.RangeType) 131 end, rangeEndErr := convert.ToTime(req.RangeEnd, req.RangeType) 132 133 if rangeStartErr != nil || rangeEndErr != nil { 134 return nil, tterrors.NewBadRequestError(xerrors.FirstError(rangeStartErr, rangeEndErr)) 135 } 136 137 q, err := convert.FromRPCQuery(req.Query) 138 if err != nil { 139 return nil, convert.ToRPCError(err) 140 } 141 142 nsID := ident.StringID(req.NameSpace) 143 opts := index.QueryOptions{ 144 StartInclusive: start, 145 EndExclusive: end, 146 } 147 148 if l := req.Limit; l != nil { 149 opts.SeriesLimit = int(*l) 150 } 151 if len(req.Source) > 0 { 152 opts.Source = req.Source 153 } 154 155 session, err := s.sessionForOpts(s.sessionOptsFromClusterQueryOpts(req.ClusterOptions)) 156 if err != nil { 157 return nil, convert.ToRPCError(err) 158 } 159 160 if req.NoData != nil && *req.NoData { 161 results, metadata, err := session.FetchTaggedIDs(tctx, nsID, 162 index.Query{Query: q}, opts) 163 if err != nil { 164 return nil, convert.ToRPCError(err) 165 } 166 167 defer results.Finalize() 168 169 result := &rpc.QueryResult_{ 170 Exhaustive: metadata.Exhaustive, 171 } 172 173 for results.Next() { 174 _, tsID, tags := results.Current() 175 curr := &rpc.QueryResultElement{ 176 ID: tsID.String(), 177 } 178 result.Results = append(result.Results, curr) 179 for tags.Next() { 180 t := tags.Current() 181 curr.Tags = append(curr.Tags, &rpc.Tag{ 182 Name: t.Name.String(), 183 Value: t.Value.String(), 184 }) 185 } 186 if err := tags.Err(); err != nil { 187 return nil, convert.ToRPCError(err) 188 } 189 } 190 191 if err := results.Err(); err != nil { 192 return nil, convert.ToRPCError(err) 193 } 194 195 return result, nil 196 } 197 198 results, metadata, err := session.FetchTagged(tctx, nsID, 199 index.Query{Query: q}, opts) 200 if err != nil { 201 return nil, convert.ToRPCError(err) 202 } 203 defer results.Close() 204 205 result := &rpc.QueryResult_{ 206 Results: make([]*rpc.QueryResultElement, 0, results.Len()), 207 Exhaustive: metadata.Exhaustive, 208 } 209 210 for _, series := range results.Iters() { 211 curr := &rpc.QueryResultElement{ 212 ID: series.ID().String(), 213 } 214 result.Results = append(result.Results, curr) 215 tags := series.Tags() 216 for tags.Next() { 217 t := tags.Current() 218 curr.Tags = append(curr.Tags, &rpc.Tag{ 219 Name: t.Name.String(), 220 Value: t.Value.String(), 221 }) 222 } 223 if err := tags.Err(); err != nil { 224 return nil, convert.ToRPCError(err) 225 } 226 227 var datapoints []*rpc.Datapoint 228 for series.Next() { 229 dp, _, annotation := series.Current() 230 231 timestamp, timestampErr := convert.ToValue(dp.TimestampNanos, req.ResultTimeType) 232 if timestampErr != nil { 233 return nil, xerrors.NewInvalidParamsError(timestampErr) 234 } 235 236 datapoints = append(datapoints, &rpc.Datapoint{ 237 Timestamp: timestamp, 238 Value: dp.Value, 239 Annotation: annotation, 240 }) 241 } 242 if err := series.Err(); err != nil { 243 return nil, convert.ToRPCError(err) 244 } 245 curr.Datapoints = datapoints 246 } 247 248 return result, nil 249 } 250 251 func (s *service) Fetch(tctx thrift.Context, req *rpc.FetchRequest) (*rpc.FetchResult_, error) { 252 session, err := s.session() 253 if err != nil { 254 return nil, tterrors.NewInternalError(err) 255 } 256 257 start, rangeStartErr := convert.ToTime(req.RangeStart, req.RangeType) 258 end, rangeEndErr := convert.ToTime(req.RangeEnd, req.RangeType) 259 if rangeStartErr != nil || rangeEndErr != nil { 260 return nil, tterrors.NewBadRequestError(xerrors.FirstError(rangeStartErr, rangeEndErr)) 261 } 262 263 ctx := tchannelthrift.Context(tctx) 264 nsID := s.idPool.GetStringID(ctx, req.NameSpace) 265 tsID := s.idPool.GetStringID(ctx, req.ID) 266 267 it, err := session.Fetch(nsID, tsID, start, end) 268 if err != nil { 269 return nil, convert.ToRPCError(err) 270 } 271 272 defer it.Close() 273 274 result := rpc.NewFetchResult_() 275 // Make datapoints an initialized empty array for JSON serialization as empty array than null 276 result.Datapoints = make([]*rpc.Datapoint, 0) 277 278 for it.Next() { 279 dp, _, annotation := it.Current() 280 ts, tsErr := convert.ToValue(dp.TimestampNanos, req.ResultTimeType) 281 if tsErr != nil { 282 return nil, tterrors.NewBadRequestError(tsErr) 283 } 284 285 datapoint := rpc.NewDatapoint() 286 datapoint.Timestamp = ts 287 datapoint.Value = dp.Value 288 datapoint.Annotation = annotation 289 result.Datapoints = append(result.Datapoints, datapoint) 290 } 291 if err := it.Err(); err != nil { 292 return nil, tterrors.NewInternalError(err) 293 } 294 295 return result, nil 296 } 297 298 func (s *service) Aggregate(ctx thrift.Context, req *rpc.AggregateQueryRequest) (*rpc.AggregateQueryResult_, error) { 299 session, err := s.session() 300 if err != nil { 301 return nil, tterrors.NewInternalError(err) 302 } 303 304 ns, query, opts, err := convert.FromRPCAggregateQueryRequest(req) 305 if err != nil { 306 return nil, tterrors.NewBadRequestError(err) 307 } 308 309 if len(req.Source) > 0 { 310 opts.Source = req.Source 311 } 312 313 iter, metadata, err := session.Aggregate(ctx, ns, query, opts) 314 if err != nil { 315 return nil, convert.ToRPCError(err) 316 } 317 defer iter.Finalize() 318 319 response := &rpc.AggregateQueryResult_{ 320 Exhaustive: metadata.Exhaustive, 321 } 322 for iter.Next() { 323 name, values := iter.Current() 324 responseElem := &rpc.AggregateQueryResultTagNameElement{ 325 TagName: name.String(), 326 } 327 responseElem.TagValues = make([]*rpc.AggregateQueryResultTagValueElement, 0, values.Remaining()) 328 for values.Next() { 329 value := values.Current() 330 responseElem.TagValues = append(responseElem.TagValues, &rpc.AggregateQueryResultTagValueElement{ 331 TagValue: value.String(), 332 }) 333 } 334 if err := values.Err(); err != nil { 335 return nil, convert.ToRPCError(err) 336 } 337 response.Results = append(response.Results, responseElem) 338 } 339 if err := iter.Err(); err != nil { 340 return nil, convert.ToRPCError(err) 341 } 342 return response, nil 343 } 344 345 func (s *service) Write(tctx thrift.Context, req *rpc.WriteRequest) error { 346 session, err := s.session() 347 if err != nil { 348 return tterrors.NewInternalError(err) 349 } 350 if req.Datapoint == nil { 351 return tterrors.NewBadRequestError(fmt.Errorf("requires datapoint")) 352 } 353 dp := req.Datapoint 354 unit, unitErr := convert.ToUnit(dp.TimestampTimeType) 355 if unitErr != nil { 356 return tterrors.NewBadRequestError(unitErr) 357 } 358 d, err := unit.Value() 359 if err != nil { 360 return tterrors.NewBadRequestError(err) 361 } 362 ts := xtime.FromNormalizedTime(dp.Timestamp, d) 363 364 ctx := tchannelthrift.Context(tctx) 365 nsID := s.idPool.GetStringID(ctx, req.NameSpace) 366 tsID := s.idPool.GetStringID(ctx, req.ID) 367 err = session.Write(nsID, tsID, ts, dp.Value, unit, dp.Annotation) 368 if err != nil { 369 return convert.ToRPCError(err) 370 } 371 return nil 372 } 373 374 func (s *service) WriteTagged(tctx thrift.Context, req *rpc.WriteTaggedRequest) error { 375 session, err := s.session() 376 if err != nil { 377 return tterrors.NewInternalError(err) 378 } 379 if req.Datapoint == nil { 380 return tterrors.NewBadRequestError(fmt.Errorf("requires datapoint")) 381 } 382 dp := req.Datapoint 383 unit, unitErr := convert.ToUnit(dp.TimestampTimeType) 384 if unitErr != nil { 385 return tterrors.NewBadRequestError(unitErr) 386 } 387 d, err := unit.Value() 388 if err != nil { 389 return tterrors.NewBadRequestError(err) 390 } 391 ts := xtime.FromNormalizedTime(dp.Timestamp, d) 392 393 ctx := tchannelthrift.Context(tctx) 394 nsID := s.idPool.GetStringID(ctx, req.NameSpace) 395 tsID := s.idPool.GetStringID(ctx, req.ID) 396 var tags ident.Tags 397 for _, tag := range req.Tags { 398 tags.Append(s.idPool.GetStringTag(ctx, tag.Name, tag.Value)) 399 } 400 err = session.WriteTagged(nsID, tsID, ident.NewTagsIterator(tags), 401 ts, dp.Value, unit, dp.Annotation) 402 if err != nil { 403 return convert.ToRPCError(err) 404 } 405 return nil 406 } 407 408 func (s *service) Truncate(tctx thrift.Context, req *rpc.TruncateRequest) (*rpc.TruncateResult_, error) { 409 session, err := s.session() 410 if err != nil { 411 return nil, tterrors.NewInternalError(err) 412 } 413 414 adminSession, ok := session.(client.AdminSession) 415 if !ok { 416 return nil, tterrors.NewInternalError(errors.New("unable to get an admin session")) 417 } 418 419 nsID := ident.BinaryID(checked.NewBytes(req.NameSpace, nil)) 420 truncated, err := adminSession.Truncate(nsID) 421 if err != nil { 422 return nil, convert.ToRPCError(err) 423 } 424 425 res := rpc.NewTruncateResult_() 426 res.NumSeries = truncated 427 return res, nil 428 } 429 430 func (s *service) sessionOptsFromClusterQueryOpts(clusterOpts *rpc.ClusterQueryOptions) sessionOpts { 431 sessOpts := sessionOpts{ 432 readConsistency: s.opts.ReadConsistencyLevel(), 433 equalTimestampStrategy: s.opts.IterationOptions().IterateEqualTimestampStrategy, 434 } 435 if clusterOpts == nil { 436 return sessOpts 437 } 438 if clusterOpts.ConflictResolutionStrategy != nil { 439 switch *clusterOpts.ConflictResolutionStrategy { 440 case rpc.EqualTimestampStrategy_LAST_PUSHED: 441 sessOpts.equalTimestampStrategy = encoding.IterateLastPushed 442 case rpc.EqualTimestampStrategy_LOWEST_VALUE: 443 sessOpts.equalTimestampStrategy = encoding.IterateLowestValue 444 case rpc.EqualTimestampStrategy_HIGHEST_VALUE: 445 sessOpts.equalTimestampStrategy = encoding.IterateHighestValue 446 case rpc.EqualTimestampStrategy_HIGHEST_FREQUENCY: 447 sessOpts.equalTimestampStrategy = encoding.IterateHighestFrequencyValue 448 default: 449 sessOpts.equalTimestampStrategy = encoding.DefaultIterateEqualTimestampStrategy 450 } 451 } 452 if clusterOpts.ReadConsistency != nil { 453 sessOpts.consistencyOverride = true 454 switch *clusterOpts.ReadConsistency { 455 case rpc.ReadConsistency_ONE: 456 sessOpts.readConsistency = topology.ReadConsistencyLevelOne 457 case rpc.ReadConsistency_UNSTRICT_MAJORITY: 458 sessOpts.readConsistency = topology.ReadConsistencyLevelUnstrictMajority 459 case rpc.ReadConsistency_MAJORITY: 460 sessOpts.readConsistency = topology.ReadConsistencyLevelMajority 461 case rpc.ReadConsistency_UNSTRICT_ALL: 462 sessOpts.readConsistency = topology.ReadConsistencyLevelUnstrictAll 463 case rpc.ReadConsistency_ALL: 464 sessOpts.readConsistency = topology.ReadConsistencyLevelAll 465 default: 466 sessOpts.readConsistency = topology.ReadConsistencyLevelNone 467 } 468 } 469 return sessOpts 470 }