forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 0
/
retrieval.go
155 lines (137 loc) · 4.66 KB
/
retrieval.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package gcsproxy
import (
"fmt"
"io"
pb "github.com/apache/beam/sdks/go/pkg/beam/model/org_apache_beam_runner_v1"
"github.com/apache/beam/sdks/go/pkg/beam/util/gcsx"
"github.com/golang/protobuf/proto"
"golang.org/x/net/context"
"google.golang.org/api/storage/v1"
)
// RetrievalServer is a artifact retrieval server backed by Google
// Cloud Storage (GCS). It serves a single manifest and ignores
// the worker id. The server performs no caching or pre-fetching.
type RetrievalServer struct {
md *pb.Manifest
blobs map[string]string
}
// ReadProxyManifest reads and parses the proxy manifest from GCS.
func ReadProxyManifest(ctx context.Context, object string) (*pb.ProxyManifest, error) {
bucket, obj, err := gcsx.ParseObject(object)
if err != nil {
return nil, fmt.Errorf("invalid manifest object %v: %v", object, err)
}
cl, err := gcsx.NewClient(ctx, storage.DevstorageReadOnlyScope)
if err != nil {
return nil, fmt.Errorf("failed to create GCS client: %v", err)
}
content, err := gcsx.ReadObject(cl, bucket, obj)
if err != nil {
return nil, fmt.Errorf("failed to read manifest %v: %v", object, err)
}
var md pb.ProxyManifest
if err := proto.Unmarshal(content, &md); err != nil {
return nil, fmt.Errorf("invalid manifest %v: %v", object, err)
}
return &md, nil
}
// NewRetrievalServer creates a artifact retrieval server for the
// given manifest. It requires that the locations are in GCS.
func NewRetrievalServer(md *pb.ProxyManifest) (*RetrievalServer, error) {
if err := validate(md); err != nil {
return nil, err
}
blobs := make(map[string]string)
for _, l := range md.GetLocation() {
if _, _, err := gcsx.ParseObject(l.GetUri()); err != nil {
return nil, fmt.Errorf("location %v is not a GCS object: %v", l.GetUri(), err)
}
blobs[l.GetName()] = l.GetUri()
}
return &RetrievalServer{md: md.GetManifest(), blobs: blobs}, nil
}
// GetManifest returns the manifest for all artifacts.
func (s *RetrievalServer) GetManifest(ctx context.Context, req *pb.GetManifestRequest) (*pb.GetManifestResponse, error) {
return &pb.GetManifestResponse{Manifest: s.md}, nil
}
// GetArtifact returns a given artifact.
func (s *RetrievalServer) GetArtifact(req *pb.GetArtifactRequest, stream pb.ArtifactRetrievalService_GetArtifactServer) error {
key := req.GetName()
blob, ok := s.blobs[key]
if !ok {
return fmt.Errorf("artifact %v not found", key)
}
bucket, object := parseObject(blob)
client, err := gcsx.NewClient(stream.Context(), storage.DevstorageReadOnlyScope)
if err != nil {
return fmt.Errorf("Failed to create client for %v: %v", key, err)
}
// Stream artifact in up to 1MB chunks.
resp, err := client.Objects.Get(bucket, object).Download()
if err != nil {
return fmt.Errorf("Failed to read object for %v: %v", key, err)
}
defer resp.Body.Close()
data := make([]byte, 1<<20)
for {
n, err := resp.Body.Read(data)
if n > 0 {
if err := stream.Send(&pb.ArtifactChunk{Data: data[:n]}); err != nil {
return fmt.Errorf("chunk send failed: %v", err)
}
}
if err == io.EOF {
break
}
if err != nil {
return fmt.Errorf("failed to read from %v: %v", blob, err)
}
}
return nil
}
func validate(md *pb.ProxyManifest) error {
keys := make(map[string]bool)
for _, a := range md.GetManifest().GetArtifact() {
if _, seen := keys[a.Name]; seen {
return fmt.Errorf("multiple artifact with name %v", a.Name)
}
keys[a.Name] = true
}
for _, l := range md.GetLocation() {
fresh, seen := keys[l.Name]
if !seen {
return fmt.Errorf("no artifact named %v for location %v", l.Name, l.Uri)
}
if !fresh {
return fmt.Errorf("multiple locations for %v:%v", l.Name, l.Uri)
}
keys[l.Name] = false
}
for key, fresh := range keys {
if fresh {
return fmt.Errorf("no location for %v", key)
}
}
return nil
}
func parseObject(blob string) (string, string) {
bucket, object, err := gcsx.ParseObject(blob)
if err != nil {
panic(err)
}
return bucket, object
}