Skip to content

Commit

Permalink
Customize error and enable DefaultHTTPErrorHandler
Browse files Browse the repository at this point in the history
1. Enable `runtime.DefaultHTTPErrorHandler` to convert gRPC status code to HTTP code.
2. Since we use custom error struct, we need to implement `GRPCStatus` to make sure grpc-go can recognize the error code in our struct,
Otherwise, it will always return codes.Unknown(2), that's the reason we always see code 2 and HTTP 500 in the past.
  • Loading branch information
Jeffwan committed Jul 10, 2022
1 parent b4f4a85 commit b597d50
Show file tree
Hide file tree
Showing 3 changed files with 30 additions and 2 deletions.
14 changes: 12 additions & 2 deletions apiserver/cmd/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,14 +4,16 @@ import (
"context"
"flag"

"google.golang.org/protobuf/encoding/protojson"

"math"
"net"
"net/http"
"path"
"strings"

assetfs "github.com/elazarl/go-bindata-assetfs"
"github.com/grpc-ecosystem/grpc-gateway/runtime"
"github.com/grpc-ecosystem/grpc-gateway/v2/runtime"

"google.golang.org/grpc"
"google.golang.org/grpc/reflection"
Expand Down Expand Up @@ -82,7 +84,15 @@ func startHttpProxy() {
defer cancel()

// Create gRPC HTTP MUX and register services.
runtimeMux := runtime.NewServeMux(runtime.WithMarshalerOption(runtime.MIMEWildcard, &runtime.JSONPb{OrigName: true, EmitDefaults: true}))
runtimeMux := runtime.NewServeMux(
runtime.WithMarshalerOption(runtime.MIMEWildcard, &runtime.JSONPb{
MarshalOptions: protojson.MarshalOptions{
UseProtoNames: true,
UseEnumNumbers: true,
},
}),
runtime.WithErrorHandler(runtime.DefaultHTTPErrorHandler),
)
registerHttpHandlerFromEndpoint(api.RegisterClusterServiceHandlerFromEndpoint, "ClusterService", ctx, runtimeMux)
registerHttpHandlerFromEndpoint(api.RegisterComputeTemplateServiceHandlerFromEndpoint, "ComputeTemplateService", ctx, runtimeMux)

Expand Down
10 changes: 10 additions & 0 deletions apiserver/pkg/manager/resource_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,8 @@ import (
"context"
"fmt"

"k8s.io/apimachinery/pkg/api/errors"

"github.com/ray-project/kuberay/apiserver/pkg/model"

"github.com/ray-project/kuberay/apiserver/pkg/util"
Expand Down Expand Up @@ -264,6 +266,10 @@ func (r *ResourceManager) DeleteComputeTemplate(ctx context.Context, name string
func getClusterByName(ctx context.Context, client rayiov1alpha1.RayClusterInterface, name string) (*v1alpha1.RayCluster, error) {
cluster, err := client.Get(ctx, name, metav1.GetOptions{})
if err != nil {
if errors.IsNotFound(err) {
return nil, util.NewNotFoundError(err, "Cluster %s not found", name)
}

return nil, util.Wrap(err, "Get Cluster failed")
}
if managedBy, ok := cluster.Labels[util.KubernetesManagedByLabelKey]; !ok || managedBy != util.ComponentName {
Expand All @@ -277,6 +283,10 @@ func getClusterByName(ctx context.Context, client rayiov1alpha1.RayClusterInterf
func getComputeTemplateByName(ctx context.Context, client clientv1.ConfigMapInterface, name string) (*v1.ConfigMap, error) {
runtime, err := client.Get(ctx, name, metav1.GetOptions{})
if err != nil {
if errors.IsNotFound(err) {
return nil, util.NewNotFoundError(err, "Cluster %s not found", name)
}

return nil, util.Wrap(err, "Get compute template failed")
}

Expand Down
8 changes: 8 additions & 0 deletions apiserver/pkg/util/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@ import (
"github.com/golang/glog"
"github.com/pkg/errors"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
k8errors "k8s.io/apimachinery/pkg/api/errors"
k8metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
)
Expand Down Expand Up @@ -215,6 +216,13 @@ func (e *UserError) String() string {
e.internalError)
}

// GRPCStatus implements `GRPCStatus` to make sure `FromError` in grpc-go can honor the code.
// Otherwise, it will always return codes.Unknown(2).
// https://github.com/grpc/grpc-go/blob/2c0949c22d46095edc579d9e66edcd025192b98c/status/status.go#L91-L92
func (e *UserError) GRPCStatus() *status.Status {
return status.New(e.externalStatusCode, e.externalMessage)
}

func (e *UserError) wrapf(format string, args ...interface{}) *UserError {
return newUserError(errors.Wrapf(e.internalError, format, args...),
e.externalMessage, e.externalStatusCode)
Expand Down

0 comments on commit b597d50

Please sign in to comment.