github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cmd/storage-consumer/main.go (about) 1 // Copyright 2022 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package main 15 16 import ( 17 "context" 18 "encoding/json" 19 "flag" 20 "fmt" 21 "net/http" 22 "net/url" 23 "os" 24 "os/signal" 25 "sort" 26 "strings" 27 "sync" 28 "syscall" 29 "time" 30 31 "github.com/pingcap/errors" 32 "github.com/pingcap/log" 33 "github.com/pingcap/tidb/br/pkg/storage" 34 "github.com/pingcap/tiflow/cdc/model" 35 "github.com/pingcap/tiflow/cdc/sink/ddlsink" 36 ddlfactory "github.com/pingcap/tiflow/cdc/sink/ddlsink/factory" 37 dmlfactory "github.com/pingcap/tiflow/cdc/sink/dmlsink/factory" 38 "github.com/pingcap/tiflow/cdc/sink/tablesink" 39 sinkutil "github.com/pingcap/tiflow/cdc/sink/util" 40 "github.com/pingcap/tiflow/pkg/cmd/util" 41 "github.com/pingcap/tiflow/pkg/config" 42 "github.com/pingcap/tiflow/pkg/logutil" 43 "github.com/pingcap/tiflow/pkg/quotes" 44 psink "github.com/pingcap/tiflow/pkg/sink" 45 "github.com/pingcap/tiflow/pkg/sink/cloudstorage" 46 "github.com/pingcap/tiflow/pkg/sink/codec" 47 "github.com/pingcap/tiflow/pkg/sink/codec/canal" 48 "github.com/pingcap/tiflow/pkg/sink/codec/common" 49 "github.com/pingcap/tiflow/pkg/sink/codec/csv" 50 "github.com/pingcap/tiflow/pkg/spanz" 51 putil "github.com/pingcap/tiflow/pkg/util" 52 "github.com/pingcap/tiflow/pkg/version" 53 "go.uber.org/zap" 54 ) 55 56 var ( 57 upstreamURIStr string 58 upstreamURI *url.URL 59 downstreamURIStr string 60 configFile string 61 logFile string 62 logLevel string 63 flushInterval time.Duration 64 fileIndexWidth int 65 enableProfiling bool 66 timezone string 67 ) 68 69 const ( 70 defaultChangefeedName = "storage-consumer" 71 defaultFlushWaitDuration = 200 * time.Millisecond 72 fakePartitionNumForSchemaFile = -1 73 ) 74 75 func init() { 76 version.LogVersionInfo("storage consumer") 77 flag.StringVar(&upstreamURIStr, "upstream-uri", "", "storage uri") 78 flag.StringVar(&downstreamURIStr, "downstream-uri", "", "downstream sink uri") 79 flag.StringVar(&configFile, "config", "", "changefeed configuration file") 80 flag.StringVar(&logFile, "log-file", "", "log file path") 81 flag.StringVar(&logLevel, "log-level", "info", "log level") 82 flag.DurationVar(&flushInterval, "flush-interval", 10*time.Second, "flush interval") 83 flag.IntVar(&fileIndexWidth, "file-index-width", 84 config.DefaultFileIndexWidth, "file index width") 85 flag.BoolVar(&enableProfiling, "enable-profiling", false, "whether to enable profiling") 86 flag.StringVar(&timezone, "tz", "System", "Specify time zone of storage consumer") 87 flag.Parse() 88 89 err := logutil.InitLogger(&logutil.Config{ 90 Level: logLevel, 91 File: logFile, 92 }) 93 if err != nil { 94 log.Error("init logger failed", zap.Error(err)) 95 os.Exit(1) 96 } 97 98 uri, err := url.Parse(upstreamURIStr) 99 if err != nil { 100 log.Error("invalid upstream-uri", zap.Error(err)) 101 os.Exit(1) 102 } 103 upstreamURI = uri 104 scheme := strings.ToLower(upstreamURI.Scheme) 105 if !psink.IsStorageScheme(scheme) { 106 log.Error("invalid storage scheme, the scheme of upstream-uri must be file/s3/azblob/gcs") 107 os.Exit(1) 108 } 109 } 110 111 // fileIndexRange defines a range of files. eg. CDC000002.csv ~ CDC000005.csv 112 type fileIndexRange struct { 113 start uint64 114 end uint64 115 } 116 117 type consumer struct { 118 sinkFactory *dmlfactory.SinkFactory 119 ddlSink ddlsink.Sink 120 replicationCfg *config.ReplicaConfig 121 codecCfg *common.Config 122 externalStorage storage.ExternalStorage 123 fileExtension string 124 // tableDMLIdxMap maintains a map of <dmlPathKey, max file index> 125 tableDMLIdxMap map[cloudstorage.DmlPathKey]uint64 126 // tableTsMap maintains a map of <TableID, max commit ts> 127 tableTsMap map[model.TableID]model.ResolvedTs 128 // tableDefMap maintains a map of <`schema`.`table`, tableDef slice sorted by TableVersion> 129 tableDefMap map[string]map[uint64]*cloudstorage.TableDefinition 130 // tableSinkMap maintains a map of <TableID, TableSink> 131 tableSinkMap map[model.TableID]tablesink.TableSink 132 tableIDGenerator *fakeTableIDGenerator 133 errCh chan error 134 } 135 136 func newConsumer(ctx context.Context) (*consumer, error) { 137 _, err := putil.GetTimezone(timezone) 138 if err != nil { 139 return nil, errors.Annotate(err, "can not load timezone") 140 } 141 serverCfg := config.GetGlobalServerConfig().Clone() 142 serverCfg.TZ = timezone 143 config.StoreGlobalServerConfig(serverCfg) 144 replicaConfig := config.GetDefaultReplicaConfig() 145 if len(configFile) > 0 { 146 err := util.StrictDecodeFile(configFile, "storage consumer", replicaConfig) 147 if err != nil { 148 log.Error("failed to decode config file", zap.Error(err)) 149 return nil, err 150 } 151 } 152 153 err = replicaConfig.ValidateAndAdjust(upstreamURI) 154 if err != nil { 155 log.Error("failed to validate replica config", zap.Error(err)) 156 return nil, err 157 } 158 159 switch putil.GetOrZero(replicaConfig.Sink.Protocol) { 160 case config.ProtocolCsv.String(): 161 case config.ProtocolCanalJSON.String(): 162 default: 163 return nil, fmt.Errorf( 164 "data encoded in protocol %s is not supported yet", 165 putil.GetOrZero(replicaConfig.Sink.Protocol), 166 ) 167 } 168 169 protocol, err := config.ParseSinkProtocolFromString(putil.GetOrZero(replicaConfig.Sink.Protocol)) 170 if err != nil { 171 return nil, err 172 } 173 174 codecConfig := common.NewConfig(protocol) 175 err = codecConfig.Apply(upstreamURI, replicaConfig) 176 if err != nil { 177 return nil, err 178 } 179 180 extension := sinkutil.GetFileExtension(protocol) 181 182 storage, err := putil.GetExternalStorageFromURI(ctx, upstreamURIStr) 183 if err != nil { 184 log.Error("failed to create external storage", zap.Error(err)) 185 return nil, err 186 } 187 188 errCh := make(chan error, 1) 189 stdCtx := ctx 190 sinkFactory, err := dmlfactory.New( 191 stdCtx, 192 model.DefaultChangeFeedID(defaultChangefeedName), 193 downstreamURIStr, 194 replicaConfig, 195 errCh, 196 nil, 197 ) 198 if err != nil { 199 log.Error("failed to create event sink factory", zap.Error(err)) 200 return nil, err 201 } 202 203 ddlSink, err := ddlfactory.New(ctx, model.DefaultChangeFeedID(defaultChangefeedName), 204 downstreamURIStr, replicaConfig) 205 if err != nil { 206 log.Error("failed to create ddl sink", zap.Error(err)) 207 return nil, err 208 } 209 210 return &consumer{ 211 sinkFactory: sinkFactory, 212 ddlSink: ddlSink, 213 replicationCfg: replicaConfig, 214 codecCfg: codecConfig, 215 externalStorage: storage, 216 fileExtension: extension, 217 errCh: errCh, 218 tableDMLIdxMap: make(map[cloudstorage.DmlPathKey]uint64), 219 tableTsMap: make(map[model.TableID]model.ResolvedTs), 220 tableDefMap: make(map[string]map[uint64]*cloudstorage.TableDefinition), 221 tableSinkMap: make(map[model.TableID]tablesink.TableSink), 222 tableIDGenerator: &fakeTableIDGenerator{ 223 tableIDs: make(map[string]int64), 224 }, 225 }, nil 226 } 227 228 // map1 - map2 229 func diffDMLMaps( 230 map1, map2 map[cloudstorage.DmlPathKey]uint64, 231 ) map[cloudstorage.DmlPathKey]fileIndexRange { 232 resMap := make(map[cloudstorage.DmlPathKey]fileIndexRange) 233 for k, v := range map1 { 234 if _, ok := map2[k]; !ok { 235 resMap[k] = fileIndexRange{ 236 start: 1, 237 end: v, 238 } 239 } else if v > map2[k] { 240 resMap[k] = fileIndexRange{ 241 start: map2[k] + 1, 242 end: v, 243 } 244 } 245 } 246 247 return resMap 248 } 249 250 // getNewFiles returns newly created dml files in specific ranges 251 func (c *consumer) getNewFiles( 252 ctx context.Context, 253 ) (map[cloudstorage.DmlPathKey]fileIndexRange, error) { 254 tableDMLMap := make(map[cloudstorage.DmlPathKey]fileIndexRange) 255 opt := &storage.WalkOption{SubDir: ""} 256 257 origDMLIdxMap := make(map[cloudstorage.DmlPathKey]uint64, len(c.tableDMLIdxMap)) 258 for k, v := range c.tableDMLIdxMap { 259 origDMLIdxMap[k] = v 260 } 261 262 err := c.externalStorage.WalkDir(ctx, opt, func(path string, size int64) error { 263 if cloudstorage.IsSchemaFile(path) { 264 err := c.parseSchemaFilePath(ctx, path) 265 if err != nil { 266 log.Error("failed to parse schema file path", zap.Error(err)) 267 // skip handling this file 268 return nil 269 } 270 } else if strings.HasSuffix(path, c.fileExtension) { 271 err := c.parseDMLFilePath(ctx, path) 272 if err != nil { 273 log.Error("failed to parse dml file path", zap.Error(err)) 274 // skip handling this file 275 return nil 276 } 277 } else { 278 log.Debug("ignore handling file", zap.String("path", path)) 279 } 280 return nil 281 }) 282 if err != nil { 283 return tableDMLMap, err 284 } 285 286 tableDMLMap = diffDMLMaps(c.tableDMLIdxMap, origDMLIdxMap) 287 return tableDMLMap, err 288 } 289 290 // emitDMLEvents decodes RowChangedEvents from file content and emit them. 291 func (c *consumer) emitDMLEvents( 292 ctx context.Context, tableID int64, 293 tableDetail cloudstorage.TableDefinition, 294 pathKey cloudstorage.DmlPathKey, 295 content []byte, 296 ) error { 297 var ( 298 decoder codec.RowEventDecoder 299 err error 300 ) 301 302 tableInfo, err := tableDetail.ToTableInfo() 303 if err != nil { 304 return errors.Trace(err) 305 } 306 307 switch c.codecCfg.Protocol { 308 case config.ProtocolCsv: 309 decoder, err = csv.NewBatchDecoder(ctx, c.codecCfg, tableInfo, content) 310 if err != nil { 311 return errors.Trace(err) 312 } 313 case config.ProtocolCanalJSON: 314 // Always enable tidb extension for canal-json protocol 315 // because we need to get the commit ts from the extension field. 316 c.codecCfg.EnableTiDBExtension = true 317 decoder, err = canal.NewBatchDecoder(ctx, c.codecCfg, nil) 318 if err != nil { 319 return errors.Trace(err) 320 } 321 err := decoder.AddKeyValue(nil, content) 322 if err != nil { 323 return errors.Trace(err) 324 } 325 } 326 327 cnt := 0 328 filteredCnt := 0 329 for { 330 tp, hasNext, err := decoder.HasNext() 331 if err != nil { 332 log.Error("failed to decode message", zap.Error(err)) 333 return err 334 } 335 if !hasNext { 336 break 337 } 338 cnt++ 339 340 if tp == model.MessageTypeRow { 341 row, err := decoder.NextRowChangedEvent() 342 if err != nil { 343 log.Error("failed to get next row changed event", zap.Error(err)) 344 return errors.Trace(err) 345 } 346 347 if _, ok := c.tableSinkMap[tableID]; !ok { 348 c.tableSinkMap[tableID] = c.sinkFactory.CreateTableSinkForConsumer( 349 model.DefaultChangeFeedID(defaultChangefeedName), 350 spanz.TableIDToComparableSpan(tableID), 351 row.CommitTs) 352 } 353 354 _, ok := c.tableTsMap[tableID] 355 if !ok || row.CommitTs > c.tableTsMap[tableID].Ts { 356 c.tableTsMap[tableID] = model.ResolvedTs{ 357 Mode: model.BatchResolvedMode, 358 Ts: row.CommitTs, 359 BatchID: 1, 360 } 361 } else if row.CommitTs == c.tableTsMap[tableID].Ts { 362 c.tableTsMap[tableID] = c.tableTsMap[tableID].AdvanceBatch() 363 } else { 364 log.Warn("row changed event commit ts fallback, ignore", 365 zap.Uint64("commitTs", row.CommitTs), 366 zap.Any("tableMaxCommitTs", c.tableTsMap[tableID]), 367 zap.Any("row", row), 368 ) 369 continue 370 } 371 row.PhysicalTableID = tableID 372 c.tableSinkMap[tableID].AppendRowChangedEvents(row) 373 filteredCnt++ 374 } 375 } 376 log.Info("decode success", zap.String("schema", pathKey.Schema), 377 zap.String("table", pathKey.Table), 378 zap.Uint64("version", pathKey.TableVersion), 379 zap.Int("decodeRowsCnt", cnt), 380 zap.Int("filteredRowsCnt", filteredCnt)) 381 382 return err 383 } 384 385 func (c *consumer) waitTableFlushComplete(ctx context.Context, tableID model.TableID) error { 386 for { 387 select { 388 case <-ctx.Done(): 389 return ctx.Err() 390 case err := <-c.errCh: 391 return err 392 default: 393 } 394 395 resolvedTs := c.tableTsMap[tableID] 396 err := c.tableSinkMap[tableID].UpdateResolvedTs(resolvedTs) 397 if err != nil { 398 return errors.Trace(err) 399 } 400 checkpoint := c.tableSinkMap[tableID].GetCheckpointTs() 401 if checkpoint.Equal(resolvedTs) { 402 c.tableTsMap[tableID] = resolvedTs.AdvanceBatch() 403 return nil 404 } 405 time.Sleep(defaultFlushWaitDuration) 406 } 407 } 408 409 func (c *consumer) syncExecDMLEvents( 410 ctx context.Context, 411 tableDef cloudstorage.TableDefinition, 412 key cloudstorage.DmlPathKey, 413 fileIdx uint64, 414 ) error { 415 filePath := key.GenerateDMLFilePath(fileIdx, c.fileExtension, fileIndexWidth) 416 log.Debug("read from dml file path", zap.String("path", filePath)) 417 content, err := c.externalStorage.ReadFile(ctx, filePath) 418 if err != nil { 419 return errors.Trace(err) 420 } 421 tableID := c.tableIDGenerator.generateFakeTableID( 422 key.Schema, key.Table, key.PartitionNum) 423 err = c.emitDMLEvents(ctx, tableID, tableDef, key, content) 424 if err != nil { 425 return errors.Trace(err) 426 } 427 428 resolvedTs := c.tableTsMap[tableID] 429 err = c.tableSinkMap[tableID].UpdateResolvedTs(resolvedTs) 430 if err != nil { 431 return errors.Trace(err) 432 } 433 err = c.waitTableFlushComplete(ctx, tableID) 434 if err != nil { 435 return errors.Trace(err) 436 } 437 438 return nil 439 } 440 441 func (c *consumer) parseDMLFilePath(_ context.Context, path string) error { 442 var dmlkey cloudstorage.DmlPathKey 443 fileIdx, err := dmlkey.ParseDMLFilePath( 444 putil.GetOrZero(c.replicationCfg.Sink.DateSeparator), 445 path, 446 ) 447 if err != nil { 448 return errors.Trace(err) 449 } 450 451 if _, ok := c.tableDMLIdxMap[dmlkey]; !ok || fileIdx >= c.tableDMLIdxMap[dmlkey] { 452 c.tableDMLIdxMap[dmlkey] = fileIdx 453 } 454 return nil 455 } 456 457 func (c *consumer) parseSchemaFilePath(ctx context.Context, path string) error { 458 var schemaKey cloudstorage.SchemaPathKey 459 checksumInFile, err := schemaKey.ParseSchemaFilePath(path) 460 if err != nil { 461 return errors.Trace(err) 462 } 463 key := schemaKey.GetKey() 464 if tableDefs, ok := c.tableDefMap[key]; ok { 465 if _, ok := tableDefs[schemaKey.TableVersion]; ok { 466 // Skip if tableDef already exists. 467 return nil 468 } 469 } else { 470 c.tableDefMap[key] = make(map[uint64]*cloudstorage.TableDefinition) 471 } 472 473 // Read tableDef from schema file and check checksum. 474 var tableDef cloudstorage.TableDefinition 475 schemaContent, err := c.externalStorage.ReadFile(ctx, path) 476 if err != nil { 477 return errors.Trace(err) 478 } 479 err = json.Unmarshal(schemaContent, &tableDef) 480 if err != nil { 481 return errors.Trace(err) 482 } 483 checksumInMem, err := tableDef.Sum32(nil) 484 if err != nil { 485 return errors.Trace(err) 486 } 487 if checksumInMem != checksumInFile || schemaKey.TableVersion != tableDef.TableVersion { 488 log.Panic("checksum mismatch", 489 zap.Uint32("checksumInMem", checksumInMem), 490 zap.Uint32("checksumInFile", checksumInFile), 491 zap.Uint64("tableversionInMem", schemaKey.TableVersion), 492 zap.Uint64("tableversionInFile", tableDef.TableVersion), 493 zap.String("path", path)) 494 } 495 496 // Update tableDefMap. 497 c.tableDefMap[key][tableDef.TableVersion] = &tableDef 498 499 // Fake a dml key for schema.json file, which is useful for putting DDL 500 // in front of the DML files when sorting. 501 // e.g, for the partitioned table: 502 // 503 // test/test1/439972354120482843/schema.json (partitionNum = -1) 504 // test/test1/439972354120482843/55/2023-03-09/CDC000001.csv (partitionNum = 55) 505 // test/test1/439972354120482843/66/2023-03-09/CDC000001.csv (partitionNum = 66) 506 // 507 // and for the non-partitioned table: 508 // test/test2/439972354120482843/schema.json (partitionNum = -1) 509 // test/test2/439972354120482843/2023-03-09/CDC000001.csv (partitionNum = 0) 510 // test/test2/439972354120482843/2023-03-09/CDC000002.csv (partitionNum = 0) 511 // 512 // the DDL event recorded in schema.json should be executed first, then the DML events 513 // in csv files can be executed. 514 dmlkey := cloudstorage.DmlPathKey{ 515 SchemaPathKey: schemaKey, 516 PartitionNum: fakePartitionNumForSchemaFile, 517 Date: "", 518 } 519 if _, ok := c.tableDMLIdxMap[dmlkey]; !ok { 520 c.tableDMLIdxMap[dmlkey] = 0 521 } else { 522 // duplicate table schema file found, this should not happen. 523 log.Panic("duplicate schema file found", 524 zap.String("path", path), zap.Any("tableDef", tableDef), 525 zap.Any("schemaKey", schemaKey), zap.Any("dmlkey", dmlkey)) 526 } 527 return nil 528 } 529 530 func (c *consumer) mustGetTableDef(key cloudstorage.SchemaPathKey) cloudstorage.TableDefinition { 531 var tableDef *cloudstorage.TableDefinition 532 if tableDefs, ok := c.tableDefMap[key.GetKey()]; ok { 533 tableDef = tableDefs[key.TableVersion] 534 } 535 if tableDef == nil { 536 log.Panic("tableDef not found", zap.Any("key", key), zap.Any("tableDefMap", c.tableDefMap)) 537 } 538 return *tableDef 539 } 540 541 func (c *consumer) handleNewFiles( 542 ctx context.Context, 543 dmlFileMap map[cloudstorage.DmlPathKey]fileIndexRange, 544 ) error { 545 keys := make([]cloudstorage.DmlPathKey, 0, len(dmlFileMap)) 546 for k := range dmlFileMap { 547 keys = append(keys, k) 548 } 549 if len(keys) == 0 { 550 log.Info("no new dml files found since last round") 551 return nil 552 } 553 sort.Slice(keys, func(i, j int) bool { 554 if keys[i].TableVersion != keys[j].TableVersion { 555 return keys[i].TableVersion < keys[j].TableVersion 556 } 557 if keys[i].PartitionNum != keys[j].PartitionNum { 558 return keys[i].PartitionNum < keys[j].PartitionNum 559 } 560 if keys[i].Date != keys[j].Date { 561 return keys[i].Date < keys[j].Date 562 } 563 if keys[i].Schema != keys[j].Schema { 564 return keys[i].Schema < keys[j].Schema 565 } 566 return keys[i].Table < keys[j].Table 567 }) 568 569 for _, key := range keys { 570 tableDef := c.mustGetTableDef(key.SchemaPathKey) 571 // if the key is a fake dml path key which is mainly used for 572 // sorting schema.json file before the dml files, then execute the ddl query. 573 if key.PartitionNum == fakePartitionNumForSchemaFile && 574 len(key.Date) == 0 && len(tableDef.Query) > 0 { 575 ddlEvent, err := tableDef.ToDDLEvent() 576 if err != nil { 577 return err 578 } 579 if err := c.ddlSink.WriteDDLEvent(ctx, ddlEvent); err != nil { 580 return errors.Trace(err) 581 } 582 // TODO: need to cleanup tableDefMap in the future. 583 log.Info("execute ddl event successfully", zap.String("query", tableDef.Query)) 584 continue 585 } 586 587 fileRange := dmlFileMap[key] 588 for i := fileRange.start; i <= fileRange.end; i++ { 589 if err := c.syncExecDMLEvents(ctx, tableDef, key, i); err != nil { 590 return err 591 } 592 } 593 } 594 595 return nil 596 } 597 598 func (c *consumer) run(ctx context.Context) error { 599 ticker := time.NewTicker(flushInterval) 600 for { 601 select { 602 case <-ctx.Done(): 603 return ctx.Err() 604 case err := <-c.errCh: 605 return err 606 case <-ticker.C: 607 } 608 609 dmlFileMap, err := c.getNewFiles(ctx) 610 if err != nil { 611 return errors.Trace(err) 612 } 613 614 err = c.handleNewFiles(ctx, dmlFileMap) 615 if err != nil { 616 return errors.Trace(err) 617 } 618 } 619 } 620 621 // copied from kafka-consumer 622 type fakeTableIDGenerator struct { 623 tableIDs map[string]int64 624 currentTableID int64 625 mu sync.Mutex 626 } 627 628 func (g *fakeTableIDGenerator) generateFakeTableID(schema, table string, partition int64) int64 { 629 g.mu.Lock() 630 defer g.mu.Unlock() 631 key := quotes.QuoteSchema(schema, table) 632 if partition != 0 { 633 key = fmt.Sprintf("%s.`%d`", key, partition) 634 } 635 if tableID, ok := g.tableIDs[key]; ok { 636 return tableID 637 } 638 g.currentTableID++ 639 g.tableIDs[key] = g.currentTableID 640 return g.currentTableID 641 } 642 643 func main() { 644 var consumer *consumer 645 var err error 646 647 if enableProfiling { 648 go func() { 649 server := &http.Server{ 650 Addr: ":6060", 651 ReadHeaderTimeout: 5 * time.Second, 652 } 653 654 if err := server.ListenAndServe(); err != nil { 655 log.Fatal("http pprof", zap.Error(err)) 656 } 657 }() 658 } 659 660 ctx, stop := signal.NotifyContext(context.Background(), syscall.SIGINT, syscall.SIGTERM) 661 deferFunc := func() int { 662 stop() 663 if consumer != nil { 664 consumer.sinkFactory.Close() 665 } 666 if err != nil && err != context.Canceled { 667 return 1 668 } 669 return 0 670 } 671 672 consumer, err = newConsumer(ctx) 673 if err != nil { 674 log.Error("failed to create storage consumer", zap.Error(err)) 675 goto EXIT 676 } 677 678 if err = consumer.run(ctx); err != nil { 679 log.Error("error occurred while running consumer", zap.Error(err)) 680 } 681 682 EXIT: 683 os.Exit(deferFunc()) 684 }