github.com/yankunsam/loki/v2@v2.6.3-0.20220817130409-389df5235c27/pkg/storage/chunk/client/gcp/bigtable_object_client.go (about) 1 package gcp 2 3 import ( 4 "context" 5 "fmt" 6 7 "cloud.google.com/go/bigtable" 8 ot "github.com/opentracing/opentracing-go" 9 otlog "github.com/opentracing/opentracing-go/log" 10 "github.com/pkg/errors" 11 12 "github.com/grafana/loki/pkg/storage/chunk" 13 "github.com/grafana/loki/pkg/storage/chunk/client" 14 "github.com/grafana/loki/pkg/storage/config" 15 "github.com/grafana/loki/pkg/util/math" 16 ) 17 18 type bigtableObjectClient struct { 19 cfg Config 20 schemaCfg config.SchemaConfig 21 client *bigtable.Client 22 } 23 24 // NewBigtableObjectClient makes a new chunk.Client that stores chunks in 25 // Bigtable. 26 func NewBigtableObjectClient(ctx context.Context, cfg Config, schemaCfg config.SchemaConfig) (client.Client, error) { 27 dialOpts, err := cfg.GRPCClientConfig.DialOption(bigtableInstrumentation()) 28 if err != nil { 29 return nil, err 30 } 31 client, err := bigtable.NewClient(ctx, cfg.Project, cfg.Instance, toOptions(dialOpts)...) 32 if err != nil { 33 return nil, err 34 } 35 return newBigtableObjectClient(cfg, schemaCfg, client), nil 36 } 37 38 func newBigtableObjectClient(cfg Config, schemaCfg config.SchemaConfig, client *bigtable.Client) client.Client { 39 return &bigtableObjectClient{ 40 cfg: cfg, 41 schemaCfg: schemaCfg, 42 client: client, 43 } 44 } 45 46 func (s *bigtableObjectClient) Stop() { 47 s.client.Close() 48 } 49 50 func (s *bigtableObjectClient) PutChunks(ctx context.Context, chunks []chunk.Chunk) error { 51 keys := map[string][]string{} 52 muts := map[string][]*bigtable.Mutation{} 53 54 for i := range chunks { 55 buf, err := chunks[i].Encoded() 56 if err != nil { 57 return err 58 } 59 key := s.schemaCfg.ExternalKey(chunks[i].ChunkRef) 60 tableName, err := s.schemaCfg.ChunkTableFor(chunks[i].From) 61 if err != nil { 62 return err 63 } 64 keys[tableName] = append(keys[tableName], key) 65 66 mut := bigtable.NewMutation() 67 mut.Set(columnFamily, column, 0, buf) 68 muts[tableName] = append(muts[tableName], mut) 69 } 70 71 for tableName := range keys { 72 table := s.client.Open(tableName) 73 errs, err := table.ApplyBulk(ctx, keys[tableName], muts[tableName]) 74 if err != nil { 75 return err 76 } 77 for _, err := range errs { 78 if err != nil { 79 return err 80 } 81 } 82 } 83 return nil 84 } 85 86 func (s *bigtableObjectClient) GetChunks(ctx context.Context, input []chunk.Chunk) ([]chunk.Chunk, error) { 87 sp, ctx := ot.StartSpanFromContext(ctx, "GetChunks") 88 defer sp.Finish() 89 sp.LogFields(otlog.Int("chunks requested", len(input))) 90 91 chunks := map[string]map[string]chunk.Chunk{} 92 keys := map[string]bigtable.RowList{} 93 for _, c := range input { 94 tableName, err := s.schemaCfg.ChunkTableFor(c.From) 95 if err != nil { 96 return nil, err 97 } 98 key := s.schemaCfg.ExternalKey(c.ChunkRef) 99 keys[tableName] = append(keys[tableName], key) 100 if _, ok := chunks[tableName]; !ok { 101 chunks[tableName] = map[string]chunk.Chunk{} 102 } 103 chunks[tableName][key] = c 104 } 105 106 outs := make(chan chunk.Chunk, len(input)) 107 errs := make(chan error, len(input)) 108 109 for tableName := range keys { 110 var ( 111 table = s.client.Open(tableName) 112 keys = keys[tableName] 113 chunks = chunks[tableName] 114 ) 115 116 for i := 0; i < len(keys); i += maxRowReads { 117 page := keys[i:math.Min(i+maxRowReads, len(keys))] 118 go func(page bigtable.RowList) { 119 decodeContext := chunk.NewDecodeContext() 120 121 var processingErr error 122 receivedChunks := 0 123 124 // rows are returned in key order, not order in row list 125 err := table.ReadRows(ctx, page, func(row bigtable.Row) bool { 126 chunk, ok := chunks[row.Key()] 127 if !ok { 128 processingErr = errors.WithStack(fmt.Errorf("Got row for unknown chunk: %s", row.Key())) 129 return false 130 } 131 132 err := chunk.Decode(decodeContext, row[columnFamily][0].Value) 133 if err != nil { 134 processingErr = err 135 return false 136 } 137 138 receivedChunks++ 139 outs <- chunk 140 return true 141 }) 142 143 if processingErr != nil { 144 errs <- processingErr 145 } else if err != nil { 146 errs <- errors.WithStack(err) 147 } else if receivedChunks < len(page) { 148 errs <- errors.WithStack(fmt.Errorf("Asked for %d chunks for Bigtable, received %d", len(page), receivedChunks)) 149 } 150 }(page) 151 } 152 } 153 154 output := make([]chunk.Chunk, 0, len(input)) 155 for i := 0; i < len(input); i++ { 156 select { 157 case c := <-outs: 158 output = append(output, c) 159 case err := <-errs: 160 return nil, err 161 case <-ctx.Done(): 162 return nil, ctx.Err() 163 } 164 } 165 166 return output, nil 167 } 168 169 func (s *bigtableObjectClient) DeleteChunk(ctx context.Context, userID, chunkID string) error { 170 chunkRef, err := chunk.ParseExternalKey(userID, chunkID) 171 if err != nil { 172 return err 173 } 174 175 tableName, err := s.schemaCfg.ChunkTableFor(chunkRef.From) 176 if err != nil { 177 return err 178 } 179 180 mut := bigtable.NewMutation() 181 mut.DeleteCellsInColumn(columnFamily, column) 182 183 return s.client.Open(tableName).Apply(ctx, chunkID, mut) 184 } 185 186 func (s *bigtableObjectClient) IsChunkNotFoundErr(_ error) bool { 187 return false 188 }