Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

add query cancel sync for presto forward #41

Open
wants to merge 1 commit into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
59 changes: 48 additions & 11 deletions cmd/forward/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,6 @@ package forward

import (
"context"
"github.com/spf13/cobra"
"net/http"
"os"
"os/signal"
Expand All @@ -16,6 +15,8 @@ import (
"sync/atomic"
"syscall"
"time"

"github.com/spf13/cobra"
)

var (
Expand All @@ -28,16 +29,27 @@ var (
ReplacePatternStrings []string
SchemaMappingStrings []string

excludePatterns []*regexp.Regexp
replacePatterns []*regexp.Regexp
replaceStrings []string
schemaMappings = make(map[string]string)
runningTasks sync.WaitGroup
failedToForward atomic.Uint32
forwarded atomic.Uint32
excludePatterns []*regexp.Regexp
replacePatterns []*regexp.Regexp
replaceStrings []string
schemaMappings = make(map[string]string)
runningTasks sync.WaitGroup
failedToForward atomic.Uint32
forwarded atomic.Uint32
runningQueriesCacheMap = make(map[string]*[]QueryCache)
)

const maxRetry = 10
const (
maxRetry = 10
queryStateErrorCancelled = "USER_CANCELED"
queryStateFailed = "FAILED"
)

type QueryCache struct {
QueryId string
NextUri string
Client *presto.Client
}

func waitForNextPoll(ctx context.Context) {
timer := time.NewTimer(PollInterval)
Expand Down Expand Up @@ -142,6 +154,10 @@ func Run(_ *cobra.Command, _ []string) {
// to process get filtered out.
newCutoffTime := lastQueryStateCheckCutoffTime
for _, state := range states {
//check if there is query in cancel status
if state.QueryState == queryStateFailed && state.ErrorCode.Name == queryStateErrorCancelled {
go checkAndCancelQuery(ctx, &state)
}
if !state.CreateTime.After(lastQueryStateCheckCutoffTime) {
// We looked at this query in the previous batch.
continue
Expand Down Expand Up @@ -171,6 +187,16 @@ func Run(_ *cobra.Command, _ []string) {
Msgf("finished forwarding queries")
}

func checkAndCancelQuery(ctx context.Context, queryState *presto.QueryStateInfo) {
if queryCaches, ok := runningQueriesCacheMap[queryState.QueryId]; ok {
for _, q := range *queryCaches {
if q.NextUri != "" {
q.Client.CancelQuery(ctx, q.NextUri)
}
}
}
}

func forwardQuery(ctx context.Context, queryState *presto.QueryStateInfo, clients []*presto.Client) {
defer runningTasks.Done()
var (
Expand Down Expand Up @@ -226,9 +252,10 @@ func forwardQuery(ctx context.Context, queryState *presto.QueryStateInfo, client
}
successful, failed := atomic.Uint32{}, atomic.Uint32{}
forwardedQueries := sync.WaitGroup{}
cachedQueries := []QueryCache{}
for i := 1; i < len(clients); i++ {
forwardedQueries.Add(1)
go func(client *presto.Client) {
go func(client *presto.Client, cachedQueries *[]QueryCache) {
defer forwardedQueries.Done()
clientResult, _, queryErr := client.Query(ctx, queryInfo.Query, func(req *http.Request) {
if queryInfo.Session.Catalog != nil {
Expand All @@ -246,6 +273,10 @@ func forwardQuery(ctx context.Context, queryState *presto.QueryStateInfo, client
failed.Add(1)
return
}
//build cache for running query
if clientResult.NextUri != nil {
*cachedQueries = append(*cachedQueries, QueryCache{QueryId: clientResult.Id, NextUri: *clientResult.NextUri, Client: client})
}
rowCount := 0
drainErr := clientResult.Drain(ctx, func(qr *presto.QueryResults) error {
rowCount += len(qr.Data)
Expand All @@ -260,10 +291,16 @@ func forwardQuery(ctx context.Context, queryState *presto.QueryStateInfo, client
successful.Add(1)
log.Info().Str("source_query_id", queryInfo.QueryId).
Str("target_host", client.GetHost()).Int("row_count", rowCount).Msg("query executed successfully")
}(clients[i])
}(clients[i], &cachedQueries)
}
//Add running query into to cache
runningQueriesCacheMap[queryState.QueryId] = &cachedQueries
log.Info().Msg("adding query to cache" + queryState.QueryId)
forwardedQueries.Wait()
log.Info().Str("source_query_id", queryInfo.QueryId).Uint32("successful", successful.Load()).
Uint32("failed", failed.Load()).Msg("query forwarding finished")
forwarded.Add(1)
//remove finished query from cache
delete(runningQueriesCacheMap, queryState.QueryId)
log.Info().Msg("removing query from cache" + queryState.QueryId)
}
12 changes: 10 additions & 2 deletions presto/query_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,8 +11,8 @@ import (
// https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/server/QueryStateInfo.java
// Unused fields are commented out for now.
type QueryStateInfo struct {
QueryId string `json:"queryId"`
//QueryState string `json:"queryState"`
QueryId string `json:"queryId"`
QueryState string `json:"queryState"`
//ResourceGroupId []string `json:"resourceGroupId"`
//Query string `json:"query"`
//QueryTruncated bool `json:"queryTruncated"`
Expand Down Expand Up @@ -41,6 +41,14 @@ type QueryStateInfo struct {
// CompletedDrivers int `json:"completedDrivers"`
//} `json:"progress"`
//WarningCodes []interface{} `json:"warningCodes"`
ErrorCode QueryStateError `json:"errorCode"`
}

type QueryStateError struct {
Code int `json:"code"`
Name string `json:"name"`
Type string `json:"type"`
Retriable bool `json:"retriable"`
}

// GetQueryStatsOptions includes parameters in https://github.com/prestodb/presto/blob/a7af002182098ba5a61248edfcaaa66e5d50e489/presto-main/src/main/java/com/facebook/presto/server/QueryStateInfoResource.java#L89-L95
Expand Down