api/router/registry: extract path based parameters from url to req (#1530)

* api/router/registry: extract path based parameters from url to req
* api/handler/rpc: fix empty body request parsing
* bundle grpc-gateway util funcs

Signed-off-by: Vasiliy Tolstov <v.tolstov@unistack.org>
This commit is contained in:
Василий Толстов 2020-04-15 17:50:51 +03:00 committed by GitHub
parent 9961ebb46e
commit 62cedf64da
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 1619 additions and 224 deletions

View File

@ -208,7 +208,6 @@ func (h *rpcHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
&request,
client.WithContentType(ct),
)
// make the call
if err := c.Call(cx, req, &response, client.WithSelectOption(so)); err != nil {
writeError(w, r, err)

View File

@ -6,16 +6,25 @@ import (
"fmt"
"net/http"
"regexp"
"strings"
"sync"
"time"
"github.com/micro/go-micro/v2/api"
"github.com/micro/go-micro/v2/api/router"
"github.com/micro/go-micro/v2/api/router/util"
"github.com/micro/go-micro/v2/logger"
"github.com/micro/go-micro/v2/metadata"
"github.com/micro/go-micro/v2/registry"
"github.com/micro/go-micro/v2/registry/cache"
)
// endpoint struct, that holds compiled pcre
type endpoint struct {
hostregs []*regexp.Regexp
pathregs []util.Pattern
}
// router is the default router
type registryRouter struct {
exit chan bool
@ -26,6 +35,8 @@ type registryRouter struct {
sync.RWMutex
eps map[string]*api.Service
// compiled regexp for host and path
ceps map[string]*endpoint
}
func (r *registryRouter) isClosed() bool {
@ -67,6 +78,7 @@ func (r *registryRouter) refresh() {
}
// refresh list in 10 minutes... cruft
// use registry watching
select {
case <-time.After(time.Minute * 10):
case <-r.exit:
@ -109,11 +121,11 @@ func (r *registryRouter) store(services []*registry.Service) {
names[service.Name] = true
// map per endpoint
for _, endpoint := range service.Endpoints {
for _, sep := range service.Endpoints {
// create a key service:endpoint_name
key := fmt.Sprintf("%s:%s", service.Name, endpoint.Name)
key := fmt.Sprintf("%s.%s", service.Name, sep.Name)
// decode endpoint
end := api.Decode(endpoint.Metadata)
end := api.Decode(sep.Metadata)
// if we got nothing skip
if err := api.Validate(end); err != nil {
@ -154,8 +166,44 @@ func (r *registryRouter) store(services []*registry.Service) {
}
// now set the eps we have
for name, endpoint := range eps {
r.eps[name] = endpoint
for name, ep := range eps {
r.eps[name] = ep
cep := &endpoint{}
for _, h := range ep.Endpoint.Host {
if h == "" || h == "*" {
continue
}
hostreg, err := regexp.CompilePOSIX(h)
if err != nil {
if logger.V(logger.TraceLevel, logger.DefaultLogger) {
logger.Tracef("endpoint have invalid host regexp: %v", err)
}
continue
}
cep.hostregs = append(cep.hostregs, hostreg)
}
for _, p := range ep.Endpoint.Path {
rule, err := util.Parse(p)
if err != nil {
if logger.V(logger.TraceLevel, logger.DefaultLogger) {
logger.Tracef("endpoint have invalid path pattern: %v", err)
}
continue
}
tpl := rule.Compile()
pathreg, err := util.NewPattern(tpl.Version, tpl.OpCodes, tpl.Pool, "")
if err != nil {
if logger.V(logger.TraceLevel, logger.DefaultLogger) {
logger.Tracef("endpoint have invalid path pattern: %v", err)
}
continue
}
cep.pathregs = append(cep.pathregs, pathreg)
}
r.ceps[name] = cep
}
}
@ -239,60 +287,89 @@ func (r *registryRouter) Endpoint(req *http.Request) (*api.Service, error) {
r.RLock()
defer r.RUnlock()
var idx int
if len(req.URL.Path) > 0 && req.URL.Path != "/" {
idx = 1
}
path := strings.Split(req.URL.Path[idx:], "/")
// use the first match
// TODO: weighted matching
for _, e := range r.eps {
for n, e := range r.eps {
cep, ok := r.ceps[n]
if !ok {
continue
}
ep := e.Endpoint
// match
var pathMatch, hostMatch, methodMatch bool
// 1. try method GET, POST, PUT, etc
// 2. try host example.com, foobar.com, etc
// 3. try path /foo/bar, /bar/baz, etc
// 1. try match method
var mMatch, hMatch, pMatch bool
// 1. try method
methodLoop:
for _, m := range ep.Method {
if req.Method == m {
methodMatch = true
break
if m == req.Method {
mMatch = true
break methodLoop
}
}
// no match on method pass
if len(ep.Method) > 0 && !methodMatch {
if !mMatch {
continue
}
// 2. try match host
for _, h := range ep.Host {
if req.Host == h {
hostMatch = true
break
}
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debugf("api method match %s", req.Method)
}
// no match on host pass
if len(ep.Host) > 0 && !hostMatch {
// 2. try host
if len(ep.Host) == 0 {
hMatch = true
} else {
hostLoop:
for idx, h := range ep.Host {
if h == "" || h == "*" {
hMatch = true
break hostLoop
} else {
if cep.hostregs[idx].MatchString(req.URL.Host) {
hMatch = true
break hostLoop
}
}
}
}
if !hMatch {
continue
}
// 3. try match paths
for _, p := range ep.Path {
re, err := regexp.CompilePOSIX(p)
if err == nil && re.MatchString(req.URL.Path) {
pathMatch = true
break
}
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debugf("api host match %s", req.URL.Host)
}
// no match pass
if len(ep.Path) > 0 && !pathMatch {
// 3. try path
// 3. try path
pathLoop:
for _, pathreg := range cep.pathregs {
matches, err := pathreg.Match(path, "")
if err != nil {
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debugf("api path not match %s != %v", path, pathreg)
}
continue
}
pMatch = true
ctx := req.Context()
md, ok := metadata.FromContext(ctx)
if !ok {
md = make(metadata.Metadata)
}
for k, v := range matches {
md[fmt.Sprintf("x-api-field-%s", k)] = v
}
md["x-api-body"] = ep.Body
*req = *req.Clone(metadata.NewContext(ctx, md))
break pathLoop
}
if !pMatch {
continue
}
// TODO: Percentage traffic
// we got here, so its a match
return e, nil
}
@ -377,6 +454,7 @@ func newRouter(opts ...router.Option) *registryRouter {
opts: options,
rc: cache.New(options.Registry),
eps: make(map[string]*api.Service),
ceps: make(map[string]*endpoint),
}
go r.watch()
go r.refresh()

View File

@ -1,136 +0,0 @@
package registry
import (
"fmt"
"net/http"
"net/url"
"testing"
"github.com/micro/go-micro/v2/api"
)
func TestRouter(t *testing.T) {
r := newRouter()
compare := func(expect, got []string) bool {
// no data to compare, return true
if len(expect) == 0 && len(got) == 0 {
return true
}
// no data expected but got some return false
if len(expect) == 0 && len(got) > 0 {
return false
}
// compare expected with what we got
for _, e := range expect {
var seen bool
for _, g := range got {
if e == g {
seen = true
break
}
}
if !seen {
return false
}
}
// we're done, return true
return true
}
testData := []struct {
e *api.Endpoint
r *http.Request
m bool
}{
{
e: &api.Endpoint{
Name: "Foo.Bar",
Host: []string{"example.com"},
Method: []string{"GET"},
Path: []string{"/foo"},
},
r: &http.Request{
Host: "example.com",
Method: "GET",
URL: &url.URL{
Path: "/foo",
},
},
m: true,
},
{
e: &api.Endpoint{
Name: "Bar.Baz",
Host: []string{"example.com", "foo.com"},
Method: []string{"GET", "POST"},
Path: []string{"/foo/bar"},
},
r: &http.Request{
Host: "foo.com",
Method: "POST",
URL: &url.URL{
Path: "/foo/bar",
},
},
m: true,
},
{
e: &api.Endpoint{
Name: "Test.Cruft",
Host: []string{"example.com", "foo.com"},
Method: []string{"GET", "POST"},
Path: []string{"/xyz"},
},
r: &http.Request{
Host: "fail.com",
Method: "DELETE",
URL: &url.URL{
Path: "/test/fail",
},
},
m: false,
},
}
for _, d := range testData {
key := fmt.Sprintf("%s:%s", "test.service", d.e.Name)
r.eps[key] = &api.Service{
Endpoint: d.e,
}
}
for _, d := range testData {
e, err := r.Endpoint(d.r)
if d.m && err != nil {
t.Fatalf("expected match, got %v", err)
}
if !d.m && err == nil {
t.Fatal("expected error got match")
}
// skip testing the non match
if !d.m {
continue
}
ep := e.Endpoint
// test the match
if d.e.Name != ep.Name {
t.Fatalf("expected %v got %v", d.e.Name, ep.Name)
}
if ok := compare(d.e.Method, ep.Method); !ok {
t.Fatalf("expected %v got %v", d.e.Method, ep.Method)
}
if ok := compare(d.e.Path, ep.Path); !ok {
t.Fatalf("expected %v got %v", d.e.Path, ep.Path)
}
if ok := compare(d.e.Host, ep.Host); !ok {
t.Fatalf("expected %v got %v", d.e.Host, ep.Host)
}
}
}

View File

@ -1,4 +1,4 @@
package api_test
package router_test
import (
"context"
@ -13,6 +13,7 @@ import (
"github.com/micro/go-micro/v2/api/handler"
"github.com/micro/go-micro/v2/api/handler/rpc"
"github.com/micro/go-micro/v2/api/router"
rregistry "github.com/micro/go-micro/v2/api/router/registry"
rstatic "github.com/micro/go-micro/v2/api/router/static"
"github.com/micro/go-micro/v2/client"
gcli "github.com/micro/go-micro/v2/client/grpc"
@ -29,11 +30,11 @@ type testServer struct {
// TestHello implements helloworld.GreeterServer
func (s *testServer) Call(ctx context.Context, req *pb.Request, rsp *pb.Response) error {
rsp.Msg = "Hello " + req.Name
rsp.Msg = "Hello " + req.Uuid
return nil
}
func TestApiAndGRPC(t *testing.T) {
func initial(t *testing.T) (server.Server, client.Client) {
r := rmemory.NewRegistry()
// create a new client
@ -53,44 +54,17 @@ func TestApiAndGRPC(t *testing.T) {
if err := s.Start(); err != nil {
t.Fatalf("failed to start: %v", err)
}
defer s.Stop()
// create a new router
router := rstatic.NewRouter(
router.WithHandler(rpc.Handler),
router.WithRegistry(r),
)
return s, c
}
err := router.Register(&api.Endpoint{
Name: "foo.Test.Call",
Method: []string{"GET"},
Path: []string{"/api/v0/test/call/{name}"},
Handler: "rpc",
})
func check(addr string, t *testing.T) {
req, err := http.NewRequest("POST", fmt.Sprintf("http://%s/api/v0/test/call/TEST", addr), nil)
if err != nil {
t.Fatal(err)
t.Fatalf("Failed to created http.Request: %v", err)
}
hrpc := rpc.NewHandler(
handler.WithClient(c),
handler.WithRouter(router),
)
hsrv := &http.Server{
Handler: hrpc,
Addr: "127.0.0.1:6543",
WriteTimeout: 15 * time.Second,
ReadTimeout: 15 * time.Second,
IdleTimeout: 20 * time.Second,
MaxHeaderBytes: 1024 * 1024 * 1, // 1Mb
}
go func() {
log.Println(hsrv.ListenAndServe())
}()
time.Sleep(1 * time.Second)
rsp, err := http.Get(fmt.Sprintf("http://%s/api/v0/test/call/TEST", hsrv.Addr))
req.Header.Set("Content-Type", "application/json")
rsp, err := (&http.Client{}).Do(req)
if err != nil {
t.Fatalf("Failed to created http.Request: %v", err)
}
@ -106,3 +80,74 @@ func TestApiAndGRPC(t *testing.T) {
t.Fatalf("invalid message received, parsing error %s != %s", buf, jsonMsg)
}
}
func TestRouterRegistry(t *testing.T) {
s, c := initial(t)
defer s.Stop()
router := rregistry.NewRouter(
router.WithHandler(rpc.Handler),
router.WithRegistry(s.Options().Registry),
)
hrpc := rpc.NewHandler(
handler.WithClient(c),
handler.WithRouter(router),
)
hsrv := &http.Server{
Handler: hrpc,
Addr: "127.0.0.1:6543",
WriteTimeout: 15 * time.Second,
ReadTimeout: 15 * time.Second,
IdleTimeout: 20 * time.Second,
MaxHeaderBytes: 1024 * 1024 * 1, // 1Mb
}
go func() {
log.Println(hsrv.ListenAndServe())
}()
defer hsrv.Close()
time.Sleep(1 * time.Second)
check(hsrv.Addr, t)
}
func TestRouterStatic(t *testing.T) {
s, c := initial(t)
defer s.Stop()
router := rstatic.NewRouter(
router.WithHandler(rpc.Handler),
router.WithRegistry(s.Options().Registry),
)
err := router.Register(&api.Endpoint{
Name: "foo.Test.Call",
Method: []string{"POST"},
Path: []string{"/api/v0/test/call/{uuid}"},
Handler: "rpc",
})
if err != nil {
t.Fatal(err)
}
hrpc := rpc.NewHandler(
handler.WithClient(c),
handler.WithRouter(router),
)
hsrv := &http.Server{
Handler: hrpc,
Addr: "127.0.0.1:6543",
WriteTimeout: 15 * time.Second,
ReadTimeout: 15 * time.Second,
IdleTimeout: 20 * time.Second,
MaxHeaderBytes: 1024 * 1024 * 1, // 1Mb
}
go func() {
log.Println(hsrv.ListenAndServe())
}()
defer hsrv.Close()
time.Sleep(1 * time.Second)
check(hsrv.Addr, t)
}

View File

@ -8,20 +8,19 @@ import (
"strings"
"sync"
"github.com/grpc-ecosystem/grpc-gateway/protoc-gen-grpc-gateway/httprule"
"github.com/grpc-ecosystem/grpc-gateway/runtime"
"github.com/micro/go-micro/v2/api"
"github.com/micro/go-micro/v2/api/router"
"github.com/micro/go-micro/v2/api/router/util"
"github.com/micro/go-micro/v2/logger"
"github.com/micro/go-micro/v2/metadata"
"github.com/micro/go-micro/v2/registry"
util "github.com/micro/go-micro/v2/util/registry"
rutil "github.com/micro/go-micro/v2/util/registry"
)
type endpoint struct {
apiep *api.Endpoint
hostregs []*regexp.Regexp
pathregs []runtime.Pattern
pathregs []util.Pattern
}
// router is the default router
@ -93,7 +92,7 @@ func (r *staticRouter) Register(ep *api.Endpoint) error {
return err
}
var pathregs []runtime.Pattern
var pathregs []util.Pattern
var hostregs []*regexp.Regexp
for _, h := range ep.Host {
@ -108,12 +107,12 @@ func (r *staticRouter) Register(ep *api.Endpoint) error {
}
for _, p := range ep.Path {
rule, err := httprule.Parse(p)
rule, err := util.Parse(p)
if err != nil {
return err
}
tpl := rule.Compile()
pathreg, err := runtime.NewPattern(tpl.Version, tpl.OpCodes, tpl.Pool, "")
pathreg, err := util.NewPattern(tpl.Version, tpl.OpCodes, tpl.Pool, "")
if err != nil {
return err
}
@ -164,7 +163,7 @@ func (r *staticRouter) Endpoint(req *http.Request) (*api.Service, error) {
// hack for stream endpoint
if ep.apiep.Stream {
svcs := util.Copy(services)
svcs := rutil.Copy(services)
for _, svc := range svcs {
if len(svc.Endpoints) == 0 {
e := &registry.Endpoint{}

View File

@ -0,0 +1,27 @@
Copyright (c) 2015, Gengo, Inc.
All rights reserved.
Redistribution and use in source and binary forms, with or without modification,
are permitted provided that the following conditions are met:
* Redistributions of source code must retain the above copyright notice,
this list of conditions and the following disclaimer.
* Redistributions in binary form must reproduce the above copyright notice,
this list of conditions and the following disclaimer in the documentation
and/or other materials provided with the distribution.
* Neither the name of Gengo, Inc. nor the names of its
contributors may be used to endorse or promote products derived from this
software without specific prior written permission.
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

115
api/router/util/compile.go Normal file
View File

@ -0,0 +1,115 @@
package util
// download from https://raw.githubusercontent.com/grpc-ecosystem/grpc-gateway/master/protoc-gen-grpc-gateway/httprule/compile.go
const (
opcodeVersion = 1
)
// Template is a compiled representation of path templates.
type Template struct {
// Version is the version number of the format.
Version int
// OpCodes is a sequence of operations.
OpCodes []int
// Pool is a constant pool
Pool []string
// Verb is a VERB part in the template.
Verb string
// Fields is a list of field paths bound in this template.
Fields []string
// Original template (example: /v1/a_bit_of_everything)
Template string
}
// Compiler compiles utilities representation of path templates into marshallable operations.
// They can be unmarshalled by runtime.NewPattern.
type Compiler interface {
Compile() Template
}
type op struct {
// code is the opcode of the operation
code OpCode
// str is a string operand of the code.
// operand is ignored if str is not empty.
str string
// operand is a numeric operand of the code.
operand int
}
func (w wildcard) compile() []op {
return []op{
{code: OpPush},
}
}
func (w deepWildcard) compile() []op {
return []op{
{code: OpPushM},
}
}
func (l literal) compile() []op {
return []op{
{
code: OpLitPush,
str: string(l),
},
}
}
func (v variable) compile() []op {
var ops []op
for _, s := range v.segments {
ops = append(ops, s.compile()...)
}
ops = append(ops, op{
code: OpConcatN,
operand: len(v.segments),
}, op{
code: OpCapture,
str: v.path,
})
return ops
}
func (t template) Compile() Template {
var rawOps []op
for _, s := range t.segments {
rawOps = append(rawOps, s.compile()...)
}
var (
ops []int
pool []string
fields []string
)
consts := make(map[string]int)
for _, op := range rawOps {
ops = append(ops, int(op.code))
if op.str == "" {
ops = append(ops, op.operand)
} else {
if _, ok := consts[op.str]; !ok {
consts[op.str] = len(pool)
pool = append(pool, op.str)
}
ops = append(ops, consts[op.str])
}
if op.code == OpCapture {
fields = append(fields, op.str)
}
}
return Template{
Version: opcodeVersion,
OpCodes: ops,
Pool: pool,
Verb: t.verb,
Fields: fields,
Template: t.template,
}
}

View File

@ -0,0 +1,122 @@
package util
// download from https://raw.githubusercontent.com/grpc-ecosystem/grpc-gateway/master/protoc-gen-grpc-gateway/httprule/compile_test.go
import (
"reflect"
"testing"
)
const (
operandFiller = 0
)
func TestCompile(t *testing.T) {
for _, spec := range []struct {
segs []segment
verb string
ops []int
pool []string
fields []string
}{
{},
{
segs: []segment{
wildcard{},
},
ops: []int{int(OpPush), operandFiller},
},
{
segs: []segment{
deepWildcard{},
},
ops: []int{int(OpPushM), operandFiller},
},
{
segs: []segment{
literal("v1"),
},
ops: []int{int(OpLitPush), 0},
pool: []string{"v1"},
},
{
segs: []segment{
literal("v1"),
},
verb: "LOCK",
ops: []int{int(OpLitPush), 0},
pool: []string{"v1"},
},
{
segs: []segment{
variable{
path: "name.nested",
segments: []segment{
wildcard{},
},
},
},
ops: []int{
int(OpPush), operandFiller,
int(OpConcatN), 1,
int(OpCapture), 0,
},
pool: []string{"name.nested"},
fields: []string{"name.nested"},
},
{
segs: []segment{
literal("obj"),
variable{
path: "name.nested",
segments: []segment{
literal("a"),
wildcard{},
literal("b"),
},
},
variable{
path: "obj",
segments: []segment{
deepWildcard{},
},
},
},
ops: []int{
int(OpLitPush), 0,
int(OpLitPush), 1,
int(OpPush), operandFiller,
int(OpLitPush), 2,
int(OpConcatN), 3,
int(OpCapture), 3,
int(OpPushM), operandFiller,
int(OpConcatN), 1,
int(OpCapture), 0,
},
pool: []string{"obj", "a", "b", "name.nested"},
fields: []string{"name.nested", "obj"},
},
} {
tmpl := template{
segments: spec.segs,
verb: spec.verb,
}
compiled := tmpl.Compile()
if got, want := compiled.Version, opcodeVersion; got != want {
t.Errorf("tmpl.Compile().Version = %d; want %d; segs=%#v, verb=%q", got, want, spec.segs, spec.verb)
}
if got, want := compiled.OpCodes, spec.ops; !reflect.DeepEqual(got, want) {
t.Errorf("tmpl.Compile().OpCodes = %v; want %v; segs=%#v, verb=%q", got, want, spec.segs, spec.verb)
}
if got, want := compiled.Pool, spec.pool; !reflect.DeepEqual(got, want) {
t.Errorf("tmpl.Compile().Pool = %q; want %q; segs=%#v, verb=%q", got, want, spec.segs, spec.verb)
}
if got, want := compiled.Verb, spec.verb; got != want {
t.Errorf("tmpl.Compile().Verb = %q; want %q; segs=%#v, verb=%q", got, want, spec.segs, spec.verb)
}
if got, want := compiled.Fields, spec.fields; !reflect.DeepEqual(got, want) {
t.Errorf("tmpl.Compile().Fields = %q; want %q; segs=%#v, verb=%q", got, want, spec.segs, spec.verb)
}
}
}

363
api/router/util/parse.go Normal file
View File

@ -0,0 +1,363 @@
package util
// download from https://raw.githubusercontent.com/grpc-ecosystem/grpc-gateway/master/protoc-gen-grpc-gateway/httprule/parse.go
import (
"fmt"
"strings"
"github.com/micro/go-micro/v2/logger"
)
// InvalidTemplateError indicates that the path template is not valid.
type InvalidTemplateError struct {
tmpl string
msg string
}
func (e InvalidTemplateError) Error() string {
return fmt.Sprintf("%s: %s", e.msg, e.tmpl)
}
// Parse parses the string representation of path template
func Parse(tmpl string) (Compiler, error) {
if !strings.HasPrefix(tmpl, "/") {
return template{}, InvalidTemplateError{tmpl: tmpl, msg: "no leading /"}
}
tokens, verb := tokenize(tmpl[1:])
p := parser{tokens: tokens}
segs, err := p.topLevelSegments()
if err != nil {
return template{}, InvalidTemplateError{tmpl: tmpl, msg: err.Error()}
}
return template{
segments: segs,
verb: verb,
template: tmpl,
}, nil
}
func tokenize(path string) (tokens []string, verb string) {
if path == "" {
return []string{eof}, ""
}
const (
init = iota
field
nested
)
var (
st = init
)
for path != "" {
var idx int
switch st {
case init:
idx = strings.IndexAny(path, "/{")
case field:
idx = strings.IndexAny(path, ".=}")
case nested:
idx = strings.IndexAny(path, "/}")
}
if idx < 0 {
tokens = append(tokens, path)
break
}
switch r := path[idx]; r {
case '/', '.':
case '{':
st = field
case '=':
st = nested
case '}':
st = init
}
if idx == 0 {
tokens = append(tokens, path[idx:idx+1])
} else {
tokens = append(tokens, path[:idx], path[idx:idx+1])
}
path = path[idx+1:]
}
l := len(tokens)
t := tokens[l-1]
if idx := strings.LastIndex(t, ":"); idx == 0 {
tokens, verb = tokens[:l-1], t[1:]
} else if idx > 0 {
tokens[l-1], verb = t[:idx], t[idx+1:]
}
tokens = append(tokens, eof)
return tokens, verb
}
// parser is a parser of the template syntax defined in github.com/googleapis/googleapis/google/api/http.proto.
type parser struct {
tokens []string
accepted []string
}
// topLevelSegments is the target of this parser.
func (p *parser) topLevelSegments() ([]segment, error) {
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debugf("Parsing %q", p.tokens)
}
segs, err := p.segments()
if err != nil {
return nil, err
}
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debugf("accept segments: %q; %q", p.accepted, p.tokens)
}
if _, err := p.accept(typeEOF); err != nil {
return nil, fmt.Errorf("unexpected token %q after segments %q", p.tokens[0], strings.Join(p.accepted, ""))
}
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debugf("accept eof: %q; %q", p.accepted, p.tokens)
}
return segs, nil
}
func (p *parser) segments() ([]segment, error) {
s, err := p.segment()
if err != nil {
return nil, err
}
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debugf("accept segment: %q; %q", p.accepted, p.tokens)
}
segs := []segment{s}
for {
if _, err := p.accept("/"); err != nil {
return segs, nil
}
s, err := p.segment()
if err != nil {
return segs, err
}
segs = append(segs, s)
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debugf("accept segment: %q; %q", p.accepted, p.tokens)
}
}
}
func (p *parser) segment() (segment, error) {
if _, err := p.accept("*"); err == nil {
return wildcard{}, nil
}
if _, err := p.accept("**"); err == nil {
return deepWildcard{}, nil
}
if l, err := p.literal(); err == nil {
return l, nil
}
v, err := p.variable()
if err != nil {
return nil, fmt.Errorf("segment neither wildcards, literal or variable: %v", err)
}
return v, err
}
func (p *parser) literal() (segment, error) {
lit, err := p.accept(typeLiteral)
if err != nil {
return nil, err
}
return literal(lit), nil
}
func (p *parser) variable() (segment, error) {
if _, err := p.accept("{"); err != nil {
return nil, err
}
path, err := p.fieldPath()
if err != nil {
return nil, err
}
var segs []segment
if _, err := p.accept("="); err == nil {
segs, err = p.segments()
if err != nil {
return nil, fmt.Errorf("invalid segment in variable %q: %v", path, err)
}
} else {
segs = []segment{wildcard{}}
}
if _, err := p.accept("}"); err != nil {
return nil, fmt.Errorf("unterminated variable segment: %s", path)
}
return variable{
path: path,
segments: segs,
}, nil
}
func (p *parser) fieldPath() (string, error) {
c, err := p.accept(typeIdent)
if err != nil {
return "", err
}
components := []string{c}
for {
if _, err = p.accept("."); err != nil {
return strings.Join(components, "."), nil
}
c, err := p.accept(typeIdent)
if err != nil {
return "", fmt.Errorf("invalid field path component: %v", err)
}
components = append(components, c)
}
}
// A termType is a type of terminal symbols.
type termType string
// These constants define some of valid values of termType.
// They improve readability of parse functions.
//
// You can also use "/", "*", "**", "." or "=" as valid values.
const (
typeIdent = termType("ident")
typeLiteral = termType("literal")
typeEOF = termType("$")
)
const (
// eof is the terminal symbol which always appears at the end of token sequence.
eof = "\u0000"
)
// accept tries to accept a token in "p".
// This function consumes a token and returns it if it matches to the specified "term".
// If it doesn't match, the function does not consume any tokens and return an error.
func (p *parser) accept(term termType) (string, error) {
t := p.tokens[0]
switch term {
case "/", "*", "**", ".", "=", "{", "}":
if t != string(term) && t != "/" {
return "", fmt.Errorf("expected %q but got %q", term, t)
}
case typeEOF:
if t != eof {
return "", fmt.Errorf("expected EOF but got %q", t)
}
case typeIdent:
if err := expectIdent(t); err != nil {
return "", err
}
case typeLiteral:
if err := expectPChars(t); err != nil {
return "", err
}
default:
return "", fmt.Errorf("unknown termType %q", term)
}
p.tokens = p.tokens[1:]
p.accepted = append(p.accepted, t)
return t, nil
}
// expectPChars determines if "t" consists of only pchars defined in RFC3986.
//
// https://www.ietf.org/rfc/rfc3986.txt, P.49
// pchar = unreserved / pct-encoded / sub-delims / ":" / "@"
// unreserved = ALPHA / DIGIT / "-" / "." / "_" / "~"
// sub-delims = "!" / "$" / "&" / "'" / "(" / ")"
// / "*" / "+" / "," / ";" / "="
// pct-encoded = "%" HEXDIG HEXDIG
func expectPChars(t string) error {
const (
init = iota
pct1
pct2
)
st := init
for _, r := range t {
if st != init {
if !isHexDigit(r) {
return fmt.Errorf("invalid hexdigit: %c(%U)", r, r)
}
switch st {
case pct1:
st = pct2
case pct2:
st = init
}
continue
}
// unreserved
switch {
case 'A' <= r && r <= 'Z':
continue
case 'a' <= r && r <= 'z':
continue
case '0' <= r && r <= '9':
continue
}
switch r {
case '-', '.', '_', '~':
// unreserved
case '!', '$', '&', '\'', '(', ')', '*', '+', ',', ';', '=':
// sub-delims
case ':', '@':
// rest of pchar
case '%':
// pct-encoded
st = pct1
default:
return fmt.Errorf("invalid character in path segment: %q(%U)", r, r)
}
}
if st != init {
return fmt.Errorf("invalid percent-encoding in %q", t)
}
return nil
}
// expectIdent determines if "ident" is a valid identifier in .proto schema ([[:alpha:]_][[:alphanum:]_]*).
func expectIdent(ident string) error {
if ident == "" {
return fmt.Errorf("empty identifier")
}
for pos, r := range ident {
switch {
case '0' <= r && r <= '9':
if pos == 0 {
return fmt.Errorf("identifier starting with digit: %s", ident)
}
continue
case 'A' <= r && r <= 'Z':
continue
case 'a' <= r && r <= 'z':
continue
case r == '_':
continue
default:
return fmt.Errorf("invalid character %q(%U) in identifier: %s", r, r, ident)
}
}
return nil
}
func isHexDigit(r rune) bool {
switch {
case '0' <= r && r <= '9':
return true
case 'A' <= r && r <= 'F':
return true
case 'a' <= r && r <= 'f':
return true
}
return false
}

View File

@ -0,0 +1,321 @@
package util
// download from https://raw.githubusercontent.com/grpc-ecosystem/grpc-gateway/master/protoc-gen-grpc-gateway/httprule/parse_test.go
import (
"flag"
"fmt"
"reflect"
"testing"
"github.com/micro/go-micro/v2/logger"
)
func TestTokenize(t *testing.T) {
for _, spec := range []struct {
src string
tokens []string
}{
{
src: "",
tokens: []string{eof},
},
{
src: "v1",
tokens: []string{"v1", eof},
},
{
src: "v1/b",
tokens: []string{"v1", "/", "b", eof},
},
{
src: "v1/endpoint/*",
tokens: []string{"v1", "/", "endpoint", "/", "*", eof},
},
{
src: "v1/endpoint/**",
tokens: []string{"v1", "/", "endpoint", "/", "**", eof},
},
{
src: "v1/b/{bucket_name=*}",
tokens: []string{
"v1", "/",
"b", "/",
"{", "bucket_name", "=", "*", "}",
eof,
},
},
{
src: "v1/b/{bucket_name=buckets/*}",
tokens: []string{
"v1", "/",
"b", "/",
"{", "bucket_name", "=", "buckets", "/", "*", "}",
eof,
},
},
{
src: "v1/b/{bucket_name=buckets/*}/o",
tokens: []string{
"v1", "/",
"b", "/",
"{", "bucket_name", "=", "buckets", "/", "*", "}", "/",
"o",
eof,
},
},
{
src: "v1/b/{bucket_name=buckets/*}/o/{name}",
tokens: []string{
"v1", "/",
"b", "/",
"{", "bucket_name", "=", "buckets", "/", "*", "}", "/",
"o", "/", "{", "name", "}",
eof,
},
},
{
src: "v1/a=b&c=d;e=f:g/endpoint.rdf",
tokens: []string{
"v1", "/",
"a=b&c=d;e=f:g", "/",
"endpoint.rdf",
eof,
},
},
} {
tokens, verb := tokenize(spec.src)
if got, want := tokens, spec.tokens; !reflect.DeepEqual(got, want) {
t.Errorf("tokenize(%q) = %q, _; want %q, _", spec.src, got, want)
}
if got, want := verb, ""; got != want {
t.Errorf("tokenize(%q) = _, %q; want _, %q", spec.src, got, want)
}
src := fmt.Sprintf("%s:%s", spec.src, "LOCK")
tokens, verb = tokenize(src)
if got, want := tokens, spec.tokens; !reflect.DeepEqual(got, want) {
t.Errorf("tokenize(%q) = %q, _; want %q, _", src, got, want)
}
if got, want := verb, "LOCK"; got != want {
t.Errorf("tokenize(%q) = _, %q; want _, %q", src, got, want)
}
}
}
func TestParseSegments(t *testing.T) {
flag.Set("v", "3")
for _, spec := range []struct {
tokens []string
want []segment
}{
{
tokens: []string{"v1", eof},
want: []segment{
literal("v1"),
},
},
{
tokens: []string{"/", eof},
want: []segment{
wildcard{},
},
},
{
tokens: []string{"-._~!$&'()*+,;=:@", eof},
want: []segment{
literal("-._~!$&'()*+,;=:@"),
},
},
{
tokens: []string{"%e7%ac%ac%e4%b8%80%e7%89%88", eof},
want: []segment{
literal("%e7%ac%ac%e4%b8%80%e7%89%88"),
},
},
{
tokens: []string{"v1", "/", "*", eof},
want: []segment{
literal("v1"),
wildcard{},
},
},
{
tokens: []string{"v1", "/", "**", eof},
want: []segment{
literal("v1"),
deepWildcard{},
},
},
{
tokens: []string{"{", "name", "}", eof},
want: []segment{
variable{
path: "name",
segments: []segment{
wildcard{},
},
},
},
},
{
tokens: []string{"{", "name", "=", "*", "}", eof},
want: []segment{
variable{
path: "name",
segments: []segment{
wildcard{},
},
},
},
},
{
tokens: []string{"{", "field", ".", "nested", ".", "nested2", "=", "*", "}", eof},
want: []segment{
variable{
path: "field.nested.nested2",
segments: []segment{
wildcard{},
},
},
},
},
{
tokens: []string{"{", "name", "=", "a", "/", "b", "/", "*", "}", eof},
want: []segment{
variable{
path: "name",
segments: []segment{
literal("a"),
literal("b"),
wildcard{},
},
},
},
},
{
tokens: []string{
"v1", "/",
"{",
"name", ".", "nested", ".", "nested2",
"=",
"a", "/", "b", "/", "*",
"}", "/",
"o", "/",
"{",
"another_name",
"=",
"a", "/", "b", "/", "*", "/", "c",
"}", "/",
"**",
eof},
want: []segment{
literal("v1"),
variable{
path: "name.nested.nested2",
segments: []segment{
literal("a"),
literal("b"),
wildcard{},
},
},
literal("o"),
variable{
path: "another_name",
segments: []segment{
literal("a"),
literal("b"),
wildcard{},
literal("c"),
},
},
deepWildcard{},
},
},
} {
p := parser{tokens: spec.tokens}
segs, err := p.topLevelSegments()
if err != nil {
t.Errorf("parser{%q}.segments() failed with %v; want success", spec.tokens, err)
continue
}
if got, want := segs, spec.want; !reflect.DeepEqual(got, want) {
t.Errorf("parser{%q}.segments() = %#v; want %#v", spec.tokens, got, want)
}
if got := p.tokens; len(got) > 0 {
t.Errorf("p.tokens = %q; want []; spec.tokens=%q", got, spec.tokens)
}
}
}
func TestParseSegmentsWithErrors(t *testing.T) {
flag.Set("v", "3")
for _, spec := range []struct {
tokens []string
}{
{
// double slash
tokens: []string{"//", eof},
},
{
// invalid literal
tokens: []string{"a?b", eof},
},
{
// invalid percent-encoding
tokens: []string{"%", eof},
},
{
// invalid percent-encoding
tokens: []string{"%2", eof},
},
{
// invalid percent-encoding
tokens: []string{"a%2z", eof},
},
{
// empty segments
tokens: []string{eof},
},
{
// unterminated variable
tokens: []string{"{", "name", eof},
},
{
// unterminated variable
tokens: []string{"{", "name", "=", eof},
},
{
// unterminated variable
tokens: []string{"{", "name", "=", "*", eof},
},
{
// empty component in field path
tokens: []string{"{", "name", ".", "}", eof},
},
{
// empty component in field path
tokens: []string{"{", "name", ".", ".", "nested", "}", eof},
},
{
// invalid character in identifier
tokens: []string{"{", "field-name", "}", eof},
},
{
// no slash between segments
tokens: []string{"v1", "endpoint", eof},
},
{
// no slash between segments
tokens: []string{"v1", "{", "name", "}", eof},
},
} {
p := parser{tokens: spec.tokens}
segs, err := p.topLevelSegments()
if err == nil {
t.Errorf("parser{%q}.segments() succeeded; want InvalidTemplateError; accepted %#v", spec.tokens, segs)
continue
}
logger.Info(err)
}
}

View File

@ -0,0 +1,24 @@
package util
// download from https://raw.githubusercontent.com/grpc-ecosystem/grpc-gateway/master/utilities/pattern.go
// An OpCode is a opcode of compiled path patterns.
type OpCode int
// These constants are the valid values of OpCode.
const (
// OpNop does nothing
OpNop = OpCode(iota)
// OpPush pushes a component to stack
OpPush
// OpLitPush pushes a component to stack if it matches to the literal
OpLitPush
// OpPushM concatenates the remaining components and pushes it to stack
OpPushM
// OpConcatN pops N items from stack, concatenates them and pushes it back to stack
OpConcatN
// OpCapture pops an item and binds it to the variable
OpCapture
// OpEnd is the least positive invalid opcode.
OpEnd
)

283
api/router/util/runtime.go Normal file
View File

@ -0,0 +1,283 @@
package util
// download from https://raw.githubusercontent.com/grpc-ecosystem/grpc-gateway/master/runtime/pattern.go
import (
"errors"
"fmt"
"strings"
"github.com/micro/go-micro/v2/logger"
)
var (
// ErrNotMatch indicates that the given HTTP request path does not match to the pattern.
ErrNotMatch = errors.New("not match to the path pattern")
// ErrInvalidPattern indicates that the given definition of Pattern is not valid.
ErrInvalidPattern = errors.New("invalid pattern")
)
type rop struct {
code OpCode
operand int
}
// Pattern is a template pattern of http request paths defined in github.com/googleapis/googleapis/google/api/http.proto.
type Pattern struct {
// ops is a list of operations
ops []rop
// pool is a constant pool indexed by the operands or vars.
pool []string
// vars is a list of variables names to be bound by this pattern
vars []string
// stacksize is the max depth of the stack
stacksize int
// tailLen is the length of the fixed-size segments after a deep wildcard
tailLen int
// verb is the VERB part of the path pattern. It is empty if the pattern does not have VERB part.
verb string
// assumeColonVerb indicates whether a path suffix after a final
// colon may only be interpreted as a verb.
assumeColonVerb bool
}
type patternOptions struct {
assumeColonVerb bool
}
// PatternOpt is an option for creating Patterns.
type PatternOpt func(*patternOptions)
// NewPattern returns a new Pattern from the given definition values.
// "ops" is a sequence of op codes. "pool" is a constant pool.
// "verb" is the verb part of the pattern. It is empty if the pattern does not have the part.
// "version" must be 1 for now.
// It returns an error if the given definition is invalid.
func NewPattern(version int, ops []int, pool []string, verb string, opts ...PatternOpt) (Pattern, error) {
options := patternOptions{
assumeColonVerb: true,
}
for _, o := range opts {
o(&options)
}
if version != 1 {
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debugf("unsupported version: %d", version)
}
return Pattern{}, ErrInvalidPattern
}
l := len(ops)
if l%2 != 0 {
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debugf("odd number of ops codes: %d", l)
}
return Pattern{}, ErrInvalidPattern
}
var (
typedOps []rop
stack, maxstack int
tailLen int
pushMSeen bool
vars []string
)
for i := 0; i < l; i += 2 {
op := rop{code: OpCode(ops[i]), operand: ops[i+1]}
switch op.code {
case OpNop:
continue
case OpPush:
if pushMSeen {
tailLen++
}
stack++
case OpPushM:
if pushMSeen {
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debug("pushM appears twice")
}
return Pattern{}, ErrInvalidPattern
}
pushMSeen = true
stack++
case OpLitPush:
if op.operand < 0 || len(pool) <= op.operand {
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debugf("negative literal index: %d", op.operand)
}
return Pattern{}, ErrInvalidPattern
}
if pushMSeen {
tailLen++
}
stack++
case OpConcatN:
if op.operand <= 0 {
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debugf("negative concat size: %d", op.operand)
}
return Pattern{}, ErrInvalidPattern
}
stack -= op.operand
if stack < 0 {
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debug("stack underflow")
}
return Pattern{}, ErrInvalidPattern
}
stack++
case OpCapture:
if op.operand < 0 || len(pool) <= op.operand {
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debugf("variable name index out of bound: %d", op.operand)
}
return Pattern{}, ErrInvalidPattern
}
v := pool[op.operand]
op.operand = len(vars)
vars = append(vars, v)
stack--
if stack < 0 {
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debug("stack underflow")
}
return Pattern{}, ErrInvalidPattern
}
default:
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Debugf("invalid opcode: %d", op.code)
}
return Pattern{}, ErrInvalidPattern
}
if maxstack < stack {
maxstack = stack
}
typedOps = append(typedOps, op)
}
return Pattern{
ops: typedOps,
pool: pool,
vars: vars,
stacksize: maxstack,
tailLen: tailLen,
verb: verb,
assumeColonVerb: options.assumeColonVerb,
}, nil
}
// MustPattern is a helper function which makes it easier to call NewPattern in variable initialization.
func MustPattern(p Pattern, err error) Pattern {
if err != nil {
if logger.V(logger.DebugLevel, logger.DefaultLogger) {
logger.Fatalf("Pattern initialization failed: %v", err)
}
}
return p
}
// Match examines components if it matches to the Pattern.
// If it matches, the function returns a mapping from field paths to their captured values.
// If otherwise, the function returns an error.
func (p Pattern) Match(components []string, verb string) (map[string]string, error) {
if p.verb != verb {
if p.assumeColonVerb || p.verb != "" {
return nil, ErrNotMatch
}
if len(components) == 0 {
components = []string{":" + verb}
} else {
components = append([]string{}, components...)
components[len(components)-1] += ":" + verb
}
verb = ""
}
var pos int
stack := make([]string, 0, p.stacksize)
captured := make([]string, len(p.vars))
l := len(components)
for _, op := range p.ops {
switch op.code {
case OpNop:
continue
case OpPush, OpLitPush:
if pos >= l {
return nil, ErrNotMatch
}
c := components[pos]
if op.code == OpLitPush {
if lit := p.pool[op.operand]; c != lit {
return nil, ErrNotMatch
}
}
stack = append(stack, c)
pos++
case OpPushM:
end := len(components)
if end < pos+p.tailLen {
return nil, ErrNotMatch
}
end -= p.tailLen
stack = append(stack, strings.Join(components[pos:end], "/"))
pos = end
case OpConcatN:
n := op.operand
l := len(stack) - n
stack = append(stack[:l], strings.Join(stack[l:], "/"))
case OpCapture:
n := len(stack) - 1
captured[op.operand] = stack[n]
stack = stack[:n]
}
}
if pos < l {
return nil, ErrNotMatch
}
bindings := make(map[string]string)
for i, val := range captured {
bindings[p.vars[i]] = val
}
return bindings, nil
}
// Verb returns the verb part of the Pattern.
func (p Pattern) Verb() string { return p.verb }
func (p Pattern) String() string {
var stack []string
for _, op := range p.ops {
switch op.code {
case OpNop:
continue
case OpPush:
stack = append(stack, "*")
case OpLitPush:
stack = append(stack, p.pool[op.operand])
case OpPushM:
stack = append(stack, "**")
case OpConcatN:
n := op.operand
l := len(stack) - n
stack = append(stack[:l], strings.Join(stack[l:], "/"))
case OpCapture:
n := len(stack) - 1
stack[n] = fmt.Sprintf("{%s=%s}", p.vars[op.operand], stack[n])
}
}
segs := strings.Join(stack, "/")
if p.verb != "" {
return fmt.Sprintf("/%s:%s", segs, p.verb)
}
return "/" + segs
}
// AssumeColonVerbOpt indicates whether a path suffix after a final
// colon may only be interpreted as a verb.
func AssumeColonVerbOpt(val bool) PatternOpt {
return PatternOpt(func(o *patternOptions) {
o.assumeColonVerb = val
})
}

62
api/router/util/types.go Normal file
View File

@ -0,0 +1,62 @@
package util
// download from https://raw.githubusercontent.com/grpc-ecosystem/grpc-gateway/master/protoc-gen-grpc-gateway/httprule/types.go
import (
"fmt"
"strings"
)
type template struct {
segments []segment
verb string
template string
}
type segment interface {
fmt.Stringer
compile() (ops []op)
}
type wildcard struct{}
type deepWildcard struct{}
type literal string
type variable struct {
path string
segments []segment
}
func (wildcard) String() string {
return "*"
}
func (deepWildcard) String() string {
return "**"
}
func (l literal) String() string {
return string(l)
}
func (v variable) String() string {
var segs []string
for _, s := range v.segments {
segs = append(segs, s.String())
}
return fmt.Sprintf("{%s=%s}", v.path, strings.Join(segs, "/"))
}
func (t template) String() string {
var segs []string
for _, s := range t.segments {
segs = append(segs, s.String())
}
str := strings.Join(segs, "/")
if t.verb != "" {
str = fmt.Sprintf("%s:%s", str, t.verb)
}
return "/" + str
}

View File

@ -0,0 +1,93 @@
package util
// download from https://raw.githubusercontent.com/grpc-ecosystem/grpc-gateway/master/protoc-gen-grpc-gateway/httprule/types_test.go
import (
"fmt"
"testing"
)
func TestTemplateStringer(t *testing.T) {
for _, spec := range []struct {
segs []segment
want string
}{
{
segs: []segment{
literal("v1"),
},
want: "/v1",
},
{
segs: []segment{
wildcard{},
},
want: "/*",
},
{
segs: []segment{
deepWildcard{},
},
want: "/**",
},
{
segs: []segment{
variable{
path: "name",
segments: []segment{
literal("a"),
},
},
},
want: "/{name=a}",
},
{
segs: []segment{
variable{
path: "name",
segments: []segment{
literal("a"),
wildcard{},
literal("b"),
},
},
},
want: "/{name=a/*/b}",
},
{
segs: []segment{
literal("v1"),
variable{
path: "name",
segments: []segment{
literal("a"),
wildcard{},
literal("b"),
},
},
literal("c"),
variable{
path: "field.nested",
segments: []segment{
wildcard{},
literal("d"),
},
},
wildcard{},
literal("e"),
deepWildcard{},
},
want: "/v1/{name=a/*/b}/c/{field.nested=*/d}/*/e/**",
},
} {
tmpl := template{segments: spec.segs}
if got, want := tmpl.String(), spec.want; got != want {
t.Errorf("%#v.String() = %q; want %q", tmpl, got, want)
}
tmpl.verb = "LOCK"
if got, want := tmpl.String(), fmt.Sprintf("%s:LOCK", spec.want); got != want {
t.Errorf("%#v.String() = %q; want %q", tmpl, got, want)
}
}
}

2
go.mod
View File

@ -34,7 +34,7 @@ require (
github.com/gorilla/websocket v1.4.1 // indirect
github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 // indirect
github.com/grpc-ecosystem/grpc-gateway v1.9.5
github.com/grpc-ecosystem/grpc-gateway v1.9.5 // indirect
github.com/hashicorp/hcl v1.0.0
github.com/hpcloud/tail v1.0.0
github.com/imdario/mergo v0.3.8