mirror of
https://github.com/minio/minio.git
synced 2025-11-07 04:42:56 -05:00
Simplify HTTP trace related code (#7833)
This commit is contained in:
committed by
kannappanr
parent
c1d2b3d5c3
commit
183ec094c4
@@ -43,6 +43,7 @@ import (
|
||||
"github.com/minio/minio/pkg/mem"
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
"github.com/minio/minio/pkg/quick"
|
||||
trace "github.com/minio/minio/pkg/trace"
|
||||
)
|
||||
|
||||
const (
|
||||
@@ -1483,11 +1484,6 @@ func (a adminAPIHandlers) TraceHandler(w http.ResponseWriter, r *http.Request) {
|
||||
return
|
||||
}
|
||||
|
||||
if globalTrace == nil {
|
||||
writeErrorResponseJSON(ctx, w, errorCodes.ToAPIErr(ErrServerNotInitialized), r.URL)
|
||||
return
|
||||
}
|
||||
|
||||
// Avoid reusing tcp connection if read timeout is hit
|
||||
// This is needed to make r.Context().Done() work as
|
||||
// expected in case of read timeout
|
||||
@@ -1496,14 +1492,33 @@ func (a adminAPIHandlers) TraceHandler(w http.ResponseWriter, r *http.Request) {
|
||||
doneCh := make(chan struct{})
|
||||
defer close(doneCh)
|
||||
|
||||
traceCh := globalTrace.Trace(doneCh, trcAll)
|
||||
// Trace Publisher and peer-trace-client uses nonblocking send and hence does not wait for slow receivers.
|
||||
// Use buffered channel to take care of burst sends or slow w.Write()
|
||||
traceCh := make(chan interface{}, 4000)
|
||||
|
||||
filter := func(entry interface{}) bool {
|
||||
if trcAll {
|
||||
return true
|
||||
}
|
||||
trcInfo := entry.(trace.Info)
|
||||
return !strings.HasPrefix(trcInfo.ReqInfo.Path, minioReservedBucketPath)
|
||||
}
|
||||
remoteHosts := getRemoteHosts(globalEndpoints)
|
||||
peers, err := getRestClients(remoteHosts)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
globalHTTPTrace.Subscribe(traceCh, doneCh, filter)
|
||||
|
||||
for _, peer := range peers {
|
||||
peer.Trace(traceCh, doneCh, trcAll)
|
||||
}
|
||||
|
||||
enc := json.NewEncoder(w)
|
||||
for {
|
||||
select {
|
||||
case entry := <-traceCh:
|
||||
if _, err := w.Write(entry); err != nil {
|
||||
return
|
||||
}
|
||||
if _, err := w.Write([]byte("\n")); err != nil {
|
||||
if err := enc.Encode(entry); err != nil {
|
||||
return
|
||||
}
|
||||
w.(http.Flusher).Flush()
|
||||
|
||||
@@ -158,9 +158,6 @@ func StartGateway(ctx *cli.Context, gw Gateway) {
|
||||
registerSTSRouter(router)
|
||||
}
|
||||
|
||||
// initialize globalTrace system
|
||||
globalTrace = NewTraceSys(context.Background(), globalEndpoints)
|
||||
|
||||
enableConfigOps := globalEtcdClient != nil && gatewayName == "nas"
|
||||
enableIAMOps := globalEtcdClient != nil
|
||||
|
||||
|
||||
@@ -35,6 +35,7 @@ import (
|
||||
"github.com/minio/minio/pkg/dns"
|
||||
iampolicy "github.com/minio/minio/pkg/iam/policy"
|
||||
"github.com/minio/minio/pkg/iam/validator"
|
||||
"github.com/minio/minio/pkg/pubsub"
|
||||
)
|
||||
|
||||
// minio configuration related constants.
|
||||
@@ -161,7 +162,7 @@ var (
|
||||
|
||||
// global Trace system to send HTTP request/response logs to
|
||||
// registered listeners
|
||||
globalTrace *HTTPTraceSys
|
||||
globalHTTPTrace = pubsub.New()
|
||||
|
||||
globalEndpoints EndpointList
|
||||
|
||||
|
||||
@@ -326,24 +326,24 @@ func extractPostPolicyFormValues(ctx context.Context, form *multipart.Form) (fil
|
||||
// Log headers and body.
|
||||
func httpTraceAll(f http.HandlerFunc) http.HandlerFunc {
|
||||
return func(w http.ResponseWriter, r *http.Request) {
|
||||
if !globalTrace.HasTraceListeners() {
|
||||
if !globalHTTPTrace.HasSubscribers() {
|
||||
f.ServeHTTP(w, r)
|
||||
return
|
||||
}
|
||||
trace := Trace(f, true, w, r)
|
||||
globalTrace.Publish(trace)
|
||||
globalHTTPTrace.Publish(trace)
|
||||
}
|
||||
}
|
||||
|
||||
// Log only the headers.
|
||||
func httpTraceHdrs(f http.HandlerFunc) http.HandlerFunc {
|
||||
return func(w http.ResponseWriter, r *http.Request) {
|
||||
if !globalTrace.HasTraceListeners() {
|
||||
if !globalHTTPTrace.HasSubscribers() {
|
||||
f.ServeHTTP(w, r)
|
||||
return
|
||||
}
|
||||
trace := Trace(f, false, w, r)
|
||||
globalTrace.Publish(trace)
|
||||
globalHTTPTrace.Publish(trace)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
115
cmd/httptrace.go
115
cmd/httptrace.go
@@ -1,115 +0,0 @@
|
||||
/*
|
||||
* MinIO Cloud Storage, (C) 2019 MinIO, Inc.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package cmd
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/json"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
"github.com/minio/minio/pkg/pubsub"
|
||||
"github.com/minio/minio/pkg/trace"
|
||||
)
|
||||
|
||||
//HTTPTraceSys holds global trace state
|
||||
type HTTPTraceSys struct {
|
||||
peers []*peerRESTClient
|
||||
pubsub *pubsub.PubSub
|
||||
}
|
||||
|
||||
// NewTraceSys - creates new HTTPTraceSys with all nodes subscribed to
|
||||
// the trace pub sub system
|
||||
func NewTraceSys(ctx context.Context, endpoints EndpointList) *HTTPTraceSys {
|
||||
remoteHosts := getRemoteHosts(endpoints)
|
||||
remoteClients, err := getRestClients(remoteHosts)
|
||||
if err != nil {
|
||||
logger.FatalIf(err, "Unable to start httptrace sub system")
|
||||
}
|
||||
|
||||
ps := pubsub.New()
|
||||
return &HTTPTraceSys{
|
||||
remoteClients, ps,
|
||||
}
|
||||
}
|
||||
|
||||
// HasTraceListeners returns true if trace listeners are registered
|
||||
// for this node or peers
|
||||
func (sys *HTTPTraceSys) HasTraceListeners() bool {
|
||||
return sys != nil && sys.pubsub.HasSubscribers()
|
||||
}
|
||||
|
||||
// Publish - publishes trace message to the http trace pubsub system
|
||||
func (sys *HTTPTraceSys) Publish(traceMsg trace.Info) {
|
||||
sys.pubsub.Publish(traceMsg)
|
||||
}
|
||||
|
||||
// Trace writes http trace to writer
|
||||
func (sys *HTTPTraceSys) Trace(doneCh chan struct{}, trcAll bool) chan []byte {
|
||||
traceCh := make(chan []byte)
|
||||
go func() {
|
||||
defer close(traceCh)
|
||||
|
||||
var wg = &sync.WaitGroup{}
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
|
||||
buf := &bytes.Buffer{}
|
||||
ch := sys.pubsub.Subscribe()
|
||||
defer sys.pubsub.Unsubscribe(ch)
|
||||
for {
|
||||
select {
|
||||
case entry := <-ch:
|
||||
trcInfo := entry.(trace.Info)
|
||||
path := strings.TrimPrefix(trcInfo.ReqInfo.Path, "/")
|
||||
// omit inter-node traffic if trcAll is false
|
||||
if !trcAll && strings.HasPrefix(path, minioReservedBucket) {
|
||||
continue
|
||||
}
|
||||
buf.Reset()
|
||||
enc := json.NewEncoder(buf)
|
||||
enc.SetEscapeHTML(false)
|
||||
if err := enc.Encode(trcInfo); err != nil {
|
||||
continue
|
||||
}
|
||||
traceCh <- buf.Bytes()
|
||||
case <-doneCh:
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
for _, peer := range sys.peers {
|
||||
wg.Add(1)
|
||||
go func(peer *peerRESTClient) {
|
||||
defer wg.Done()
|
||||
ch, err := peer.Trace(doneCh, trcAll)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
for entry := range ch {
|
||||
traceCh <- entry
|
||||
}
|
||||
}(peer)
|
||||
}
|
||||
wg.Wait()
|
||||
}()
|
||||
return traceCh
|
||||
}
|
||||
@@ -17,7 +17,6 @@
|
||||
package cmd
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"bytes"
|
||||
"context"
|
||||
"crypto/tls"
|
||||
@@ -34,6 +33,7 @@ import (
|
||||
"github.com/minio/minio/pkg/madmin"
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
"github.com/minio/minio/pkg/policy"
|
||||
trace "github.com/minio/minio/pkg/trace"
|
||||
)
|
||||
|
||||
// client to talk to peer Nodes.
|
||||
@@ -435,52 +435,59 @@ func (client *peerRESTClient) BackgroundHealStatus() (madmin.BgHealState, error)
|
||||
return state, err
|
||||
}
|
||||
|
||||
// Trace - send http trace request to peer nodes
|
||||
func (client *peerRESTClient) Trace(doneCh chan struct{}, trcAll bool) (chan []byte, error) {
|
||||
ch := make(chan []byte)
|
||||
func (client *peerRESTClient) doTrace(traceCh chan interface{}, doneCh chan struct{}, trcAll bool) {
|
||||
values := make(url.Values)
|
||||
values.Set(peerRESTTraceAll, strconv.FormatBool(trcAll))
|
||||
|
||||
// To cancel the REST request in case doneCh gets closed.
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
cancelCh := make(chan struct{})
|
||||
defer close(cancelCh)
|
||||
go func() {
|
||||
cleanupFn := func(cancel context.CancelFunc, ch chan []byte, respBody io.ReadCloser) {
|
||||
close(ch)
|
||||
if cancel != nil {
|
||||
cancel()
|
||||
}
|
||||
http.DrainBody(respBody)
|
||||
select {
|
||||
case <-doneCh:
|
||||
case <-cancelCh:
|
||||
// There was an error in the REST request.
|
||||
}
|
||||
cancel()
|
||||
}()
|
||||
|
||||
respBody, err := client.callWithContext(ctx, peerRESTMethodTrace, values, nil, -1)
|
||||
defer http.DrainBody(respBody)
|
||||
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
dec := gob.NewDecoder(respBody)
|
||||
for {
|
||||
var info trace.Info
|
||||
if err = dec.Decode(&info); err != nil {
|
||||
return
|
||||
}
|
||||
select {
|
||||
case traceCh <- info:
|
||||
default:
|
||||
// Do not block on slow receivers.
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Trace - send http trace request to peer nodes
|
||||
func (client *peerRESTClient) Trace(traceCh chan interface{}, doneCh chan struct{}, trcAll bool) {
|
||||
go func() {
|
||||
for {
|
||||
values := make(url.Values)
|
||||
values.Set(peerRESTTraceAll, strconv.FormatBool(trcAll))
|
||||
// get cancellation context to properly unsubscribe peers
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
respBody, err := client.callWithContext(ctx, peerRESTMethodTrace, values, nil, -1)
|
||||
if err != nil {
|
||||
//retry
|
||||
time.Sleep(5 * time.Second)
|
||||
select {
|
||||
case <-doneCh:
|
||||
cleanupFn(cancel, ch, respBody)
|
||||
return
|
||||
default:
|
||||
}
|
||||
continue
|
||||
}
|
||||
bio := bufio.NewScanner(respBody)
|
||||
go func() {
|
||||
<-doneCh
|
||||
cancel()
|
||||
}()
|
||||
// Unmarshal each line, returns marshaled values.
|
||||
for bio.Scan() {
|
||||
ch <- bio.Bytes()
|
||||
}
|
||||
client.doTrace(traceCh, doneCh, trcAll)
|
||||
select {
|
||||
case <-doneCh:
|
||||
cleanupFn(cancel, ch, respBody)
|
||||
return
|
||||
default:
|
||||
// There was error in the REST request, retry after sometime as probably the peer is down.
|
||||
time.Sleep(5 * time.Second)
|
||||
}
|
||||
}
|
||||
}()
|
||||
return ch, nil
|
||||
}
|
||||
|
||||
func getRemoteHosts(endpoints EndpointList) []*xnet.Host {
|
||||
|
||||
@@ -19,7 +19,6 @@ package cmd
|
||||
import (
|
||||
"context"
|
||||
"encoding/gob"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"net/http"
|
||||
@@ -679,32 +678,33 @@ func (s *peerRESTServer) TraceHandler(w http.ResponseWriter, r *http.Request) {
|
||||
w.Header().Set("Connection", "close")
|
||||
w.WriteHeader(http.StatusOK)
|
||||
w.(http.Flusher).Flush()
|
||||
ch := globalTrace.pubsub.Subscribe()
|
||||
defer globalTrace.pubsub.Unsubscribe(ch)
|
||||
|
||||
enc := json.NewEncoder(w)
|
||||
enc.SetEscapeHTML(false)
|
||||
filter := func(entry interface{}) bool {
|
||||
if trcAll {
|
||||
return true
|
||||
}
|
||||
trcInfo := entry.(trace.Info)
|
||||
return !strings.HasPrefix(trcInfo.ReqInfo.Path, minioReservedBucketPath)
|
||||
}
|
||||
|
||||
doneCh := make(chan struct{})
|
||||
defer close(doneCh)
|
||||
|
||||
// Trace Publisher uses nonblocking publish and hence does not wait for slow subscribers.
|
||||
// Use buffered channel to take care of burst sends or slow w.Write()
|
||||
ch := make(chan interface{}, 2000)
|
||||
globalHTTPTrace.Subscribe(ch, doneCh, filter)
|
||||
|
||||
enc := gob.NewEncoder(w)
|
||||
for {
|
||||
select {
|
||||
case entry := <-ch:
|
||||
trcInfo := entry.(trace.Info)
|
||||
path := strings.TrimPrefix(trcInfo.ReqInfo.Path, "/")
|
||||
// omit inter-node traffic if trcAll is false
|
||||
if !trcAll && strings.HasPrefix(path, minioReservedBucket) {
|
||||
continue
|
||||
}
|
||||
|
||||
if err := enc.Encode(trcInfo); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
if _, err := w.Write([]byte("\n")); err != nil {
|
||||
if err := enc.Encode(entry); err != nil {
|
||||
return
|
||||
}
|
||||
w.(http.Flusher).Flush()
|
||||
case <-r.Context().Done():
|
||||
return
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -294,9 +294,6 @@ func serverMain(ctx *cli.Context) {
|
||||
globalSweepHealState = initHealState()
|
||||
}
|
||||
|
||||
// initialize globalTrace system
|
||||
globalTrace = NewTraceSys(context.Background(), globalEndpoints)
|
||||
|
||||
// Configure server.
|
||||
var handler http.Handler
|
||||
handler, err = configureServerHandler(globalEndpoints)
|
||||
|
||||
Reference in New Issue
Block a user