github.com/google/trillian-examples@v0.0.0-20240520080811-0d40d35cef0e/experimental/batchmap/sumdb/build/pipeline/log.go (about) 1 // Copyright 2020 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 // http://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 package pipeline 16 17 import ( 18 "fmt" 19 "reflect" 20 "sort" 21 22 "github.com/apache/beam/sdks/v2/go/pkg/beam" 23 24 "github.com/google/trillian/experimental/batchmap" 25 "github.com/google/trillian/merkle/coniks" 26 "github.com/google/trillian/merkle/smt/node" 27 "github.com/transparency-dev/merkle/compact" 28 29 "golang.org/x/mod/sumdb/tlog" 30 ) 31 32 func init() { 33 beam.RegisterFunction(makeModuleVersionLogFn) 34 beam.RegisterType(reflect.TypeOf((*moduleLogHashFn)(nil)).Elem()) 35 beam.RegisterFunction(metadataKeyFn) 36 } 37 38 // ModuleVersionLog represents the versions found for a single 39 // Go Module within the SumDB log. The versions are sorted by 40 // the order they are logged in SumDB. 41 type ModuleVersionLog struct { 42 Module string 43 Versions []string 44 } 45 46 // MakeVersionLogs takes the Metadata for all modules and processes this by 47 // module in order to create logs of versions. The versions for each module 48 // are sorted (by ID in the original log), and a log is constructed for each 49 // module. This method returns two PCollections: the first is of type Entry 50 // and is the key/value data to include in the map, the second is of type 51 // ModuleVersionLog. 52 func MakeVersionLogs(s beam.Scope, treeID int64, metadata beam.PCollection) (beam.PCollection, beam.PCollection) { 53 keyed := beam.ParDo(s, metadataKeyFn, metadata) 54 logs := beam.ParDo(s, makeModuleVersionLogFn, beam.GroupByKey(s, keyed)) 55 return beam.ParDo(s, &moduleLogHashFn{TreeID: treeID}, logs), logs 56 } 57 58 func metadataKeyFn(m Metadata) (string, Metadata) { return m.Module, m } 59 60 type moduleLogHashFn struct { 61 TreeID int64 62 63 rf *compact.RangeFactory 64 } 65 66 func (fn *moduleLogHashFn) Setup() { 67 fn.rf = &compact.RangeFactory{ 68 Hash: func(left, right []byte) []byte { 69 // There is no particular need for using this hash function, but it was convenient. 70 var lHash, rHash tlog.Hash 71 copy(lHash[:], left) 72 copy(rHash[:], right) 73 thash := tlog.NodeHash(lHash, rHash) 74 return thash[:] 75 }, 76 } 77 } 78 79 func (fn *moduleLogHashFn) ProcessElement(log *ModuleVersionLog) (*batchmap.Entry, error) { 80 logRange := fn.rf.NewEmptyRange(0) 81 for _, v := range log.Versions { 82 h := tlog.RecordHash([]byte(v)) 83 if err := logRange.Append(h[:], nil); err != nil { 84 return nil, fmt.Errorf("logRange.Append(): %v", err) 85 } 86 } 87 logRoot, err := logRange.GetRootHash(nil) 88 if err != nil { 89 return nil, fmt.Errorf("failed to create log for %q: %v", log.Module, err) 90 } 91 h := hash.New() 92 h.Write([]byte(log.Module)) 93 logKey := h.Sum(nil) 94 leafID := node.NewID(string(logKey), uint(len(logKey)*8)) 95 96 return &batchmap.Entry{ 97 HashKey: h.Sum(nil), 98 HashValue: coniks.Default.HashLeaf(fn.TreeID, leafID, logRoot), 99 }, nil 100 } 101 102 func makeModuleVersionLogFn(module string, metadata func(*Metadata) bool) (*ModuleVersionLog, error) { 103 // We need to ensure ordering by ID in the original log for stability. 104 105 // First build up a map from ID to version. 106 mm := make(map[int64]string) 107 var m Metadata 108 for metadata(&m) { 109 mm[m.ID] = m.Version 110 } 111 112 // Now order the keyset. 113 keys := make([]int64, 0, len(mm)) 114 for k := range mm { 115 keys = append(keys, k) 116 } 117 sort.Slice(keys, func(i, j int) bool { return keys[i] < keys[j] }) 118 119 // Now iterate the map in the right order to build the log. 120 versions := make([]string, 0, len(keys)) 121 for _, id := range keys { 122 versions = append(versions, mm[id]) 123 } 124 125 return &ModuleVersionLog{ 126 Module: module, 127 Versions: versions, 128 }, nil 129 }