github.com/matrixorigin/matrixone@v1.2.0/pkg/fileservice/s3_fs.go (about) 1 // Copyright 2022 Matrix Origin 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 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 15 package fileservice 16 17 import ( 18 "bytes" 19 "context" 20 "crypto/tls" 21 "errors" 22 "io" 23 "net/http/httptrace" 24 pathpkg "path" 25 "sort" 26 "strings" 27 "sync/atomic" 28 "time" 29 30 "github.com/matrixorigin/matrixone/pkg/common/moerr" 31 "github.com/matrixorigin/matrixone/pkg/common/reuse" 32 "github.com/matrixorigin/matrixone/pkg/logutil" 33 "github.com/matrixorigin/matrixone/pkg/perfcounter" 34 metric "github.com/matrixorigin/matrixone/pkg/util/metric/v2" 35 "github.com/matrixorigin/matrixone/pkg/util/trace" 36 "github.com/matrixorigin/matrixone/pkg/util/trace/impl/motrace/statistic" 37 "go.uber.org/zap" 38 ) 39 40 // S3FS is a FileService implementation backed by S3 41 type S3FS struct { 42 name string 43 storage ObjectStorage 44 keyPrefix string 45 46 allocator CacheDataAllocator 47 memCache *MemCache 48 diskCache *DiskCache 49 remoteCache *RemoteCache 50 asyncUpdate bool 51 52 perfCounterSets []*perfcounter.CounterSet 53 54 ioMerger *IOMerger 55 } 56 57 // key mapping scheme: 58 // <KeyPrefix>/<file path> -> file content 59 60 var _ FileService = new(S3FS) 61 62 func NewS3FS( 63 ctx context.Context, 64 args ObjectStorageArguments, 65 cacheConfig CacheConfig, 66 perfCounterSets []*perfcounter.CounterSet, 67 noCache bool, 68 noDefaultCredential bool, 69 ) (*S3FS, error) { 70 71 args.NoDefaultCredentials = noDefaultCredential 72 73 fs := &S3FS{ 74 name: args.Name, 75 keyPrefix: args.KeyPrefix, 76 asyncUpdate: true, 77 perfCounterSets: perfCounterSets, 78 ioMerger: NewIOMerger(), 79 } 80 81 var err error 82 switch { 83 84 case strings.Contains(args.Endpoint, "ctyunapi.cn"): 85 fs.storage, err = NewMinioSDK(ctx, args, perfCounterSets) 86 if err != nil { 87 return nil, err 88 } 89 90 case strings.Contains(args.Endpoint, "aliyuncs.com"): 91 fs.storage, err = NewAliyunSDK(ctx, args, perfCounterSets) 92 if err != nil { 93 return nil, err 94 } 95 96 default: 97 fs.storage, err = NewAwsSDKv2(ctx, args, perfCounterSets) 98 if err != nil { 99 return nil, err 100 } 101 102 } 103 104 if !noCache { 105 if err := fs.initCaches(ctx, cacheConfig); err != nil { 106 return nil, err 107 } 108 } 109 if fs.memCache != nil { 110 fs.allocator = fs.memCache 111 } else { 112 fs.allocator = DefaultCacheDataAllocator 113 } 114 115 return fs, nil 116 } 117 118 func (s *S3FS) initCaches(ctx context.Context, config CacheConfig) error { 119 config.setDefaults() 120 121 // Init the remote cache first, because the callback needs to be set for mem and disk cache. 122 if config.RemoteCacheEnabled { 123 if config.QueryClient == nil { 124 return moerr.NewInternalError(ctx, "query client is nil") 125 } 126 s.remoteCache = NewRemoteCache(config.QueryClient, config.KeyRouterFactory) 127 logutil.Info("fileservice: remote cache initialized", 128 zap.Any("fs-name", s.name), 129 ) 130 } 131 132 // memory cache 133 if *config.MemoryCapacity > DisableCacheCapacity { 134 s.memCache = NewMemCache( 135 NewMemoryCache(int64(*config.MemoryCapacity), true, &config.CacheCallbacks), 136 s.perfCounterSets, 137 ) 138 logutil.Info("fileservice: memory cache initialized", 139 zap.Any("fs-name", s.name), 140 zap.Any("capacity", config.MemoryCapacity), 141 ) 142 } 143 144 // disk cache 145 if *config.DiskCapacity > DisableCacheCapacity && config.DiskPath != nil { 146 var err error 147 s.diskCache, err = NewDiskCache( 148 ctx, 149 *config.DiskPath, 150 int(*config.DiskCapacity), 151 s.perfCounterSets, 152 ) 153 if err != nil { 154 return err 155 } 156 logutil.Info("fileservice: disk cache initialized", 157 zap.Any("fs-name", s.name), 158 zap.Any("config", config), 159 ) 160 } 161 162 return nil 163 } 164 165 func (s *S3FS) Name() string { 166 return s.name 167 } 168 169 func (s *S3FS) pathToKey(filePath string) string { 170 return pathpkg.Join(s.keyPrefix, filePath) 171 } 172 173 func (s *S3FS) keyToPath(key string) string { 174 path := strings.TrimPrefix(key, s.keyPrefix) 175 path = strings.TrimLeft(path, "/") 176 return path 177 } 178 179 func (s *S3FS) List(ctx context.Context, dirPath string) (entries []DirEntry, err error) { 180 ctx, span := trace.Start(ctx, "S3FS.List") 181 defer span.End() 182 start := time.Now() 183 defer func() { 184 metric.S3ListIODurationHistogram.Observe(time.Since(start).Seconds()) 185 }() 186 187 path, err := ParsePathAtService(dirPath, s.name) 188 if err != nil { 189 return nil, err 190 } 191 prefix := s.pathToKey(path.File) 192 if prefix != "" { 193 prefix += "/" 194 } 195 196 if err := s.storage.List(ctx, prefix, func(isPrefix bool, key string, size int64) (bool, error) { 197 198 if isPrefix { 199 filePath := s.keyToPath(key) 200 filePath = strings.TrimRight(filePath, "/") 201 _, name := pathpkg.Split(filePath) 202 entries = append(entries, DirEntry{ 203 Name: name, 204 IsDir: true, 205 }) 206 207 } else { 208 filePath := s.keyToPath(key) 209 filePath = strings.TrimRight(filePath, "/") 210 _, name := pathpkg.Split(filePath) 211 entries = append(entries, DirEntry{ 212 Name: name, 213 IsDir: false, 214 Size: size, 215 }) 216 } 217 218 return true, nil 219 }); err != nil { 220 return nil, err 221 } 222 223 return 224 } 225 226 func (s *S3FS) StatFile(ctx context.Context, filePath string) (*DirEntry, error) { 227 ctx, span := trace.Start(ctx, "S3FS.StatFile") 228 defer span.End() 229 start := time.Now() 230 defer func() { 231 metric.S3StatIODurationHistogram.Observe(time.Since(start).Seconds()) 232 }() 233 path, err := ParsePathAtService(filePath, s.name) 234 if err != nil { 235 return nil, err 236 } 237 key := s.pathToKey(path.File) 238 239 size, err := s.storage.Stat(ctx, key) 240 if err != nil { 241 return nil, err 242 } 243 244 return &DirEntry{ 245 Name: pathpkg.Base(filePath), 246 IsDir: false, 247 Size: size, 248 }, nil 249 } 250 251 func (s *S3FS) PrefetchFile(ctx context.Context, filePath string) error { 252 253 path, err := ParsePathAtService(filePath, s.name) 254 if err != nil { 255 return err 256 } 257 258 startLock := time.Now() 259 done, wait := s.ioMerger.Merge(IOMergeKey{ 260 Path: filePath, 261 }) 262 if done != nil { 263 defer done() 264 } else { 265 wait() 266 } 267 statistic.StatsInfoFromContext(ctx).AddS3FSPrefetchFileIOMergerTimeConsumption(time.Since(startLock)) 268 269 // load to disk cache 270 if s.diskCache != nil { 271 if err := s.diskCache.SetFile( 272 ctx, path.File, 273 func(ctx context.Context) (io.ReadCloser, error) { 274 return s.newReadCloser(ctx, filePath) 275 }, 276 ); err != nil { 277 return err 278 } 279 } 280 281 return nil 282 } 283 284 func (s *S3FS) newReadCloser(ctx context.Context, filePath string) (io.ReadCloser, error) { 285 if err := ctx.Err(); err != nil { 286 return nil, err 287 } 288 289 key := s.pathToKey(filePath) 290 r, err := s.storage.Read(ctx, key, ptrTo[int64](0), (*int64)(nil)) 291 if err != nil { 292 return nil, err 293 } 294 295 return r, nil 296 } 297 298 func (s *S3FS) Write(ctx context.Context, vector IOVector) error { 299 if err := ctx.Err(); err != nil { 300 return err 301 } 302 metric.FSWriteS3Counter.Add(float64(len(vector.Entries))) 303 304 tp := reuse.Alloc[tracePoint](nil) 305 defer reuse.Free(tp, nil) 306 ctx = httptrace.WithClientTrace(ctx, tp.getClientTrace()) 307 308 var bytesWritten int 309 start := time.Now() 310 defer func() { 311 metric.S3WriteIODurationHistogram.Observe(time.Since(start).Seconds()) 312 metric.S3WriteIOBytesHistogram.Observe(float64(bytesWritten)) 313 }() 314 315 // check existence 316 path, err := ParsePathAtService(vector.FilePath, s.name) 317 if err != nil { 318 return err 319 } 320 key := s.pathToKey(path.File) 321 exists, err := s.storage.Exists(ctx, key) 322 if err != nil { 323 return err 324 } 325 if exists { 326 return moerr.NewFileAlreadyExistsNoCtx(vector.FilePath) 327 } 328 329 bytesWritten, err = s.write(ctx, vector) 330 return err 331 } 332 333 func (s *S3FS) write(ctx context.Context, vector IOVector) (bytesWritten int, err error) { 334 ctx, span := trace.Start(ctx, "S3FS.write") 335 defer span.End() 336 337 path, err := ParsePathAtService(vector.FilePath, s.name) 338 if err != nil { 339 return 0, err 340 } 341 342 // sort 343 sort.Slice(vector.Entries, func(i, j int) bool { 344 return vector.Entries[i].Offset < vector.Entries[j].Offset 345 }) 346 347 // size 348 var size int64 349 if len(vector.Entries) > 0 { 350 last := vector.Entries[len(vector.Entries)-1] 351 size = int64(last.Offset + last.Size) 352 } 353 354 // content 355 var content []byte 356 if len(vector.Entries) == 1 && 357 vector.Entries[0].Size > 0 && 358 int(vector.Entries[0].Size) == len(vector.Entries[0].Data) { 359 // one piece of data 360 content = vector.Entries[0].Data 361 362 } else { 363 r := newIOEntriesReader(ctx, vector.Entries) 364 content, err = io.ReadAll(r) 365 if err != nil { 366 return 0, err 367 } 368 } 369 370 r := bytes.NewReader(content) 371 var expire *time.Time 372 if !vector.ExpireAt.IsZero() { 373 expire = &vector.ExpireAt 374 } 375 key := s.pathToKey(path.File) 376 if err := s.storage.Write(ctx, key, r, size, expire); err != nil { 377 return 0, err 378 } 379 380 // write to disk cache 381 if s.diskCache != nil && !vector.Policy.Any(SkipDiskCacheWrites) { 382 if err := s.diskCache.SetFile(ctx, vector.FilePath, func(context.Context) (io.ReadCloser, error) { 383 return io.NopCloser(bytes.NewReader(content)), nil 384 }); err != nil { 385 return 0, err 386 } 387 } 388 389 return len(content), nil 390 } 391 392 func (s *S3FS) Read(ctx context.Context, vector *IOVector) (err error) { 393 if err := ctx.Err(); err != nil { 394 return err 395 } 396 397 tp := reuse.Alloc[tracePoint](nil) 398 defer reuse.Free(tp, nil) 399 ctx = httptrace.WithClientTrace(ctx, tp.getClientTrace()) 400 401 if len(vector.Entries) == 0 { 402 return moerr.NewEmptyVectorNoCtx() 403 } 404 405 startLock := time.Now() 406 done, wait := s.ioMerger.Merge(vector.ioMergeKey()) 407 if done != nil { 408 defer done() 409 } else { 410 wait() 411 } 412 stats := statistic.StatsInfoFromContext(ctx) 413 stats.AddS3FSReadIOMergerTimeConsumption(time.Since(startLock)) 414 415 allocator := s.allocator 416 if vector.Policy.Any(SkipMemoryCache) { 417 allocator = DefaultCacheDataAllocator 418 } 419 for i := range vector.Entries { 420 vector.Entries[i].allocator = allocator 421 } 422 423 for _, cache := range vector.Caches { 424 cache := cache 425 if err := readCache(ctx, cache, vector); err != nil { 426 return err 427 } 428 defer func() { 429 if err != nil { 430 return 431 } 432 err = cache.Update(ctx, vector, false) 433 }() 434 } 435 436 if s.memCache != nil { 437 if err := readCache(ctx, s.memCache, vector); err != nil { 438 return err 439 } 440 defer func() { 441 if err != nil { 442 return 443 } 444 err = s.memCache.Update(ctx, vector, s.asyncUpdate) 445 }() 446 } 447 448 ioStart := time.Now() 449 defer func() { 450 stats.AddIOAccessTimeConsumption(time.Since(ioStart)) 451 }() 452 453 if s.diskCache != nil { 454 if err := readCache(ctx, s.diskCache, vector); err != nil { 455 return err 456 } 457 // try to cache IOEntry if not caching the full file 458 if vector.Policy.CacheIOEntry() { 459 defer func() { 460 if err != nil { 461 return 462 } 463 err = s.diskCache.Update(ctx, vector, s.asyncUpdate) 464 }() 465 } 466 } 467 468 if s.remoteCache != nil { 469 if err := readCache(ctx, s.remoteCache, vector); err != nil { 470 return err 471 } 472 } 473 474 return s.read(ctx, vector) 475 } 476 477 func (s *S3FS) ReadCache(ctx context.Context, vector *IOVector) (err error) { 478 if err := ctx.Err(); err != nil { 479 return err 480 } 481 482 if len(vector.Entries) == 0 { 483 return moerr.NewEmptyVectorNoCtx() 484 } 485 486 startLock := time.Now() 487 done, wait := s.ioMerger.Merge(vector.ioMergeKey()) 488 if done != nil { 489 defer done() 490 } else { 491 wait() 492 } 493 statistic.StatsInfoFromContext(ctx).AddS3FSReadCacheIOMergerTimeConsumption(time.Since(startLock)) 494 495 for _, cache := range vector.Caches { 496 cache := cache 497 if err := readCache(ctx, cache, vector); err != nil { 498 return err 499 } 500 defer func() { 501 if err != nil { 502 return 503 } 504 err = cache.Update(ctx, vector, false) 505 }() 506 } 507 508 if s.memCache != nil { 509 if err := readCache(ctx, s.memCache, vector); err != nil { 510 return err 511 } 512 } 513 514 return nil 515 } 516 517 func (s *S3FS) read(ctx context.Context, vector *IOVector) (err error) { 518 if vector.allDone() { 519 // all cache hit 520 return nil 521 } 522 523 path, err := ParsePathAtService(vector.FilePath, s.name) 524 if err != nil { 525 return err 526 } 527 528 min, max, readFullObject := vector.readRange() 529 530 // a function to get an io.ReadCloser 531 getReader := func(ctx context.Context, min *int64, max *int64) (io.ReadCloser, error) { 532 t0 := time.Now() 533 bytesCounter := new(atomic.Int64) 534 ctx, spanR := trace.Start(ctx, "S3FS.read.getReader") 535 defer spanR.End() 536 key := s.pathToKey(path.File) 537 r, err := s.storage.Read(ctx, key, min, max) 538 if err != nil { 539 return nil, err 540 } 541 return &readCloser{ 542 r: &countingReader{ 543 R: r, 544 C: bytesCounter, 545 }, 546 closeFunc: func() error { 547 s3ReadIODuration := time.Since(t0) 548 549 metric.S3ReadIODurationHistogram.Observe(s3ReadIODuration.Seconds()) 550 metric.S3ReadIOBytesHistogram.Observe(float64(bytesCounter.Load())) 551 return r.Close() 552 }, 553 }, nil 554 } 555 556 // a function to get data lazily 557 var contentBytes []byte 558 var contentErr error 559 var getContentDone bool 560 getContent := func(ctx context.Context) (bs []byte, err error) { 561 ctx, spanC := trace.Start(ctx, "S3FS.read.getContent") 562 defer spanC.End() 563 if getContentDone { 564 return contentBytes, contentErr 565 } 566 defer func() { 567 contentBytes = bs 568 contentErr = err 569 getContentDone = true 570 }() 571 572 reader, err := getReader(ctx, min, max) 573 if err != nil { 574 return nil, err 575 } 576 defer reader.Close() 577 bs, err = io.ReadAll(reader) 578 if err != nil { 579 return nil, err 580 } 581 582 return 583 } 584 585 numNotDoneEntries := 0 586 defer func() { 587 metric.FSReadS3Counter.Add(float64(numNotDoneEntries)) 588 }() 589 for i, entry := range vector.Entries { 590 if entry.done { 591 continue 592 } 593 entry := entry 594 numNotDoneEntries++ 595 596 start := entry.Offset - *min 597 598 if entry.Size == 0 { 599 return moerr.NewEmptyRangeNoCtx(path.File) 600 } 601 602 // a function to get entry data lazily 603 getData := func(ctx context.Context) ([]byte, error) { 604 ctx, spanD := trace.Start(ctx, "S3FS.reader.getData") 605 defer spanD.End() 606 if entry.Size < 0 { 607 // read to end 608 content, err := getContent(ctx) 609 if err != nil { 610 return nil, err 611 } 612 if start >= int64(len(content)) { 613 return nil, moerr.NewEmptyRangeNoCtx(path.File) 614 } 615 return content[start:], nil 616 } 617 content, err := getContent(ctx) 618 if err != nil { 619 return nil, err 620 } 621 end := start + entry.Size 622 if end > int64(len(content)) { 623 return nil, moerr.NewUnexpectedEOFNoCtx(path.File) 624 } 625 if start == end { 626 return nil, moerr.NewEmptyRangeNoCtx(path.File) 627 } 628 return content[start:end], nil 629 } 630 631 setData := true 632 var data []byte 633 634 if w := vector.Entries[i].WriterForRead; w != nil { 635 setData = false 636 if getContentDone { 637 // data is ready 638 data, err = getData(ctx) 639 if err != nil { 640 return err 641 } 642 _, err = w.Write(data) 643 if err != nil { 644 return err 645 } 646 647 } else { 648 // get a reader and copy 649 min := &entry.Offset 650 var max *int64 651 if entry.Size > 0 { 652 max = ptrTo(entry.Offset + entry.Size) 653 } 654 reader, err := getReader(ctx, min, max) 655 if err != nil { 656 return err 657 } 658 defer reader.Close() 659 var buf []byte 660 put := ioBufferPool.Get(&buf) 661 defer put.Put() 662 _, err = io.CopyBuffer(w, reader, buf) 663 if err != nil { 664 return err 665 } 666 } 667 } 668 669 if ptr := vector.Entries[i].ReadCloserForRead; ptr != nil { 670 setData = false 671 if getContentDone { 672 // data is ready 673 data, err = getData(ctx) 674 if err != nil { 675 return err 676 } 677 *ptr = io.NopCloser(bytes.NewReader(data)) 678 679 } else { 680 // get a new reader 681 min := &entry.Offset 682 var max *int64 683 if entry.Size > 0 { 684 max = ptrTo(entry.Offset + entry.Size) 685 } 686 reader, err := getReader(ctx, min, max) 687 if err != nil { 688 return err 689 } 690 *ptr = &readCloser{ 691 r: reader, 692 closeFunc: reader.Close, 693 } 694 } 695 } 696 697 // set Data field 698 if setData { 699 data, err = getData(ctx) 700 if err != nil { 701 return err 702 } 703 if int64(len(entry.Data)) < entry.Size || entry.Size < 0 { 704 entry.Data = data 705 if entry.Size < 0 { 706 entry.Size = int64(len(data)) 707 } 708 } else { 709 copy(entry.Data, data) 710 } 711 } 712 713 if err = entry.setCachedData(); err != nil { 714 return err 715 } 716 717 vector.Entries[i] = entry 718 } 719 720 // write to disk cache 721 if readFullObject && 722 contentErr == nil && 723 len(contentBytes) > 0 && 724 s.diskCache != nil && 725 !vector.Policy.Any(SkipDiskCacheWrites) { 726 if err := s.diskCache.SetFile(ctx, vector.FilePath, func(context.Context) (io.ReadCloser, error) { 727 return io.NopCloser(bytes.NewReader(contentBytes)), nil 728 }); err != nil { 729 return err 730 } 731 } 732 733 return nil 734 } 735 736 func (s *S3FS) Delete(ctx context.Context, filePaths ...string) error { 737 ctx, span := trace.Start(ctx, "S3FS.Delete") 738 defer span.End() 739 740 keys := make([]string, 0, len(filePaths)) 741 for _, filePath := range filePaths { 742 path, err := ParsePathAtService(filePath, s.name) 743 if err != nil { 744 return err 745 } 746 keys = append(keys, s.pathToKey(path.File)) 747 } 748 749 return errors.Join( 750 s.storage.Delete(ctx, keys...), 751 func() error { 752 if s.memCache == nil { 753 return nil 754 } 755 return s.memCache.DeletePaths(ctx, filePaths) 756 }(), 757 func() error { 758 if s.diskCache == nil { 759 return nil 760 } 761 return s.diskCache.DeletePaths(ctx, filePaths) 762 }(), 763 func() error { 764 if s.remoteCache == nil { 765 return nil 766 } 767 return s.remoteCache.DeletePaths(ctx, filePaths) 768 }(), 769 ) 770 } 771 772 var _ ETLFileService = new(S3FS) 773 774 func (*S3FS) ETLCompatible() {} 775 776 var _ CachingFileService = new(S3FS) 777 778 func (s *S3FS) Close() { 779 s.FlushCache() 780 } 781 782 func (s *S3FS) FlushCache() { 783 if s.memCache != nil { 784 s.memCache.Flush() 785 } 786 } 787 788 func (s *S3FS) SetAsyncUpdate(b bool) { 789 s.asyncUpdate = b 790 } 791 792 type tracePoint struct { 793 start time.Time 794 dnsStart time.Time 795 connectStart time.Time 796 tlsHandshakeStart time.Time 797 ct *httptrace.ClientTrace 798 } 799 800 func newTracePoint() *tracePoint { 801 tp := &tracePoint{ 802 ct: &httptrace.ClientTrace{}, 803 } 804 tp.ct.GetConn = tp.getConnPoint 805 tp.ct.GotConn = tp.gotConnPoint 806 tp.ct.DNSStart = tp.dnsStartPoint 807 tp.ct.DNSDone = tp.dnsDonePoint 808 tp.ct.ConnectStart = tp.connectStartPoint 809 tp.ct.ConnectDone = tp.connectDonePoint 810 tp.ct.TLSHandshakeStart = tp.tlsHandshakeStartPoint 811 tp.ct.TLSHandshakeDone = tp.tlsHandshakeDonePoint 812 return tp 813 } 814 815 func (tp tracePoint) TypeName() string { 816 return "fileservice.tracePoint" 817 } 818 819 func resetTracePoint(tp *tracePoint) { 820 tp.start = time.Time{} 821 tp.dnsStart = time.Time{} 822 tp.connectStart = time.Time{} 823 tp.tlsHandshakeStart = time.Time{} 824 } 825 826 func (tp *tracePoint) getClientTrace() *httptrace.ClientTrace { 827 return tp.ct 828 } 829 830 func (tp *tracePoint) getConnPoint(hostPort string) { 831 tp.start = time.Now() 832 } 833 834 func (tp *tracePoint) gotConnPoint(info httptrace.GotConnInfo) { 835 metric.S3GetConnDurationHistogram.Observe(time.Since(tp.start).Seconds()) 836 } 837 838 func (tp *tracePoint) dnsStartPoint(di httptrace.DNSStartInfo) { 839 metric.S3DNSResolveCounter.Inc() 840 tp.dnsStart = time.Now() 841 } 842 843 func (tp *tracePoint) dnsDonePoint(di httptrace.DNSDoneInfo) { 844 metric.S3DNSResolveDurationHistogram.Observe(time.Since(tp.dnsStart).Seconds()) 845 } 846 847 func (tp *tracePoint) connectStartPoint(network, addr string) { 848 metric.S3ConnectCounter.Inc() 849 tp.connectStart = time.Now() 850 } 851 852 func (tp *tracePoint) connectDonePoint(network, addr string, err error) { 853 metric.S3ConnectDurationHistogram.Observe(time.Since(tp.connectStart).Seconds()) 854 } 855 856 func (tp *tracePoint) tlsHandshakeStartPoint() { 857 tp.tlsHandshakeStart = time.Now() 858 } 859 860 func (tp *tracePoint) tlsHandshakeDonePoint(cs tls.ConnectionState, err error) { 861 metric.S3TLSHandshakeDurationHistogram.Observe(time.Since(tp.tlsHandshakeStart).Seconds()) 862 }