mirror of
				https://github.com/caddyserver/caddy.git
				synced 2025-11-04 03:27:23 -05:00 
			
		
		
		
	Differentiating middleware and responders has one benefit, namely that it's clear which module provides the response, but even then it's not a great advantage. Linear handler config makes a little more sense, giving greater flexibility and simplifying the core a bit, even though it's slightly awkward that handlers which are responders may not use the 'next' handler that is passed in at all.
This commit is contained in:
		
							parent
							
								
									6dfba5fda8
								
							
						
					
					
						commit
						9343403358
					
				@ -322,14 +322,16 @@ func (app *App) automaticHTTPS() error {
 | 
			
		||||
							MatchHost(domains),
 | 
			
		||||
						},
 | 
			
		||||
					},
 | 
			
		||||
					responder: Static{
 | 
			
		||||
						StatusCode: http.StatusTemporaryRedirect, // TODO: use permanent redirect instead
 | 
			
		||||
					handlers: []MiddlewareHandler{
 | 
			
		||||
						Static{
 | 
			
		||||
							StatusCode: strconv.Itoa(http.StatusTemporaryRedirect), // TODO: use permanent redirect instead
 | 
			
		||||
							Headers: http.Header{
 | 
			
		||||
								"Location":   []string{redirTo},
 | 
			
		||||
								"Connection": []string{"close"},
 | 
			
		||||
							},
 | 
			
		||||
							Close: true,
 | 
			
		||||
						},
 | 
			
		||||
					},
 | 
			
		||||
				})
 | 
			
		||||
			}
 | 
			
		||||
		}
 | 
			
		||||
@ -381,34 +383,18 @@ func (app *App) listenerTaken(network, address string) bool {
 | 
			
		||||
var defaultALPN = []string{"h2", "http/1.1"}
 | 
			
		||||
 | 
			
		||||
// RequestMatcher is a type that can match to a request.
 | 
			
		||||
// A route matcher MUST NOT modify the request.
 | 
			
		||||
// A route matcher MUST NOT modify the request, with the
 | 
			
		||||
// only exception being its context.
 | 
			
		||||
type RequestMatcher interface {
 | 
			
		||||
	Match(*http.Request) bool
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// Middleware chains one Handler to the next by being passed
 | 
			
		||||
// the next Handler in the chain.
 | 
			
		||||
type Middleware func(HandlerFunc) HandlerFunc
 | 
			
		||||
 | 
			
		||||
// MiddlewareHandler is a Handler that includes a reference
 | 
			
		||||
// to the next middleware handler in the chain. Middleware
 | 
			
		||||
// handlers MUST NOT call Write() or WriteHeader() on the
 | 
			
		||||
// response writer; doing so will panic. See Handler godoc
 | 
			
		||||
// for more information.
 | 
			
		||||
type MiddlewareHandler interface {
 | 
			
		||||
	ServeHTTP(http.ResponseWriter, *http.Request, Handler) error
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// Handler is like http.Handler except ServeHTTP may return an error.
 | 
			
		||||
//
 | 
			
		||||
// Middleware and responder handlers both implement this method.
 | 
			
		||||
// Middleware must not call Write or WriteHeader on the ResponseWriter;
 | 
			
		||||
// doing so will cause a panic. Responders should write to the response
 | 
			
		||||
// if there was not an error.
 | 
			
		||||
//
 | 
			
		||||
// If any handler encounters an error, it should be returned for proper
 | 
			
		||||
// handling. Return values should be propagated down the middleware chain
 | 
			
		||||
// by returning it unchanged. Returned errors should not be re-wrapped.
 | 
			
		||||
// by returning it unchanged. Returned errors should not be re-wrapped
 | 
			
		||||
// if they are already HandlerError values.
 | 
			
		||||
type Handler interface {
 | 
			
		||||
	ServeHTTP(http.ResponseWriter, *http.Request) error
 | 
			
		||||
}
 | 
			
		||||
@ -421,9 +407,25 @@ func (f HandlerFunc) ServeHTTP(w http.ResponseWriter, r *http.Request) error {
 | 
			
		||||
	return f(w, r)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// emptyHandler is used as a no-op handler, which is
 | 
			
		||||
// sometimes better than a nil Handler pointer.
 | 
			
		||||
var emptyHandler HandlerFunc = func(w http.ResponseWriter, r *http.Request) error { return nil }
 | 
			
		||||
// Middleware chains one Handler to the next by being passed
 | 
			
		||||
// the next Handler in the chain.
 | 
			
		||||
type Middleware func(HandlerFunc) HandlerFunc
 | 
			
		||||
 | 
			
		||||
// MiddlewareHandler is like Handler except it takes as a third
 | 
			
		||||
// argument the next handler in the chain. The next handler will
 | 
			
		||||
// never be nil, but may be a no-op handler if this is the last
 | 
			
		||||
// handler in the chain. Handlers which act as middleware should
 | 
			
		||||
// call the next handler's ServeHTTP method so as to propagate
 | 
			
		||||
// the request down the chain properly. Handlers which act as
 | 
			
		||||
// responders (content origins) need not invoke the next handler,
 | 
			
		||||
// since the last handler in the chain should be the first to
 | 
			
		||||
// write the response.
 | 
			
		||||
type MiddlewareHandler interface {
 | 
			
		||||
	ServeHTTP(http.ResponseWriter, *http.Request, Handler) error
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// emptyHandler is used as a no-op handler.
 | 
			
		||||
var emptyHandler HandlerFunc = func(http.ResponseWriter, *http.Request) error { return nil }
 | 
			
		||||
 | 
			
		||||
const (
 | 
			
		||||
	// DefaultHTTPPort is the default port for HTTP.
 | 
			
		||||
 | 
			
		||||
@ -36,7 +36,7 @@ import (
 | 
			
		||||
 | 
			
		||||
func init() {
 | 
			
		||||
	caddy.RegisterModule(caddy.Module{
 | 
			
		||||
		Name: "http.middleware.encode",
 | 
			
		||||
		Name: "http.handlers.encode",
 | 
			
		||||
		New:  func() interface{} { return new(Encode) },
 | 
			
		||||
	})
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@ -35,7 +35,7 @@ func init() {
 | 
			
		||||
	weakrand.Seed(time.Now().UnixNano())
 | 
			
		||||
 | 
			
		||||
	caddy.RegisterModule(caddy.Module{
 | 
			
		||||
		Name: "http.responders.file_server",
 | 
			
		||||
		Name: "http.handlers.file_server",
 | 
			
		||||
		New:  func() interface{} { return new(FileServer) },
 | 
			
		||||
	})
 | 
			
		||||
}
 | 
			
		||||
@ -108,7 +108,7 @@ func (fsrv *FileServer) Validate() error {
 | 
			
		||||
	return nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (fsrv *FileServer) ServeHTTP(w http.ResponseWriter, r *http.Request) error {
 | 
			
		||||
func (fsrv *FileServer) ServeHTTP(w http.ResponseWriter, r *http.Request, _ caddyhttp.Handler) error {
 | 
			
		||||
	repl := r.Context().Value(caddy.ReplacerCtxKey).(caddy.Replacer)
 | 
			
		||||
 | 
			
		||||
	filesToHide := fsrv.transformHidePaths(repl)
 | 
			
		||||
@ -119,7 +119,7 @@ func (fsrv *FileServer) ServeHTTP(w http.ResponseWriter, r *http.Request) error
 | 
			
		||||
	if filename == "" {
 | 
			
		||||
		// no files worked, so resort to fallback
 | 
			
		||||
		if fsrv.Fallback != nil {
 | 
			
		||||
			fallback, w := fsrv.Fallback.BuildCompositeRoute(w, r)
 | 
			
		||||
			fallback := fsrv.Fallback.BuildCompositeRoute(w, r)
 | 
			
		||||
			return fallback.ServeHTTP(w, r)
 | 
			
		||||
		}
 | 
			
		||||
		return caddyhttp.Error(http.StatusNotFound, nil)
 | 
			
		||||
@ -454,5 +454,5 @@ const minBackoff, maxBackoff = 2, 5
 | 
			
		||||
var (
 | 
			
		||||
	_ caddy.Provisioner           = (*FileServer)(nil)
 | 
			
		||||
	_ caddy.Validator             = (*FileServer)(nil)
 | 
			
		||||
	_ caddyhttp.Handler = (*FileServer)(nil)
 | 
			
		||||
	_ caddyhttp.MiddlewareHandler = (*FileServer)(nil)
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
@ -25,7 +25,7 @@ import (
 | 
			
		||||
 | 
			
		||||
func init() {
 | 
			
		||||
	caddy.RegisterModule(caddy.Module{
 | 
			
		||||
		Name: "http.middleware.rewrite",
 | 
			
		||||
		Name: "http.handlers.rewrite",
 | 
			
		||||
		New:  func() interface{} { return new(Rewrite) },
 | 
			
		||||
	})
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@ -28,25 +28,19 @@ import (
 | 
			
		||||
type ServerRoute struct {
 | 
			
		||||
	Group       string                       `json:"group,omitempty"`
 | 
			
		||||
	MatcherSets []map[string]json.RawMessage `json:"match,omitempty"`
 | 
			
		||||
	Apply       []json.RawMessage            `json:"apply,omitempty"`
 | 
			
		||||
	Respond     json.RawMessage              `json:"respond,omitempty"`
 | 
			
		||||
 | 
			
		||||
	Handle      []json.RawMessage            `json:"handle,omitempty"`
 | 
			
		||||
	Terminal    bool                         `json:"terminal,omitempty"`
 | 
			
		||||
 | 
			
		||||
	// decoded values
 | 
			
		||||
	matcherSets []MatcherSet
 | 
			
		||||
	middleware  []MiddlewareHandler
 | 
			
		||||
	responder   Handler
 | 
			
		||||
	handlers    []MiddlewareHandler
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// Empty returns true if the route has all zero/default values.
 | 
			
		||||
func (sr ServerRoute) Empty() bool {
 | 
			
		||||
	return len(sr.MatcherSets) == 0 &&
 | 
			
		||||
		len(sr.Apply) == 0 &&
 | 
			
		||||
		len(sr.Respond) == 0 &&
 | 
			
		||||
		len(sr.matcherSets) == 0 &&
 | 
			
		||||
		len(sr.middleware) == 0 &&
 | 
			
		||||
		sr.responder == nil &&
 | 
			
		||||
		len(sr.Handle) == 0 &&
 | 
			
		||||
		len(sr.handlers) == 0 &&
 | 
			
		||||
		!sr.Terminal &&
 | 
			
		||||
		sr.Group == ""
 | 
			
		||||
}
 | 
			
		||||
@ -98,40 +92,27 @@ func (routes RouteList) Provision(ctx caddy.Context) error {
 | 
			
		||||
		}
 | 
			
		||||
		routes[i].MatcherSets = nil // allow GC to deallocate - TODO: Does this help?
 | 
			
		||||
 | 
			
		||||
		// middleware
 | 
			
		||||
		for j, rawMsg := range route.Apply {
 | 
			
		||||
			mid, err := ctx.LoadModuleInline("middleware", "http.middleware", rawMsg)
 | 
			
		||||
		// handlers
 | 
			
		||||
		for j, rawMsg := range route.Handle {
 | 
			
		||||
			mh, err := ctx.LoadModuleInline("handler", "http.handlers", rawMsg)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				return fmt.Errorf("loading middleware module in position %d: %v", j, err)
 | 
			
		||||
				return fmt.Errorf("loading handler module in position %d: %v", j, err)
 | 
			
		||||
			}
 | 
			
		||||
			routes[i].middleware = append(routes[i].middleware, mid.(MiddlewareHandler))
 | 
			
		||||
			routes[i].handlers = append(routes[i].handlers, mh.(MiddlewareHandler))
 | 
			
		||||
		}
 | 
			
		||||
		routes[i].Apply = nil // allow GC to deallocate - TODO: Does this help?
 | 
			
		||||
 | 
			
		||||
		// responder
 | 
			
		||||
		if route.Respond != nil {
 | 
			
		||||
			resp, err := ctx.LoadModuleInline("responder", "http.responders", route.Respond)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				return fmt.Errorf("loading responder module: %v", err)
 | 
			
		||||
			}
 | 
			
		||||
			routes[i].responder = resp.(Handler)
 | 
			
		||||
		}
 | 
			
		||||
		routes[i].Respond = nil // allow GC to deallocate - TODO: Does this help?
 | 
			
		||||
		routes[i].Handle = nil // allow GC to deallocate - TODO: Does this help?
 | 
			
		||||
	}
 | 
			
		||||
	return nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// BuildCompositeRoute creates a chain of handlers by applying all the matching
 | 
			
		||||
// routes. The returned ResponseWriter should be used instead of rw.
 | 
			
		||||
func (routes RouteList) BuildCompositeRoute(rw http.ResponseWriter, req *http.Request) (Handler, http.ResponseWriter) {
 | 
			
		||||
	mrw := &middlewareResponseWriter{ResponseWriterWrapper: &ResponseWriterWrapper{rw}}
 | 
			
		||||
 | 
			
		||||
// BuildCompositeRoute creates a chain of handlers by
 | 
			
		||||
// applying all of the matching routes.
 | 
			
		||||
func (routes RouteList) BuildCompositeRoute(rw http.ResponseWriter, req *http.Request) Handler {
 | 
			
		||||
	if len(routes) == 0 {
 | 
			
		||||
		return emptyHandler, mrw
 | 
			
		||||
		return emptyHandler
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	var mid []Middleware
 | 
			
		||||
	var responder Handler
 | 
			
		||||
	groups := make(map[string]struct{})
 | 
			
		||||
 | 
			
		||||
	for _, route := range routes {
 | 
			
		||||
@ -140,9 +121,8 @@ func (routes RouteList) BuildCompositeRoute(rw http.ResponseWriter, req *http.Re
 | 
			
		||||
			continue
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// if route is part of a group, ensure only
 | 
			
		||||
		// the first matching route in the group is
 | 
			
		||||
		// applied
 | 
			
		||||
		// if route is part of a group, ensure only the
 | 
			
		||||
		// first matching route in the group is applied
 | 
			
		||||
		if route.Group != "" {
 | 
			
		||||
			_, ok := groups[route.Group]
 | 
			
		||||
			if ok {
 | 
			
		||||
@ -155,78 +135,48 @@ func (routes RouteList) BuildCompositeRoute(rw http.ResponseWriter, req *http.Re
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// apply the rest of the route
 | 
			
		||||
		for _, m := range route.middleware {
 | 
			
		||||
			// we have to be sure to wrap m outside
 | 
			
		||||
			// of our current scope so that the
 | 
			
		||||
			// reference to this m isn't overwritten
 | 
			
		||||
			// on the next iteration, leaving only
 | 
			
		||||
			// the last middleware in the chain as
 | 
			
		||||
			// the ONLY middleware in the chain!
 | 
			
		||||
			mid = append(mid, wrapMiddleware(m))
 | 
			
		||||
		}
 | 
			
		||||
		if responder == nil {
 | 
			
		||||
			responder = route.responder
 | 
			
		||||
		for _, mh := range route.handlers {
 | 
			
		||||
			// we have to be sure to wrap mh outside
 | 
			
		||||
			// of our current stack frame so that the
 | 
			
		||||
			// reference to this mh isn't overwritten
 | 
			
		||||
			// on the next iteration, leaving the last
 | 
			
		||||
			// middleware in the chain as the ONLY
 | 
			
		||||
			// middleware in the chain!
 | 
			
		||||
			mid = append(mid, wrapMiddleware(mh))
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// if this route is supposed to be last, don't
 | 
			
		||||
		// compile any more into the chain
 | 
			
		||||
		if route.Terminal {
 | 
			
		||||
			break
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// build the middleware stack, with the responder at the end
 | 
			
		||||
	stack := HandlerFunc(func(w http.ResponseWriter, r *http.Request) error {
 | 
			
		||||
		if responder == nil {
 | 
			
		||||
			return nil
 | 
			
		||||
		}
 | 
			
		||||
		mrw.allowWrites = true
 | 
			
		||||
		return responder.ServeHTTP(w, r)
 | 
			
		||||
	})
 | 
			
		||||
	// build the middleware chain, with the responder at the end
 | 
			
		||||
	stack := emptyHandler
 | 
			
		||||
	for i := len(mid) - 1; i >= 0; i-- {
 | 
			
		||||
		stack = mid[i](stack)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	return stack, mrw
 | 
			
		||||
	return stack
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// wrapMiddleware wraps m such that it can be correctly
 | 
			
		||||
// appended to a list of middleware. This separate closure
 | 
			
		||||
// is necessary so that only the last middleware in a loop
 | 
			
		||||
// does not become the only middleware of the stack,
 | 
			
		||||
// repeatedly executed (i.e. it is necessary to keep a
 | 
			
		||||
// reference to this m outside of the scope of a loop)!
 | 
			
		||||
func wrapMiddleware(m MiddlewareHandler) Middleware {
 | 
			
		||||
// appended to a list of middleware. We can't do this
 | 
			
		||||
// directly in a loop because it relies on a reference
 | 
			
		||||
// to mh not changing until the execution of its handler,
 | 
			
		||||
// which is deferred by multiple func closures. In other
 | 
			
		||||
// words, we need to pull this particular MiddlewareHandler
 | 
			
		||||
// pointer into its own stack frame to preserve it so it
 | 
			
		||||
// won't be overwritten in future loop iterations.
 | 
			
		||||
func wrapMiddleware(mh MiddlewareHandler) Middleware {
 | 
			
		||||
	return func(next HandlerFunc) HandlerFunc {
 | 
			
		||||
		return func(w http.ResponseWriter, r *http.Request) error {
 | 
			
		||||
			// TODO: This is where request tracing could be implemented; also
 | 
			
		||||
			// see below to trace the responder as well
 | 
			
		||||
			// TODO: Trace a diff of the request, would be cool too! see what changed since the last middleware (host, headers, URI...)
 | 
			
		||||
			// TODO: see what the std lib gives us in terms of stack tracing too
 | 
			
		||||
			return m.ServeHTTP(w, r, next)
 | 
			
		||||
			return mh.ServeHTTP(w, r, next)
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
type middlewareResponseWriter struct {
 | 
			
		||||
	*ResponseWriterWrapper
 | 
			
		||||
	allowWrites bool
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (mrw middlewareResponseWriter) WriteHeader(statusCode int) {
 | 
			
		||||
	if !mrw.allowWrites {
 | 
			
		||||
		// technically, this is not true: middleware can write headers,
 | 
			
		||||
		// but only after the responder handler has returned; either the
 | 
			
		||||
		// responder did nothing with the response (sad face), or the
 | 
			
		||||
		// middleware wrapped the response and deferred the write
 | 
			
		||||
		panic("WriteHeader: middleware cannot write response headers")
 | 
			
		||||
	}
 | 
			
		||||
	mrw.ResponseWriterWrapper.WriteHeader(statusCode)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (mrw middlewareResponseWriter) Write(b []byte) (int, error) {
 | 
			
		||||
	if !mrw.allowWrites {
 | 
			
		||||
		panic("Write: middleware cannot write to the response before responder")
 | 
			
		||||
	}
 | 
			
		||||
	return mrw.ResponseWriterWrapper.Write(b)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// Interface guard
 | 
			
		||||
var _ HTTPInterfaces = (*middlewareResponseWriter)(nil)
 | 
			
		||||
 | 
			
		||||
@ -65,9 +65,9 @@ func (s *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) {
 | 
			
		||||
	addHTTPVarsToReplacer(repl, r, w)
 | 
			
		||||
 | 
			
		||||
	// build and execute the main handler chain
 | 
			
		||||
	stack, wrappedWriter := s.Routes.BuildCompositeRoute(w, r)
 | 
			
		||||
	stack := s.Routes.BuildCompositeRoute(w, r)
 | 
			
		||||
	stack = s.wrapPrimaryRoute(stack)
 | 
			
		||||
	err := s.executeCompositeRoute(wrappedWriter, r, stack)
 | 
			
		||||
	err := s.executeCompositeRoute(w, r, stack)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		// add the raw error value to the request context
 | 
			
		||||
		// so it can be accessed by error handlers
 | 
			
		||||
@ -85,8 +85,8 @@ func (s *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) {
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		if s.Errors != nil && len(s.Errors.Routes) > 0 {
 | 
			
		||||
			errStack, wrappedWriter := s.Errors.Routes.BuildCompositeRoute(w, r)
 | 
			
		||||
			err := s.executeCompositeRoute(wrappedWriter, r, errStack)
 | 
			
		||||
			errStack := s.Errors.Routes.BuildCompositeRoute(w, r)
 | 
			
		||||
			err := s.executeCompositeRoute(w, r, errStack)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				// TODO: what should we do if the error handler has an error?
 | 
			
		||||
				log.Printf("[ERROR] [%s %s] handling error: %v", r.Method, r.RequestURI, err)
 | 
			
		||||
@ -154,6 +154,8 @@ func (s *Server) enforcementHandler(w http.ResponseWriter, r *http.Request, next
 | 
			
		||||
	return next.ServeHTTP(w, r)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// listenersUseAnyPortOtherThan returns true if there are any
 | 
			
		||||
// listeners in s that use a port which is not otherPort.
 | 
			
		||||
func (s *Server) listenersUseAnyPortOtherThan(otherPort int) bool {
 | 
			
		||||
	for _, lnAddr := range s.Listen {
 | 
			
		||||
		_, addrs, err := caddy.ParseListenAddr(lnAddr)
 | 
			
		||||
 | 
			
		||||
@ -24,21 +24,20 @@ import (
 | 
			
		||||
 | 
			
		||||
func init() {
 | 
			
		||||
	caddy.RegisterModule(caddy.Module{
 | 
			
		||||
		Name: "http.responders.static",
 | 
			
		||||
		Name: "http.handlers.static",
 | 
			
		||||
		New:  func() interface{} { return new(Static) },
 | 
			
		||||
	})
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// Static implements a simple responder for static responses.
 | 
			
		||||
type Static struct {
 | 
			
		||||
	StatusCode    int         `json:"status_code"` // TODO: should we turn this into a string so that only one field is needed? (string allows replacements)
 | 
			
		||||
	StatusCodeStr string      `json:"status_code_str"`
 | 
			
		||||
	StatusCode string      `json:"status_code"`
 | 
			
		||||
	Headers    http.Header `json:"headers"`
 | 
			
		||||
	Body       string      `json:"body"`
 | 
			
		||||
	Close      bool        `json:"close"`
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (s Static) ServeHTTP(w http.ResponseWriter, r *http.Request) error {
 | 
			
		||||
func (s Static) ServeHTTP(w http.ResponseWriter, r *http.Request, _ Handler) error {
 | 
			
		||||
	repl := r.Context().Value(caddy.ReplacerCtxKey).(caddy.Replacer)
 | 
			
		||||
 | 
			
		||||
	// close the connection after responding
 | 
			
		||||
@ -60,16 +59,13 @@ func (s Static) ServeHTTP(w http.ResponseWriter, r *http.Request) error {
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// get the status code
 | 
			
		||||
	statusCode := s.StatusCode
 | 
			
		||||
	if statusCode == 0 && s.StatusCodeStr != "" {
 | 
			
		||||
		intVal, err := strconv.Atoi(repl.ReplaceAll(s.StatusCodeStr, ""))
 | 
			
		||||
	statusCode := http.StatusOK
 | 
			
		||||
	if s.StatusCode != "" {
 | 
			
		||||
		intVal, err := strconv.Atoi(repl.ReplaceAll(s.StatusCode, ""))
 | 
			
		||||
		if err == nil {
 | 
			
		||||
			statusCode = intVal
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	if statusCode == 0 {
 | 
			
		||||
		statusCode = http.StatusOK
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// write headers
 | 
			
		||||
	w.WriteHeader(statusCode)
 | 
			
		||||
@ -83,4 +79,4 @@ func (s Static) ServeHTTP(w http.ResponseWriter, r *http.Request) error {
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// Interface guard
 | 
			
		||||
var _ Handler = (*Static)(nil)
 | 
			
		||||
var _ MiddlewareHandler = (*Static)(nil)
 | 
			
		||||
 | 
			
		||||
@ -19,6 +19,7 @@ import (
 | 
			
		||||
	"io/ioutil"
 | 
			
		||||
	"net/http"
 | 
			
		||||
	"net/http/httptest"
 | 
			
		||||
	"strconv"
 | 
			
		||||
	"testing"
 | 
			
		||||
 | 
			
		||||
	"github.com/caddyserver/caddy/v2"
 | 
			
		||||
@ -29,7 +30,7 @@ func TestStaticResponseHandler(t *testing.T) {
 | 
			
		||||
	w := httptest.NewRecorder()
 | 
			
		||||
 | 
			
		||||
	s := Static{
 | 
			
		||||
		StatusCode: http.StatusNotFound,
 | 
			
		||||
		StatusCode: strconv.Itoa(http.StatusNotFound),
 | 
			
		||||
		Headers: http.Header{
 | 
			
		||||
			"X-Test": []string{"Testing"},
 | 
			
		||||
		},
 | 
			
		||||
@ -37,7 +38,7 @@ func TestStaticResponseHandler(t *testing.T) {
 | 
			
		||||
		Close: true,
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	err := s.ServeHTTP(w, r)
 | 
			
		||||
	err := s.ServeHTTP(w, r, nil)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		t.Errorf("did not expect an error, but got: %v", err)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
@ -28,7 +28,7 @@ import (
 | 
			
		||||
 | 
			
		||||
func init() {
 | 
			
		||||
	caddy.RegisterModule(caddy.Module{
 | 
			
		||||
		Name: "http.middleware.templates",
 | 
			
		||||
		Name: "http.handlers.templates",
 | 
			
		||||
		New:  func() interface{} { return new(Templates) },
 | 
			
		||||
	})
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
		Loading…
	
	
			
			x
			
			
		
	
		Reference in New Issue
	
	Block a user