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:
Mohammed Diaa 2024-08-01 18:13:47 +03:00 committed by GitHub
parent 2655c29458
commit ff23949bb0
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 55 additions and 78 deletions

View File

@ -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
} }

View File

@ -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", ""},

View File

@ -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()
} }

View File

@ -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 {

View File

@ -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()
}

View File

@ -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
} }

View File

@ -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

View File

@ -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")
}

View File

@ -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

View File

@ -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
} }
} }

View File

@ -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())
}