/
build.go
235 lines (203 loc) · 6.71 KB
/
build.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
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
// Copyright 2021 Google LLC. All Rights Reserved.
//
// Licensed 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
//
// https://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.
// build is a tool to build a map from a given clone of a log.
package main
import (
"bufio"
"context"
"flag"
"fmt"
"path"
"reflect"
"github.com/apache/beam/sdks/v2/go/pkg/beam"
"github.com/apache/beam/sdks/v2/go/pkg/beam/io/databaseio"
"github.com/apache/beam/sdks/v2/go/pkg/beam/io/filesystem"
"github.com/apache/beam/sdks/v2/go/pkg/beam/io/textio"
beamlog "github.com/apache/beam/sdks/v2/go/pkg/beam/log"
"github.com/apache/beam/sdks/v2/go/pkg/beam/x/beamx"
"github.com/golang/glog"
"github.com/google/trillian-examples/clone/logdb"
"github.com/google/trillian-examples/experimental/batchmap/ctmap/internal/pipeline"
"github.com/google/trillian/experimental/batchmap"
_ "github.com/go-sql-driver/mysql"
)
var (
mysqlURI = flag.String("mysql_log_uri", "", "URL of a MySQL database to read the log from.")
mapOutputRootDir = flag.String("map_output_root_dir", "", "Filesystem root for this map. Cannot be shared with other maps.")
treeID = flag.Int64("tree_id", 12345, "The ID of the tree. Used as a salt in hashing.")
prefixStrata = flag.Int("prefix_strata", 2, "The number of strata of 8-bit strata before the final strata.")
count = flag.Int64("count", 1, "The total number of entries starting from the beginning of the log to use")
)
func init() {
beam.RegisterType(reflect.TypeOf((*writeTileFn)(nil)).Elem())
beam.RegisterType(reflect.TypeOf((*writeCheckpointFn)(nil)).Elem())
}
func main() {
flag.Parse()
beam.Init()
ctx := context.Background()
if len(*mapOutputRootDir) == 0 {
glog.Exit("Required flag 'map_output_root_dir' required")
}
if len(*mysqlURI) == 0 {
glog.Exit("Missing required flag 'mysql_log_uri'")
}
db, err := logdb.NewDatabase(*mysqlURI)
if err != nil {
glog.Exitf("Failed to connect to database: %q", err)
}
beamlog.SetLogger(&BeamGLogger{InfoLogAtVerbosity: 2})
p, s := beam.NewPipelineWithRoot()
log := mySQLLog{
db: db,
dbString: *mysqlURI,
}
mb := pipeline.NewMapBuilder(&log, *treeID, *prefixStrata)
r, err := mb.Create(ctx, s, *count)
if err != nil {
glog.Exitf("Failed to create pipeline: %v", err)
}
// Write out the leaf values, i.e. the logs.
// This currently writes a single large file containing all the results.
textio.Write(s, path.Join(*mapOutputRootDir, "logs.txt"), beam.ParDo(s, formatFn, r.DomainCertIndexLogs))
// Write out all of the tiles that represent the map.
beam.ParDo0(s, &writeTileFn{*mapOutputRootDir}, r.MapTiles)
// Write out the map checkpoint.
beam.ParDo0(s, &writeCheckpointFn{
RootDir: *mapOutputRootDir,
LogCheckpoint: r.Metadata.Checkpoint,
EntryCount: uint64(r.Metadata.Entries),
}, r.MapTiles)
glog.Info("Pipeline constructed, calling beamx.Run()")
// All of the above constructs the pipeline but doesn't run it. Now we run it.
if err := beamx.Run(ctx, p); err != nil {
glog.Exitf("Failed to execute job: %q", err)
}
}
// BeamGLogger allows Beam to log via the glog mechanism.
// This is used to allow the very verbose logging output from Beam to be switched off.
type BeamGLogger struct {
InfoLogAtVerbosity glog.Level
}
// Log logs.
func (l *BeamGLogger) Log(ctx context.Context, sev beamlog.Severity, _ int, msg string) {
switch sev {
case beamlog.SevDebug:
glog.V(3).Info(msg)
case beamlog.SevInfo:
glog.V(l.InfoLogAtVerbosity).Info(msg)
case beamlog.SevError:
glog.Error(msg)
case beamlog.SevWarn:
glog.Warning(msg)
default:
glog.V(5).Infof("?? %s", msg)
}
}
type mySQLLog struct {
db *logdb.Database
dbString string
}
// Head returns the metadata of available entries.
func (l *mySQLLog) Head(ctx context.Context) ([]byte, int64, error) {
// There may be more leaves than `size`, but any leaves at a higher index than
// this have not been verified, and are not committed to by the checkpoint so
// we can't use them in the map.
size, cp, _, err := l.db.GetLatestCheckpoint(ctx)
return cp, int64(size), err
}
// Entries returns a PCollection of InputLogLeaf, containing entries in range [start, end).
func (l *mySQLLog) Entries(s beam.Scope, start, end int64) beam.PCollection {
return databaseio.Query(s, "mysql", l.dbString, fmt.Sprintf("SELECT id AS Seq, data AS Data FROM leaves WHERE id >= %d AND id < %d", start, end), reflect.TypeOf(pipeline.InputLogLeaf{}))
}
// formatFn is a DoFn that formats a domain's log as a string.
func formatFn(l *pipeline.DomainCertIndexLog) string {
r := l.Domain
for _, i := range l.Indices {
r = fmt.Sprintf("%s,%d", r, i)
}
return r
}
type writeTileFn struct {
RootDir string
}
func (w *writeTileFn) ProcessElement(ctx context.Context, t *batchmap.Tile) error {
filename := path.Join(w.RootDir, fmt.Sprintf("tile-%x", t.Path))
fs, err := filesystem.New(ctx, filename)
if err != nil {
return err
}
defer func() {
if err := fs.Close(); err != nil {
glog.Errorf("fs.Close(): %v", err)
}
}()
fd, err := fs.OpenWrite(ctx, filename)
if err != nil {
return err
}
buf := bufio.NewWriterSize(fd, 1<<20) // use 1MB buffer
beamlog.Infof(ctx, "Writing to %v", filename)
for _, l := range t.Leaves {
if _, err := buf.Write(l.Path); err != nil {
return err
}
if _, err := buf.Write([]byte{'\t'}); err != nil {
return err
}
if _, err := buf.Write(l.Hash); err != nil {
return err
}
if _, err := buf.Write([]byte{'\n'}); err != nil {
return err
}
}
if err := buf.Flush(); err != nil {
return err
}
return fd.Close()
}
type writeCheckpointFn struct {
RootDir string
LogCheckpoint []byte
EntryCount uint64
}
func (w *writeCheckpointFn) ProcessElement(ctx context.Context, t *batchmap.Tile) error {
if len(t.Path) > 0 {
return nil
}
root := t.RootHash
filename := path.Join(w.RootDir, "checkpoint")
fs, err := filesystem.New(ctx, filename)
if err != nil {
return err
}
defer func() {
if err := fs.Close(); err != nil {
glog.Errorf("fs.Close(): %v", err)
}
}()
fd, err := fs.OpenWrite(ctx, filename)
if err != nil {
return err
}
if _, err := fd.Write([]byte(fmt.Sprintf("%d\n%x\n", w.EntryCount, root))); err != nil {
return err
}
if _, err := fd.Write(w.LogCheckpoint); err != nil {
return err
}
// TODO(mhutchinson): Add signature to the map root.
return fd.Close()
}