github.com/google/trillian-examples@v0.0.0-20240520080811-0d40d35cef0e/experimental/batchmap/ctmap/cmd/build/build.go (about) 1 // Copyright 2021 Google LLC. All Rights Reserved. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // https://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 15 // build is a tool to build a map from a given clone of a log. 16 package main 17 18 import ( 19 "bufio" 20 "context" 21 "flag" 22 "fmt" 23 "path" 24 "reflect" 25 26 "github.com/apache/beam/sdks/v2/go/pkg/beam" 27 "github.com/apache/beam/sdks/v2/go/pkg/beam/io/databaseio" 28 "github.com/apache/beam/sdks/v2/go/pkg/beam/io/filesystem" 29 "github.com/apache/beam/sdks/v2/go/pkg/beam/io/textio" 30 beamlog "github.com/apache/beam/sdks/v2/go/pkg/beam/log" 31 "github.com/apache/beam/sdks/v2/go/pkg/beam/x/beamx" 32 "github.com/golang/glog" 33 "github.com/google/trillian-examples/clone/logdb" 34 "github.com/google/trillian-examples/experimental/batchmap/ctmap/internal/pipeline" 35 "github.com/google/trillian/experimental/batchmap" 36 37 _ "github.com/go-sql-driver/mysql" 38 ) 39 40 var ( 41 mysqlURI = flag.String("mysql_log_uri", "", "URL of a MySQL database to read the log from.") 42 mapOutputRootDir = flag.String("map_output_root_dir", "", "Filesystem root for this map. Cannot be shared with other maps.") 43 treeID = flag.Int64("tree_id", 12345, "The ID of the tree. Used as a salt in hashing.") 44 prefixStrata = flag.Int("prefix_strata", 2, "The number of strata of 8-bit strata before the final strata.") 45 count = flag.Int64("count", 1, "The total number of entries starting from the beginning of the log to use") 46 ) 47 48 func init() { 49 beam.RegisterType(reflect.TypeOf((*writeTileFn)(nil)).Elem()) 50 beam.RegisterType(reflect.TypeOf((*writeCheckpointFn)(nil)).Elem()) 51 } 52 53 func main() { 54 flag.Parse() 55 beam.Init() 56 57 ctx := context.Background() 58 if len(*mapOutputRootDir) == 0 { 59 glog.Exit("Required flag 'map_output_root_dir' required") 60 } 61 if len(*mysqlURI) == 0 { 62 glog.Exit("Missing required flag 'mysql_log_uri'") 63 } 64 db, err := logdb.NewDatabase(*mysqlURI) 65 if err != nil { 66 glog.Exitf("Failed to connect to database: %q", err) 67 } 68 69 beamlog.SetLogger(&BeamGLogger{InfoLogAtVerbosity: 2}) 70 p, s := beam.NewPipelineWithRoot() 71 72 log := mySQLLog{ 73 db: db, 74 dbString: *mysqlURI, 75 } 76 mb := pipeline.NewMapBuilder(&log, *treeID, *prefixStrata) 77 r, err := mb.Create(ctx, s, *count) 78 if err != nil { 79 glog.Exitf("Failed to create pipeline: %v", err) 80 } 81 // Write out the leaf values, i.e. the logs. 82 // This currently writes a single large file containing all the results. 83 textio.Write(s, path.Join(*mapOutputRootDir, "logs.txt"), beam.ParDo(s, formatFn, r.DomainCertIndexLogs)) 84 85 // Write out all of the tiles that represent the map. 86 beam.ParDo0(s, &writeTileFn{*mapOutputRootDir}, r.MapTiles) 87 88 // Write out the map checkpoint. 89 beam.ParDo0(s, &writeCheckpointFn{ 90 RootDir: *mapOutputRootDir, 91 LogCheckpoint: r.Metadata.Checkpoint, 92 EntryCount: uint64(r.Metadata.Entries), 93 }, r.MapTiles) 94 95 glog.Info("Pipeline constructed, calling beamx.Run()") 96 // All of the above constructs the pipeline but doesn't run it. Now we run it. 97 if err := beamx.Run(ctx, p); err != nil { 98 glog.Exitf("Failed to execute job: %q", err) 99 } 100 } 101 102 // BeamGLogger allows Beam to log via the glog mechanism. 103 // This is used to allow the very verbose logging output from Beam to be switched off. 104 type BeamGLogger struct { 105 InfoLogAtVerbosity glog.Level 106 } 107 108 // Log logs. 109 func (l *BeamGLogger) Log(ctx context.Context, sev beamlog.Severity, _ int, msg string) { 110 switch sev { 111 case beamlog.SevDebug: 112 glog.V(3).Info(msg) 113 case beamlog.SevInfo: 114 glog.V(l.InfoLogAtVerbosity).Info(msg) 115 case beamlog.SevError: 116 glog.Error(msg) 117 case beamlog.SevWarn: 118 glog.Warning(msg) 119 default: 120 glog.V(5).Infof("?? %s", msg) 121 } 122 } 123 124 type mySQLLog struct { 125 db *logdb.Database 126 dbString string 127 } 128 129 // Head returns the metadata of available entries. 130 func (l *mySQLLog) Head(ctx context.Context) ([]byte, int64, error) { 131 // There may be more leaves than `size`, but any leaves at a higher index than 132 // this have not been verified, and are not committed to by the checkpoint so 133 // we can't use them in the map. 134 size, cp, _, err := l.db.GetLatestCheckpoint(ctx) 135 return cp, int64(size), err 136 } 137 138 // Entries returns a PCollection of InputLogLeaf, containing entries in range [start, end). 139 func (l *mySQLLog) Entries(s beam.Scope, start, end int64) beam.PCollection { 140 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{})) 141 } 142 143 // formatFn is a DoFn that formats a domain's log as a string. 144 func formatFn(l *pipeline.DomainCertIndexLog) string { 145 r := l.Domain 146 for _, i := range l.Indices { 147 r = fmt.Sprintf("%s,%d", r, i) 148 } 149 return r 150 } 151 152 type writeTileFn struct { 153 RootDir string 154 } 155 156 func (w *writeTileFn) ProcessElement(ctx context.Context, t *batchmap.Tile) error { 157 filename := path.Join(w.RootDir, fmt.Sprintf("tile-%x", t.Path)) 158 fs, err := filesystem.New(ctx, filename) 159 if err != nil { 160 return err 161 } 162 defer func() { 163 if err := fs.Close(); err != nil { 164 glog.Errorf("fs.Close(): %v", err) 165 } 166 }() 167 168 fd, err := fs.OpenWrite(ctx, filename) 169 if err != nil { 170 return err 171 } 172 buf := bufio.NewWriterSize(fd, 1<<20) // use 1MB buffer 173 174 beamlog.Infof(ctx, "Writing to %v", filename) 175 176 for _, l := range t.Leaves { 177 if _, err := buf.Write(l.Path); err != nil { 178 return err 179 } 180 if _, err := buf.Write([]byte{'\t'}); err != nil { 181 return err 182 } 183 if _, err := buf.Write(l.Hash); err != nil { 184 return err 185 } 186 if _, err := buf.Write([]byte{'\n'}); err != nil { 187 return err 188 } 189 } 190 191 if err := buf.Flush(); err != nil { 192 return err 193 } 194 return fd.Close() 195 } 196 197 type writeCheckpointFn struct { 198 RootDir string 199 LogCheckpoint []byte 200 EntryCount uint64 201 } 202 203 func (w *writeCheckpointFn) ProcessElement(ctx context.Context, t *batchmap.Tile) error { 204 if len(t.Path) > 0 { 205 return nil 206 } 207 root := t.RootHash 208 209 filename := path.Join(w.RootDir, "checkpoint") 210 fs, err := filesystem.New(ctx, filename) 211 if err != nil { 212 return err 213 } 214 defer func() { 215 if err := fs.Close(); err != nil { 216 glog.Errorf("fs.Close(): %v", err) 217 } 218 }() 219 220 fd, err := fs.OpenWrite(ctx, filename) 221 if err != nil { 222 return err 223 } 224 225 if _, err := fd.Write([]byte(fmt.Sprintf("%d\n%x\n", w.EntryCount, root))); err != nil { 226 return err 227 } 228 if _, err := fd.Write(w.LogCheckpoint); err != nil { 229 return err 230 } 231 232 // TODO(mhutchinson): Add signature to the map root. 233 234 return fd.Close() 235 }