github.com/google/trillian-examples@v0.0.0-20240520080811-0d40d35cef0e/binary_transparency/firmware/internal/ftmap/pipeline.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  //     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 ftmap contains Beam pipeline library functions for the FT
    16  // verifiable map.
    17  package ftmap
    18  
    19  import (
    20  	"crypto"
    21  	"encoding/json"
    22  	"fmt"
    23  	"reflect"
    24  
    25  	"github.com/apache/beam/sdks/v2/go/pkg/beam"
    26  	"github.com/golang/glog"
    27  	"github.com/google/trillian-examples/binary_transparency/firmware/api"
    28  	"github.com/google/trillian/experimental/batchmap"
    29  )
    30  
    31  func init() {
    32  	beam.RegisterFunction(parseStatementFn)
    33  	beam.RegisterFunction(parseFirmwareFn)
    34  	beam.RegisterFunction(parseAnnotationMalwareFn)
    35  	beam.RegisterFunction(partitionFn)
    36  	beam.RegisterType(reflect.TypeOf((*InputLogMetadata)(nil)).Elem())
    37  	beam.RegisterType(reflect.TypeOf((*InputLogLeaf)(nil)).Elem())
    38  	beam.RegisterType(reflect.TypeOf((*PipelineResult)(nil)).Elem())
    39  	beam.RegisterType(reflect.TypeOf((*loggedStatement)(nil)).Elem())
    40  	beam.RegisterType(reflect.TypeOf((*firmwareLogEntry)(nil)).Elem())
    41  	beam.RegisterType(reflect.TypeOf((*annotationMalwareLogEntry)(nil)).Elem())
    42  }
    43  
    44  // InputLog allows access to entries from the FT Log.
    45  type InputLog interface {
    46  	// Head returns the metadata of available entries.
    47  	// The log checkpoint is a serialized LogRootV1.
    48  	Head() (checkpoint []byte, count int64, err error)
    49  	// Entries returns a PCollection of InputLogLeaf, containing entries in range [start, end).
    50  	Entries(s beam.Scope, start, end int64) beam.PCollection
    51  }
    52  
    53  // InputLogMetadata describes the provenance information of the input
    54  // log to be passed around atomically.
    55  type InputLogMetadata struct {
    56  	Checkpoint []byte
    57  	Entries    int64
    58  }
    59  
    60  // InputLogLeaf is a leaf in an input log, with its sequence index and data.
    61  type InputLogLeaf struct {
    62  	Seq  int64
    63  	Data []byte
    64  }
    65  
    66  // PipelineResult is returned on successful run of the pipeline. It primarily
    67  // exists to name the output and aid readability, as PCollections are untyped
    68  // in code, so having them as named fields at least aids a little.
    69  type PipelineResult struct {
    70  	MapTiles           beam.PCollection
    71  	DeviceLogs         beam.PCollection
    72  	AggregatedFirmware beam.PCollection
    73  	Metadata           InputLogMetadata
    74  }
    75  
    76  // MapBuilder contains the static configuration for a map, and allows
    77  // maps at different log sizes to be built using its methods.
    78  type MapBuilder struct {
    79  	source       InputLog
    80  	treeID       int64
    81  	prefixStrata int
    82  }
    83  
    84  // NewMapBuilder returns a MapBuilder for a map with the given configuration.
    85  func NewMapBuilder(source InputLog, treeID int64, prefixStrata int) MapBuilder {
    86  	return MapBuilder{
    87  		source:       source,
    88  		treeID:       treeID,
    89  		prefixStrata: prefixStrata,
    90  	}
    91  }
    92  
    93  // Create builds a map from scratch, using the first `size` entries in the
    94  // input log. If there aren't enough entries then it will fail.
    95  // The results of the pipeline on success are returned as a PipelineResult.
    96  func (b *MapBuilder) Create(s beam.Scope, size int64) (*PipelineResult, error) {
    97  	var tiles, logs beam.PCollection
    98  
    99  	endID, golden, err := b.getLogEnd(size)
   100  	if err != nil {
   101  		return nil, err
   102  	}
   103  
   104  	// Read the log as a collection of InputLogLeaf.
   105  	inputLeaves := b.source.Entries(s.Scope("source"), 0, endID)
   106  	// Parse these into their loggedStatements.
   107  	statements := beam.ParDo(s.Scope("parseStatement"), parseStatementFn, inputLeaves)
   108  
   109  	// Partition into:
   110  	// 0: FW Metadata
   111  	// 1: Annotation malware
   112  	// 2: Everything else
   113  	partitions := beam.Partition(s.Scope("partition"), MaxPartitions, partitionFn, statements)
   114  
   115  	fws := beam.ParDo(s, parseFirmwareFn, partitions[FirmwareMetaPartition])
   116  	ams := beam.ParDo(s, parseAnnotationMalwareFn, partitions[MalwareStatementPartition])
   117  
   118  	// Branch 1: create the logs of firmware releases.
   119  	logEntries, logs := MakeReleaseLogs(s.Scope("makeLogs"), b.treeID, fws)
   120  
   121  	// Branch 2: aggregate firmware releases with their annotations.
   122  	annotationEntries, aggregated := Aggregate(s, b.treeID, fws, ams)
   123  
   124  	// Flatten the entries together to create a single unified map.
   125  	entries := beam.Flatten(s, logEntries, annotationEntries)
   126  
   127  	glog.Infof("Creating new map revision from range [0, %d)", endID)
   128  	tiles, err = batchmap.Create(s, entries, b.treeID, crypto.SHA512_256, b.prefixStrata)
   129  
   130  	return &PipelineResult{
   131  		MapTiles:           tiles,
   132  		DeviceLogs:         logs,
   133  		AggregatedFirmware: aggregated,
   134  		Metadata: InputLogMetadata{
   135  			Checkpoint: golden,
   136  			Entries:    endID,
   137  		},
   138  	}, err
   139  }
   140  
   141  func (b *MapBuilder) getLogEnd(requiredEntries int64) (int64, []byte, error) {
   142  	golden, totalLeaves, err := b.source.Head()
   143  	if err != nil {
   144  		return 0, nil, fmt.Errorf("failed to get Head of input log: %v", err)
   145  	}
   146  
   147  	if requiredEntries < 0 {
   148  		return totalLeaves, golden, nil
   149  	}
   150  
   151  	if totalLeaves < requiredEntries {
   152  		return 0, nil, fmt.Errorf("wanted %d leaves but only %d available", requiredEntries, totalLeaves)
   153  	}
   154  
   155  	return requiredEntries, golden, nil
   156  }
   157  
   158  const (
   159  	// FirmwareMetaPartition is the partition index for partition containing FirmwareMetadata
   160  	FirmwareMetaPartition = iota
   161  	// MalwareStatementPartition is the partition index for partition containing MalwareStatement
   162  	MalwareStatementPartition
   163  	// UnclassifiedStatementPartition is the partition index for partition containing anything not classified above
   164  	UnclassifiedStatementPartition
   165  	// MaxPartitions is the total number of supported partitions. Add new partitions here.
   166  	MaxPartitions
   167  )
   168  
   169  // partitionFn partitions the statements by type.
   170  func partitionFn(s *loggedStatement) int {
   171  	switch s.Statement.Type {
   172  	case api.FirmwareMetadataType:
   173  		return FirmwareMetaPartition
   174  	case api.MalwareStatementType:
   175  		return MalwareStatementPartition
   176  	default:
   177  		return UnclassifiedStatementPartition
   178  	}
   179  }
   180  
   181  // loggedStatement is a parsed version of InputLogLeaf.
   182  type loggedStatement struct {
   183  	Index     int64
   184  	Statement api.SignedStatement
   185  }
   186  
   187  func parseStatementFn(l InputLogLeaf) (*loggedStatement, error) {
   188  	var s api.SignedStatement
   189  	if err := json.Unmarshal(l.Data, &s); err != nil {
   190  		return nil, err
   191  	}
   192  
   193  	return &loggedStatement{
   194  		Index:     l.Seq,
   195  		Statement: s,
   196  	}, nil
   197  }
   198  
   199  type firmwareLogEntry struct {
   200  	Index    int64
   201  	Firmware api.FirmwareMetadata
   202  }
   203  
   204  func parseFirmwareFn(s *loggedStatement) (*firmwareLogEntry, error) {
   205  	var m api.FirmwareMetadata
   206  	if err := json.Unmarshal(s.Statement.Statement, &m); err != nil {
   207  		return nil, err
   208  	}
   209  	return &firmwareLogEntry{
   210  		Index:    s.Index,
   211  		Firmware: m,
   212  	}, nil
   213  }
   214  
   215  type annotationMalwareLogEntry struct {
   216  	Index      int64
   217  	Annotation api.MalwareStatement
   218  }
   219  
   220  func parseAnnotationMalwareFn(s *loggedStatement) (*annotationMalwareLogEntry, error) {
   221  	var a api.MalwareStatement
   222  	if err := json.Unmarshal(s.Statement.Statement, &a); err != nil {
   223  		return nil, err
   224  	}
   225  	return &annotationMalwareLogEntry{
   226  		Index:      s.Index,
   227  		Annotation: a,
   228  	}, nil
   229  }