mirror of
https://github.com/projectdiscovery/nuclei.git
synced 2025-12-17 20:15:27 +00:00
Apply input transformation to multi-protocol templates (#5426)
* Apply input transformation to multi-protocol template execution * Remove ad hoc input transoformation from DNS protocol * Add SSL protocol input transformer * Remove ad hoc input transoformation from SSL protocol * Remove unused function extractDomain from the DNS protocol engine * transform in flow as well * bug fix + update test * bug fix multi proto : * bug fix multi proto input * bug fixes in input transform --------- Co-authored-by: Tarun Koyalwar <tarun@projectdiscovery.io>
This commit is contained in:
parent
2655c29458
commit
ff23949bb0
@ -47,6 +47,8 @@ func (h *Helper) Transform(input string, protocol templateTypes.ProtocolType) st
|
|||||||
return h.convertInputToType(input, typeHostWithOptionalPort, "")
|
return h.convertInputToType(input, typeHostWithOptionalPort, "")
|
||||||
case templateTypes.WebsocketProtocol:
|
case templateTypes.WebsocketProtocol:
|
||||||
return h.convertInputToType(input, typeWebsocket, "")
|
return h.convertInputToType(input, typeWebsocket, "")
|
||||||
|
case templateTypes.SSLProtocol:
|
||||||
|
return h.convertInputToType(input, typeHostWithPort, "443")
|
||||||
}
|
}
|
||||||
return input
|
return input
|
||||||
}
|
}
|
||||||
@ -94,6 +96,8 @@ func (h *Helper) convertInputToType(input string, inputType inputType, defaultPo
|
|||||||
if _, err := filepath.Match(input, ""); err != filepath.ErrBadPattern && !isURL {
|
if _, err := filepath.Match(input, ""); err != filepath.ErrBadPattern && !isURL {
|
||||||
return input
|
return input
|
||||||
}
|
}
|
||||||
|
// if none of these satisfy the condition return empty
|
||||||
|
return ""
|
||||||
case typeHostOnly:
|
case typeHostOnly:
|
||||||
if hasHost {
|
if hasHost {
|
||||||
return host
|
return host
|
||||||
@ -111,6 +115,10 @@ func (h *Helper) convertInputToType(input string, inputType inputType, defaultPo
|
|||||||
return string(probed)
|
return string(probed)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
// try to parse it as absolute url and return
|
||||||
|
if absUrl, err := urlutil.ParseAbsoluteURL(input, false); err == nil {
|
||||||
|
return absUrl.String()
|
||||||
|
}
|
||||||
case typeHostWithPort, typeHostWithOptionalPort:
|
case typeHostWithPort, typeHostWithOptionalPort:
|
||||||
if hasHost && hasPort {
|
if hasHost && hasPort {
|
||||||
return net.JoinHostPort(host, port)
|
return net.JoinHostPort(host, port)
|
||||||
@ -128,6 +136,9 @@ func (h *Helper) convertInputToType(input string, inputType inputType, defaultPo
|
|||||||
if uri != nil && stringsutil.EqualFoldAny(uri.Scheme, "ws", "wss") {
|
if uri != nil && stringsutil.EqualFoldAny(uri.Scheme, "ws", "wss") {
|
||||||
return input
|
return input
|
||||||
}
|
}
|
||||||
}
|
// empty if prefix is not given
|
||||||
return ""
|
return ""
|
||||||
}
|
}
|
||||||
|
// do not return empty
|
||||||
|
return input
|
||||||
|
}
|
||||||
|
|||||||
@ -30,7 +30,7 @@ func TestConvertInputToType(t *testing.T) {
|
|||||||
{"https://google.com:443", typeHostOnly, "google.com", ""},
|
{"https://google.com:443", typeHostOnly, "google.com", ""},
|
||||||
|
|
||||||
// url
|
// url
|
||||||
{"test.com", typeURL, "", ""},
|
{"test.com", typeURL, "test.com", ""},
|
||||||
{"google.com", typeURL, "https://google.com", ""},
|
{"google.com", typeURL, "https://google.com", ""},
|
||||||
{"https://google.com", typeURL, "https://google.com", ""},
|
{"https://google.com", typeURL, "https://google.com", ""},
|
||||||
|
|
||||||
@ -43,7 +43,7 @@ func TestConvertInputToType(t *testing.T) {
|
|||||||
{"input_test.*", typeFilepath, "input_test.*", ""},
|
{"input_test.*", typeFilepath, "input_test.*", ""},
|
||||||
|
|
||||||
// host-port
|
// host-port
|
||||||
{"google.com", typeHostWithPort, "", ""},
|
{"google.com", typeHostWithPort, "google.com", ""},
|
||||||
{"google.com:443", typeHostWithPort, "google.com:443", ""},
|
{"google.com:443", typeHostWithPort, "google.com:443", ""},
|
||||||
{"https://google.com", typeHostWithPort, "google.com:443", ""},
|
{"https://google.com", typeHostWithPort, "google.com:443", ""},
|
||||||
{"https://google.com:443", typeHostWithPort, "google.com:443", ""},
|
{"https://google.com:443", typeHostWithPort, "google.com:443", ""},
|
||||||
|
|||||||
@ -145,6 +145,7 @@ func (metaInput *MetaInput) Clone() *MetaInput {
|
|||||||
input := NewMetaInput()
|
input := NewMetaInput()
|
||||||
input.Input = metaInput.Input
|
input.Input = metaInput.Input
|
||||||
input.CustomIP = metaInput.CustomIP
|
input.CustomIP = metaInput.CustomIP
|
||||||
|
input.hash = metaInput.hash
|
||||||
if metaInput.ReqResp != nil {
|
if metaInput.ReqResp != nil {
|
||||||
input.ReqResp = metaInput.ReqResp.Clone()
|
input.ReqResp = metaInput.ReqResp.Clone()
|
||||||
}
|
}
|
||||||
|
|||||||
@ -141,12 +141,6 @@ func (request *Request) Compile(options *protocols.ExecutorOptions) error {
|
|||||||
recursion := true
|
recursion := true
|
||||||
request.Recursion = &recursion
|
request.Recursion = &recursion
|
||||||
}
|
}
|
||||||
dnsClientOptions := &dnsclientpool.Configuration{
|
|
||||||
Retries: request.Retries,
|
|
||||||
}
|
|
||||||
if len(request.Resolvers) > 0 {
|
|
||||||
dnsClientOptions.Resolvers = request.Resolvers
|
|
||||||
}
|
|
||||||
// Create a dns client for the class
|
// Create a dns client for the class
|
||||||
client, err := request.getDnsClient(options, nil)
|
client, err := request.getDnsClient(options, nil)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
|||||||
@ -3,7 +3,6 @@ package dns
|
|||||||
import (
|
import (
|
||||||
"encoding/hex"
|
"encoding/hex"
|
||||||
"fmt"
|
"fmt"
|
||||||
"net/url"
|
|
||||||
"strings"
|
"strings"
|
||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
@ -23,7 +22,6 @@ import (
|
|||||||
"github.com/projectdiscovery/nuclei/v3/pkg/protocols/common/utils/vardump"
|
"github.com/projectdiscovery/nuclei/v3/pkg/protocols/common/utils/vardump"
|
||||||
protocolutils "github.com/projectdiscovery/nuclei/v3/pkg/protocols/utils"
|
protocolutils "github.com/projectdiscovery/nuclei/v3/pkg/protocols/utils"
|
||||||
templateTypes "github.com/projectdiscovery/nuclei/v3/pkg/templates/types"
|
templateTypes "github.com/projectdiscovery/nuclei/v3/pkg/templates/types"
|
||||||
"github.com/projectdiscovery/nuclei/v3/pkg/utils"
|
|
||||||
"github.com/projectdiscovery/retryabledns"
|
"github.com/projectdiscovery/retryabledns"
|
||||||
iputil "github.com/projectdiscovery/utils/ip"
|
iputil "github.com/projectdiscovery/utils/ip"
|
||||||
syncutil "github.com/projectdiscovery/utils/sync"
|
syncutil "github.com/projectdiscovery/utils/sync"
|
||||||
@ -38,16 +36,8 @@ func (request *Request) Type() templateTypes.ProtocolType {
|
|||||||
|
|
||||||
// ExecuteWithResults executes the protocol requests and returns results instead of writing them.
|
// ExecuteWithResults executes the protocol requests and returns results instead of writing them.
|
||||||
func (request *Request) ExecuteWithResults(input *contextargs.Context, metadata, previous output.InternalEvent, callback protocols.OutputEventCallback) error {
|
func (request *Request) ExecuteWithResults(input *contextargs.Context, metadata, previous output.InternalEvent, callback protocols.OutputEventCallback) error {
|
||||||
// Parse the URL and return domain if URL.
|
|
||||||
var domain string
|
|
||||||
if utils.IsURL(input.MetaInput.Input) {
|
|
||||||
domain = extractDomain(input.MetaInput.Input)
|
|
||||||
} else {
|
|
||||||
domain = input.MetaInput.Input
|
|
||||||
}
|
|
||||||
|
|
||||||
var err error
|
var err error
|
||||||
domain, err = request.parseDNSInput(domain)
|
domain, err := request.parseDNSInput(input.MetaInput.Input)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Wrap(err, "could not build request")
|
return errors.Wrap(err, "could not build request")
|
||||||
}
|
}
|
||||||
@ -230,7 +220,7 @@ func (request *Request) parseDNSInput(host string) (string, error) {
|
|||||||
return host, nil
|
return host, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func dumpResponse(event *output.InternalWrappedEvent, request *Request, requestOptions *protocols.ExecutorOptions, response, domain string) {
|
func dumpResponse(event *output.InternalWrappedEvent, request *Request, _ *protocols.ExecutorOptions, response, domain string) {
|
||||||
cliOptions := request.options.Options
|
cliOptions := request.options.Options
|
||||||
if cliOptions.Debug || cliOptions.DebugResponse || cliOptions.StoreResponse {
|
if cliOptions.Debug || cliOptions.DebugResponse || cliOptions.StoreResponse {
|
||||||
hexDump := false
|
hexDump := false
|
||||||
@ -261,12 +251,3 @@ func dumpTraceData(event *output.InternalWrappedEvent, requestOptions *protocols
|
|||||||
gologger.Debug().Msgf("[%s] Dumped DNS Trace data for %s\n\n%s", requestOptions.TemplateID, domain, highlightedResponse)
|
gologger.Debug().Msgf("[%s] Dumped DNS Trace data for %s\n\n%s", requestOptions.TemplateID, domain, highlightedResponse)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// extractDomain extracts the domain name of a URL
|
|
||||||
func extractDomain(theURL string) string {
|
|
||||||
u, err := url.Parse(theURL)
|
|
||||||
if err != nil {
|
|
||||||
return ""
|
|
||||||
}
|
|
||||||
return u.Hostname()
|
|
||||||
}
|
|
||||||
|
|||||||
@ -67,19 +67,5 @@ func TestDNSExecuteWithResults(t *testing.T) {
|
|||||||
require.Equal(t, 1, len(finalEvent.Results[0].ExtractedResults), "could not get correct number of extracted results")
|
require.Equal(t, 1, len(finalEvent.Results[0].ExtractedResults), "could not get correct number of extracted results")
|
||||||
require.Equal(t, "93.184.215.14", finalEvent.Results[0].ExtractedResults[0], "could not get correct extracted results")
|
require.Equal(t, "93.184.215.14", finalEvent.Results[0].ExtractedResults[0], "could not get correct extracted results")
|
||||||
finalEvent = nil
|
finalEvent = nil
|
||||||
|
// Note: changing url to domain is responsible at tmplexec package and is implemented there
|
||||||
t.Run("url-to-domain", func(t *testing.T) {
|
|
||||||
metadata := make(output.InternalEvent)
|
|
||||||
previous := make(output.InternalEvent)
|
|
||||||
err := request.ExecuteWithResults(contextargs.NewWithInput(context.Background(), "https://example.com"), metadata, previous, func(event *output.InternalWrappedEvent) {
|
|
||||||
finalEvent = event
|
|
||||||
})
|
|
||||||
require.Nil(t, err, "could not execute dns request")
|
|
||||||
})
|
|
||||||
require.NotNil(t, finalEvent, "could not get event output from request")
|
|
||||||
require.Equal(t, 1, len(finalEvent.Results), "could not get correct number of results")
|
|
||||||
require.Equal(t, "test", finalEvent.Results[0].MatcherName, "could not get correct matcher name of results")
|
|
||||||
require.Equal(t, 1, len(finalEvent.Results[0].ExtractedResults), "could not get correct number of extracted results")
|
|
||||||
require.Equal(t, "93.184.215.14", finalEvent.Results[0].ExtractedResults[0], "could not get correct extracted results")
|
|
||||||
finalEvent = nil
|
|
||||||
}
|
}
|
||||||
|
|||||||
@ -34,7 +34,6 @@ import (
|
|||||||
"github.com/projectdiscovery/tlsx/pkg/tlsx/openssl"
|
"github.com/projectdiscovery/tlsx/pkg/tlsx/openssl"
|
||||||
errorutil "github.com/projectdiscovery/utils/errors"
|
errorutil "github.com/projectdiscovery/utils/errors"
|
||||||
stringsutil "github.com/projectdiscovery/utils/strings"
|
stringsutil "github.com/projectdiscovery/utils/strings"
|
||||||
urlutil "github.com/projectdiscovery/utils/url"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// Request is a request for the SSL protocol
|
// Request is a request for the SSL protocol
|
||||||
@ -199,10 +198,7 @@ func (request *Request) GetID() string {
|
|||||||
|
|
||||||
// ExecuteWithResults executes the protocol requests and returns results instead of writing them.
|
// ExecuteWithResults executes the protocol requests and returns results instead of writing them.
|
||||||
func (request *Request) ExecuteWithResults(input *contextargs.Context, dynamicValues, previous output.InternalEvent, callback protocols.OutputEventCallback) error {
|
func (request *Request) ExecuteWithResults(input *contextargs.Context, dynamicValues, previous output.InternalEvent, callback protocols.OutputEventCallback) error {
|
||||||
hostPort, err := getAddress(input.MetaInput.Input)
|
hostPort := input.MetaInput.Input
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
hostname, port, _ := net.SplitHostPort(hostPort)
|
hostname, port, _ := net.SplitHostPort(hostPort)
|
||||||
|
|
||||||
requestOptions := request.options
|
requestOptions := request.options
|
||||||
@ -358,19 +354,6 @@ var RequestPartDefinitions = map[string]string{
|
|||||||
"matched": "Matched is the input which was matched upon",
|
"matched": "Matched is the input which was matched upon",
|
||||||
}
|
}
|
||||||
|
|
||||||
// getAddress returns the address of the host to make request to
|
|
||||||
func getAddress(toTest string) (string, error) {
|
|
||||||
urlx, err := urlutil.Parse(toTest)
|
|
||||||
if err != nil {
|
|
||||||
// use given input instead of url parsing failure
|
|
||||||
return toTest, nil
|
|
||||||
}
|
|
||||||
if urlx.Port() == "" {
|
|
||||||
urlx.UpdatePort("443")
|
|
||||||
}
|
|
||||||
return urlx.Host, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// Match performs matching operation for a matcher on model and returns:
|
// Match performs matching operation for a matcher on model and returns:
|
||||||
// true and a list of matched snippets if the matcher type is supports it
|
// true and a list of matched snippets if the matcher type is supports it
|
||||||
// otherwise false and an empty string slice
|
// otherwise false and an empty string slice
|
||||||
|
|||||||
@ -36,8 +36,3 @@ func TestSSLProtocol(t *testing.T) {
|
|||||||
require.Nil(t, err, "could not run ssl request")
|
require.Nil(t, err, "could not run ssl request")
|
||||||
require.NotEmpty(t, gotEvent, "could not get event items")
|
require.NotEmpty(t, gotEvent, "could not get event items")
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestGetAddress(t *testing.T) {
|
|
||||||
address, _ := getAddress("https://scanme.sh")
|
|
||||||
require.Equal(t, "scanme.sh:443", address, "could not get correct address")
|
|
||||||
}
|
|
||||||
|
|||||||
@ -29,7 +29,15 @@ func (f *FlowExecutor) requestExecutor(runtime *goja.Runtime, reqMap mapsutil.Ma
|
|||||||
// execution logic for http()/dns() etc
|
// execution logic for http()/dns() etc
|
||||||
for index := range f.allProtocols[opts.protoName] {
|
for index := range f.allProtocols[opts.protoName] {
|
||||||
req := f.allProtocols[opts.protoName][index]
|
req := f.allProtocols[opts.protoName][index]
|
||||||
err := req.ExecuteWithResults(f.ctx.Input, output.InternalEvent(f.options.GetTemplateCtx(f.ctx.Input.MetaInput).GetAll()), nil, f.protocolResultCallback(req, matcherStatus, opts))
|
// transform input if required
|
||||||
|
inputItem := f.ctx.Input.Clone()
|
||||||
|
if f.options.InputHelper != nil && f.ctx.Input.MetaInput.Input != "" {
|
||||||
|
if inputItem.MetaInput.Input = f.options.InputHelper.Transform(inputItem.MetaInput.Input, req.Type()); inputItem.MetaInput.Input == "" {
|
||||||
|
f.ctx.LogError(fmt.Errorf("failed to transform input for protocol %s", req.Type()))
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
}
|
||||||
|
err := req.ExecuteWithResults(inputItem, output.InternalEvent(f.options.GetTemplateCtx(f.ctx.Input.MetaInput).GetAll()), nil, f.protocolResultCallback(req, matcherStatus, opts))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
// save all errors in a map with id as key
|
// save all errors in a map with id as key
|
||||||
// its less likely that there will be race condition but just in case
|
// its less likely that there will be race condition but just in case
|
||||||
@ -58,7 +66,15 @@ func (f *FlowExecutor) requestExecutor(runtime *goja.Runtime, reqMap mapsutil.Ma
|
|||||||
}
|
}
|
||||||
return matcherStatus.Load()
|
return matcherStatus.Load()
|
||||||
}
|
}
|
||||||
err := req.ExecuteWithResults(f.ctx.Input, output.InternalEvent(f.options.GetTemplateCtx(f.ctx.Input.MetaInput).GetAll()), nil, f.protocolResultCallback(req, matcherStatus, opts))
|
// transform input if required
|
||||||
|
inputItem := f.ctx.Input.Clone()
|
||||||
|
if f.options.InputHelper != nil && f.ctx.Input.MetaInput.Input != "" {
|
||||||
|
if inputItem.MetaInput.Input = f.options.InputHelper.Transform(inputItem.MetaInput.Input, req.Type()); inputItem.MetaInput.Input == "" {
|
||||||
|
f.ctx.LogError(fmt.Errorf("failed to transform input for protocol %s", req.Type()))
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
}
|
||||||
|
err := req.ExecuteWithResults(inputItem, output.InternalEvent(f.options.GetTemplateCtx(f.ctx.Input.MetaInput).GetAll()), nil, f.protocolResultCallback(req, matcherStatus, opts))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
index := id
|
index := id
|
||||||
err = f.allErrs.Set(opts.protoName+":"+index, err)
|
err = f.allErrs.Set(opts.protoName+":"+index, err)
|
||||||
@ -72,7 +88,7 @@ func (f *FlowExecutor) requestExecutor(runtime *goja.Runtime, reqMap mapsutil.Ma
|
|||||||
|
|
||||||
// protocolResultCallback returns a callback that is executed
|
// protocolResultCallback returns a callback that is executed
|
||||||
// after execution of each protocol request
|
// after execution of each protocol request
|
||||||
func (f *FlowExecutor) protocolResultCallback(req protocols.Request, matcherStatus *atomic.Bool, opts *ProtoOptions) func(result *output.InternalWrappedEvent) {
|
func (f *FlowExecutor) protocolResultCallback(req protocols.Request, matcherStatus *atomic.Bool, _ *ProtoOptions) func(result *output.InternalWrappedEvent) {
|
||||||
return func(result *output.InternalWrappedEvent) {
|
return func(result *output.InternalWrappedEvent) {
|
||||||
if result != nil {
|
if result != nil {
|
||||||
// Note: flow specific implicit behaviours should be handled here
|
// Note: flow specific implicit behaviours should be handled here
|
||||||
|
|||||||
@ -109,14 +109,19 @@ func (m *MultiProtocol) ExecuteWithResults(ctx *scan.ScanContext) error {
|
|||||||
return ctx.Context().Err()
|
return ctx.Context().Err()
|
||||||
default:
|
default:
|
||||||
}
|
}
|
||||||
|
inputItem := ctx.Input.Clone()
|
||||||
values := m.options.GetTemplateCtx(ctx.Input.MetaInput).GetAll()
|
if m.options.InputHelper != nil && ctx.Input.MetaInput.Input != "" {
|
||||||
err := req.ExecuteWithResults(ctx.Input, output.InternalEvent(values), nil, multiProtoCallback)
|
if inputItem.MetaInput.Input = m.options.InputHelper.Transform(inputItem.MetaInput.Input, req.Type()); inputItem.MetaInput.Input == "" {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// FIXME: this hack of using hash to get templateCtx has known issues scan context based approach should be adopted ASAP
|
||||||
|
values := m.options.GetTemplateCtx(inputItem.MetaInput).GetAll()
|
||||||
|
err := req.ExecuteWithResults(inputItem, output.InternalEvent(values), nil, multiProtoCallback)
|
||||||
// in case of fatal error skip execution of next protocols
|
// in case of fatal error skip execution of next protocols
|
||||||
if err != nil {
|
if err != nil {
|
||||||
// always log errors
|
// always log errors
|
||||||
ctx.LogError(err)
|
ctx.LogError(err)
|
||||||
|
|
||||||
// for some classes of protocols (i.e ssl) errors like tls handshake are a legitimate behavior so we don't stop execution
|
// for some classes of protocols (i.e ssl) errors like tls handshake are a legitimate behavior so we don't stop execution
|
||||||
// connection failures are already tracked by the internal host error cache
|
// connection failures are already tracked by the internal host error cache
|
||||||
// we use strings comparison as the error is not formalized into instance within the standard library
|
// we use strings comparison as the error is not formalized into instance within the standard library
|
||||||
@ -124,7 +129,6 @@ func (m *MultiProtocol) ExecuteWithResults(ctx *scan.ScanContext) error {
|
|||||||
if req.Type() == types.SSLProtocol && stringsutil.ContainsAnyI(err.Error(), "protocol version not supported", "could not do tls handshake") {
|
if req.Type() == types.SSLProtocol && stringsutil.ContainsAnyI(err.Error(), "protocol version not supported", "could not do tls handshake") {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@ -3,11 +3,13 @@ package multiproto_test
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"log"
|
"log"
|
||||||
|
"os"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/projectdiscovery/nuclei/v3/pkg/catalog/config"
|
"github.com/projectdiscovery/nuclei/v3/pkg/catalog/config"
|
||||||
"github.com/projectdiscovery/nuclei/v3/pkg/catalog/disk"
|
"github.com/projectdiscovery/nuclei/v3/pkg/catalog/disk"
|
||||||
|
"github.com/projectdiscovery/nuclei/v3/pkg/input"
|
||||||
"github.com/projectdiscovery/nuclei/v3/pkg/loader/workflow"
|
"github.com/projectdiscovery/nuclei/v3/pkg/loader/workflow"
|
||||||
"github.com/projectdiscovery/nuclei/v3/pkg/progress"
|
"github.com/projectdiscovery/nuclei/v3/pkg/progress"
|
||||||
"github.com/projectdiscovery/nuclei/v3/pkg/protocols"
|
"github.com/projectdiscovery/nuclei/v3/pkg/protocols"
|
||||||
@ -36,6 +38,7 @@ func setup() {
|
|||||||
Catalog: disk.NewCatalog(config.DefaultConfig.TemplatesDirectory),
|
Catalog: disk.NewCatalog(config.DefaultConfig.TemplatesDirectory),
|
||||||
RateLimiter: ratelimit.New(context.Background(), uint(options.RateLimit), time.Second),
|
RateLimiter: ratelimit.New(context.Background(), uint(options.RateLimit), time.Second),
|
||||||
Parser: templates.NewParser(),
|
Parser: templates.NewParser(),
|
||||||
|
InputHelper: input.NewHelper(),
|
||||||
}
|
}
|
||||||
workflowLoader, err := workflow.NewLoader(&executerOpts)
|
workflowLoader, err := workflow.NewLoader(&executerOpts)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -45,7 +48,6 @@ func setup() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func TestMultiProtoWithDynamicExtractor(t *testing.T) {
|
func TestMultiProtoWithDynamicExtractor(t *testing.T) {
|
||||||
setup()
|
|
||||||
Template, err := templates.Parse("testcases/multiprotodynamic.yaml", nil, executerOpts)
|
Template, err := templates.Parse("testcases/multiprotodynamic.yaml", nil, executerOpts)
|
||||||
require.Nil(t, err, "could not parse template")
|
require.Nil(t, err, "could not parse template")
|
||||||
|
|
||||||
@ -62,7 +64,6 @@ func TestMultiProtoWithDynamicExtractor(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func TestMultiProtoWithProtoPrefix(t *testing.T) {
|
func TestMultiProtoWithProtoPrefix(t *testing.T) {
|
||||||
setup()
|
|
||||||
Template, err := templates.Parse("testcases/multiprotowithprefix.yaml", nil, executerOpts)
|
Template, err := templates.Parse("testcases/multiprotowithprefix.yaml", nil, executerOpts)
|
||||||
require.Nil(t, err, "could not parse template")
|
require.Nil(t, err, "could not parse template")
|
||||||
|
|
||||||
@ -77,3 +78,8 @@ func TestMultiProtoWithProtoPrefix(t *testing.T) {
|
|||||||
require.Nil(t, err, "could not execute template")
|
require.Nil(t, err, "could not execute template")
|
||||||
require.True(t, gotresults)
|
require.True(t, gotresults)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestMain(m *testing.M) {
|
||||||
|
setup()
|
||||||
|
os.Exit(m.Run())
|
||||||
|
}
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user