From d02b74a5dcfed4bfc8f2f8e545bca4d2afabb296 Mon Sep 17 00:00:00 2001 From: Googler Date: Fri, 27 Apr 2018 10:37:02 -0700 Subject: Check in gVisor. PiperOrigin-RevId: 194583126 Change-Id: Ica1d8821a90f74e7e745962d71801c598c652463 --- pkg/state/BUILD | 77 ++++ pkg/state/decode.go | 594 ++++++++++++++++++++++++++++ pkg/state/encode.go | 454 +++++++++++++++++++++ pkg/state/encode_unsafe.go | 81 ++++ pkg/state/map.go | 221 +++++++++++ pkg/state/object.proto | 140 +++++++ pkg/state/printer.go | 188 +++++++++ pkg/state/state.go | 349 +++++++++++++++++ pkg/state/state_test.go | 719 ++++++++++++++++++++++++++++++++++ pkg/state/statefile/BUILD | 23 ++ pkg/state/statefile/statefile.go | 233 +++++++++++ pkg/state/statefile/statefile_test.go | 299 ++++++++++++++ pkg/state/stats.go | 133 +++++++ 13 files changed, 3511 insertions(+) create mode 100644 pkg/state/BUILD create mode 100644 pkg/state/decode.go create mode 100644 pkg/state/encode.go create mode 100644 pkg/state/encode_unsafe.go create mode 100644 pkg/state/map.go create mode 100644 pkg/state/object.proto create mode 100644 pkg/state/printer.go create mode 100644 pkg/state/state.go create mode 100644 pkg/state/state_test.go create mode 100644 pkg/state/statefile/BUILD create mode 100644 pkg/state/statefile/statefile.go create mode 100644 pkg/state/statefile/statefile_test.go create mode 100644 pkg/state/stats.go (limited to 'pkg/state') diff --git a/pkg/state/BUILD b/pkg/state/BUILD new file mode 100644 index 000000000..bb6415d9b --- /dev/null +++ b/pkg/state/BUILD @@ -0,0 +1,77 @@ +package(licenses = ["notice"]) # Apache 2.0 + +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_generics:defs.bzl", "go_template_instance") + +go_template_instance( + name = "addr_range", + out = "addr_range.go", + package = "state", + prefix = "addr", + template = "//pkg/segment:generic_range", + types = { + "T": "uintptr", + }, +) + +go_template_instance( + name = "addr_set", + out = "addr_set.go", + consts = { + "minDegree": "10", + }, + imports = { + "reflect": "reflect", + }, + package = "state", + prefix = "addr", + template = "//pkg/segment:generic_set", + types = { + "Key": "uintptr", + "Range": "addrRange", + "Value": "reflect.Value", + "Functions": "addrSetFunctions", + }, +) + +go_library( + name = "state", + srcs = [ + "addr_range.go", + "addr_set.go", + "decode.go", + "encode.go", + "encode_unsafe.go", + "map.go", + "printer.go", + "state.go", + "stats.go", + ], + importpath = "gvisor.googlesource.com/gvisor/pkg/state", + visibility = ["//:sandbox"], + deps = [ + ":object_go_proto", + "@com_github_golang_protobuf//proto:go_default_library", + ], +) + +proto_library( + name = "object_proto", + srcs = ["object.proto"], + visibility = ["//:sandbox"], +) + +go_proto_library( + name = "object_go_proto", + importpath = "gvisor.googlesource.com/gvisor/pkg/state/object_go_proto", + proto = ":object_proto", + visibility = ["//:sandbox"], +) + +go_test( + name = "state_test", + size = "small", + srcs = ["state_test.go"], + embed = [":state"], +) diff --git a/pkg/state/decode.go b/pkg/state/decode.go new file mode 100644 index 000000000..05758495b --- /dev/null +++ b/pkg/state/decode.go @@ -0,0 +1,594 @@ +// Copyright 2018 Google Inc. +// +// 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 +// +// 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 state + +import ( + "bytes" + "encoding/binary" + "errors" + "fmt" + "io" + "reflect" + "sort" + + "github.com/golang/protobuf/proto" + pb "gvisor.googlesource.com/gvisor/pkg/state/object_go_proto" +) + +// objectState represents an object that may be in the process of being +// decoded. Specifically, it represents either a decoded object, or an an +// interest in a future object that will be decoded. When that interest is +// registered (via register), the storage for the object will be created, but +// it will not be decoded until the object is encountered in the stream. +type objectState struct { + // id is the id for this object. + // + // If this field is zero, then this is an anonymous (unregistered, + // non-reference primitive) object. This is immutable. + id uint64 + + // obj is the object. This may or may not be valid yet, depending on + // whether complete returns true. However, regardless of whether the + // object is valid, obj contains a final storage location for the + // object. This is immutable. + // + // Note that this must be addressable (obj.Addr() must not panic). + // + // The obj passed to the decode methods below will equal this obj only + // in the case of decoding the top-level object. However, the passed + // obj may represent individual fields, elements of a slice, etc. that + // are effectively embedded within the reflect.Value below but with + // distinct types. + obj reflect.Value + + // blockedBy is the number of dependencies this object has. + blockedBy int + + // blocking is a list of the objects blocked by this one. + blocking []*objectState + + // callbacks is a set of callbacks to execute on load. + callbacks []func() + + // path is the decoding path to the object. + path recoverable +} + +// complete indicates the object is complete. +func (os *objectState) complete() bool { + return os.blockedBy == 0 && len(os.callbacks) == 0 +} + +// checkComplete checks for completion. If the object is complete, pending +// callbacks will be executed and checkComplete will be called on downstream +// objects (those depending on this one). +func (os *objectState) checkComplete(stats *Stats) { + if os.blockedBy > 0 { + return + } + + // Fire all callbacks. + for _, fn := range os.callbacks { + stats.Start(os.obj) + fn() + stats.Done() + } + os.callbacks = nil + + // Clear all blocked objects. + for _, other := range os.blocking { + other.blockedBy-- + other.checkComplete(stats) + } + os.blocking = nil +} + +// waitFor queues a dependency on the given object. +func (os *objectState) waitFor(other *objectState, callback func()) { + os.blockedBy++ + other.blocking = append(other.blocking, os) + if callback != nil { + other.callbacks = append(other.callbacks, callback) + } +} + +// findCycleFor returns when the given object is found in the blocking set. +func (os *objectState) findCycleFor(target *objectState) []*objectState { + for _, other := range os.blocking { + if other == target { + return []*objectState{target} + } else if childList := other.findCycleFor(target); childList != nil { + return append(childList, other) + } + } + return nil +} + +// findCycle finds a dependency cycle. +func (os *objectState) findCycle() []*objectState { + return append(os.findCycleFor(os), os) +} + +// decodeState is a graph of objects in the process of being decoded. +// +// The decode process involves loading the breadth-first graph generated by +// encode. This graph is read in it's entirety, ensuring that all object +// storage is complete. +// +// As the graph is being serialized, a set of completion callbacks are +// executed. These completion callbacks should form a set of acyclic subgraphs +// over the original one. After decoding is complete, the objects are scanned +// to ensure that all callbacks are executed, otherwise the callback graph was +// not acyclic. +type decodeState struct { + // objectByID is the set of objects in progress. + objectsByID map[uint64]*objectState + + // deferred are objects that have been read, by no interest has been + // registered yet. These will be decoded once interest in registered. + deferred map[uint64]*pb.Object + + // outstanding is the number of outstanding objects. + outstanding uint32 + + // r is the input stream. + r io.Reader + + // stats is the passed stats object. + stats *Stats + + // recoverable is the panic recover facility. + recoverable +} + +// lookup looks up an object in decodeState or returns nil if no such object +// has been previously registered. +func (ds *decodeState) lookup(id uint64) *objectState { + return ds.objectsByID[id] +} + +// wait registers a dependency on an object. +// +// As a special case, we always allow _useable_ references back to the first +// decoding object because it may have fields that are already decoded. We also +// allow trivial self reference, since they can be handled internally. +func (ds *decodeState) wait(waiter *objectState, id uint64, callback func()) { + switch id { + case 0: + // Nil pointer; nothing to wait for. + fallthrough + case waiter.id: + // Trivial self reference. + fallthrough + case 1: + // Root object; see above. + if callback != nil { + callback() + } + return + } + + // No nil can be returned here. + waiter.waitFor(ds.lookup(id), callback) +} + +// waitObject notes a blocking relationship. +func (ds *decodeState) waitObject(os *objectState, p *pb.Object, callback func()) { + if rv, ok := p.Value.(*pb.Object_RefValue); ok { + // Refs can encode pointers and maps. + ds.wait(os, rv.RefValue, callback) + } else if sv, ok := p.Value.(*pb.Object_SliceValue); ok { + // See decodeObject; we need to wait for the array (if non-nil). + ds.wait(os, sv.SliceValue.RefValue, callback) + } else if iv, ok := p.Value.(*pb.Object_InterfaceValue); ok { + // It's an interface (wait recurisvely). + ds.waitObject(os, iv.InterfaceValue.Value, callback) + } else if callback != nil { + // Nothing to wait for: execute the callback immediately. + callback() + } +} + +// register registers a decode with a type. +// +// This type is only used to instantiate a new object if it has not been +// registered previously. +func (ds *decodeState) register(id uint64, typ reflect.Type) *objectState { + os, ok := ds.objectsByID[id] + if ok { + return os + } + + // Record in the object index. + if typ.Kind() == reflect.Map { + os = &objectState{id: id, obj: reflect.MakeMap(typ), path: ds.recoverable.copy()} + } else { + os = &objectState{id: id, obj: reflect.New(typ).Elem(), path: ds.recoverable.copy()} + } + ds.objectsByID[id] = os + + if o, ok := ds.deferred[id]; ok { + // There is a deferred object. + delete(ds.deferred, id) // Free memory. + ds.decodeObject(os, os.obj, o, "", nil) + } else { + // There is no deferred object. + ds.outstanding++ + } + + return os +} + +// decodeStruct decodes a struct value. +func (ds *decodeState) decodeStruct(os *objectState, obj reflect.Value, s *pb.Struct) { + // Set the fields. + m := Map{newInternalMap(nil, ds, os)} + defer internalMapPool.Put(m.internalMap) + for _, field := range s.Fields { + m.data = append(m.data, entry{ + name: field.Name, + object: field.Value, + }) + } + + // Sort the fields for efficient searching. + // + // Technically, these should already appear in sorted order in the + // state ordering, so this cost is effectively a single scan to ensure + // that the order is correct. + if len(m.data) > 1 { + sort.Slice(m.data, func(i, j int) bool { + return m.data[i].name < m.data[j].name + }) + } + + // Invoke the load; this will recursively decode other objects. + fns, ok := registeredTypes.lookupFns(obj.Addr().Type()) + if ok { + // Invoke the loader. + fns.invokeLoad(obj.Addr(), m) + } else if obj.NumField() == 0 { + // Allow anonymous empty structs. + return + } else { + // Propagate an error. + panic(fmt.Errorf("unregistered type %s", obj.Type())) + } +} + +// decodeMap decodes a map value. +func (ds *decodeState) decodeMap(os *objectState, obj reflect.Value, m *pb.Map) { + if obj.IsNil() { + obj.Set(reflect.MakeMap(obj.Type())) + } + for i := 0; i < len(m.Keys); i++ { + // Decode the objects. + kv := reflect.New(obj.Type().Key()).Elem() + vv := reflect.New(obj.Type().Elem()).Elem() + ds.decodeObject(os, kv, m.Keys[i], ".(key %d)", i) + ds.decodeObject(os, vv, m.Values[i], "[%#v]", kv.Interface()) + ds.waitObject(os, m.Keys[i], nil) + ds.waitObject(os, m.Values[i], nil) + + // Set in the map. + obj.SetMapIndex(kv, vv) + } +} + +// decodeArray decodes an array value. +func (ds *decodeState) decodeArray(os *objectState, obj reflect.Value, a *pb.Array) { + if len(a.Contents) != obj.Len() { + panic(fmt.Errorf("mismatching array length expect=%d, actual=%d", obj.Len(), len(a.Contents))) + } + // Decode the contents into the array. + for i := 0; i < len(a.Contents); i++ { + ds.decodeObject(os, obj.Index(i), a.Contents[i], "[%d]", i) + ds.waitObject(os, a.Contents[i], nil) + } +} + +// decodeInterface decodes an interface value. +func (ds *decodeState) decodeInterface(os *objectState, obj reflect.Value, i *pb.Interface) { + // Is this a nil value? + if i.Type == "" { + return // Just leave obj alone. + } + + // Get the dispatchable type. This may not be used if the given + // reference has already been resolved, but if not we need to know the + // type to create. + t, ok := registeredTypes.lookupType(i.Type) + if !ok { + panic(fmt.Errorf("no valid type for %q", i.Type)) + } + + if obj.Kind() != reflect.Map { + // Set the obj to be the given typed value; this actually sets + // obj to be a non-zero value -- namely, it inserts type + // information. There's no need to do this for maps. + obj.Set(reflect.Zero(t)) + } + + // Decode the dereferenced element; there is no need to wait here, as + // the interface object shares the current object state. + ds.decodeObject(os, obj, i.Value, ".(%s)", i.Type) +} + +// decodeObject decodes a object value. +func (ds *decodeState) decodeObject(os *objectState, obj reflect.Value, object *pb.Object, format string, param interface{}) { + ds.push(false, format, param) + ds.stats.Start(obj) + + switch x := object.GetValue().(type) { + case *pb.Object_BoolValue: + obj.SetBool(x.BoolValue) + case *pb.Object_StringValue: + obj.SetString(x.StringValue) + case *pb.Object_Int64Value: + obj.SetInt(x.Int64Value) + if obj.Int() != x.Int64Value { + panic(fmt.Errorf("signed integer truncated in %v for %s", object, obj.Type())) + } + case *pb.Object_Uint64Value: + obj.SetUint(x.Uint64Value) + if obj.Uint() != x.Uint64Value { + panic(fmt.Errorf("unsigned integer truncated in %v for %s", object, obj.Type())) + } + case *pb.Object_DoubleValue: + obj.SetFloat(x.DoubleValue) + if obj.Float() != x.DoubleValue { + panic(fmt.Errorf("float truncated in %v for %s", object, obj.Type())) + } + case *pb.Object_RefValue: + // Resolve the pointer itself, even though the object may not + // be decoded yet. You need to use wait() in order to ensure + // that is the case. See wait above, and Map.Barrier. + if id := x.RefValue; id != 0 { + // Decoding the interface should have imparted type + // information, so from this point it's safe to resolve + // and use this dynamic information for actually + // creating the object in register. + // + // (For non-interfaces this is a no-op). + dyntyp := reflect.TypeOf(obj.Interface()) + if dyntyp.Kind() == reflect.Map { + obj.Set(ds.register(id, dyntyp).obj) + } else if dyntyp.Kind() == reflect.Ptr { + ds.push(true /* dereference */, "", nil) + obj.Set(ds.register(id, dyntyp.Elem()).obj.Addr()) + ds.pop() + } else { + obj.Set(ds.register(id, dyntyp.Elem()).obj.Addr()) + } + } else { + // We leave obj alone here. That's because if obj + // represents an interface, it may have been embued + // with type information in decodeInterface, and we + // don't want to destroy that information. + } + case *pb.Object_SliceValue: + // It's okay to slice the array here, since the contents will + // still be provided later on. These semantics are a bit + // strange but they are handled in the Map.Barrier properly. + // + // The special semantics of zero ref apply here too. + if id := x.SliceValue.RefValue; id != 0 && x.SliceValue.Capacity > 0 { + v := reflect.ArrayOf(int(x.SliceValue.Capacity), obj.Type().Elem()) + obj.Set(ds.register(id, v).obj.Slice3(0, int(x.SliceValue.Length), int(x.SliceValue.Capacity))) + } + case *pb.Object_ArrayValue: + ds.decodeArray(os, obj, x.ArrayValue) + case *pb.Object_StructValue: + ds.decodeStruct(os, obj, x.StructValue) + case *pb.Object_MapValue: + ds.decodeMap(os, obj, x.MapValue) + case *pb.Object_InterfaceValue: + ds.decodeInterface(os, obj, x.InterfaceValue) + case *pb.Object_ByteArrayValue: + copyArray(obj, reflect.ValueOf(x.ByteArrayValue)) + case *pb.Object_Uint16ArrayValue: + // 16-bit slices are serialized as 32-bit slices. + // See object.proto for details. + s := x.Uint16ArrayValue.Values + t := obj.Slice(0, obj.Len()).Interface().([]uint16) + if len(t) != len(s) { + panic(fmt.Errorf("mismatching array length expect=%d, actual=%d", len(t), len(s))) + } + for i := range s { + t[i] = uint16(s[i]) + } + case *pb.Object_Uint32ArrayValue: + copyArray(obj, reflect.ValueOf(x.Uint32ArrayValue.Values)) + case *pb.Object_Uint64ArrayValue: + copyArray(obj, reflect.ValueOf(x.Uint64ArrayValue.Values)) + case *pb.Object_UintptrArrayValue: + copyArray(obj, castSlice(reflect.ValueOf(x.UintptrArrayValue.Values), reflect.TypeOf(uintptr(0)))) + case *pb.Object_Int8ArrayValue: + copyArray(obj, castSlice(reflect.ValueOf(x.Int8ArrayValue.Values), reflect.TypeOf(int8(0)))) + case *pb.Object_Int16ArrayValue: + // 16-bit slices are serialized as 32-bit slices. + // See object.proto for details. + s := x.Int16ArrayValue.Values + t := obj.Slice(0, obj.Len()).Interface().([]int16) + if len(t) != len(s) { + panic(fmt.Errorf("mismatching array length expect=%d, actual=%d", len(t), len(s))) + } + for i := range s { + t[i] = int16(s[i]) + } + case *pb.Object_Int32ArrayValue: + copyArray(obj, reflect.ValueOf(x.Int32ArrayValue.Values)) + case *pb.Object_Int64ArrayValue: + copyArray(obj, reflect.ValueOf(x.Int64ArrayValue.Values)) + case *pb.Object_BoolArrayValue: + copyArray(obj, reflect.ValueOf(x.BoolArrayValue.Values)) + case *pb.Object_Float64ArrayValue: + copyArray(obj, reflect.ValueOf(x.Float64ArrayValue.Values)) + case *pb.Object_Float32ArrayValue: + copyArray(obj, reflect.ValueOf(x.Float32ArrayValue.Values)) + default: + // Shoud not happen, not propagated as an error. + panic(fmt.Sprintf("unknown object %v for %s", object, obj.Type())) + } + + ds.stats.Done() + ds.pop() +} + +func copyArray(dest reflect.Value, src reflect.Value) { + if dest.Len() != src.Len() { + panic(fmt.Errorf("mismatching array length expect=%d, actual=%d", dest.Len(), src.Len())) + } + reflect.Copy(dest, castSlice(src, dest.Type().Elem())) +} + +// Deserialize deserializes the object state. +// +// This function may panic and should be run in safely(). +func (ds *decodeState) Deserialize(obj reflect.Value) { + ds.objectsByID[1] = &objectState{id: 1, obj: obj, path: ds.recoverable.copy()} + ds.outstanding = 1 // The root object. + + // Decode all objects in the stream. + // + // See above, we never process objects while we have no outstanding + // interests (other than the very first object). + for id := uint64(1); ds.outstanding > 0; id++ { + o, err := ds.readObject() + if err != nil { + panic(err) + } + + os := ds.lookup(id) + if os != nil { + // Decode the object. + ds.from = &os.path + ds.decodeObject(os, os.obj, o, "", nil) + ds.outstanding-- + } else { + // If an object hasn't had interest registered + // previously, we deferred decoding until interest is + // registered. + ds.deferred[id] = o + } + } + + // Check the zero-length header at the end. + length, object, err := ReadHeader(ds.r) + if err != nil { + panic(err) + } + if length != 0 { + panic(fmt.Sprintf("expected zero-length terminal, got %d", length)) + } + if object { + panic("expected non-object terminal") + } + + // Check if we have any deferred objects. + if count := len(ds.deferred); count > 0 { + // Shoud not happen, not propagated as an error. + panic(fmt.Sprintf("still have %d deferred objects", count)) + } + + // Scan and fire all callbacks. + for _, os := range ds.objectsByID { + os.checkComplete(ds.stats) + } + + // Check if we have any remaining dependency cycles. + for _, os := range ds.objectsByID { + if !os.complete() { + // This must be the result of a dependency cycle. + cycle := os.findCycle() + var buf bytes.Buffer + buf.WriteString("dependency cycle: {") + for i, cycleOS := range cycle { + if i > 0 { + buf.WriteString(" => ") + } + buf.WriteString(fmt.Sprintf("%s", cycleOS.obj.Type())) + } + buf.WriteString("}") + // Panic as an error; propagate to the caller. + panic(errors.New(string(buf.Bytes()))) + } + } +} + +type byteReader struct { + io.Reader +} + +// ReadByte implements io.ByteReader. +func (br byteReader) ReadByte() (byte, error) { + var b [1]byte + n, err := br.Reader.Read(b[:]) + if n > 0 { + return b[0], nil + } else if err != nil { + return 0, err + } else { + return 0, io.ErrUnexpectedEOF + } +} + +// ReadHeader reads an object header. +// +// Each object written to the statefile is prefixed with a header. See +// WriteHeader for more information; these functions are exported to allow +// non-state writes to the file to play nice with debugging tools. +func ReadHeader(r io.Reader) (length uint64, object bool, err error) { + // Read the header. + length, err = binary.ReadUvarint(byteReader{r}) + if err != nil { + return + } + + // Decode whether the object is valid. + object = length&0x1 != 0 + length = length >> 1 + return +} + +// readObject reads an object from the stream. +func (ds *decodeState) readObject() (*pb.Object, error) { + // Read the header. + length, object, err := ReadHeader(ds.r) + if err != nil { + return nil, err + } + if !object { + return nil, fmt.Errorf("invalid object header") + } + + // Read the object. + buf := make([]byte, length) + for done := 0; done < len(buf); { + n, err := ds.r.Read(buf[done:]) + done += n + if n == 0 && err != nil { + return nil, err + } + } + + // Unmarshal. + obj := new(pb.Object) + if err := proto.Unmarshal(buf, obj); err != nil { + return nil, err + } + + return obj, nil +} diff --git a/pkg/state/encode.go b/pkg/state/encode.go new file mode 100644 index 000000000..eb6527afc --- /dev/null +++ b/pkg/state/encode.go @@ -0,0 +1,454 @@ +// Copyright 2018 Google Inc. +// +// 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 +// +// 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 state + +import ( + "container/list" + "encoding/binary" + "fmt" + "io" + "reflect" + "sort" + + "github.com/golang/protobuf/proto" + pb "gvisor.googlesource.com/gvisor/pkg/state/object_go_proto" +) + +// queuedObject is an object queued for encoding. +type queuedObject struct { + id uint64 + obj reflect.Value + path recoverable +} + +// encodeState is state used for encoding. +// +// The encoding process is a breadth-first traversal of the object graph. The +// inherent races and dependencies are much simpler than the decode case. +type encodeState struct { + // lastID is the last object ID. + // + // See idsByObject for context. Because of the special zero encoding + // used for reference values, the first ID must be 1. + lastID uint64 + + // idsByObject is a set of objects, indexed via: + // + // reflect.ValueOf(x).UnsafeAddr + // + // This provides IDs for objects. + idsByObject map[uintptr]uint64 + + // values stores values that span the addresses. + // + // addrSet is a a generated type which efficiently stores ranges of + // addresses. When encoding pointers, these ranges are filled in and + // used to check for overlapping or conflicting pointers. This would + // indicate a pointer to an field, or a non-type safe value, neither of + // which are currently decodable. + // + // See the usage of values below for more context. + values addrSet + + // w is the output stream. + w io.Writer + + // pending is the list of objects to be serialized. + // + // This is a set of queuedObjects. + pending list.List + + // done is the a list of finished objects. + // + // This is kept to prevent garbage collection and address reuse. + done list.List + + // stats is the passed stats object. + stats *Stats + + // recoverable is the panic recover facility. + recoverable +} + +// register looks up an ID, registering if necessary. +// +// If the object was not previosly registered, it is enqueued to be serialized. +// See the documentation for idsByObject for more information. +func (es *encodeState) register(obj reflect.Value) uint64 { + // It is not legal to call register for any non-pointer objects (see + // below), so we panic with a recoverable error if this is a mismatch. + if obj.Kind() != reflect.Ptr && obj.Kind() != reflect.Map { + panic(fmt.Errorf("non-pointer %#v registered", obj.Interface())) + } + + addr := obj.Pointer() + if obj.Kind() == reflect.Ptr && obj.Elem().Type().Size() == 0 { + // For zero-sized objects, we always provide a unique ID. + // That's because the runtime internally multiplexes pointers + // to the same address. We can't be certain what the intent is + // with pointers to zero-sized objects, so we just give them + // all unique identities. + } else if id, ok := es.idsByObject[addr]; ok { + // Already registered. + return id + } + + // Ensure that the first ID given out is one. See note on lastID. The + // ID zero is used to indicate nil values. + es.lastID++ + id := es.lastID + es.idsByObject[addr] = id + if obj.Kind() == reflect.Ptr { + // Dereference and treat as a pointer. + es.pending.PushBack(queuedObject{id: id, obj: obj.Elem(), path: es.recoverable.copy()}) + + // Register this object at all addresses. + typ := obj.Elem().Type() + if size := typ.Size(); size > 0 { + r := addrRange{addr, addr + size} + if !es.values.IsEmptyRange(r) { + panic(fmt.Errorf("overlapping objects: [new object] %#v [existing object] %#v", obj.Interface(), es.values.FindSegment(addr).Value().Elem().Interface())) + } + es.values.Add(r, obj) + } + } else { + // Push back the map itself; when maps are encoded from the + // top-level, forceMap will be equal to true. + es.pending.PushBack(queuedObject{id: id, obj: obj, path: es.recoverable.copy()}) + } + + return id +} + +// encodeMap encodes a map. +func (es *encodeState) encodeMap(obj reflect.Value) *pb.Map { + var ( + keys []*pb.Object + values []*pb.Object + ) + for i, k := range obj.MapKeys() { + v := obj.MapIndex(k) + kp := es.encodeObject(k, false, ".(key %d)", i) + vp := es.encodeObject(v, false, "[%#v]", k.Interface()) + keys = append(keys, kp) + values = append(values, vp) + } + return &pb.Map{Keys: keys, Values: values} +} + +// encodeStruct encodes a composite object. +func (es *encodeState) encodeStruct(obj reflect.Value) *pb.Struct { + // Invoke the save. + m := Map{newInternalMap(es, nil, nil)} + defer internalMapPool.Put(m.internalMap) + if !obj.CanAddr() { + // Force it to a * type of the above; this involves a copy. + localObj := reflect.New(obj.Type()) + localObj.Elem().Set(obj) + obj = localObj.Elem() + } + fns, ok := registeredTypes.lookupFns(obj.Addr().Type()) + if ok { + // Invoke the provided saver. + fns.invokeSave(obj.Addr(), m) + } else if obj.NumField() == 0 { + // Allow unregistered anonymous, empty structs. + return &pb.Struct{} + } else { + // Propagate an error. + panic(fmt.Errorf("unregistered type %T", obj.Interface())) + } + + // Sort the underlying slice, and check for duplicates. This is done + // once instead of on each add, because performing this sort once is + // far more efficient. + if len(m.data) > 1 { + sort.Slice(m.data, func(i, j int) bool { + return m.data[i].name < m.data[j].name + }) + for i := range m.data { + if i > 0 && m.data[i-1].name == m.data[i].name { + panic(fmt.Errorf("duplicate name %s", m.data[i].name)) + } + } + } + + // Encode the resulting fields. + fields := make([]*pb.Field, 0, len(m.data)) + for _, e := range m.data { + fields = append(fields, &pb.Field{ + Name: e.name, + Value: e.object, + }) + } + + // Return the encoded object. + return &pb.Struct{Fields: fields} +} + +// encodeArray encodes an array. +func (es *encodeState) encodeArray(obj reflect.Value) *pb.Array { + var ( + contents []*pb.Object + ) + for i := 0; i < obj.Len(); i++ { + entry := es.encodeObject(obj.Index(i), false, "[%d]", i) + contents = append(contents, entry) + } + return &pb.Array{Contents: contents} +} + +// encodeInterface encodes an interface. +// +// Precondition: the value is not nil. +func (es *encodeState) encodeInterface(obj reflect.Value) *pb.Interface { + // Check for the nil interface. + obj = reflect.ValueOf(obj.Interface()) + if !obj.IsValid() { + return &pb.Interface{ + Type: "", // left alone in decode. + Value: &pb.Object{Value: &pb.Object_RefValue{0}}, + } + } + // We have an interface value here. How do we save that? We + // resolve the underlying type and save it as a dispatchable. + typName, ok := registeredTypes.lookupName(obj.Type()) + if !ok { + panic(fmt.Errorf("type %s is not registered", obj.Type())) + } + + // Encode the object again. + return &pb.Interface{ + Type: typName, + Value: es.encodeObject(obj, false, ".(%s)", typName), + } +} + +// encodeObject encodes an object. +// +// If mapAsValue is true, then a map will be encoded directly. +func (es *encodeState) encodeObject(obj reflect.Value, mapAsValue bool, format string, param interface{}) (object *pb.Object) { + es.push(false, format, param) + es.stats.Start(obj) + + switch obj.Kind() { + case reflect.Bool: + object = &pb.Object{Value: &pb.Object_BoolValue{obj.Bool()}} + case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64: + object = &pb.Object{Value: &pb.Object_Int64Value{obj.Int()}} + case reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64, reflect.Uintptr: + object = &pb.Object{Value: &pb.Object_Uint64Value{obj.Uint()}} + case reflect.Float32, reflect.Float64: + object = &pb.Object{Value: &pb.Object_DoubleValue{obj.Float()}} + case reflect.Array: + switch obj.Type().Elem().Kind() { + case reflect.Uint8: + object = &pb.Object{Value: &pb.Object_ByteArrayValue{pbSlice(obj).Interface().([]byte)}} + case reflect.Uint16: + // 16-bit slices are serialized as 32-bit slices. + // See object.proto for details. + s := pbSlice(obj).Interface().([]uint16) + t := make([]uint32, len(s)) + for i := range s { + t[i] = uint32(s[i]) + } + object = &pb.Object{Value: &pb.Object_Uint16ArrayValue{&pb.Uint16S{Values: t}}} + case reflect.Uint32: + object = &pb.Object{Value: &pb.Object_Uint32ArrayValue{&pb.Uint32S{Values: pbSlice(obj).Interface().([]uint32)}}} + case reflect.Uint64: + object = &pb.Object{Value: &pb.Object_Uint64ArrayValue{&pb.Uint64S{Values: pbSlice(obj).Interface().([]uint64)}}} + case reflect.Uintptr: + object = &pb.Object{Value: &pb.Object_UintptrArrayValue{&pb.Uintptrs{Values: pbSlice(obj).Interface().([]uint64)}}} + case reflect.Int8: + object = &pb.Object{Value: &pb.Object_Int8ArrayValue{&pb.Int8S{Values: pbSlice(obj).Interface().([]byte)}}} + case reflect.Int16: + // 16-bit slices are serialized as 32-bit slices. + // See object.proto for details. + s := pbSlice(obj).Interface().([]int16) + t := make([]int32, len(s)) + for i := range s { + t[i] = int32(s[i]) + } + object = &pb.Object{Value: &pb.Object_Int16ArrayValue{&pb.Int16S{Values: t}}} + case reflect.Int32: + object = &pb.Object{Value: &pb.Object_Int32ArrayValue{&pb.Int32S{Values: pbSlice(obj).Interface().([]int32)}}} + case reflect.Int64: + object = &pb.Object{Value: &pb.Object_Int64ArrayValue{&pb.Int64S{Values: pbSlice(obj).Interface().([]int64)}}} + case reflect.Bool: + object = &pb.Object{Value: &pb.Object_BoolArrayValue{&pb.Bools{Values: pbSlice(obj).Interface().([]bool)}}} + case reflect.Float32: + object = &pb.Object{Value: &pb.Object_Float32ArrayValue{&pb.Float32S{Values: pbSlice(obj).Interface().([]float32)}}} + case reflect.Float64: + object = &pb.Object{Value: &pb.Object_Float64ArrayValue{&pb.Float64S{Values: pbSlice(obj).Interface().([]float64)}}} + default: + object = &pb.Object{Value: &pb.Object_ArrayValue{es.encodeArray(obj)}} + } + case reflect.Slice: + if obj.IsNil() || obj.Cap() == 0 { + // Handled specially in decode; store as nil value. + object = &pb.Object{Value: &pb.Object_RefValue{0}} + } else { + // Serialize a slice as the array plus length and capacity. + object = &pb.Object{Value: &pb.Object_SliceValue{&pb.Slice{ + Capacity: uint32(obj.Cap()), + Length: uint32(obj.Len()), + RefValue: es.register(arrayFromSlice(obj)), + }}} + } + case reflect.String: + object = &pb.Object{Value: &pb.Object_StringValue{obj.String()}} + case reflect.Ptr: + if obj.IsNil() { + // Handled specially in decode; store as a nil value. + object = &pb.Object{Value: &pb.Object_RefValue{0}} + } else { + es.push(true /* dereference */, "", nil) + object = &pb.Object{Value: &pb.Object_RefValue{es.register(obj)}} + es.pop() + } + case reflect.Interface: + // We don't check for IsNil here, as we want to encode type + // information. The case of the empty interface (no type, no + // value) is handled by encodeInteface. + object = &pb.Object{Value: &pb.Object_InterfaceValue{es.encodeInterface(obj)}} + case reflect.Struct: + object = &pb.Object{Value: &pb.Object_StructValue{es.encodeStruct(obj)}} + case reflect.Map: + if obj.IsNil() { + // Handled specially in decode; store as a nil value. + object = &pb.Object{Value: &pb.Object_RefValue{0}} + } else if mapAsValue { + // Encode the map directly. + object = &pb.Object{Value: &pb.Object_MapValue{es.encodeMap(obj)}} + } else { + // Encode a reference to the map. + object = &pb.Object{Value: &pb.Object_RefValue{es.register(obj)}} + } + default: + panic(fmt.Errorf("unknown primitive %#v", obj.Interface())) + } + + es.stats.Done() + es.pop() + return +} + +// Serialize serializes the object state. +// +// This function may panic and should be run in safely(). +func (es *encodeState) Serialize(obj reflect.Value) { + es.register(obj.Addr()) + + // Pop off the list until we're done. + for es.pending.Len() > 0 { + e := es.pending.Front() + es.pending.Remove(e) + + // Extract the queued object. + qo := e.Value.(queuedObject) + es.from = &qo.path + o := es.encodeObject(qo.obj, true, "", nil) + + // Emit to our output stream. + if err := es.writeObject(qo.id, o); err != nil { + panic(err) + } + + // Mark as done. + es.done.PushBack(e) + } + + // Write a zero-length terminal at the end; this is a sanity check + // applied at decode time as well (see decode.go). + if err := WriteHeader(es.w, 0, false); err != nil { + panic(err) + } +} + +// WriteHeader writes a header. +// +// Each object written to the statefile should be prefixed with a header. In +// order to generate statefiles that play nicely with debugging tools, raw +// writes should be prefixed with a header with object set to false and the +// appropriate length. This will allow tools to skip these regions. +func WriteHeader(w io.Writer, length uint64, object bool) error { + // The lowest-order bit encodes whether this is a valid object. This is + // a purely internal convention, but allows the object flag to be + // returned from ReadHeader. + length = length << 1 + if object { + length |= 0x1 + } + + // Write a header. + var hdr [32]byte + encodedLen := binary.PutUvarint(hdr[:], length) + for done := 0; done < encodedLen; { + n, err := w.Write(hdr[done:encodedLen]) + done += n + if n == 0 && err != nil { + return err + } + } + + return nil +} + +// writeObject writes an object to the stream. +func (es *encodeState) writeObject(id uint64, obj *pb.Object) error { + // Marshal the proto. + buf, err := proto.Marshal(obj) + if err != nil { + return err + } + + // Write the object header. + if err := WriteHeader(es.w, uint64(len(buf)), true); err != nil { + return err + } + + // Write the object. + for done := 0; done < len(buf); { + n, err := es.w.Write(buf[done:]) + done += n + if n == 0 && err != nil { + return err + } + } + + return nil +} + +// addrSetFunctions is used by addrSet. +type addrSetFunctions struct{} + +func (addrSetFunctions) MinKey() uintptr { + return 0 +} + +func (addrSetFunctions) MaxKey() uintptr { + return ^uintptr(0) +} + +func (addrSetFunctions) ClearValue(val *reflect.Value) { +} + +func (addrSetFunctions) Merge(_ addrRange, val1 reflect.Value, _ addrRange, val2 reflect.Value) (reflect.Value, bool) { + return val1, val1 == val2 +} + +func (addrSetFunctions) Split(_ addrRange, val reflect.Value, _ uintptr) (reflect.Value, reflect.Value) { + return val, val +} diff --git a/pkg/state/encode_unsafe.go b/pkg/state/encode_unsafe.go new file mode 100644 index 000000000..d96ba56d4 --- /dev/null +++ b/pkg/state/encode_unsafe.go @@ -0,0 +1,81 @@ +// Copyright 2018 Google Inc. +// +// 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 +// +// 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 state + +import ( + "reflect" + "unsafe" +) + +// arrayFromSlice constructs a new pointer to the slice data. +// +// It would be similar to the following: +// +// x := make([]Foo, l, c) +// a := ([l]Foo*)(unsafe.Pointer(x[0])) +// +func arrayFromSlice(obj reflect.Value) reflect.Value { + return reflect.NewAt( + reflect.ArrayOf(obj.Cap(), obj.Type().Elem()), + unsafe.Pointer(obj.Pointer())) +} + +// pbSlice returns a protobuf-supported slice of the array and erase the +// original element type (which could be a defined type or non-supported type). +func pbSlice(obj reflect.Value) reflect.Value { + var typ reflect.Type + switch obj.Type().Elem().Kind() { + case reflect.Uint8: + typ = reflect.TypeOf(byte(0)) + case reflect.Uint16: + typ = reflect.TypeOf(uint16(0)) + case reflect.Uint32: + typ = reflect.TypeOf(uint32(0)) + case reflect.Uint64: + typ = reflect.TypeOf(uint64(0)) + case reflect.Uintptr: + typ = reflect.TypeOf(uint64(0)) + case reflect.Int8: + typ = reflect.TypeOf(byte(0)) + case reflect.Int16: + typ = reflect.TypeOf(int16(0)) + case reflect.Int32: + typ = reflect.TypeOf(int32(0)) + case reflect.Int64: + typ = reflect.TypeOf(int64(0)) + case reflect.Bool: + typ = reflect.TypeOf(bool(false)) + case reflect.Float32: + typ = reflect.TypeOf(float32(0)) + case reflect.Float64: + typ = reflect.TypeOf(float64(0)) + default: + panic("slice element is not of basic value type") + } + return reflect.NewAt( + reflect.ArrayOf(obj.Len(), typ), + unsafe.Pointer(obj.Slice(0, obj.Len()).Pointer()), + ).Elem().Slice(0, obj.Len()) +} + +func castSlice(obj reflect.Value, elemTyp reflect.Type) reflect.Value { + if obj.Type().Elem().Size() != elemTyp.Size() { + panic("cannot cast slice into other element type of different size") + } + return reflect.NewAt( + reflect.ArrayOf(obj.Len(), elemTyp), + unsafe.Pointer(obj.Slice(0, obj.Len()).Pointer()), + ).Elem() +} diff --git a/pkg/state/map.go b/pkg/state/map.go new file mode 100644 index 000000000..c3d165501 --- /dev/null +++ b/pkg/state/map.go @@ -0,0 +1,221 @@ +// Copyright 2018 Google Inc. +// +// 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 +// +// 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 state + +import ( + "fmt" + "reflect" + "sort" + "sync" + + pb "gvisor.googlesource.com/gvisor/pkg/state/object_go_proto" +) + +// entry is a single map entry. +type entry struct { + name string + object *pb.Object +} + +// internalMap is the internal Map state. +// +// These are recycled via a pool to avoid churn. +type internalMap struct { + // es is encodeState. + es *encodeState + + // ds is decodeState. + ds *decodeState + + // os is current object being decoded. + // + // This will always be nil during encode. + os *objectState + + // data stores the encoded values. + data []entry +} + +var internalMapPool = sync.Pool{ + New: func() interface{} { + return new(internalMap) + }, +} + +// newInternalMap returns a cached map. +func newInternalMap(es *encodeState, ds *decodeState, os *objectState) *internalMap { + m := internalMapPool.Get().(*internalMap) + m.es = es + m.ds = ds + m.os = os + if m.data != nil { + m.data = m.data[:0] + } + return m +} + +// Map is a generic state container. +// +// This is the object passed to Save and Load in order to store their state. +// +// Detailed documentation is available in individual methods. +type Map struct { + *internalMap +} + +// Save adds the given object to the map. +// +// You should pass always pointers to the object you are saving. For example: +// +// type X struct { +// A int +// B *int +// } +// +// func (x *X) Save(m Map) { +// m.Save("A", &x.A) +// m.Save("B", &x.B) +// } +// +// func (x *X) Load(m Map) { +// m.Load("A", &x.A) +// m.Load("B", &x.B) +// } +func (m Map) Save(name string, objPtr interface{}) { + m.save(name, reflect.ValueOf(objPtr).Elem(), ".%s") +} + +// SaveValue adds the given object value to the map. +// +// This should be used for values where pointers are not available, or casts +// are required during Save/Load. +// +// For example, if we want to cast external package type P.Foo to int64: +// +// type X struct { +// A P.Foo +// } +// +// func (x *X) Save(m Map) { +// m.SaveValue("A", int64(x.A)) +// } +// +// func (x *X) Load(m Map) { +// m.LoadValue("A", new(int64), func(x interface{}) { +// x.A = P.Foo(x.(int64)) +// }) +// } +func (m Map) SaveValue(name string, obj interface{}) { + m.save(name, reflect.ValueOf(obj), ".(value %s)") +} + +// save is helper for the above. It takes the name of value to save the field +// to, the field object (obj), and a format string that specifies how the +// field's saving logic is dispatched from the struct (normal, value, etc.). The +// format string should expect one string parameter, which is the name of the +// field. +func (m Map) save(name string, obj reflect.Value, format string) { + if m.es == nil { + // Not currently encoding. + m.Failf("no encode state for %q", name) + } + + // Attempt the encode. + // + // These are sorted at the end, after all objects are added and will be + // sorted and checked for duplicates (see encodeStruct). + m.data = append(m.data, entry{ + name: name, + object: m.es.encodeObject(obj, false, format, name), + }) +} + +// Load loads the given object from the map. +// +// See Save for an example. +func (m Map) Load(name string, objPtr interface{}) { + m.load(name, reflect.ValueOf(objPtr), false, nil, ".%s") +} + +// LoadWait loads the given objects from the map, and marks it as requiring all +// AfterLoad executions to complete prior to running this object's AfterLoad. +// +// See Save for an example. +func (m Map) LoadWait(name string, objPtr interface{}) { + m.load(name, reflect.ValueOf(objPtr), true, nil, ".(wait %s)") +} + +// LoadValue loads the given object value from the map. +// +// See SaveValue for an example. +func (m Map) LoadValue(name string, objPtr interface{}, fn func(interface{})) { + o := reflect.ValueOf(objPtr) + m.load(name, o, true, func() { fn(o.Elem().Interface()) }, ".(value %s)") +} + +// load is helper for the above. It takes the name of value to load the field +// from, the target field pointer (objPtr), whether load completion of the +// struct depends on the field's load completion (wait), the load completion +// logic (fn), and a format string that specifies how the field's loading logic +// is dispatched from the struct (normal, wait, value, etc.). The format string +// should expect one string parameter, which is the name of the field. +func (m Map) load(name string, objPtr reflect.Value, wait bool, fn func(), format string) { + if m.ds == nil { + // Not currently decoding. + m.Failf("no decode state for %q", name) + } + + // Find the object. + // + // These are sorted up front (and should appear in the state file + // sorted as well), so we can do a binary search here to ensure that + // large structs don't behave badly. + i := sort.Search(len(m.data), func(i int) bool { + return m.data[i].name >= name + }) + if i >= len(m.data) || m.data[i].name != name { + // There is no data for this name? + m.Failf("no data found for %q", name) + } + + // Perform the decode. + m.ds.decodeObject(m.os, objPtr.Elem(), m.data[i].object, format, name) + if wait { + // Mark this individual object a blocker. + m.ds.waitObject(m.os, m.data[i].object, fn) + } +} + +// Failf fails the save or restore with the provided message. Processing will +// stop after calling Failf, as the state package uses a panic & recover +// mechanism for state errors. You should defer any cleanup required. +func (m Map) Failf(format string, args ...interface{}) { + panic(fmt.Errorf(format, args...)) +} + +// AfterLoad schedules a function execution when all objects have been allocated +// and their automated loading and customized load logic have been executed. fn +// will not be executed until all of current object's dependencies' AfterLoad() +// logic, if exist, have been executed. +func (m Map) AfterLoad(fn func()) { + if m.ds == nil { + // Not currently decoding. + m.Failf("not decoding") + } + + // Queue the local callback; this will execute when all of the above + // data dependencies have been cleared. + m.os.callbacks = append(m.os.callbacks, fn) +} diff --git a/pkg/state/object.proto b/pkg/state/object.proto new file mode 100644 index 000000000..6595c5519 --- /dev/null +++ b/pkg/state/object.proto @@ -0,0 +1,140 @@ +// Copyright 2018 Google Inc. +// +// 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 +// +// 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. + +syntax = "proto3"; + +package gvisor.state.statefile; + +// Slice is a slice value. +message Slice { + uint32 length = 1; + uint32 capacity = 2; + uint64 ref_value = 3; +} + +// Array is an array value. +message Array { + repeated Object contents = 1; +} + +// Map is a map value. +message Map { + repeated Object keys = 1; + repeated Object values = 2; +} + +// Interface is an interface value. +message Interface { + string type = 1; + Object value = 2; +} + +// Struct is a basic composite value. +message Struct { + repeated Field fields = 1; +} + +// Field encodes a single field. +message Field { + string name = 1; + Object value = 2; +} + +// Uint16s encodes an uint16 array. To be used inside oneof structure. +message Uint16s { + // There is no 16-bit type in protobuf so we use variable length 32-bit here. + repeated uint32 values = 1; +} + +// Uint32s encodes an uint32 array. To be used inside oneof structure. +message Uint32s { + repeated fixed32 values = 1; +} + +// Uint64s encodes an uint64 array. To be used inside oneof structure. +message Uint64s { + repeated fixed64 values = 1; +} + +// Uintptrs encodes an uintptr array. To be used inside oneof structure. +message Uintptrs { + repeated fixed64 values = 1; +} + +// Int8s encodes an int8 array. To be used inside oneof structure. +message Int8s { + bytes values = 1; +} + +// Int16s encodes an int16 array. To be used inside oneof structure. +message Int16s { + // There is no 16-bit type in protobuf so we use variable length 32-bit here. + repeated int32 values = 1; +} + +// Int32s encodes an int32 array. To be used inside oneof structure. +message Int32s { + repeated sfixed32 values = 1; +} + +// Int64s encodes an int64 array. To be used inside oneof structure. +message Int64s { + repeated sfixed64 values = 1; +} + +// Bools encodes a boolean array. To be used inside oneof structure. +message Bools { + repeated bool values = 1; +} + +// Float64s encodes a float64 array. To be used inside oneof structure. +message Float64s { + repeated double values = 1; +} + +// Float32s encodes a float32 array. To be used inside oneof structure. +message Float32s { + repeated float values = 1; +} + +// Object are primitive encodings. +// +// Note that ref_value references an Object.id, below. +message Object { + oneof value { + bool bool_value = 1; + string string_value = 2; + int64 int64_value = 3; + uint64 uint64_value = 4; + double double_value = 5; + uint64 ref_value = 6; + Slice slice_value = 7; + Array array_value = 8; + Interface interface_value = 9; + Struct struct_value = 10; + Map map_value = 11; + bytes byte_array_value = 12; + Uint16s uint16_array_value = 13; + Uint32s uint32_array_value = 14; + Uint64s uint64_array_value = 15; + Uintptrs uintptr_array_value = 16; + Int8s int8_array_value = 17; + Int16s int16_array_value = 18; + Int32s int32_array_value = 19; + Int64s int64_array_value = 20; + Bools bool_array_value = 21; + Float64s float64_array_value = 22; + Float32s float32_array_value = 23; + } +} diff --git a/pkg/state/printer.go b/pkg/state/printer.go new file mode 100644 index 000000000..c61ec4a26 --- /dev/null +++ b/pkg/state/printer.go @@ -0,0 +1,188 @@ +// Copyright 2018 Google Inc. +// +// 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 +// +// 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 state + +import ( + "fmt" + "io" + "io/ioutil" + "strings" + + "github.com/golang/protobuf/proto" + pb "gvisor.googlesource.com/gvisor/pkg/state/object_go_proto" +) + +// format formats a single object, for pretty-printing. +func format(graph uint64, depth int, object *pb.Object, html bool) (string, bool) { + switch x := object.GetValue().(type) { + case *pb.Object_BoolValue: + return fmt.Sprintf("%t", x.BoolValue), x.BoolValue != false + case *pb.Object_StringValue: + return fmt.Sprintf("\"%s\"", x.StringValue), x.StringValue != "" + case *pb.Object_Int64Value: + return fmt.Sprintf("%d", x.Int64Value), x.Int64Value != 0 + case *pb.Object_Uint64Value: + return fmt.Sprintf("%du", x.Uint64Value), x.Uint64Value != 0 + case *pb.Object_DoubleValue: + return fmt.Sprintf("%f", x.DoubleValue), x.DoubleValue != 0.0 + case *pb.Object_RefValue: + if x.RefValue == 0 { + return "nil", false + } + ref := fmt.Sprintf("g%dr%d", graph, x.RefValue) + if html { + ref = fmt.Sprintf("%s", ref, ref) + } + return ref, true + case *pb.Object_SliceValue: + if x.SliceValue.RefValue == 0 { + return "nil", false + } + ref := fmt.Sprintf("g%dr%d", graph, x.SliceValue.RefValue) + if html { + ref = fmt.Sprintf("%s", ref, ref) + } + return fmt.Sprintf("%s[:%d:%d]", ref, x.SliceValue.Length, x.SliceValue.Capacity), true + case *pb.Object_ArrayValue: + if len(x.ArrayValue.Contents) == 0 { + return "[]", false + } + items := make([]string, 0, len(x.ArrayValue.Contents)+2) + zeros := make([]string, 0) // used to eliminate zero entries. + items = append(items, "[") + tabs := "\n" + strings.Repeat("\t", depth) + for i := 0; i < len(x.ArrayValue.Contents); i++ { + item, ok := format(graph, depth+1, x.ArrayValue.Contents[i], html) + if ok { + if len(zeros) > 0 { + items = append(items, zeros...) + zeros = nil + } + items = append(items, fmt.Sprintf("\t%s,", item)) + } else { + zeros = append(zeros, fmt.Sprintf("\t%s,", item)) + } + } + if len(zeros) > 0 { + items = append(items, fmt.Sprintf("\t... (%d zero),", len(zeros))) + } + items = append(items, "]") + return strings.Join(items, tabs), len(zeros) < len(x.ArrayValue.Contents) + case *pb.Object_StructValue: + if len(x.StructValue.Fields) == 0 { + return "struct{}", false + } + items := make([]string, 0, len(x.StructValue.Fields)+2) + items = append(items, "struct{") + tabs := "\n" + strings.Repeat("\t", depth) + allZero := true + for _, field := range x.StructValue.Fields { + element, ok := format(graph, depth+1, field.Value, html) + allZero = allZero && !ok + items = append(items, fmt.Sprintf("\t%s: %s,", field.Name, element)) + } + items = append(items, "}") + return strings.Join(items, tabs), !allZero + case *pb.Object_MapValue: + if len(x.MapValue.Keys) == 0 { + return "map{}", false + } + items := make([]string, 0, len(x.MapValue.Keys)+2) + items = append(items, "map{") + tabs := "\n" + strings.Repeat("\t", depth) + for i := 0; i < len(x.MapValue.Keys); i++ { + key, _ := format(graph, depth+1, x.MapValue.Keys[i], html) + value, _ := format(graph, depth+1, x.MapValue.Values[i], html) + items = append(items, fmt.Sprintf("\t%s: %s,", key, value)) + } + items = append(items, "}") + return strings.Join(items, tabs), true + case *pb.Object_InterfaceValue: + if x.InterfaceValue.Type == "" { + return "interface(nil){}", false + } + element, _ := format(graph, depth+1, x.InterfaceValue.Value, html) + return fmt.Sprintf("interface(\"%s\"){%s}", x.InterfaceValue.Type, element), true + } + + // Should not happen, but tolerate. + return fmt.Sprintf("(unknown proto type: %T)", object.GetValue()), true +} + +// PrettyPrint reads the state stream from r, and pretty prints to w. +func PrettyPrint(w io.Writer, r io.Reader, html bool) error { + var ( + // current graph ID. + graph uint64 + + // current object ID. + id uint64 + ) + + if html { + fmt.Fprintf(w, "
")
+		defer fmt.Fprintf(w, "
") + } + + for { + // Find the first object to begin generation. + length, object, err := ReadHeader(r) + if err == io.EOF { + // Nothing else to do. + break + } else if err != nil { + return err + } + if !object { + // Increment the graph number & reset the ID. + graph++ + id = 0 + if length > 0 { + fmt.Fprintf(w, "(%d bytes non-object data)\n", length) + io.Copy(ioutil.Discard, &io.LimitedReader{ + R: r, + N: int64(length), + }) + } + continue + } + + // Read & unmarshal the object. + buf := make([]byte, length) + for done := 0; done < len(buf); { + n, err := r.Read(buf[done:]) + done += n + if n == 0 && err != nil { + return err + } + } + obj := new(pb.Object) + if err := proto.Unmarshal(buf, obj); err != nil { + return err + } + + id++ // First object must be one. + str, _ := format(graph, 0, obj, html) + tag := fmt.Sprintf("g%dr%d", graph, id) + if html { + tag = fmt.Sprintf("%s", tag, tag) + } + if _, err := fmt.Fprintf(w, "%s = %s\n", tag, str); err != nil { + return err + } + } + + return nil +} diff --git a/pkg/state/state.go b/pkg/state/state.go new file mode 100644 index 000000000..23a0b5922 --- /dev/null +++ b/pkg/state/state.go @@ -0,0 +1,349 @@ +// Copyright 2018 Google Inc. +// +// 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 +// +// 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 state provides functionality related to saving and loading object +// graphs. For most types, it provides a set of default saving / loading logic +// that will be invoked automatically if custom logic is not defined. +// +// Kind Support +// ---- ------- +// Bool default +// Int default +// Int8 default +// Int16 default +// Int32 default +// Int64 default +// Uint default +// Uint8 default +// Uint16 default +// Uint32 default +// Uint64 default +// Float32 default +// Float64 default +// Complex64 custom +// Complex128 custom +// Array default +// Chan custom +// Func custom +// Interface custom +// Map default (*) +// Ptr default +// Slice default +// String default +// Struct custom +// UnsafePointer custom +// +// (*) Maps are treated as value types by this package, even if they are +// pointers internally. If you want to save two independent references +// to the same map value, you must explicitly use a pointer to a map. +package state + +import ( + "fmt" + "io" + "reflect" + "runtime" + + pb "gvisor.googlesource.com/gvisor/pkg/state/object_go_proto" +) + +// ErrState is returned when an error is encountered during encode/decode. +type ErrState struct { + // Err is the underlying error. + Err error + + // path is the visit path from root to the current object. + path string + + // trace is the stack trace. + trace string +} + +// Error returns a sensible description of the state error. +func (e *ErrState) Error() string { + return fmt.Sprintf("%v:\nstate path: %s\n%s", e.Err, e.path, e.trace) +} + +// Save saves the given object state. +func Save(w io.Writer, rootPtr interface{}, stats *Stats) error { + // Create the encoding state. + es := &encodeState{ + idsByObject: make(map[uintptr]uint64), + w: w, + stats: stats, + } + + // Perform the encoding. + return es.safely(func() { + es.Serialize(reflect.ValueOf(rootPtr).Elem()) + }) +} + +// Load loads a checkpoint. +func Load(r io.Reader, rootPtr interface{}, stats *Stats) error { + // Create the decoding state. + ds := &decodeState{ + objectsByID: make(map[uint64]*objectState), + deferred: make(map[uint64]*pb.Object), + r: r, + stats: stats, + } + + // Attempt our decode. + return ds.safely(func() { + ds.Deserialize(reflect.ValueOf(rootPtr).Elem()) + }) +} + +// Fns are the state dispatch functions. +type Fns struct { + // Save is a function like Save(concreteType, Map). + Save interface{} + + // Load is a function like Load(concreteType, Map). + Load interface{} +} + +// Save executes the save function. +func (fns *Fns) invokeSave(obj reflect.Value, m Map) { + reflect.ValueOf(fns.Save).Call([]reflect.Value{obj, reflect.ValueOf(m)}) +} + +// Load executes the load function. +func (fns *Fns) invokeLoad(obj reflect.Value, m Map) { + reflect.ValueOf(fns.Load).Call([]reflect.Value{obj, reflect.ValueOf(m)}) +} + +// validateStateFn ensures types are correct. +func validateStateFn(fn interface{}, typ reflect.Type) bool { + fnTyp := reflect.TypeOf(fn) + if fnTyp.Kind() != reflect.Func { + return false + } + if fnTyp.NumIn() != 2 { + return false + } + if fnTyp.NumOut() != 0 { + return false + } + if fnTyp.In(0) != typ { + return false + } + if fnTyp.In(1) != reflect.TypeOf(Map{}) { + return false + } + return true +} + +// Validate validates all state functions. +func (fns *Fns) Validate(typ reflect.Type) bool { + return validateStateFn(fns.Save, typ) && validateStateFn(fns.Load, typ) +} + +type typeDatabase struct { + // nameToType is a forward lookup table. + nameToType map[string]reflect.Type + + // typeToName is the reverse lookup table. + typeToName map[reflect.Type]string + + // typeToFns is the function lookup table. + typeToFns map[reflect.Type]Fns +} + +// registeredTypes is a database used for SaveInterface and LoadInterface. +var registeredTypes = typeDatabase{ + nameToType: make(map[string]reflect.Type), + typeToName: make(map[reflect.Type]string), + typeToFns: make(map[reflect.Type]Fns), +} + +// register registers a type under the given name. This will generally be +// called via init() methods, and therefore uses panic to propagate errors. +func (t *typeDatabase) register(name string, typ reflect.Type, fns Fns) { + // We can't allow name collisions. + if ot, ok := t.nameToType[name]; ok { + panic(fmt.Sprintf("type %q can't use name %q, already in use by type %q", typ.Name(), name, ot.Name())) + } + + // Or multiple registrations. + if on, ok := t.typeToName[typ]; ok { + panic(fmt.Sprintf("type %q can't be registered as %q, already registered as %q", typ.Name(), name, on)) + } + + t.nameToType[name] = typ + t.typeToName[typ] = name + t.typeToFns[typ] = fns +} + +// lookupType finds a type given a name. +func (t *typeDatabase) lookupType(name string) (reflect.Type, bool) { + typ, ok := t.nameToType[name] + return typ, ok +} + +// lookupName finds a name given a type. +func (t *typeDatabase) lookupName(typ reflect.Type) (string, bool) { + name, ok := t.typeToName[typ] + return name, ok +} + +// lookupFns finds functions given a type. +func (t *typeDatabase) lookupFns(typ reflect.Type) (Fns, bool) { + fns, ok := t.typeToFns[typ] + return fns, ok +} + +// Register must be called for any interface implementation types that +// implements Loader. +// +// Register should be called either immediately after startup or via init() +// methods. Double registration of either names or types will result in a panic. +// +// No synchronization is provided; this should only be called in init. +// +// Example usage: +// +// state.Register("Foo", (*Foo)(nil), state.Fns{ +// Save: (*Foo).Save, +// Load: (*Foo).Load, +// }) +// +func Register(name string, instance interface{}, fns Fns) { + registeredTypes.register(name, reflect.TypeOf(instance), fns) +} + +// IsZeroValue checks if the given value is the zero value. +// +// This function is used by the stateify tool. +func IsZeroValue(val interface{}) bool { + if val == nil { + return true + } + return reflect.DeepEqual(val, reflect.Zero(reflect.TypeOf(val)).Interface()) +} + +// step captures one encoding / decoding step. On each step, there is up to one +// choice made, which is captured by non-nil param. We intentionally do not +// eagerly create the final path string, as that will only be needed upon panic. +type step struct { + // dereference indicate if the current object is obtained by + // dereferencing a pointer. + dereference bool + + // format is the formatting string that takes param below, if + // non-nil. For example, in array indexing case, we have "[%d]". + format string + + // param stores the choice made at the current encoding / decoding step. + // For eaxmple, in array indexing case, param stores the index. When no + // choice is made, e.g. dereference, param should be nil. + param interface{} +} + +// recoverable is the state encoding / decoding panic recovery facility. It is +// also used to store encoding / decoding steps as well as the reference to the +// original queued object from which the current object is dispatched. The +// complete encoding / decoding path is synthesised from the steps in all queued +// objects leading to the current object. +type recoverable struct { + from *recoverable + steps []step +} + +// push enters a new context level. +func (sr *recoverable) push(dereference bool, format string, param interface{}) { + sr.steps = append(sr.steps, step{dereference, format, param}) +} + +// pop exits the current context level. +func (sr *recoverable) pop() { + if len(sr.steps) <= 1 { + return + } + sr.steps = sr.steps[:len(sr.steps)-1] +} + +// path returns the complete encoding / decoding path from root. This is only +// called upon panic. +func (sr *recoverable) path() string { + if sr.from == nil { + return "root" + } + p := sr.from.path() + for _, s := range sr.steps { + if s.dereference { + p = fmt.Sprintf("*(%s)", p) + } + if s.param == nil { + p += s.format + } else { + p += fmt.Sprintf(s.format, s.param) + } + } + return p +} + +func (sr *recoverable) copy() recoverable { + return recoverable{from: sr.from, steps: append([]step(nil), sr.steps...)} +} + +// safely executes the given function, catching a panic and unpacking as an error. +// +// The error flow through the state package uses panic and recover. There are +// two important reasons for this: +// +// 1) Many of the reflection methods will already panic with invalid data or +// violated assumptions. We would want to recover anyways here. +// +// 2) It allows us to eliminate boilerplate within Save() and Load() functions. +// In nearly all cases, when the low-level serialization functions fail, you +// will want the checkpoint to fail anyways. Plumbing errors through every +// method doesn't add a lot of value. If there are specific error conditions +// that you'd like to handle, you should add appropriate functionality to +// objects themselves prior to calling Save() and Load(). +func (sr *recoverable) safely(fn func()) (err error) { + defer func() { + if r := recover(); r != nil { + es := new(ErrState) + if e, ok := r.(error); ok { + es.Err = e + } else { + es.Err = fmt.Errorf("%v", r) + } + + es.path = sr.path() + + // Make a stack. We don't know how big it will be ahead + // of time, but want to make sure we get the whole + // thing. So we just do a stupid brute force approach. + var stack []byte + for sz := 1024; ; sz *= 2 { + stack = make([]byte, sz) + n := runtime.Stack(stack, false) + if n < sz { + es.trace = string(stack[:n]) + break + } + } + + // Set the error. + err = es + } + }() + + // Execute the function. + fn() + return nil +} diff --git a/pkg/state/state_test.go b/pkg/state/state_test.go new file mode 100644 index 000000000..d5a739f18 --- /dev/null +++ b/pkg/state/state_test.go @@ -0,0 +1,719 @@ +// Copyright 2018 Google Inc. +// +// 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 +// +// 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 state + +import ( + "bytes" + "io/ioutil" + "math" + "reflect" + "testing" +) + +// TestCase is used to define a single success/failure testcase of +// serialization of a set of objects. +type TestCase struct { + // Name is the name of the test case. + Name string + + // Objects is the list of values to serialize. + Objects []interface{} + + // Fail is whether the test case is supposed to fail or not. + Fail bool +} + +// runTest runs all testcases. +func runTest(t *testing.T, tests []TestCase) { + for _, test := range tests { + t.Logf("TEST %s:", test.Name) + for i, root := range test.Objects { + t.Logf(" case#%d: %#v", i, root) + + // Save the passed object. + saveBuffer := &bytes.Buffer{} + saveObjectPtr := reflect.New(reflect.TypeOf(root)) + saveObjectPtr.Elem().Set(reflect.ValueOf(root)) + if err := Save(saveBuffer, saveObjectPtr.Interface(), nil); err != nil && !test.Fail { + t.Errorf(" FAIL: Save failed unexpectedly: %v", err) + continue + } else if err != nil { + t.Logf(" PASS: Save failed as expected: %v", err) + continue + } + + // Load a new copy of the object. + loadObjectPtr := reflect.New(reflect.TypeOf(root)) + if err := Load(bytes.NewReader(saveBuffer.Bytes()), loadObjectPtr.Interface(), nil); err != nil && !test.Fail { + t.Errorf(" FAIL: Load failed unexpectedly: %v", err) + continue + } else if err != nil { + t.Logf(" PASS: Load failed as expected: %v", err) + continue + } + + // Compare the values. + loadedValue := loadObjectPtr.Elem().Interface() + if eq := reflect.DeepEqual(root, loadedValue); !eq && !test.Fail { + t.Errorf(" FAIL: Objects differs; got %#v", loadedValue) + continue + } else if !eq { + t.Logf(" PASS: Object different as expected.") + continue + } + + // Everything went okay. Is that good? + if test.Fail { + t.Errorf(" FAIL: Unexpected success.") + } else { + t.Logf(" PASS: Success.") + } + } + } +} + +// dumbStruct is a struct which does not implement the loader/saver interface. +// We expect that serialization of this struct will fail. +type dumbStruct struct { + A int + B int +} + +// smartStruct is a struct which does implement the loader/saver interface. +// We expect that serialization of this struct will succeed. +type smartStruct struct { + A int + B int +} + +func (s *smartStruct) save(m Map) { + m.Save("A", &s.A) + m.Save("B", &s.B) +} + +func (s *smartStruct) load(m Map) { + m.Load("A", &s.A) + m.Load("B", &s.B) +} + +// valueLoadStruct uses a value load. +type valueLoadStruct struct { + v int +} + +func (v *valueLoadStruct) save(m Map) { + m.SaveValue("v", v.v) +} + +func (v *valueLoadStruct) load(m Map) { + m.LoadValue("v", new(int), func(value interface{}) { + v.v = value.(int) + }) +} + +// afterLoadStruct has an AfterLoad function. +type afterLoadStruct struct { + v int +} + +func (a *afterLoadStruct) save(m Map) { +} + +func (a *afterLoadStruct) load(m Map) { + m.AfterLoad(func() { + a.v++ + }) +} + +// genericContainer is a generic dispatcher. +type genericContainer struct { + v interface{} +} + +func (g *genericContainer) save(m Map) { + m.Save("v", &g.v) +} + +func (g *genericContainer) load(m Map) { + m.Load("v", &g.v) +} + +// sliceContainer is a generic slice. +type sliceContainer struct { + v []interface{} +} + +func (s *sliceContainer) save(m Map) { + m.Save("v", &s.v) +} + +func (s *sliceContainer) load(m Map) { + m.Load("v", &s.v) +} + +// mapContainer is a generic map. +type mapContainer struct { + v map[int]interface{} +} + +func (mc *mapContainer) save(m Map) { + m.Save("v", &mc.v) +} + +func (mc *mapContainer) load(m Map) { + // Some of the test cases below assume legacy behavior wherein maps + // will automatically inherit dependencies. + m.LoadWait("v", &mc.v) +} + +// dumbMap is a map which does not implement the loader/saver interface. +// Serialization of this map will default to the standard encode/decode logic. +type dumbMap map[string]int + +// pointerStruct contains various pointers, shared and non-shared, and pointers +// to pointers. We expect that serialization will respect the structure. +type pointerStruct struct { + A *int + B *int + C *int + D *int + + AA **int + BB **int +} + +func (p *pointerStruct) save(m Map) { + m.Save("A", &p.A) + m.Save("B", &p.B) + m.Save("C", &p.C) + m.Save("D", &p.D) + m.Save("AA", &p.AA) + m.Save("BB", &p.BB) +} + +func (p *pointerStruct) load(m Map) { + m.Load("A", &p.A) + m.Load("B", &p.B) + m.Load("C", &p.C) + m.Load("D", &p.D) + m.Load("AA", &p.AA) + m.Load("BB", &p.BB) +} + +// testInterface is a trivial interface example. +type testInterface interface { + Foo() +} + +// testImpl is a trivial implementation of testInterface. +type testImpl struct { +} + +// Foo satisfies testInterface. +func (t *testImpl) Foo() { +} + +// testImpl is trivially serializable. +func (t *testImpl) save(m Map) { +} + +// testImpl is trivially serializable. +func (t *testImpl) load(m Map) { +} + +// testI demonstrates interface dispatching. +type testI struct { + I testInterface +} + +func (t *testI) save(m Map) { + m.Save("I", &t.I) +} + +func (t *testI) load(m Map) { + m.Load("I", &t.I) +} + +// cycleStruct is used to implement basic cycles. +type cycleStruct struct { + c *cycleStruct +} + +func (c *cycleStruct) save(m Map) { + m.Save("c", &c.c) +} + +func (c *cycleStruct) load(m Map) { + m.Load("c", &c.c) +} + +// badCycleStruct actually has deadlocking dependencies. +// +// This should pass if b.b = {nil|b} and fail otherwise. +type badCycleStruct struct { + b *badCycleStruct +} + +func (b *badCycleStruct) save(m Map) { + m.Save("b", &b.b) +} + +func (b *badCycleStruct) load(m Map) { + m.LoadWait("b", &b.b) + m.AfterLoad(func() { + // This is not executable, since AfterLoad requires that the + // object and all dependencies are complete. This should cause + // a deadlock error during load. + }) +} + +// emptyStructPointer points to an empty struct. +type emptyStructPointer struct { + nothing *struct{} +} + +func (e *emptyStructPointer) save(m Map) { + m.Save("nothing", &e.nothing) +} + +func (e *emptyStructPointer) load(m Map) { + m.Load("nothing", &e.nothing) +} + +// truncateInteger truncates an integer. +type truncateInteger struct { + v int64 + v2 int32 +} + +func (t *truncateInteger) save(m Map) { + t.v2 = int32(t.v) + m.Save("v", &t.v) +} + +func (t *truncateInteger) load(m Map) { + m.Load("v", &t.v2) + t.v = int64(t.v2) +} + +// truncateUnsignedInteger truncates an unsigned integer. +type truncateUnsignedInteger struct { + v uint64 + v2 uint32 +} + +func (t *truncateUnsignedInteger) save(m Map) { + t.v2 = uint32(t.v) + m.Save("v", &t.v) +} + +func (t *truncateUnsignedInteger) load(m Map) { + m.Load("v", &t.v2) + t.v = uint64(t.v2) +} + +// truncateFloat truncates a floating point number. +type truncateFloat struct { + v float64 + v2 float32 +} + +func (t *truncateFloat) save(m Map) { + t.v2 = float32(t.v) + m.Save("v", &t.v) +} + +func (t *truncateFloat) load(m Map) { + m.Load("v", &t.v2) + t.v = float64(t.v2) +} + +func TestTypes(t *testing.T) { + // x and y are basic integers, while xp points to x. + x := 1 + y := 2 + xp := &x + + // cs is a single object cycle. + cs := cycleStruct{nil} + cs.c = &cs + + // cs1 and cs2 are in a two object cycle. + cs1 := cycleStruct{nil} + cs2 := cycleStruct{nil} + cs1.c = &cs2 + cs2.c = &cs1 + + // bs is a single object cycle. + bs := badCycleStruct{nil} + bs.b = &bs + + // bs2 and bs2 are in a deadlocking cycle. + bs1 := badCycleStruct{nil} + bs2 := badCycleStruct{nil} + bs1.b = &bs2 + bs2.b = &bs1 + + // regular nils. + var ( + nilmap dumbMap + nilslice []byte + ) + + // embed points to embedded fields. + embed1 := pointerStruct{} + embed1.AA = &embed1.A + embed2 := pointerStruct{} + embed2.BB = &embed2.B + + // es1 contains two structs pointing to the same empty struct. + es := emptyStructPointer{new(struct{})} + es1 := []emptyStructPointer{es, es} + + tests := []TestCase{ + { + Name: "bool", + Objects: []interface{}{ + true, + false, + }, + }, + { + Name: "integers", + Objects: []interface{}{ + int(0), + int(1), + int(-1), + int8(0), + int8(1), + int8(-1), + int16(0), + int16(1), + int16(-1), + int32(0), + int32(1), + int32(-1), + int64(0), + int64(1), + int64(-1), + }, + }, + { + Name: "unsigned integers", + Objects: []interface{}{ + uint(0), + uint(1), + uint8(0), + uint8(1), + uint16(0), + uint16(1), + uint32(1), + uint64(0), + uint64(1), + }, + }, + { + Name: "strings", + Objects: []interface{}{ + "", + "foo", + "bar", + }, + }, + { + Name: "slices", + Objects: []interface{}{ + []int{-1, 0, 1}, + []*int{&x, &x, &x}, + []int{1, 2, 3}[0:1], + []int{1, 2, 3}[1:2], + make([]byte, 32), + make([]byte, 32)[:16], + make([]byte, 32)[:16:20], + nilslice, + }, + }, + { + Name: "arrays", + Objects: []interface{}{ + &[1048576]bool{false, true, false, true}, + &[1048576]uint8{0, 1, 2, 3}, + &[1048576]byte{0, 1, 2, 3}, + &[1048576]uint16{0, 1, 2, 3}, + &[1048576]uint{0, 1, 2, 3}, + &[1048576]uint32{0, 1, 2, 3}, + &[1048576]uint64{0, 1, 2, 3}, + &[1048576]uintptr{0, 1, 2, 3}, + &[1048576]int8{0, -1, -2, -3}, + &[1048576]int16{0, -1, -2, -3}, + &[1048576]int32{0, -1, -2, -3}, + &[1048576]int64{0, -1, -2, -3}, + &[1048576]float32{0, 1.1, 2.2, 3.3}, + &[1048576]float64{0, 1.1, 2.2, 3.3}, + }, + }, + { + Name: "pointers", + Objects: []interface{}{ + &pointerStruct{A: &x, B: &x, C: &y, D: &y, AA: &xp, BB: &xp}, + &pointerStruct{}, + }, + }, + { + Name: "empty struct", + Objects: []interface{}{ + struct{}{}, + }, + }, + { + Name: "unenlightened structs", + Objects: []interface{}{ + &dumbStruct{A: 1, B: 2}, + }, + Fail: true, + }, + { + Name: "enlightened structs", + Objects: []interface{}{ + &smartStruct{A: 1, B: 2}, + }, + }, + { + Name: "load-hooks", + Objects: []interface{}{ + &afterLoadStruct{v: 1}, + &valueLoadStruct{v: 1}, + &genericContainer{v: &afterLoadStruct{v: 1}}, + &genericContainer{v: &valueLoadStruct{v: 1}}, + &sliceContainer{v: []interface{}{&afterLoadStruct{v: 1}}}, + &sliceContainer{v: []interface{}{&valueLoadStruct{v: 1}}}, + &mapContainer{v: map[int]interface{}{0: &afterLoadStruct{v: 1}}}, + &mapContainer{v: map[int]interface{}{0: &valueLoadStruct{v: 1}}}, + }, + }, + { + Name: "maps", + Objects: []interface{}{ + dumbMap{"a": -1, "b": 0, "c": 1}, + map[smartStruct]int{{}: 0, {A: 1}: 1}, + nilmap, + &mapContainer{v: map[int]interface{}{0: &smartStruct{A: 1}}}, + }, + }, + { + Name: "interfaces", + Objects: []interface{}{ + &testI{&testImpl{}}, + &testI{nil}, + &testI{(*testImpl)(nil)}, + }, + }, + { + Name: "unregistered-interfaces", + Objects: []interface{}{ + &genericContainer{v: afterLoadStruct{v: 1}}, + &genericContainer{v: valueLoadStruct{v: 1}}, + &sliceContainer{v: []interface{}{afterLoadStruct{v: 1}}}, + &sliceContainer{v: []interface{}{valueLoadStruct{v: 1}}}, + &mapContainer{v: map[int]interface{}{0: afterLoadStruct{v: 1}}}, + &mapContainer{v: map[int]interface{}{0: valueLoadStruct{v: 1}}}, + }, + Fail: true, + }, + { + Name: "cycles", + Objects: []interface{}{ + &cs, + &cs1, + &cycleStruct{&cs1}, + &cycleStruct{&cs}, + &badCycleStruct{nil}, + &bs, + }, + }, + { + Name: "deadlock", + Objects: []interface{}{ + &bs1, + }, + Fail: true, + }, + { + Name: "embed", + Objects: []interface{}{ + &embed1, + &embed2, + }, + Fail: true, + }, + { + Name: "empty structs", + Objects: []interface{}{ + new(struct{}), + es, + es1, + }, + }, + { + Name: "truncated okay", + Objects: []interface{}{ + &truncateInteger{v: 1}, + &truncateUnsignedInteger{v: 1}, + &truncateFloat{v: 1.0}, + }, + }, + { + Name: "truncated bad", + Objects: []interface{}{ + &truncateInteger{v: math.MaxInt32 + 1}, + &truncateUnsignedInteger{v: math.MaxUint32 + 1}, + &truncateFloat{v: math.MaxFloat32 * 2}, + }, + Fail: true, + }, + } + + runTest(t, tests) +} + +// benchStruct is used for benchmarking. +type benchStruct struct { + b *benchStruct + + // Dummy data is included to ensure that these objects are large. + // This is to detect possible regression when registering objects. + _ [4096]byte +} + +func (b *benchStruct) save(m Map) { + m.Save("b", &b.b) +} + +func (b *benchStruct) load(m Map) { + m.LoadWait("b", &b.b) + m.AfterLoad(b.afterLoad) +} + +func (b *benchStruct) afterLoad() { + // Do nothing, just force scheduling. +} + +// buildObject builds a benchmark object. +func buildObject(n int) (b *benchStruct) { + for i := 0; i < n; i++ { + b = &benchStruct{b: b} + } + return +} + +func BenchmarkEncoding(b *testing.B) { + b.StopTimer() + bs := buildObject(b.N) + var stats Stats + b.StartTimer() + if err := Save(ioutil.Discard, bs, &stats); err != nil { + b.Errorf("save failed: %v", err) + } + b.StopTimer() + if b.N > 1000 { + b.Logf("breakdown (n=%d): %s", b.N, &stats) + } +} + +func BenchmarkDecoding(b *testing.B) { + b.StopTimer() + bs := buildObject(b.N) + var newBS benchStruct + buf := &bytes.Buffer{} + if err := Save(buf, bs, nil); err != nil { + b.Errorf("save failed: %v", err) + } + var stats Stats + b.StartTimer() + if err := Load(buf, &newBS, &stats); err != nil { + b.Errorf("load failed: %v", err) + } + b.StopTimer() + if b.N > 1000 { + b.Logf("breakdown (n=%d): %s", b.N, &stats) + } +} + +func init() { + Register("stateTest.smartStruct", (*smartStruct)(nil), Fns{ + Save: (*smartStruct).save, + Load: (*smartStruct).load, + }) + Register("stateTest.afterLoadStruct", (*afterLoadStruct)(nil), Fns{ + Save: (*afterLoadStruct).save, + Load: (*afterLoadStruct).load, + }) + Register("stateTest.valueLoadStruct", (*valueLoadStruct)(nil), Fns{ + Save: (*valueLoadStruct).save, + Load: (*valueLoadStruct).load, + }) + Register("stateTest.genericContainer", (*genericContainer)(nil), Fns{ + Save: (*genericContainer).save, + Load: (*genericContainer).load, + }) + Register("stateTest.sliceContainer", (*sliceContainer)(nil), Fns{ + Save: (*sliceContainer).save, + Load: (*sliceContainer).load, + }) + Register("stateTest.mapContainer", (*mapContainer)(nil), Fns{ + Save: (*mapContainer).save, + Load: (*mapContainer).load, + }) + Register("stateTest.pointerStruct", (*pointerStruct)(nil), Fns{ + Save: (*pointerStruct).save, + Load: (*pointerStruct).load, + }) + Register("stateTest.testImpl", (*testImpl)(nil), Fns{ + Save: (*testImpl).save, + Load: (*testImpl).load, + }) + Register("stateTest.testI", (*testI)(nil), Fns{ + Save: (*testI).save, + Load: (*testI).load, + }) + Register("stateTest.cycleStruct", (*cycleStruct)(nil), Fns{ + Save: (*cycleStruct).save, + Load: (*cycleStruct).load, + }) + Register("stateTest.badCycleStruct", (*badCycleStruct)(nil), Fns{ + Save: (*badCycleStruct).save, + Load: (*badCycleStruct).load, + }) + Register("stateTest.emptyStructPointer", (*emptyStructPointer)(nil), Fns{ + Save: (*emptyStructPointer).save, + Load: (*emptyStructPointer).load, + }) + Register("stateTest.truncateInteger", (*truncateInteger)(nil), Fns{ + Save: (*truncateInteger).save, + Load: (*truncateInteger).load, + }) + Register("stateTest.truncateUnsignedInteger", (*truncateUnsignedInteger)(nil), Fns{ + Save: (*truncateUnsignedInteger).save, + Load: (*truncateUnsignedInteger).load, + }) + Register("stateTest.truncateFloat", (*truncateFloat)(nil), Fns{ + Save: (*truncateFloat).save, + Load: (*truncateFloat).load, + }) + Register("stateTest.benchStruct", (*benchStruct)(nil), Fns{ + Save: (*benchStruct).save, + Load: (*benchStruct).load, + }) +} diff --git a/pkg/state/statefile/BUILD b/pkg/state/statefile/BUILD new file mode 100644 index 000000000..df2c6a578 --- /dev/null +++ b/pkg/state/statefile/BUILD @@ -0,0 +1,23 @@ +package(licenses = ["notice"]) # Apache 2.0 + +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "statefile", + srcs = ["statefile.go"], + importpath = "gvisor.googlesource.com/gvisor/pkg/state/statefile", + visibility = ["//:sandbox"], + deps = [ + "//pkg/binary", + "//pkg/compressio", + "//pkg/hashio", + ], +) + +go_test( + name = "statefile_test", + size = "small", + srcs = ["statefile_test.go"], + embed = [":statefile"], + deps = ["//pkg/hashio"], +) diff --git a/pkg/state/statefile/statefile.go b/pkg/state/statefile/statefile.go new file mode 100644 index 000000000..b25b743b7 --- /dev/null +++ b/pkg/state/statefile/statefile.go @@ -0,0 +1,233 @@ +// Copyright 2018 Google Inc. +// +// 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 +// +// 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 statefile defines the state file data stream. +// +// This package currently does not include any details regarding the state +// encoding itself, only details regarding state metadata and data layout. +// +// The file format is defined as follows. +// +// /------------------------------------------------------\ +// | header (8-bytes) | +// +------------------------------------------------------+ +// | metadata length (8-bytes) | +// +------------------------------------------------------+ +// | metadata | +// +------------------------------------------------------+ +// | data | +// \------------------------------------------------------/ +// +// First, it includes a 8-byte magic header which is the following +// sequence of bytes [0x67, 0x56, 0x69, 0x73, 0x6f, 0x72, 0x53, 0x46] +// +// This header is followed by an 8-byte length N (big endian), and an +// ASCII-encoded JSON map that is exactly N bytes long. +// +// This map includes only strings for keys and strings for values. Keys in the +// map that begin with "_" are for internal use only. They may be read, but may +// not be provided by the user. In the future, this metadata may contain some +// information relating to the state encoding itself. +// +// After the map, the remainder of the file is the state data. +package statefile + +import ( + "bytes" + "crypto/hmac" + "crypto/sha256" + "encoding/json" + "fmt" + "hash" + "io" + "strings" + "time" + + "gvisor.googlesource.com/gvisor/pkg/binary" + "gvisor.googlesource.com/gvisor/pkg/compressio" + "gvisor.googlesource.com/gvisor/pkg/hashio" +) + +// keySize is the AES-256 key length. +const keySize = 32 + +// compressionChunkSize is the chunk size for compression. +const compressionChunkSize = 1024 * 1024 + +// maxMetadataSize is the size limit of metadata section. +const maxMetadataSize = 16 * 1024 * 1024 + +// magicHeader is the byte sequence beginning each file. +var magicHeader = []byte("\x67\x56\x69\x73\x6f\x72\x53\x46") + +// ErrBadMagic is returned if the header does not match. +var ErrBadMagic = fmt.Errorf("bad magic header") + +// ErrMetadataMissing is returned if the state file is missing mandatory metadata. +var ErrMetadataMissing = fmt.Errorf("missing metadata") + +// ErrInvalidMetadataLength is returned if the metadata length is too large. +var ErrInvalidMetadataLength = fmt.Errorf("metadata length invalid, maximum size is %d", maxMetadataSize) + +// ErrMetadataInvalid is returned if passed metadata is invalid. +var ErrMetadataInvalid = fmt.Errorf("metadata invalid, can't start with _") + +// NewWriter returns a state data writer for a statefile. +// +// Note that the returned WriteCloser must be closed. +func NewWriter(w io.Writer, key []byte, metadata map[string]string, compressionLevel int) (io.WriteCloser, error) { + if metadata == nil { + metadata = make(map[string]string) + } + for k := range metadata { + if strings.HasPrefix(k, "_") { + return nil, ErrMetadataInvalid + } + } + + // Create our HMAC function. + h := hmac.New(sha256.New, key) + mw := io.MultiWriter(w, h) + + // First, write the header. + if _, err := mw.Write(magicHeader); err != nil { + return nil, err + } + + // Generate a timestamp, for convenience only. + metadata["_timestamp"] = time.Now().UTC().String() + defer delete(metadata, "_timestamp") + + // Write the metadata. + b, err := json.Marshal(metadata) + if err != nil { + return nil, err + } + + if len(b) > maxMetadataSize { + return nil, ErrInvalidMetadataLength + } + + // Metadata length. + if err := binary.WriteUint64(mw, binary.BigEndian, uint64(len(b))); err != nil { + return nil, err + } + // Metadata bytes; io.MultiWriter will return a short write error if + // any of the writers returns < n. + if _, err := mw.Write(b); err != nil { + return nil, err + } + // Write the current hash. + cur := h.Sum(nil) + for done := 0; done < len(cur); { + n, err := mw.Write(cur[done:]) + done += n + if err != nil { + return nil, err + } + } + + w = hashio.NewWriter(w, h) + + // Wrap in compression. + return compressio.NewWriter(w, compressionChunkSize, compressionLevel) +} + +// MetadataUnsafe reads out the metadata from a state file without verifying any +// HMAC. This function shouldn't be called for untrusted input files. +func MetadataUnsafe(r io.Reader) (map[string]string, error) { + return metadata(r, nil) +} + +// metadata validates the magic header and reads out the metadata from a state +// data stream. +func metadata(r io.Reader, h hash.Hash) (map[string]string, error) { + if h != nil { + r = io.TeeReader(r, h) + } + + // Read and validate magic header. + b := make([]byte, len(magicHeader)) + if _, err := r.Read(b); err != nil { + return nil, err + } + if !bytes.Equal(b, magicHeader) { + return nil, ErrBadMagic + } + + // Read and validate metadata. + b, err := func() (b []byte, err error) { + defer func() { + if r := recover(); r != nil { + b = nil + err = fmt.Errorf("%v", r) + } + }() + + metadataLen, err := binary.ReadUint64(r, binary.BigEndian) + if err != nil { + return nil, err + } + if metadataLen > maxMetadataSize { + return nil, ErrInvalidMetadataLength + } + b = make([]byte, int(metadataLen)) + if _, err := io.ReadFull(r, b); err != nil { + return nil, err + } + return b, nil + }() + if err != nil { + return nil, err + } + + if h != nil { + // Check the hash prior to decoding. + cur := h.Sum(nil) + buf := make([]byte, len(cur)) + if _, err := io.ReadFull(r, buf); err != nil { + return nil, err + } + if !hmac.Equal(cur, buf) { + return nil, hashio.ErrHashMismatch + } + } + + // Decode the metadata. + metadata := make(map[string]string) + if err := json.Unmarshal(b, &metadata); err != nil { + return nil, err + } + + return metadata, nil +} + +// NewReader returns a reader for a statefile. +func NewReader(r io.Reader, key []byte) (io.Reader, map[string]string, error) { + // Read the metadata with the hash. + h := hmac.New(sha256.New, key) + metadata, err := metadata(r, h) + if err != nil { + return nil, nil, err + } + + r = hashio.NewReader(r, h) + + // Wrap in compression. + rc, err := compressio.NewReader(r) + if err != nil { + return nil, nil, err + } + return rc, metadata, nil +} diff --git a/pkg/state/statefile/statefile_test.go b/pkg/state/statefile/statefile_test.go new file mode 100644 index 000000000..6e67b51de --- /dev/null +++ b/pkg/state/statefile/statefile_test.go @@ -0,0 +1,299 @@ +// Copyright 2018 Google Inc. +// +// 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 +// +// 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 statefile + +import ( + "bytes" + "compress/flate" + crand "crypto/rand" + "encoding/base64" + "io" + "math/rand" + "testing" + + "gvisor.googlesource.com/gvisor/pkg/hashio" +) + +func randomKey() ([]byte, error) { + r := make([]byte, base64.RawStdEncoding.DecodedLen(keySize)) + if _, err := io.ReadFull(crand.Reader, r); err != nil { + return nil, err + } + key := make([]byte, keySize) + base64.RawStdEncoding.Encode(key, r) + return key, nil +} + +type testCase struct { + name string + data []byte + metadata map[string]string +} + +func TestStatefile(t *testing.T) { + cases := []testCase{ + // Various data sizes. + {"nil", nil, nil}, + {"empty", []byte(""), nil}, + {"some", []byte("_"), nil}, + {"one", []byte("0"), nil}, + {"two", []byte("01"), nil}, + {"three", []byte("012"), nil}, + {"four", []byte("0123"), nil}, + {"five", []byte("01234"), nil}, + {"six", []byte("012356"), nil}, + {"seven", []byte("0123567"), nil}, + {"eight", []byte("01235678"), nil}, + + // Make sure we have one longer than the hash length. + {"longer than hash", []byte("012356asdjflkasjlk3jlk23j4lkjaso0d789f0aujw3lkjlkxsdf78asdful2kj3ljka78"), nil}, + + // Make sure we have one longer than the segment size. + {"segments", make([]byte, 3*hashio.SegmentSize), nil}, + {"segments minus one", make([]byte, 3*hashio.SegmentSize-1), nil}, + {"segments plus one", make([]byte, 3*hashio.SegmentSize+1), nil}, + {"segments minus hash", make([]byte, 3*hashio.SegmentSize-32), nil}, + {"segments plus hash", make([]byte, 3*hashio.SegmentSize+32), nil}, + {"large", make([]byte, 30*hashio.SegmentSize), nil}, + + // Different metadata. + {"one metadata", []byte("data"), map[string]string{"foo": "bar"}}, + {"two metadata", []byte("data"), map[string]string{"foo": "bar", "one": "two"}}, + } + + for _, c := range cases { + // Generate a key. + integrityKey, err := randomKey() + if err != nil { + t.Errorf("can't generate key: got %v, excepted nil", err) + continue + } + + t.Run(c.name, func(t *testing.T) { + for _, key := range [][]byte{nil, integrityKey} { + t.Run("key="+string(key), func(t *testing.T) { + // Encoding happens via a buffer. + var bufEncoded bytes.Buffer + var bufDecoded bytes.Buffer + + // Do all the writing. + w, err := NewWriter(&bufEncoded, key, c.metadata, flate.BestSpeed) + if err != nil { + t.Fatalf("error creating writer: got %v, expected nil", err) + } + if _, err := io.Copy(w, bytes.NewBuffer(c.data)); err != nil { + t.Fatalf("error during write: got %v, expected nil", err) + } + + // Finish the sum. + if err := w.Close(); err != nil { + t.Fatalf("error during close: got %v, expected nil", err) + } + + t.Logf("original data: %d bytes, encoded: %d bytes.", + len(c.data), len(bufEncoded.Bytes())) + + // Do all the reading. + r, metadata, err := NewReader(bytes.NewReader(bufEncoded.Bytes()), key) + if err != nil { + t.Fatalf("error creating reader: got %v, expected nil", err) + } + if _, err := io.Copy(&bufDecoded, r); err != nil { + t.Fatalf("error during read: got %v, expected nil", err) + } + + // Check that the data matches. + if !bytes.Equal(c.data, bufDecoded.Bytes()) { + t.Fatalf("data didn't match (%d vs %d bytes)", len(bufDecoded.Bytes()), len(c.data)) + } + + // Check that the metadata matches. + for k, v := range c.metadata { + nv, ok := metadata[k] + if !ok { + t.Fatalf("missing metadata: %s", k) + } + if v != nv { + t.Fatalf("mismatched metdata for %s: got %s, expected %s", k, nv, v) + } + } + + // Change the data and verify that it fails. + b := append([]byte(nil), bufEncoded.Bytes()...) + b[rand.Intn(len(b))]++ + r, _, err = NewReader(bytes.NewReader(b), key) + if err == nil { + _, err = io.Copy(&bufDecoded, r) + } + if err == nil { + t.Error("got no error: expected error on data corruption") + } + + // Change the key and verify that it fails. + if key == nil { + key = integrityKey + } else { + key[rand.Intn(len(key))]++ + } + r, _, err = NewReader(bytes.NewReader(bufEncoded.Bytes()), key) + if err == nil { + _, err = io.Copy(&bufDecoded, r) + } + if err != hashio.ErrHashMismatch { + t.Errorf("got error: %v, expected ErrHashMismatch on key mismatch", err) + } + }) + } + }) + } +} + +const benchmarkDataSize = 10 * 1024 * 1024 + +func benchmark(b *testing.B, size int, write bool, compressible bool) { + b.StopTimer() + b.SetBytes(benchmarkDataSize) + + // Generate source data. + var source []byte + if compressible { + // For compressible data, we use essentially all zeros. + source = make([]byte, benchmarkDataSize) + } else { + // For non-compressible data, we use random base64 data (to + // make it marginally compressible, a ratio of 75%). + var sourceBuf bytes.Buffer + bufW := base64.NewEncoder(base64.RawStdEncoding, &sourceBuf) + bufR := rand.New(rand.NewSource(0)) + if _, err := io.CopyN(bufW, bufR, benchmarkDataSize); err != nil { + b.Fatalf("unable to seed random data: %v", err) + } + source = sourceBuf.Bytes() + } + + // Generate a random key for integrity check. + key, err := randomKey() + if err != nil { + b.Fatalf("error generating key: %v", err) + } + + // Define our benchmark functions. Prior to running the readState + // function here, you must execute the writeState function at least + // once (done below). + var stateBuf bytes.Buffer + writeState := func() { + stateBuf.Reset() + w, err := NewWriter(&stateBuf, key, nil, flate.BestSpeed) + if err != nil { + b.Fatalf("error creating writer: %v", err) + } + for done := 0; done < len(source); { + chunk := size // limit size. + if done+chunk > len(source) { + chunk = len(source) - done + } + n, err := w.Write(source[done : done+chunk]) + done += n + if n == 0 && err != nil { + b.Fatalf("error during write: %v", err) + } + } + if err := w.Close(); err != nil { + b.Fatalf("error closing writer: %v", err) + } + } + readState := func() { + tmpBuf := bytes.NewBuffer(stateBuf.Bytes()) + r, _, err := NewReader(tmpBuf, key) + if err != nil { + b.Fatalf("error creating reader: %v", err) + } + for done := 0; done < len(source); { + chunk := size // limit size. + if done+chunk > len(source) { + chunk = len(source) - done + } + n, err := r.Read(source[done : done+chunk]) + done += n + if n == 0 && err != nil { + b.Fatalf("error during read: %v", err) + } + } + } + // Generate the state once without timing to ensure that buffers have + // been appropriately allocated. + writeState() + if write { + b.StartTimer() + for i := 0; i < b.N; i++ { + writeState() + } + b.StopTimer() + } else { + b.StartTimer() + for i := 0; i < b.N; i++ { + readState() + } + b.StopTimer() + } +} + +func BenchmarkWrite1BCompressible(b *testing.B) { + benchmark(b, 1, true, true) +} + +func BenchmarkWrite1BNoncompressible(b *testing.B) { + benchmark(b, 1, true, false) +} + +func BenchmarkWrite4KCompressible(b *testing.B) { + benchmark(b, 4096, true, true) +} + +func BenchmarkWrite4KNoncompressible(b *testing.B) { + benchmark(b, 4096, true, false) +} + +func BenchmarkWrite1MCompressible(b *testing.B) { + benchmark(b, 1024*1024, true, true) +} + +func BenchmarkWrite1MNoncompressible(b *testing.B) { + benchmark(b, 1024*1024, true, false) +} + +func BenchmarkRead1BCompressible(b *testing.B) { + benchmark(b, 1, false, true) +} + +func BenchmarkRead1BNoncompressible(b *testing.B) { + benchmark(b, 1, false, false) +} + +func BenchmarkRead4KCompressible(b *testing.B) { + benchmark(b, 4096, false, true) +} + +func BenchmarkRead4KNoncompressible(b *testing.B) { + benchmark(b, 4096, false, false) +} + +func BenchmarkRead1MCompressible(b *testing.B) { + benchmark(b, 1024*1024, false, true) +} + +func BenchmarkRead1MNoncompressible(b *testing.B) { + benchmark(b, 1024*1024, false, false) +} diff --git a/pkg/state/stats.go b/pkg/state/stats.go new file mode 100644 index 000000000..1ebd8ebb4 --- /dev/null +++ b/pkg/state/stats.go @@ -0,0 +1,133 @@ +// Copyright 2018 Google Inc. +// +// 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 +// +// 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 state + +import ( + "bytes" + "fmt" + "reflect" + "sort" + "time" +) + +type statEntry struct { + count uint + total time.Duration +} + +// Stats tracks encode / decode timing. +// +// This currently provides a meaningful String function and no other way to +// extract stats about individual types. +// +// All exported receivers accept nil. +type Stats struct { + // byType contains a breakdown of time spent by type. + byType map[reflect.Type]*statEntry + + // stack contains objects in progress. + stack []reflect.Type + + // last is the last start time. + last time.Time +} + +// sample adds the given number of samples to the given object. +func (s *Stats) sample(typ reflect.Type, count uint) { + if s.byType == nil { + s.byType = make(map[reflect.Type]*statEntry) + } + entry, ok := s.byType[typ] + if !ok { + entry = new(statEntry) + s.byType[typ] = entry + } + now := time.Now() + entry.count += count + entry.total += now.Sub(s.last) + s.last = now +} + +// Start starts a sample. +func (s *Stats) Start(obj reflect.Value) { + if s == nil { + return + } + if len(s.stack) > 0 { + last := s.stack[len(s.stack)-1] + s.sample(last, 0) + } else { + // First time sample. + s.last = time.Now() + } + s.stack = append(s.stack, obj.Type()) +} + +// Done finishes the current sample. +func (s *Stats) Done() { + if s == nil { + return + } + last := s.stack[len(s.stack)-1] + s.sample(last, 1) + s.stack = s.stack[:len(s.stack)-1] +} + +type sliceEntry struct { + typ reflect.Type + entry *statEntry +} + +// String returns a table representation of the stats. +func (s *Stats) String() string { + if s == nil || len(s.byType) == 0 { + return "(no data)" + } + + // Build a list of stat entries. + ss := make([]sliceEntry, 0, len(s.byType)) + for typ, entry := range s.byType { + ss = append(ss, sliceEntry{ + typ: typ, + entry: entry, + }) + } + + // Sort by total time (descending). + sort.Slice(ss, func(i, j int) bool { + return ss[i].entry.total > ss[j].entry.total + }) + + // Print the stat results. + var ( + buf bytes.Buffer + count uint + total time.Duration + ) + buf.WriteString("\n") + buf.WriteString(fmt.Sprintf("%12s | %8s | %8s | %s\n", "total", "count", "per", "type")) + buf.WriteString("-------------+----------+----------+-------------\n") + for _, se := range ss { + count += se.entry.count + total += se.entry.total + per := se.entry.total / time.Duration(se.entry.count) + buf.WriteString(fmt.Sprintf("%12s | %8d | %8s | %s\n", + se.entry.total, se.entry.count, per, se.typ.String())) + } + buf.WriteString("-------------+----------+----------+-------------\n") + buf.WriteString(fmt.Sprintf("%12s | %8d | %8s | [all]", + total, count, total/time.Duration(count))) + return string(buf.Bytes()) +} -- cgit v1.2.3 From fea624b37a90c0e1efc0c1e7ae7dda7b2d1a0050 Mon Sep 17 00:00:00 2001 From: Zhaozhong Ni Date: Tue, 8 May 2018 10:06:14 -0700 Subject: Sentry: always use "best speed" compression for save and remove the option. PiperOrigin-RevId: 195835861 Change-Id: Ib696b1b571a6b061725a33c535cd7215fe518b97 --- pkg/sentry/state/state.go | 7 +------ pkg/state/statefile/statefile.go | 10 +++++++--- pkg/state/statefile/statefile_test.go | 5 ++--- 3 files changed, 10 insertions(+), 12 deletions(-) (limited to 'pkg/state') diff --git a/pkg/sentry/state/state.go b/pkg/sentry/state/state.go index c306091da..393289926 100644 --- a/pkg/sentry/state/state.go +++ b/pkg/sentry/state/state.go @@ -50,11 +50,6 @@ type SaveOpts struct { // Metadata is save metadata. Metadata map[string]string - // CompressionLevel is the compression level to use. - // - // See statefile.NewWriter for details. - CompressionLevel int - // Callback is called prior to unpause, with any save error. Callback func(err error) } @@ -76,7 +71,7 @@ func (opts SaveOpts) Save(k *kernel.Kernel, w *watchdog.Watchdog) error { addSaveMetadata(opts.Metadata) // Open the statefile. - wc, err := statefile.NewWriter(opts.Destination, opts.Key, opts.Metadata, opts.CompressionLevel) + wc, err := statefile.NewWriter(opts.Destination, opts.Key, opts.Metadata) if err != nil { err = ErrStateFile{err} } else { diff --git a/pkg/state/statefile/statefile.go b/pkg/state/statefile/statefile.go index b25b743b7..64b0a6312 100644 --- a/pkg/state/statefile/statefile.go +++ b/pkg/state/statefile/statefile.go @@ -45,6 +45,7 @@ package statefile import ( "bytes" + "compress/flate" "crypto/hmac" "crypto/sha256" "encoding/json" @@ -86,7 +87,7 @@ var ErrMetadataInvalid = fmt.Errorf("metadata invalid, can't start with _") // NewWriter returns a state data writer for a statefile. // // Note that the returned WriteCloser must be closed. -func NewWriter(w io.Writer, key []byte, metadata map[string]string, compressionLevel int) (io.WriteCloser, error) { +func NewWriter(w io.Writer, key []byte, metadata map[string]string) (io.WriteCloser, error) { if metadata == nil { metadata = make(map[string]string) } @@ -140,8 +141,11 @@ func NewWriter(w io.Writer, key []byte, metadata map[string]string, compressionL w = hashio.NewWriter(w, h) - // Wrap in compression. - return compressio.NewWriter(w, compressionChunkSize, compressionLevel) + // Wrap in compression. We always use "best speed" mode here. When using + // "best compression" mode, there is usually only a little gain in file + // size reduction, which translate to even smaller gain in restore + // latency reduction, while inccuring much more CPU usage at save time. + return compressio.NewWriter(w, compressionChunkSize, flate.BestSpeed) } // MetadataUnsafe reads out the metadata from a state file without verifying any diff --git a/pkg/state/statefile/statefile_test.go b/pkg/state/statefile/statefile_test.go index 6e67b51de..66d9581ed 100644 --- a/pkg/state/statefile/statefile_test.go +++ b/pkg/state/statefile/statefile_test.go @@ -16,7 +16,6 @@ package statefile import ( "bytes" - "compress/flate" crand "crypto/rand" "encoding/base64" "io" @@ -89,7 +88,7 @@ func TestStatefile(t *testing.T) { var bufDecoded bytes.Buffer // Do all the writing. - w, err := NewWriter(&bufEncoded, key, c.metadata, flate.BestSpeed) + w, err := NewWriter(&bufEncoded, key, c.metadata) if err != nil { t.Fatalf("error creating writer: got %v, expected nil", err) } @@ -195,7 +194,7 @@ func benchmark(b *testing.B, size int, write bool, compressible bool) { var stateBuf bytes.Buffer writeState := func() { stateBuf.Reset() - w, err := NewWriter(&stateBuf, key, nil, flate.BestSpeed) + w, err := NewWriter(&stateBuf, key, nil) if err != nil { b.Fatalf("error creating writer: %v", err) } -- cgit v1.2.3 From ad278d69447ddca9c6923e5e830c12f1329438b9 Mon Sep 17 00:00:00 2001 From: Zhaozhong Ni Date: Tue, 8 May 2018 17:23:06 -0700 Subject: state: serialize string as bytes instead of protobuf string. Protobuf strings have to be UTF-8 encoded or 7-bit ASCII. PiperOrigin-RevId: 195902557 Change-Id: I9800afd47ecfa6615e28a2cce7f2532f04f10763 --- pkg/state/decode.go | 2 +- pkg/state/encode.go | 2 +- pkg/state/object.proto | 2 +- pkg/state/printer.go | 2 +- pkg/state/state_test.go | 1 + 5 files changed, 5 insertions(+), 4 deletions(-) (limited to 'pkg/state') diff --git a/pkg/state/decode.go b/pkg/state/decode.go index 05758495b..33ec926c7 100644 --- a/pkg/state/decode.go +++ b/pkg/state/decode.go @@ -335,7 +335,7 @@ func (ds *decodeState) decodeObject(os *objectState, obj reflect.Value, object * case *pb.Object_BoolValue: obj.SetBool(x.BoolValue) case *pb.Object_StringValue: - obj.SetString(x.StringValue) + obj.SetString(string(x.StringValue)) case *pb.Object_Int64Value: obj.SetInt(x.Int64Value) if obj.Int() != x.Int64Value { diff --git a/pkg/state/encode.go b/pkg/state/encode.go index eb6527afc..1cec14f24 100644 --- a/pkg/state/encode.go +++ b/pkg/state/encode.go @@ -308,7 +308,7 @@ func (es *encodeState) encodeObject(obj reflect.Value, mapAsValue bool, format s }}} } case reflect.String: - object = &pb.Object{Value: &pb.Object_StringValue{obj.String()}} + object = &pb.Object{Value: &pb.Object_StringValue{[]byte(obj.String())}} case reflect.Ptr: if obj.IsNil() { // Handled specially in decode; store as a nil value. diff --git a/pkg/state/object.proto b/pkg/state/object.proto index 6595c5519..c78efed2a 100644 --- a/pkg/state/object.proto +++ b/pkg/state/object.proto @@ -114,7 +114,7 @@ message Float32s { message Object { oneof value { bool bool_value = 1; - string string_value = 2; + bytes string_value = 2; int64 int64_value = 3; uint64 uint64_value = 4; double double_value = 5; diff --git a/pkg/state/printer.go b/pkg/state/printer.go index c61ec4a26..881125ddd 100644 --- a/pkg/state/printer.go +++ b/pkg/state/printer.go @@ -30,7 +30,7 @@ func format(graph uint64, depth int, object *pb.Object, html bool) (string, bool case *pb.Object_BoolValue: return fmt.Sprintf("%t", x.BoolValue), x.BoolValue != false case *pb.Object_StringValue: - return fmt.Sprintf("\"%s\"", x.StringValue), x.StringValue != "" + return fmt.Sprintf("\"%s\"", string(x.StringValue)), len(x.StringValue) != 0 case *pb.Object_Int64Value: return fmt.Sprintf("%d", x.Int64Value), x.Int64Value != 0 case *pb.Object_Uint64Value: diff --git a/pkg/state/state_test.go b/pkg/state/state_test.go index d5a739f18..38ad9da9c 100644 --- a/pkg/state/state_test.go +++ b/pkg/state/state_test.go @@ -430,6 +430,7 @@ func TestTypes(t *testing.T) { "", "foo", "bar", + "\xa0", }, }, { -- cgit v1.2.3 From 5581256f879f4249de5ebffddaf0626fcb39eebd Mon Sep 17 00:00:00 2001 From: Zhaozhong Ni Date: Tue, 19 Jun 2018 11:04:05 -0700 Subject: state: include I/O and protobuf time in kernel S/R timing stats. PiperOrigin-RevId: 201205733 Change-Id: I300307b0668989ba7776ab9e3faee71efdd33f46 --- pkg/state/decode.go | 11 ++++++++--- pkg/state/encode.go | 7 ++++++- pkg/state/stats.go | 24 ++++++++++++++++-------- 3 files changed, 30 insertions(+), 12 deletions(-) (limited to 'pkg/state') diff --git a/pkg/state/decode.go b/pkg/state/decode.go index 33ec926c7..b3fbc793c 100644 --- a/pkg/state/decode.go +++ b/pkg/state/decode.go @@ -78,12 +78,11 @@ func (os *objectState) checkComplete(stats *Stats) { if os.blockedBy > 0 { return } + stats.Start(os.obj) // Fire all callbacks. for _, fn := range os.callbacks { - stats.Start(os.obj) fn() - stats.Done() } os.callbacks = nil @@ -93,6 +92,7 @@ func (os *objectState) checkComplete(stats *Stats) { other.checkComplete(stats) } os.blocking = nil + stats.Done() } // waitFor queues a dependency on the given object. @@ -329,6 +329,7 @@ func (ds *decodeState) decodeInterface(os *objectState, obj reflect.Value, i *pb // decodeObject decodes a object value. func (ds *decodeState) decodeObject(os *objectState, obj reflect.Value, object *pb.Object, format string, param interface{}) { ds.push(false, format, param) + ds.stats.Add(obj) ds.stats.Start(obj) switch x := object.GetValue().(type) { @@ -466,12 +467,14 @@ func (ds *decodeState) Deserialize(obj reflect.Value) { // See above, we never process objects while we have no outstanding // interests (other than the very first object). for id := uint64(1); ds.outstanding > 0; id++ { + os := ds.lookup(id) + ds.stats.Start(os.obj) + o, err := ds.readObject() if err != nil { panic(err) } - os := ds.lookup(id) if os != nil { // Decode the object. ds.from = &os.path @@ -483,6 +486,8 @@ func (ds *decodeState) Deserialize(obj reflect.Value) { // registered. ds.deferred[id] = o } + + ds.stats.Done() } // Check the zero-length header at the end. diff --git a/pkg/state/encode.go b/pkg/state/encode.go index 1cec14f24..59914bef4 100644 --- a/pkg/state/encode.go +++ b/pkg/state/encode.go @@ -241,6 +241,7 @@ func (es *encodeState) encodeInterface(obj reflect.Value) *pb.Interface { // If mapAsValue is true, then a map will be encoded directly. func (es *encodeState) encodeObject(obj reflect.Value, mapAsValue bool, format string, param interface{}) (object *pb.Object) { es.push(false, format, param) + es.stats.Add(obj) es.stats.Start(obj) switch obj.Kind() { @@ -354,10 +355,13 @@ func (es *encodeState) Serialize(obj reflect.Value) { // Pop off the list until we're done. for es.pending.Len() > 0 { e := es.pending.Front() - es.pending.Remove(e) // Extract the queued object. qo := e.Value.(queuedObject) + es.stats.Start(qo.obj) + + es.pending.Remove(e) + es.from = &qo.path o := es.encodeObject(qo.obj, true, "", nil) @@ -368,6 +372,7 @@ func (es *encodeState) Serialize(obj reflect.Value) { // Mark as done. es.done.PushBack(e) + es.stats.Done() } // Write a zero-length terminal at the end; this is a sanity check diff --git a/pkg/state/stats.go b/pkg/state/stats.go index 1ebd8ebb4..c4135a889 100644 --- a/pkg/state/stats.go +++ b/pkg/state/stats.go @@ -44,20 +44,28 @@ type Stats struct { last time.Time } -// sample adds the given number of samples to the given object. -func (s *Stats) sample(typ reflect.Type, count uint) { +// sample adds the samples to the given object. +func (s *Stats) sample(typ reflect.Type) { + now := time.Now() + s.byType[typ].total += now.Sub(s.last) + s.last = now +} + +// Add adds a sample count. +func (s *Stats) Add(obj reflect.Value) { + if s == nil { + return + } if s.byType == nil { s.byType = make(map[reflect.Type]*statEntry) } + typ := obj.Type() entry, ok := s.byType[typ] if !ok { entry = new(statEntry) s.byType[typ] = entry } - now := time.Now() - entry.count += count - entry.total += now.Sub(s.last) - s.last = now + entry.count++ } // Start starts a sample. @@ -67,7 +75,7 @@ func (s *Stats) Start(obj reflect.Value) { } if len(s.stack) > 0 { last := s.stack[len(s.stack)-1] - s.sample(last, 0) + s.sample(last) } else { // First time sample. s.last = time.Now() @@ -81,7 +89,7 @@ func (s *Stats) Done() { return } last := s.stack[len(s.stack)-1] - s.sample(last, 1) + s.sample(last) s.stack = s.stack[:len(s.stack)-1] } -- cgit v1.2.3 From 18d899245329daf472c322c81af356958b3e2613 Mon Sep 17 00:00:00 2001 From: Zhaozhong Ni Date: Tue, 19 Jun 2018 17:12:48 -0700 Subject: state: pretty-print primitive type arrays. PiperOrigin-RevId: 201269072 Change-Id: Ia542c5a42b5b5d21c1104a003ddff5279644d309 --- pkg/state/printer.go | 67 ++++++++++++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 65 insertions(+), 2 deletions(-) (limited to 'pkg/state') diff --git a/pkg/state/printer.go b/pkg/state/printer.go index 881125ddd..2c8ce60a5 100644 --- a/pkg/state/printer.go +++ b/pkg/state/printer.go @@ -18,13 +18,15 @@ import ( "fmt" "io" "io/ioutil" + "reflect" "strings" "github.com/golang/protobuf/proto" pb "gvisor.googlesource.com/gvisor/pkg/state/object_go_proto" ) -// format formats a single object, for pretty-printing. +// format formats a single object, for pretty-printing. It also returns whether +// the value is a non-zero value. func format(graph uint64, depth int, object *pb.Object, html bool) (string, bool) { switch x := object.GetValue().(type) { case *pb.Object_BoolValue: @@ -76,7 +78,7 @@ func format(graph uint64, depth int, object *pb.Object, html bool) (string, bool } } if len(zeros) > 0 { - items = append(items, fmt.Sprintf("\t... (%d zero),", len(zeros))) + items = append(items, fmt.Sprintf("\t... (%d zeros),", len(zeros))) } items = append(items, "]") return strings.Join(items, tabs), len(zeros) < len(x.ArrayValue.Contents) @@ -115,6 +117,30 @@ func format(graph uint64, depth int, object *pb.Object, html bool) (string, bool } element, _ := format(graph, depth+1, x.InterfaceValue.Value, html) return fmt.Sprintf("interface(\"%s\"){%s}", x.InterfaceValue.Type, element), true + case *pb.Object_ByteArrayValue: + return printArray(reflect.ValueOf(x.ByteArrayValue)) + case *pb.Object_Uint16ArrayValue: + return printArray(reflect.ValueOf(x.Uint16ArrayValue.Values)) + case *pb.Object_Uint32ArrayValue: + return printArray(reflect.ValueOf(x.Uint32ArrayValue.Values)) + case *pb.Object_Uint64ArrayValue: + return printArray(reflect.ValueOf(x.Uint64ArrayValue.Values)) + case *pb.Object_UintptrArrayValue: + return printArray(castSlice(reflect.ValueOf(x.UintptrArrayValue.Values), reflect.TypeOf(uintptr(0)))) + case *pb.Object_Int8ArrayValue: + return printArray(castSlice(reflect.ValueOf(x.Int8ArrayValue.Values), reflect.TypeOf(int8(0)))) + case *pb.Object_Int16ArrayValue: + return printArray(reflect.ValueOf(x.Int16ArrayValue.Values)) + case *pb.Object_Int32ArrayValue: + return printArray(reflect.ValueOf(x.Int32ArrayValue.Values)) + case *pb.Object_Int64ArrayValue: + return printArray(reflect.ValueOf(x.Int64ArrayValue.Values)) + case *pb.Object_BoolArrayValue: + return printArray(reflect.ValueOf(x.BoolArrayValue.Values)) + case *pb.Object_Float64ArrayValue: + return printArray(reflect.ValueOf(x.Float64ArrayValue.Values)) + case *pb.Object_Float32ArrayValue: + return printArray(reflect.ValueOf(x.Float32ArrayValue.Values)) } // Should not happen, but tolerate. @@ -186,3 +212,40 @@ func PrettyPrint(w io.Writer, r io.Reader, html bool) error { return nil } + +func printArray(s reflect.Value) (string, bool) { + zero := reflect.Zero(s.Type().Elem()).Interface() + z := "0" + switch s.Type().Elem().Kind() { + case reflect.Bool: + z = "false" + case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64: + case reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64, reflect.Uintptr: + case reflect.Float32, reflect.Float64: + default: + return fmt.Sprintf("unexpected non-primitive type array: %#v", s.Interface()), true + } + + zeros := 0 + items := make([]string, 0, s.Len()) + for i := 0; i <= s.Len(); i++ { + if i < s.Len() && reflect.DeepEqual(s.Index(i).Interface(), zero) { + zeros++ + continue + } + if zeros > 0 { + if zeros <= 4 { + for ; zeros > 0; zeros-- { + items = append(items, z) + } + } else { + items = append(items, fmt.Sprintf("(%d %ss)", zeros, z)) + zeros = 0 + } + } + if i < s.Len() { + items = append(items, fmt.Sprintf("%v", s.Index(i).Interface())) + } + } + return "[" + strings.Join(items, ",") + "]", zeros < s.Len() +} -- cgit v1.2.3 From 9cd69c2f3db7fd4b30d14d86be5fb5a3401054e5 Mon Sep 17 00:00:00 2001 From: Michael Pratt Date: Tue, 10 Jul 2018 15:54:17 -0700 Subject: Internal change PiperOrigin-RevId: 204028082 Change-Id: I4251cce10aace43f9b9a80c36204ef66f1b329df --- pkg/hashio/hashio.go | 3 ++- pkg/state/statefile/statefile.go | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) (limited to 'pkg/state') diff --git a/pkg/hashio/hashio.go b/pkg/hashio/hashio.go index d97948850..e0e8ef413 100644 --- a/pkg/hashio/hashio.go +++ b/pkg/hashio/hashio.go @@ -37,11 +37,12 @@ passed hash verification. Hence the client code can safely do any kind of package hashio import ( - "crypto/hmac" "errors" "hash" "io" "sync" + + "crypto/hmac" ) // SegmentSize is the unit we split payload data and insert hash at. diff --git a/pkg/state/statefile/statefile.go b/pkg/state/statefile/statefile.go index 64b0a6312..0b4eff8fa 100644 --- a/pkg/state/statefile/statefile.go +++ b/pkg/state/statefile/statefile.go @@ -46,8 +46,6 @@ package statefile import ( "bytes" "compress/flate" - "crypto/hmac" - "crypto/sha256" "encoding/json" "fmt" "hash" @@ -55,6 +53,8 @@ import ( "strings" "time" + "crypto/hmac" + "crypto/sha256" "gvisor.googlesource.com/gvisor/pkg/binary" "gvisor.googlesource.com/gvisor/pkg/compressio" "gvisor.googlesource.com/gvisor/pkg/hashio" -- cgit v1.2.3 From be7fcbc5582fe831b5ec63f773d867d7591e27a1 Mon Sep 17 00:00:00 2001 From: Zhaozhong Ni Date: Fri, 27 Jul 2018 10:16:27 -0700 Subject: stateify: support explicit annotation mode; convert refs and stack packages. We have been unnecessarily creating too many savable types implicitly. PiperOrigin-RevId: 206334201 Change-Id: Idc5a3a14bfb7ee125c4f2bb2b1c53164e46f29a8 --- pkg/abi/BUILD | 3 +- pkg/abi/linux/BUILD | 3 +- pkg/amutex/BUILD | 2 +- pkg/atomicbitops/BUILD | 2 +- pkg/binary/BUILD | 2 +- pkg/bits/BUILD | 2 +- pkg/bpf/BUILD | 3 +- pkg/compressio/BUILD | 2 +- pkg/control/client/BUILD | 2 +- pkg/control/server/BUILD | 2 +- pkg/cpuid/BUILD | 3 +- pkg/dhcp/BUILD | 2 +- pkg/eventchannel/BUILD | 2 +- pkg/fd/BUILD | 2 +- pkg/gate/BUILD | 2 +- pkg/hashio/BUILD | 2 +- pkg/ilist/BUILD | 3 +- pkg/linewriter/BUILD | 2 +- pkg/log/BUILD | 2 +- pkg/metric/BUILD | 2 +- pkg/p9/BUILD | 2 +- pkg/p9/p9test/BUILD | 2 +- pkg/rand/BUILD | 2 +- pkg/refs/BUILD | 20 +------- pkg/refs/refcounter.go | 4 ++ pkg/refs/refcounter_state.go | 1 + pkg/seccomp/BUILD | 3 +- pkg/secio/BUILD | 2 +- pkg/segment/test/BUILD | 2 +- pkg/sentry/arch/BUILD | 3 +- pkg/sentry/context/BUILD | 2 +- pkg/sentry/context/contexttest/BUILD | 3 +- pkg/sentry/control/BUILD | 2 +- pkg/sentry/device/BUILD | 2 +- pkg/sentry/fs/BUILD | 3 +- pkg/sentry/fs/anon/BUILD | 2 +- pkg/sentry/fs/ashmem/BUILD | 3 +- pkg/sentry/fs/binder/BUILD | 3 +- pkg/sentry/fs/dev/BUILD | 3 +- pkg/sentry/fs/fdpipe/BUILD | 3 +- pkg/sentry/fs/filetest/BUILD | 3 +- pkg/sentry/fs/fsutil/BUILD | 3 +- pkg/sentry/fs/gofer/BUILD | 3 +- pkg/sentry/fs/host/BUILD | 3 +- pkg/sentry/fs/lock/BUILD | 3 +- pkg/sentry/fs/proc/BUILD | 3 +- pkg/sentry/fs/proc/device/BUILD | 2 +- pkg/sentry/fs/proc/seqfile/BUILD | 3 +- pkg/sentry/fs/ramfs/BUILD | 3 +- pkg/sentry/fs/ramfs/test/BUILD | 3 +- pkg/sentry/fs/sys/BUILD | 3 +- pkg/sentry/fs/timerfd/BUILD | 3 +- pkg/sentry/fs/tmpfs/BUILD | 3 +- pkg/sentry/fs/tty/BUILD | 3 +- pkg/sentry/hostcpu/BUILD | 2 +- pkg/sentry/inet/BUILD | 4 +- pkg/sentry/kernel/BUILD | 3 +- pkg/sentry/kernel/auth/BUILD | 3 +- pkg/sentry/kernel/epoll/BUILD | 3 +- pkg/sentry/kernel/eventfd/BUILD | 3 +- pkg/sentry/kernel/fasync/BUILD | 2 +- pkg/sentry/kernel/futex/BUILD | 3 +- pkg/sentry/kernel/kdefs/BUILD | 2 +- pkg/sentry/kernel/memevent/BUILD | 2 +- pkg/sentry/kernel/pipe/BUILD | 3 +- pkg/sentry/kernel/sched/BUILD | 2 +- pkg/sentry/kernel/semaphore/BUILD | 3 +- pkg/sentry/kernel/shm/BUILD | 3 +- pkg/sentry/kernel/time/BUILD | 3 +- pkg/sentry/limits/BUILD | 3 +- pkg/sentry/loader/BUILD | 4 +- pkg/sentry/memmap/BUILD | 3 +- pkg/sentry/memutil/BUILD | 2 +- pkg/sentry/mm/BUILD | 3 +- pkg/sentry/platform/BUILD | 3 +- pkg/sentry/platform/filemem/BUILD | 3 +- pkg/sentry/platform/interrupt/BUILD | 2 +- pkg/sentry/platform/kvm/BUILD | 2 +- pkg/sentry/platform/kvm/testutil/BUILD | 2 +- pkg/sentry/platform/procid/BUILD | 2 +- pkg/sentry/platform/ptrace/BUILD | 2 +- pkg/sentry/platform/ring0/BUILD | 2 +- pkg/sentry/platform/ring0/pagetables/BUILD | 2 +- pkg/sentry/platform/safecopy/BUILD | 2 +- pkg/sentry/safemem/BUILD | 2 +- pkg/sentry/sighandling/BUILD | 2 +- pkg/sentry/socket/BUILD | 3 +- pkg/sentry/socket/control/BUILD | 3 +- pkg/sentry/socket/epsocket/BUILD | 3 +- pkg/sentry/socket/hostinet/BUILD | 3 +- pkg/sentry/socket/netlink/BUILD | 3 +- pkg/sentry/socket/netlink/port/BUILD | 3 +- pkg/sentry/socket/netlink/route/BUILD | 3 +- pkg/sentry/socket/rpcinet/BUILD | 2 +- pkg/sentry/socket/rpcinet/conn/BUILD | 2 +- pkg/sentry/socket/rpcinet/notifier/BUILD | 2 +- pkg/sentry/socket/unix/BUILD | 3 +- pkg/sentry/state/BUILD | 2 +- pkg/sentry/strace/BUILD | 2 +- pkg/sentry/syscalls/BUILD | 2 +- pkg/sentry/syscalls/linux/BUILD | 3 +- pkg/sentry/time/BUILD | 2 +- pkg/sentry/uniqueid/BUILD | 2 +- pkg/sentry/usage/BUILD | 3 +- pkg/sentry/usermem/BUILD | 3 +- pkg/sentry/watchdog/BUILD | 2 +- pkg/sleep/BUILD | 2 +- pkg/state/BUILD | 2 +- pkg/state/statefile/BUILD | 2 +- pkg/sync/BUILD | 2 +- pkg/sync/seqatomictest/BUILD | 2 +- pkg/syserr/BUILD | 2 +- pkg/syserror/BUILD | 2 +- pkg/tcpip/BUILD | 3 +- pkg/tcpip/adapters/gonet/BUILD | 2 +- pkg/tcpip/buffer/BUILD | 3 +- pkg/tcpip/checker/BUILD | 2 +- pkg/tcpip/header/BUILD | 3 +- pkg/tcpip/link/channel/BUILD | 2 +- pkg/tcpip/link/fdbased/BUILD | 2 +- pkg/tcpip/link/loopback/BUILD | 2 +- pkg/tcpip/link/rawfile/BUILD | 2 +- pkg/tcpip/link/sharedmem/BUILD | 2 +- pkg/tcpip/link/sharedmem/pipe/BUILD | 2 +- pkg/tcpip/link/sharedmem/queue/BUILD | 2 +- pkg/tcpip/link/sniffer/BUILD | 2 +- pkg/tcpip/link/tun/BUILD | 2 +- pkg/tcpip/link/waitable/BUILD | 2 +- pkg/tcpip/network/BUILD | 2 +- pkg/tcpip/network/arp/BUILD | 2 +- pkg/tcpip/network/fragmentation/BUILD | 3 +- pkg/tcpip/network/hash/BUILD | 2 +- pkg/tcpip/network/ipv4/BUILD | 2 +- pkg/tcpip/network/ipv6/BUILD | 2 +- pkg/tcpip/ports/BUILD | 2 +- pkg/tcpip/seqnum/BUILD | 3 +- pkg/tcpip/stack/BUILD | 15 +----- pkg/tcpip/stack/registration.go | 2 + pkg/tcpip/transport/ping/BUILD | 3 +- pkg/tcpip/transport/queue/BUILD | 3 +- pkg/tcpip/transport/tcp/BUILD | 3 +- pkg/tcpip/transport/tcp/testing/context/BUILD | 2 +- pkg/tcpip/transport/tcpconntrack/BUILD | 2 +- pkg/tcpip/transport/udp/BUILD | 3 +- pkg/tcpip/transport/unix/BUILD | 3 +- pkg/tmutex/BUILD | 2 +- pkg/unet/BUILD | 2 +- pkg/urpc/BUILD | 2 +- pkg/waiter/BUILD | 3 +- pkg/waiter/fdnotifier/BUILD | 2 +- tools/go_stateify/defs.bzl | 58 ++++++++++++++++++----- tools/go_stateify/main.go | 66 +++++++++++++++++++++------ 152 files changed, 255 insertions(+), 267 deletions(-) (limited to 'pkg/state') diff --git a/pkg/abi/BUILD b/pkg/abi/BUILD index 4d507161f..f1e6bac67 100644 --- a/pkg/abi/BUILD +++ b/pkg/abi/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "abi_state", diff --git a/pkg/abi/linux/BUILD b/pkg/abi/linux/BUILD index ae7e4378c..38b4829c9 100644 --- a/pkg/abi/linux/BUILD +++ b/pkg/abi/linux/BUILD @@ -4,8 +4,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "linux_state", diff --git a/pkg/amutex/BUILD b/pkg/amutex/BUILD index 442096319..84e6b79a5 100644 --- a/pkg/amutex/BUILD +++ b/pkg/amutex/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "amutex", diff --git a/pkg/atomicbitops/BUILD b/pkg/atomicbitops/BUILD index f20a9f855..a8dd17825 100644 --- a/pkg/atomicbitops/BUILD +++ b/pkg/atomicbitops/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "atomicbitops", diff --git a/pkg/binary/BUILD b/pkg/binary/BUILD index 16f08b13f..586d05634 100644 --- a/pkg/binary/BUILD +++ b/pkg/binary/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "binary", diff --git a/pkg/bits/BUILD b/pkg/bits/BUILD index 9897e5dc3..8c943b615 100644 --- a/pkg/bits/BUILD +++ b/pkg/bits/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") go_library( diff --git a/pkg/bpf/BUILD b/pkg/bpf/BUILD index d4f12f13a..403270049 100644 --- a/pkg/bpf/BUILD +++ b/pkg/bpf/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "bpf_state", diff --git a/pkg/compressio/BUILD b/pkg/compressio/BUILD index 721b2d983..d70f982c1 100644 --- a/pkg/compressio/BUILD +++ b/pkg/compressio/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "compressio", diff --git a/pkg/control/client/BUILD b/pkg/control/client/BUILD index 9e1c058e4..d58cd1b71 100644 --- a/pkg/control/client/BUILD +++ b/pkg/control/client/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "client", diff --git a/pkg/control/server/BUILD b/pkg/control/server/BUILD index 2d0fdd8b8..c3f74a532 100644 --- a/pkg/control/server/BUILD +++ b/pkg/control/server/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "server", diff --git a/pkg/cpuid/BUILD b/pkg/cpuid/BUILD index a503b7ae8..9a0ca1b33 100644 --- a/pkg/cpuid/BUILD +++ b/pkg/cpuid/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "cpuid_state", diff --git a/pkg/dhcp/BUILD b/pkg/dhcp/BUILD index f56969ad8..bd9f592b4 100644 --- a/pkg/dhcp/BUILD +++ b/pkg/dhcp/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "dhcp", diff --git a/pkg/eventchannel/BUILD b/pkg/eventchannel/BUILD index ea0c587be..ac2ea869d 100644 --- a/pkg/eventchannel/BUILD +++ b/pkg/eventchannel/BUILD @@ -1,7 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 +load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "eventchannel", diff --git a/pkg/fd/BUILD b/pkg/fd/BUILD index e69d83d06..435b6fa34 100644 --- a/pkg/fd/BUILD +++ b/pkg/fd/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "fd", diff --git a/pkg/gate/BUILD b/pkg/gate/BUILD index 0b8b01da8..872eff531 100644 --- a/pkg/gate/BUILD +++ b/pkg/gate/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "gate", diff --git a/pkg/hashio/BUILD b/pkg/hashio/BUILD index aaa58b58f..5736e2e73 100644 --- a/pkg/hashio/BUILD +++ b/pkg/hashio/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "hashio", diff --git a/pkg/ilist/BUILD b/pkg/ilist/BUILD index 16a738e89..e32f26ffa 100644 --- a/pkg/ilist/BUILD +++ b/pkg/ilist/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "list_state", diff --git a/pkg/linewriter/BUILD b/pkg/linewriter/BUILD index 4a96c6f1d..6c3795432 100644 --- a/pkg/linewriter/BUILD +++ b/pkg/linewriter/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "linewriter", diff --git a/pkg/log/BUILD b/pkg/log/BUILD index 2530cfd18..fc9281079 100644 --- a/pkg/log/BUILD +++ b/pkg/log/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "log", diff --git a/pkg/metric/BUILD b/pkg/metric/BUILD index e3f50d528..c0cd40c7b 100644 --- a/pkg/metric/BUILD +++ b/pkg/metric/BUILD @@ -1,7 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "metric", diff --git a/pkg/p9/BUILD b/pkg/p9/BUILD index f348ff2e9..1cf5c6458 100644 --- a/pkg/p9/BUILD +++ b/pkg/p9/BUILD @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") package( default_visibility = ["//visibility:public"], diff --git a/pkg/p9/p9test/BUILD b/pkg/p9/p9test/BUILD index 339c86089..d6f428e11 100644 --- a/pkg/p9/p9test/BUILD +++ b/pkg/p9/p9test/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_test( name = "p9test_test", diff --git a/pkg/rand/BUILD b/pkg/rand/BUILD index 2bb59f895..12e6cf25a 100644 --- a/pkg/rand/BUILD +++ b/pkg/rand/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "rand", diff --git a/pkg/refs/BUILD b/pkg/refs/BUILD index 4b7c9345d..3ea877ccf 100644 --- a/pkg/refs/BUILD +++ b/pkg/refs/BUILD @@ -1,32 +1,16 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") - -go_stateify( - name = "refs_state", - srcs = [ - "refcounter.go", - "refcounter_state.go", - ], - out = "refs_state.go", - package = "refs", -) +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "refs", srcs = [ "refcounter.go", "refcounter_state.go", - "refs_state.go", ], importpath = "gvisor.googlesource.com/gvisor/pkg/refs", visibility = ["//:sandbox"], - deps = [ - "//pkg/ilist", - "//pkg/log", - "//pkg/state", - ], + deps = ["//pkg/ilist"], ) go_test( diff --git a/pkg/refs/refcounter.go b/pkg/refs/refcounter.go index 3162001e1..0d44c2499 100644 --- a/pkg/refs/refcounter.go +++ b/pkg/refs/refcounter.go @@ -58,6 +58,8 @@ type WeakRefUser interface { } // WeakRef is a weak reference. +// +// +stateify savable type WeakRef struct { ilist.Entry `state:"nosave"` @@ -177,6 +179,8 @@ func (w *WeakRef) zap() { // // N.B. To allow the zero-object to be initialized, the count is offset by // 1, that is, when refCount is n, there are really n+1 references. +// +// +stateify savable type AtomicRefCount struct { // refCount is composed of two fields: // diff --git a/pkg/refs/refcounter_state.go b/pkg/refs/refcounter_state.go index 1be67f951..093eae785 100644 --- a/pkg/refs/refcounter_state.go +++ b/pkg/refs/refcounter_state.go @@ -14,6 +14,7 @@ package refs +// +stateify savable type savedReference struct { obj interface{} } diff --git a/pkg/seccomp/BUILD b/pkg/seccomp/BUILD index cadd24505..b3e2f0b38 100644 --- a/pkg/seccomp/BUILD +++ b/pkg/seccomp/BUILD @@ -1,6 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_embed_data", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_embed_data") go_binary( name = "victim", diff --git a/pkg/secio/BUILD b/pkg/secio/BUILD index 9a28d2c1f..0ed38c64a 100644 --- a/pkg/secio/BUILD +++ b/pkg/secio/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "secio", diff --git a/pkg/segment/test/BUILD b/pkg/segment/test/BUILD index 9d398d71a..bdf53e24e 100644 --- a/pkg/segment/test/BUILD +++ b/pkg/segment/test/BUILD @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") package( default_visibility = ["//visibility:private"], diff --git a/pkg/sentry/arch/BUILD b/pkg/sentry/arch/BUILD index a88f57ac7..0a2a35400 100644 --- a/pkg/sentry/arch/BUILD +++ b/pkg/sentry/arch/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "arch_state", diff --git a/pkg/sentry/context/BUILD b/pkg/sentry/context/BUILD index ff39f94ba..2a7a6df23 100644 --- a/pkg/sentry/context/BUILD +++ b/pkg/sentry/context/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "context", diff --git a/pkg/sentry/context/contexttest/BUILD b/pkg/sentry/context/contexttest/BUILD index 5977344de..591b11a4d 100644 --- a/pkg/sentry/context/contexttest/BUILD +++ b/pkg/sentry/context/contexttest/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "contexttest_state", diff --git a/pkg/sentry/control/BUILD b/pkg/sentry/control/BUILD index 6169891f7..fbdde0721 100644 --- a/pkg/sentry/control/BUILD +++ b/pkg/sentry/control/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "control", diff --git a/pkg/sentry/device/BUILD b/pkg/sentry/device/BUILD index 1a8b461ba..69c99b0b3 100644 --- a/pkg/sentry/device/BUILD +++ b/pkg/sentry/device/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "device", diff --git a/pkg/sentry/fs/BUILD b/pkg/sentry/fs/BUILD index 9b7264753..e3c9a9b70 100644 --- a/pkg/sentry/fs/BUILD +++ b/pkg/sentry/fs/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "fs_state", diff --git a/pkg/sentry/fs/anon/BUILD b/pkg/sentry/fs/anon/BUILD index 6b18aee47..ff4ab850a 100644 --- a/pkg/sentry/fs/anon/BUILD +++ b/pkg/sentry/fs/anon/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "anon", diff --git a/pkg/sentry/fs/ashmem/BUILD b/pkg/sentry/fs/ashmem/BUILD index e20e22a0f..9f166799a 100644 --- a/pkg/sentry/fs/ashmem/BUILD +++ b/pkg/sentry/fs/ashmem/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_stateify( diff --git a/pkg/sentry/fs/binder/BUILD b/pkg/sentry/fs/binder/BUILD index 15f91699f..ec3928baf 100644 --- a/pkg/sentry/fs/binder/BUILD +++ b/pkg/sentry/fs/binder/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "binder_state", diff --git a/pkg/sentry/fs/dev/BUILD b/pkg/sentry/fs/dev/BUILD index d33a19c2f..ea41615fd 100644 --- a/pkg/sentry/fs/dev/BUILD +++ b/pkg/sentry/fs/dev/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "dev_state", diff --git a/pkg/sentry/fs/fdpipe/BUILD b/pkg/sentry/fs/fdpipe/BUILD index 9e1f65d3e..4fcb06f1f 100644 --- a/pkg/sentry/fs/fdpipe/BUILD +++ b/pkg/sentry/fs/fdpipe/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "pipe_state", diff --git a/pkg/sentry/fs/filetest/BUILD b/pkg/sentry/fs/filetest/BUILD index 51a390d77..f481c57fb 100644 --- a/pkg/sentry/fs/filetest/BUILD +++ b/pkg/sentry/fs/filetest/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "filetest_state", diff --git a/pkg/sentry/fs/fsutil/BUILD b/pkg/sentry/fs/fsutil/BUILD index 4fa6395f7..6eea64298 100644 --- a/pkg/sentry/fs/fsutil/BUILD +++ b/pkg/sentry/fs/fsutil/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "fsutil_state", diff --git a/pkg/sentry/fs/gofer/BUILD b/pkg/sentry/fs/gofer/BUILD index e6f659c53..1277379e7 100644 --- a/pkg/sentry/fs/gofer/BUILD +++ b/pkg/sentry/fs/gofer/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "gofer_state", diff --git a/pkg/sentry/fs/host/BUILD b/pkg/sentry/fs/host/BUILD index 97b64daed..23ec66f50 100644 --- a/pkg/sentry/fs/host/BUILD +++ b/pkg/sentry/fs/host/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "host_state", diff --git a/pkg/sentry/fs/lock/BUILD b/pkg/sentry/fs/lock/BUILD index c15dde800..2607d7ed3 100644 --- a/pkg/sentry/fs/lock/BUILD +++ b/pkg/sentry/fs/lock/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "lock_state", diff --git a/pkg/sentry/fs/proc/BUILD b/pkg/sentry/fs/proc/BUILD index 21b5fc0c3..870df47b2 100644 --- a/pkg/sentry/fs/proc/BUILD +++ b/pkg/sentry/fs/proc/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "proc_state", diff --git a/pkg/sentry/fs/proc/device/BUILD b/pkg/sentry/fs/proc/device/BUILD index b62062bd7..34582f275 100644 --- a/pkg/sentry/fs/proc/device/BUILD +++ b/pkg/sentry/fs/proc/device/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "device", diff --git a/pkg/sentry/fs/proc/seqfile/BUILD b/pkg/sentry/fs/proc/seqfile/BUILD index 48dd25e5b..c84f7e20d 100644 --- a/pkg/sentry/fs/proc/seqfile/BUILD +++ b/pkg/sentry/fs/proc/seqfile/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "seqfile_state", diff --git a/pkg/sentry/fs/ramfs/BUILD b/pkg/sentry/fs/ramfs/BUILD index 663a1aeb9..d84f2c624 100644 --- a/pkg/sentry/fs/ramfs/BUILD +++ b/pkg/sentry/fs/ramfs/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "ramfs_state", diff --git a/pkg/sentry/fs/ramfs/test/BUILD b/pkg/sentry/fs/ramfs/test/BUILD index 074b0f5ad..57fee45e2 100644 --- a/pkg/sentry/fs/ramfs/test/BUILD +++ b/pkg/sentry/fs/ramfs/test/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "test_state", diff --git a/pkg/sentry/fs/sys/BUILD b/pkg/sentry/fs/sys/BUILD index 0ae2cbac8..095ff1f25 100644 --- a/pkg/sentry/fs/sys/BUILD +++ b/pkg/sentry/fs/sys/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "sys_state", diff --git a/pkg/sentry/fs/timerfd/BUILD b/pkg/sentry/fs/timerfd/BUILD index 7fddc29f4..8b1b7872e 100644 --- a/pkg/sentry/fs/timerfd/BUILD +++ b/pkg/sentry/fs/timerfd/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "timerfd_state", diff --git a/pkg/sentry/fs/tmpfs/BUILD b/pkg/sentry/fs/tmpfs/BUILD index be4e695d3..473ab4296 100644 --- a/pkg/sentry/fs/tmpfs/BUILD +++ b/pkg/sentry/fs/tmpfs/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "tmpfs_state", diff --git a/pkg/sentry/fs/tty/BUILD b/pkg/sentry/fs/tty/BUILD index fce327dfe..363897b2c 100644 --- a/pkg/sentry/fs/tty/BUILD +++ b/pkg/sentry/fs/tty/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "tty_state", diff --git a/pkg/sentry/hostcpu/BUILD b/pkg/sentry/hostcpu/BUILD index 9457618d8..f362d15c8 100644 --- a/pkg/sentry/hostcpu/BUILD +++ b/pkg/sentry/hostcpu/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "hostcpu", diff --git a/pkg/sentry/inet/BUILD b/pkg/sentry/inet/BUILD index 1150ced57..eaf8f15b2 100644 --- a/pkg/sentry/inet/BUILD +++ b/pkg/sentry/inet/BUILD @@ -1,11 +1,9 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - package( default_visibility = ["//:sandbox"], licenses = ["notice"], # Apache 2.0 ) -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "inet_state", diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD index 07568b47c..c4a7dacb2 100644 --- a/pkg/sentry/kernel/BUILD +++ b/pkg/sentry/kernel/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "kernel_state", diff --git a/pkg/sentry/kernel/auth/BUILD b/pkg/sentry/kernel/auth/BUILD index 7f0680b88..5b7b30557 100644 --- a/pkg/sentry/kernel/auth/BUILD +++ b/pkg/sentry/kernel/auth/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "auth_state", diff --git a/pkg/sentry/kernel/epoll/BUILD b/pkg/sentry/kernel/epoll/BUILD index 04651d961..7d491efbc 100644 --- a/pkg/sentry/kernel/epoll/BUILD +++ b/pkg/sentry/kernel/epoll/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "epoll_autogen_state", diff --git a/pkg/sentry/kernel/eventfd/BUILD b/pkg/sentry/kernel/eventfd/BUILD index 561ced852..7ec179bd8 100644 --- a/pkg/sentry/kernel/eventfd/BUILD +++ b/pkg/sentry/kernel/eventfd/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "eventfd_state", diff --git a/pkg/sentry/kernel/fasync/BUILD b/pkg/sentry/kernel/fasync/BUILD index 8d06e1182..17749c0de 100644 --- a/pkg/sentry/kernel/fasync/BUILD +++ b/pkg/sentry/kernel/fasync/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "fasync", diff --git a/pkg/sentry/kernel/futex/BUILD b/pkg/sentry/kernel/futex/BUILD index de9897c58..a97a43549 100644 --- a/pkg/sentry/kernel/futex/BUILD +++ b/pkg/sentry/kernel/futex/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_template_instance( name = "waiter_list", diff --git a/pkg/sentry/kernel/kdefs/BUILD b/pkg/sentry/kernel/kdefs/BUILD index b6c00042a..fe6fa2260 100644 --- a/pkg/sentry/kernel/kdefs/BUILD +++ b/pkg/sentry/kernel/kdefs/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "kdefs", diff --git a/pkg/sentry/kernel/memevent/BUILD b/pkg/sentry/kernel/memevent/BUILD index c7779e1d5..66899910c 100644 --- a/pkg/sentry/kernel/memevent/BUILD +++ b/pkg/sentry/kernel/memevent/BUILD @@ -1,7 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 +load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "memevent", diff --git a/pkg/sentry/kernel/pipe/BUILD b/pkg/sentry/kernel/pipe/BUILD index ca9825f9d..4600d19bd 100644 --- a/pkg/sentry/kernel/pipe/BUILD +++ b/pkg/sentry/kernel/pipe/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "pipe_state", diff --git a/pkg/sentry/kernel/sched/BUILD b/pkg/sentry/kernel/sched/BUILD index b533c51c4..125792f39 100644 --- a/pkg/sentry/kernel/sched/BUILD +++ b/pkg/sentry/kernel/sched/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "sched", diff --git a/pkg/sentry/kernel/semaphore/BUILD b/pkg/sentry/kernel/semaphore/BUILD index 1656ad126..969145fe1 100644 --- a/pkg/sentry/kernel/semaphore/BUILD +++ b/pkg/sentry/kernel/semaphore/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_template_instance( name = "waiter_list", diff --git a/pkg/sentry/kernel/shm/BUILD b/pkg/sentry/kernel/shm/BUILD index 182cc1c76..0f88eb0ac 100644 --- a/pkg/sentry/kernel/shm/BUILD +++ b/pkg/sentry/kernel/shm/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "shm_state", diff --git a/pkg/sentry/kernel/time/BUILD b/pkg/sentry/kernel/time/BUILD index 84f31b2dc..b3ed42aa4 100644 --- a/pkg/sentry/kernel/time/BUILD +++ b/pkg/sentry/kernel/time/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "time_state", diff --git a/pkg/sentry/limits/BUILD b/pkg/sentry/limits/BUILD index 06c3e72b0..3ce41cacc 100644 --- a/pkg/sentry/limits/BUILD +++ b/pkg/sentry/limits/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "limits_state", diff --git a/pkg/sentry/loader/BUILD b/pkg/sentry/loader/BUILD index 01a0ec426..e63052c6d 100644 --- a/pkg/sentry/loader/BUILD +++ b/pkg/sentry/loader/BUILD @@ -1,7 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_embed_data", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("@io_bazel_rules_go//go:def.bzl", "go_embed_data") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_embed_data( name = "vdso_bin", diff --git a/pkg/sentry/memmap/BUILD b/pkg/sentry/memmap/BUILD index 7525fea45..2e367e189 100644 --- a/pkg/sentry/memmap/BUILD +++ b/pkg/sentry/memmap/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "memmap_state", diff --git a/pkg/sentry/memutil/BUILD b/pkg/sentry/memutil/BUILD index a387a0c9f..341b30b98 100644 --- a/pkg/sentry/memutil/BUILD +++ b/pkg/sentry/memutil/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "memutil", diff --git a/pkg/sentry/mm/BUILD b/pkg/sentry/mm/BUILD index 258389bb2..3f396986a 100644 --- a/pkg/sentry/mm/BUILD +++ b/pkg/sentry/mm/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "mm_state", diff --git a/pkg/sentry/platform/BUILD b/pkg/sentry/platform/BUILD index d5be81f8d..15a7fbbc3 100644 --- a/pkg/sentry/platform/BUILD +++ b/pkg/sentry/platform/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "platform_state", diff --git a/pkg/sentry/platform/filemem/BUILD b/pkg/sentry/platform/filemem/BUILD index 3c4d5b0b6..dadba1d38 100644 --- a/pkg/sentry/platform/filemem/BUILD +++ b/pkg/sentry/platform/filemem/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "filemem_autogen_state", diff --git a/pkg/sentry/platform/interrupt/BUILD b/pkg/sentry/platform/interrupt/BUILD index 33dde2a31..35121321a 100644 --- a/pkg/sentry/platform/interrupt/BUILD +++ b/pkg/sentry/platform/interrupt/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "interrupt", diff --git a/pkg/sentry/platform/kvm/BUILD b/pkg/sentry/platform/kvm/BUILD index 673393fad..4ef9e20d7 100644 --- a/pkg/sentry/platform/kvm/BUILD +++ b/pkg/sentry/platform/kvm/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/sentry/platform/kvm/testutil/BUILD b/pkg/sentry/platform/kvm/testutil/BUILD index 8533a8d89..e779e3893 100644 --- a/pkg/sentry/platform/kvm/testutil/BUILD +++ b/pkg/sentry/platform/kvm/testutil/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "testutil", diff --git a/pkg/sentry/platform/procid/BUILD b/pkg/sentry/platform/procid/BUILD index 5db4f6261..ba68d48f4 100644 --- a/pkg/sentry/platform/procid/BUILD +++ b/pkg/sentry/platform/procid/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "procid", diff --git a/pkg/sentry/platform/ptrace/BUILD b/pkg/sentry/platform/ptrace/BUILD index 16b0b3c69..ceee895dc 100644 --- a/pkg/sentry/platform/ptrace/BUILD +++ b/pkg/sentry/platform/ptrace/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "ptrace", diff --git a/pkg/sentry/platform/ring0/BUILD b/pkg/sentry/platform/ring0/BUILD index 2df232a64..2485eb2eb 100644 --- a/pkg/sentry/platform/ring0/BUILD +++ b/pkg/sentry/platform/ring0/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") go_template( diff --git a/pkg/sentry/platform/ring0/pagetables/BUILD b/pkg/sentry/platform/ring0/pagetables/BUILD index 023e298a0..7a86e2234 100644 --- a/pkg/sentry/platform/ring0/pagetables/BUILD +++ b/pkg/sentry/platform/ring0/pagetables/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") go_template( diff --git a/pkg/sentry/platform/safecopy/BUILD b/pkg/sentry/platform/safecopy/BUILD index 8b9f29403..7dcf6e561 100644 --- a/pkg/sentry/platform/safecopy/BUILD +++ b/pkg/sentry/platform/safecopy/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "safecopy", diff --git a/pkg/sentry/safemem/BUILD b/pkg/sentry/safemem/BUILD index dc4cfce41..e96509ce1 100644 --- a/pkg/sentry/safemem/BUILD +++ b/pkg/sentry/safemem/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "safemem", diff --git a/pkg/sentry/sighandling/BUILD b/pkg/sentry/sighandling/BUILD index daaad7c90..f480f0735 100644 --- a/pkg/sentry/sighandling/BUILD +++ b/pkg/sentry/sighandling/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "sighandling", diff --git a/pkg/sentry/socket/BUILD b/pkg/sentry/socket/BUILD index 5500a676e..929787aa0 100644 --- a/pkg/sentry/socket/BUILD +++ b/pkg/sentry/socket/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "socket_state", diff --git a/pkg/sentry/socket/control/BUILD b/pkg/sentry/socket/control/BUILD index 25de2f655..faf2b4c27 100644 --- a/pkg/sentry/socket/control/BUILD +++ b/pkg/sentry/socket/control/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "control_state", diff --git a/pkg/sentry/socket/epsocket/BUILD b/pkg/sentry/socket/epsocket/BUILD index 8430886cb..7ad5e88c5 100644 --- a/pkg/sentry/socket/epsocket/BUILD +++ b/pkg/sentry/socket/epsocket/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "epsocket_state", diff --git a/pkg/sentry/socket/hostinet/BUILD b/pkg/sentry/socket/hostinet/BUILD index 60ec265ba..227ca3926 100644 --- a/pkg/sentry/socket/hostinet/BUILD +++ b/pkg/sentry/socket/hostinet/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "hostinet_state", diff --git a/pkg/sentry/socket/netlink/BUILD b/pkg/sentry/socket/netlink/BUILD index 9df3ab17c..b23a243f7 100644 --- a/pkg/sentry/socket/netlink/BUILD +++ b/pkg/sentry/socket/netlink/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "netlink_state", diff --git a/pkg/sentry/socket/netlink/port/BUILD b/pkg/sentry/socket/netlink/port/BUILD index 7340b95c9..ba6f686e4 100644 --- a/pkg/sentry/socket/netlink/port/BUILD +++ b/pkg/sentry/socket/netlink/port/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "port_state", diff --git a/pkg/sentry/socket/netlink/route/BUILD b/pkg/sentry/socket/netlink/route/BUILD index ff3f7b7a4..726469fc9 100644 --- a/pkg/sentry/socket/netlink/route/BUILD +++ b/pkg/sentry/socket/netlink/route/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "route_state", diff --git a/pkg/sentry/socket/rpcinet/BUILD b/pkg/sentry/socket/rpcinet/BUILD index 8973453f9..288199779 100644 --- a/pkg/sentry/socket/rpcinet/BUILD +++ b/pkg/sentry/socket/rpcinet/BUILD @@ -1,7 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 +load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "rpcinet", diff --git a/pkg/sentry/socket/rpcinet/conn/BUILD b/pkg/sentry/socket/rpcinet/conn/BUILD index 4923dee4b..c51ca14b1 100644 --- a/pkg/sentry/socket/rpcinet/conn/BUILD +++ b/pkg/sentry/socket/rpcinet/conn/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # BSD -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "conn", diff --git a/pkg/sentry/socket/rpcinet/notifier/BUILD b/pkg/sentry/socket/rpcinet/notifier/BUILD index 6f3b06a05..2ae902b3f 100644 --- a/pkg/sentry/socket/rpcinet/notifier/BUILD +++ b/pkg/sentry/socket/rpcinet/notifier/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # BSD -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "notifier", diff --git a/pkg/sentry/socket/unix/BUILD b/pkg/sentry/socket/unix/BUILD index 1ec6eb7ed..7d04d6b6b 100644 --- a/pkg/sentry/socket/unix/BUILD +++ b/pkg/sentry/socket/unix/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "unix_state", diff --git a/pkg/sentry/state/BUILD b/pkg/sentry/state/BUILD index 9bd98f445..a57a8298e 100644 --- a/pkg/sentry/state/BUILD +++ b/pkg/sentry/state/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "state", diff --git a/pkg/sentry/strace/BUILD b/pkg/sentry/strace/BUILD index c5946a564..e1c8db67a 100644 --- a/pkg/sentry/strace/BUILD +++ b/pkg/sentry/strace/BUILD @@ -1,7 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 +load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "strace", diff --git a/pkg/sentry/syscalls/BUILD b/pkg/sentry/syscalls/BUILD index d667b42c8..22a757095 100644 --- a/pkg/sentry/syscalls/BUILD +++ b/pkg/sentry/syscalls/BUILD @@ -1,7 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 +load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "syscalls", diff --git a/pkg/sentry/syscalls/linux/BUILD b/pkg/sentry/syscalls/linux/BUILD index d3f3cc459..574621ad2 100644 --- a/pkg/sentry/syscalls/linux/BUILD +++ b/pkg/sentry/syscalls/linux/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "linux_state", diff --git a/pkg/sentry/time/BUILD b/pkg/sentry/time/BUILD index cbcd699d5..9452787fb 100644 --- a/pkg/sentry/time/BUILD +++ b/pkg/sentry/time/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/sentry/uniqueid/BUILD b/pkg/sentry/uniqueid/BUILD index c8ab03c3d..8eba3609e 100644 --- a/pkg/sentry/uniqueid/BUILD +++ b/pkg/sentry/uniqueid/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "uniqueid", diff --git a/pkg/sentry/usage/BUILD b/pkg/sentry/usage/BUILD index a0fe0aa07..edee44d96 100644 --- a/pkg/sentry/usage/BUILD +++ b/pkg/sentry/usage/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "usage_state", diff --git a/pkg/sentry/usermem/BUILD b/pkg/sentry/usermem/BUILD index 36c0760dd..9dd1cd2b5 100644 --- a/pkg/sentry/usermem/BUILD +++ b/pkg/sentry/usermem/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "usermem_state", diff --git a/pkg/sentry/watchdog/BUILD b/pkg/sentry/watchdog/BUILD index 28fae4490..13bc33eb1 100644 --- a/pkg/sentry/watchdog/BUILD +++ b/pkg/sentry/watchdog/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "watchdog", diff --git a/pkg/sleep/BUILD b/pkg/sleep/BUILD index f2b69b225..05e4ca540 100644 --- a/pkg/sleep/BUILD +++ b/pkg/sleep/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "sleep", diff --git a/pkg/state/BUILD b/pkg/state/BUILD index bb6415d9b..012b0484e 100644 --- a/pkg/state/BUILD +++ b/pkg/state/BUILD @@ -1,7 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/state/statefile/BUILD b/pkg/state/statefile/BUILD index df2c6a578..16abe1930 100644 --- a/pkg/state/statefile/BUILD +++ b/pkg/state/statefile/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "statefile", diff --git a/pkg/sync/BUILD b/pkg/sync/BUILD index 1fc0c25b5..3959fea36 100644 --- a/pkg/sync/BUILD +++ b/pkg/sync/BUILD @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") package( default_visibility = ["//:sandbox"], diff --git a/pkg/sync/seqatomictest/BUILD b/pkg/sync/seqatomictest/BUILD index 9d6ee2dfb..07b4f85ab 100644 --- a/pkg/sync/seqatomictest/BUILD +++ b/pkg/sync/seqatomictest/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/syserr/BUILD b/pkg/syserr/BUILD index e5ce48412..c0850f3d9 100644 --- a/pkg/syserr/BUILD +++ b/pkg/syserr/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "syserr", diff --git a/pkg/syserror/BUILD b/pkg/syserror/BUILD index 68ddec786..e050c2043 100644 --- a/pkg/syserror/BUILD +++ b/pkg/syserror/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "syserror", diff --git a/pkg/tcpip/BUILD b/pkg/tcpip/BUILD index 186a0d3bf..391d801d0 100644 --- a/pkg/tcpip/BUILD +++ b/pkg/tcpip/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "tcpip_state", diff --git a/pkg/tcpip/adapters/gonet/BUILD b/pkg/tcpip/adapters/gonet/BUILD index 97da46776..bf618831a 100644 --- a/pkg/tcpip/adapters/gonet/BUILD +++ b/pkg/tcpip/adapters/gonet/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "gonet", diff --git a/pkg/tcpip/buffer/BUILD b/pkg/tcpip/buffer/BUILD index 08adf18cd..efeb6a448 100644 --- a/pkg/tcpip/buffer/BUILD +++ b/pkg/tcpip/buffer/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "buffer_state", diff --git a/pkg/tcpip/checker/BUILD b/pkg/tcpip/checker/BUILD index 5447cfbf4..e8a524918 100644 --- a/pkg/tcpip/checker/BUILD +++ b/pkg/tcpip/checker/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "checker", diff --git a/pkg/tcpip/header/BUILD b/pkg/tcpip/header/BUILD index 859c2a106..3aa2cfb24 100644 --- a/pkg/tcpip/header/BUILD +++ b/pkg/tcpip/header/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "tcp_header_state", diff --git a/pkg/tcpip/link/channel/BUILD b/pkg/tcpip/link/channel/BUILD index f2f0c8b6f..9a6f49c45 100644 --- a/pkg/tcpip/link/channel/BUILD +++ b/pkg/tcpip/link/channel/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "channel", diff --git a/pkg/tcpip/link/fdbased/BUILD b/pkg/tcpip/link/fdbased/BUILD index aca3b14ca..6e75e9f47 100644 --- a/pkg/tcpip/link/fdbased/BUILD +++ b/pkg/tcpip/link/fdbased/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "fdbased", diff --git a/pkg/tcpip/link/loopback/BUILD b/pkg/tcpip/link/loopback/BUILD index 9714e93db..cc4247ffd 100644 --- a/pkg/tcpip/link/loopback/BUILD +++ b/pkg/tcpip/link/loopback/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "loopback", diff --git a/pkg/tcpip/link/rawfile/BUILD b/pkg/tcpip/link/rawfile/BUILD index 4b30c7c1c..10b35a37e 100644 --- a/pkg/tcpip/link/rawfile/BUILD +++ b/pkg/tcpip/link/rawfile/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "rawfile", diff --git a/pkg/tcpip/link/sharedmem/BUILD b/pkg/tcpip/link/sharedmem/BUILD index 1bd79a3f4..5390257c5 100644 --- a/pkg/tcpip/link/sharedmem/BUILD +++ b/pkg/tcpip/link/sharedmem/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "sharedmem", diff --git a/pkg/tcpip/link/sharedmem/pipe/BUILD b/pkg/tcpip/link/sharedmem/pipe/BUILD index e6c658071..ff798ae6f 100644 --- a/pkg/tcpip/link/sharedmem/pipe/BUILD +++ b/pkg/tcpip/link/sharedmem/pipe/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "pipe", diff --git a/pkg/tcpip/link/sharedmem/queue/BUILD b/pkg/tcpip/link/sharedmem/queue/BUILD index 80cedade1..c4a7879c4 100644 --- a/pkg/tcpip/link/sharedmem/queue/BUILD +++ b/pkg/tcpip/link/sharedmem/queue/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "queue", diff --git a/pkg/tcpip/link/sniffer/BUILD b/pkg/tcpip/link/sniffer/BUILD index d14f150d1..1e844f949 100644 --- a/pkg/tcpip/link/sniffer/BUILD +++ b/pkg/tcpip/link/sniffer/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "sniffer", diff --git a/pkg/tcpip/link/tun/BUILD b/pkg/tcpip/link/tun/BUILD index 21da7d57e..a8bb03661 100644 --- a/pkg/tcpip/link/tun/BUILD +++ b/pkg/tcpip/link/tun/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "tun", diff --git a/pkg/tcpip/link/waitable/BUILD b/pkg/tcpip/link/waitable/BUILD index 3b513383a..7582df32e 100644 --- a/pkg/tcpip/link/waitable/BUILD +++ b/pkg/tcpip/link/waitable/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "waitable", diff --git a/pkg/tcpip/network/BUILD b/pkg/tcpip/network/BUILD index 963857f51..9a26b46c4 100644 --- a/pkg/tcpip/network/BUILD +++ b/pkg/tcpip/network/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_test") +load("//tools/go_stateify:defs.bzl", "go_test") go_test( name = "ip_test", diff --git a/pkg/tcpip/network/arp/BUILD b/pkg/tcpip/network/arp/BUILD index 689f66d6e..44f2b66e5 100644 --- a/pkg/tcpip/network/arp/BUILD +++ b/pkg/tcpip/network/arp/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "arp", diff --git a/pkg/tcpip/network/fragmentation/BUILD b/pkg/tcpip/network/fragmentation/BUILD index a173f87fb..ac97ebe43 100644 --- a/pkg/tcpip/network/fragmentation/BUILD +++ b/pkg/tcpip/network/fragmentation/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "fragmentation_state", diff --git a/pkg/tcpip/network/hash/BUILD b/pkg/tcpip/network/hash/BUILD index e1b5f26c4..1c22c52fc 100644 --- a/pkg/tcpip/network/hash/BUILD +++ b/pkg/tcpip/network/hash/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "hash", diff --git a/pkg/tcpip/network/ipv4/BUILD b/pkg/tcpip/network/ipv4/BUILD index ae42b662f..19314e9bd 100644 --- a/pkg/tcpip/network/ipv4/BUILD +++ b/pkg/tcpip/network/ipv4/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "ipv4", diff --git a/pkg/tcpip/network/ipv6/BUILD b/pkg/tcpip/network/ipv6/BUILD index d008ac7fb..1c3eccae0 100644 --- a/pkg/tcpip/network/ipv6/BUILD +++ b/pkg/tcpip/network/ipv6/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "ipv6", diff --git a/pkg/tcpip/ports/BUILD b/pkg/tcpip/ports/BUILD index 710c283f7..3c3374275 100644 --- a/pkg/tcpip/ports/BUILD +++ b/pkg/tcpip/ports/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "ports", diff --git a/pkg/tcpip/seqnum/BUILD b/pkg/tcpip/seqnum/BUILD index 6d28dbc3f..a75869dac 100644 --- a/pkg/tcpip/seqnum/BUILD +++ b/pkg/tcpip/seqnum/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "seqnum_state", diff --git a/pkg/tcpip/stack/BUILD b/pkg/tcpip/stack/BUILD index 6d201d0a2..5e7355135 100644 --- a/pkg/tcpip/stack/BUILD +++ b/pkg/tcpip/stack/BUILD @@ -1,17 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") - -go_stateify( - name = "stack_state", - srcs = [ - "registration.go", - "stack.go", - ], - out = "stack_state.go", - package = "stack", -) +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "stack", @@ -22,7 +11,6 @@ go_library( "route.go", "stack.go", "stack_global_state.go", - "stack_state.go", "transport_demuxer.go", ], importpath = "gvisor.googlesource.com/gvisor/pkg/tcpip/stack", @@ -32,7 +20,6 @@ go_library( deps = [ "//pkg/ilist", "//pkg/sleep", - "//pkg/state", "//pkg/tcpip", "//pkg/tcpip/buffer", "//pkg/tcpip/header", diff --git a/pkg/tcpip/stack/registration.go b/pkg/tcpip/stack/registration.go index e9550a062..c66f925a8 100644 --- a/pkg/tcpip/stack/registration.go +++ b/pkg/tcpip/stack/registration.go @@ -31,6 +31,8 @@ type NetworkEndpointID struct { } // TransportEndpointID is the identifier of a transport layer protocol endpoint. +// +// +stateify savable type TransportEndpointID struct { // LocalPort is the local port associated with the endpoint. LocalPort uint16 diff --git a/pkg/tcpip/transport/ping/BUILD b/pkg/tcpip/transport/ping/BUILD index 1febbf7f5..28e3e1700 100644 --- a/pkg/tcpip/transport/ping/BUILD +++ b/pkg/tcpip/transport/ping/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "ping_state", diff --git a/pkg/tcpip/transport/queue/BUILD b/pkg/tcpip/transport/queue/BUILD index 7e8ee1f66..fb878ad36 100644 --- a/pkg/tcpip/transport/queue/BUILD +++ b/pkg/tcpip/transport/queue/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "queue_state", diff --git a/pkg/tcpip/transport/tcp/BUILD b/pkg/tcpip/transport/tcp/BUILD index 53623787d..6a7153e4d 100644 --- a/pkg/tcpip/transport/tcp/BUILD +++ b/pkg/tcpip/transport/tcp/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "tcp_state", diff --git a/pkg/tcpip/transport/tcp/testing/context/BUILD b/pkg/tcpip/transport/tcp/testing/context/BUILD index 3caa38bcb..7a95594ef 100644 --- a/pkg/tcpip/transport/tcp/testing/context/BUILD +++ b/pkg/tcpip/transport/tcp/testing/context/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "context", diff --git a/pkg/tcpip/transport/tcpconntrack/BUILD b/pkg/tcpip/transport/tcpconntrack/BUILD index 3d748528e..46da3e6f1 100644 --- a/pkg/tcpip/transport/tcpconntrack/BUILD +++ b/pkg/tcpip/transport/tcpconntrack/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "tcpconntrack", diff --git a/pkg/tcpip/transport/udp/BUILD b/pkg/tcpip/transport/udp/BUILD index 4f7a47973..790dd55a3 100644 --- a/pkg/tcpip/transport/udp/BUILD +++ b/pkg/tcpip/transport/udp/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "udp_state", diff --git a/pkg/tcpip/transport/unix/BUILD b/pkg/tcpip/transport/unix/BUILD index d58f06544..676f2cf92 100644 --- a/pkg/tcpip/transport/unix/BUILD +++ b/pkg/tcpip/transport/unix/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "unix_state", diff --git a/pkg/tmutex/BUILD b/pkg/tmutex/BUILD index d9a2c5ae5..d18338fff 100644 --- a/pkg/tmutex/BUILD +++ b/pkg/tmutex/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "tmutex", diff --git a/pkg/unet/BUILD b/pkg/unet/BUILD index e8e40315a..acdfd7cb6 100644 --- a/pkg/unet/BUILD +++ b/pkg/unet/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "unet", diff --git a/pkg/urpc/BUILD b/pkg/urpc/BUILD index b29b25637..d32c57d1a 100644 --- a/pkg/urpc/BUILD +++ b/pkg/urpc/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "urpc", diff --git a/pkg/waiter/BUILD b/pkg/waiter/BUILD index 032ec3237..8256acdb4 100644 --- a/pkg/waiter/BUILD +++ b/pkg/waiter/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "waiter_state", diff --git a/pkg/waiter/fdnotifier/BUILD b/pkg/waiter/fdnotifier/BUILD index d5b5ee82d..4e582755d 100644 --- a/pkg/waiter/fdnotifier/BUILD +++ b/pkg/waiter/fdnotifier/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "fdnotifier", diff --git a/tools/go_stateify/defs.bzl b/tools/go_stateify/defs.bzl index 60a9895ff..2b2582b7a 100644 --- a/tools/go_stateify/defs.bzl +++ b/tools/go_stateify/defs.bzl @@ -22,6 +22,8 @@ go_library( ) """ +load("@io_bazel_rules_go//go:def.bzl", _go_library = "go_library", _go_test = "go_test") + def _go_stateify_impl(ctx): """Implementation for the stateify tool.""" output = ctx.outputs.out @@ -33,6 +35,8 @@ def _go_stateify_impl(ctx): args += ["-statepkg=%s" % ctx.attr._statepkg] if ctx.attr.imports: args += ["-imports=%s" % ",".join(ctx.attr.imports)] + if ctx.attr.explicit: + args += ["-explicit=true"] args += ["--"] for src in ctx.attr.srcs: args += [f.path for f in src.files] @@ -45,17 +49,15 @@ def _go_stateify_impl(ctx): executable = ctx.executable._tool, ) -""" -Generates save and restore logic from a set of Go files. - - -Args: - name: the name of the rule. - srcs: the input source files. These files should include all structs in the package that need to be saved. - imports: an optional list of extra non-aliased, Go-style absolute import paths. - out: the name of the generated file output. This must not conflict with any other files and must be added to the srcs of the relevant go_library. - package: the package name for the input sources. -""" +# Generates save and restore logic from a set of Go files. +# +# Args: +# name: the name of the rule. +# srcs: the input source files. These files should include all structs in the package that need to be saved. +# imports: an optional list of extra non-aliased, Go-style absolute import paths. +# out: the name of the generated file output. This must not conflict with any other files and must be added to the srcs of the relevant go_library. +# package: the package name for the input sources. +# explicit: only generate for types explicitly annotated as savable. go_stateify = rule( implementation = _go_stateify_impl, attrs = { @@ -63,7 +65,41 @@ go_stateify = rule( "imports": attr.string_list(mandatory = False), "package": attr.string(mandatory = True), "out": attr.output(mandatory = True), + "explicit": attr.bool(default = False), "_tool": attr.label(executable = True, cfg = "host", default = Label("//tools/go_stateify:stateify")), "_statepkg": attr.string(default = "gvisor.googlesource.com/gvisor/pkg/state"), }, ) + +def go_library(name, srcs, deps = [], imports = [], **kwargs): + """wraps the standard go_library and does stateification.""" + if "encode_unsafe.go" not in srcs and (name + "_state_autogen.go") not in srcs: + # Only do stateification for non-state packages without manual autogen. + go_stateify( + name = name + "_state_autogen", + srcs = [src for src in srcs if src.endswith(".go")], + imports = imports, + package = name, + out = name + "_state_autogen.go", + explicit = True, + ) + all_srcs = srcs + [name + "_state_autogen.go"] + if "//pkg/state" not in deps: + all_deps = deps + ["//pkg/state"] + else: + all_deps = deps + else: + all_deps = deps + all_srcs = srcs + _go_library( + name = name, + srcs = all_srcs, + deps = all_deps, + **kwargs + ) + +def go_test(**kwargs): + """Wraps the standard go_test.""" + _go_test( + **kwargs + ) diff --git a/tools/go_stateify/main.go b/tools/go_stateify/main.go index 6c3583c62..231c6d80b 100644 --- a/tools/go_stateify/main.go +++ b/tools/go_stateify/main.go @@ -25,6 +25,7 @@ import ( "os" "reflect" "strings" + "sync" ) var ( @@ -32,6 +33,7 @@ var ( imports = flag.String("imports", "", "extra imports for the output file") output = flag.String("output", "", "output file") statePkg = flag.String("statepkg", "", "state import package; defaults to empty") + explicit = flag.Bool("explicit", false, "only generate for types explicitly tagged '// +stateify savable'") ) // resolveTypeName returns a qualified type name. @@ -224,16 +226,24 @@ func main() { // Emit the package name. fmt.Fprint(outputFile, "// automatically generated by stateify.\n\n") fmt.Fprintf(outputFile, "package %s\n\n", *pkg) - fmt.Fprint(outputFile, "import (\n") - if *statePkg != "" { - fmt.Fprintf(outputFile, " \"%s\"\n", *statePkg) - } - if *imports != "" { - for _, i := range strings.Split(*imports, ",") { - fmt.Fprintf(outputFile, " \"%s\"\n", i) - } + + // Emit the imports lazily. + var once sync.Once + maybeEmitImports := func() { + once.Do(func() { + // Emit the imports. + fmt.Fprint(outputFile, "import (\n") + if *statePkg != "" { + fmt.Fprintf(outputFile, " \"%s\"\n", *statePkg) + } + if *imports != "" { + for _, i := range strings.Split(*imports, ",") { + fmt.Fprintf(outputFile, " \"%s\"\n", i) + } + } + fmt.Fprint(outputFile, ")\n\n") + }) } - fmt.Fprint(outputFile, ")\n\n") files := make([]*ast.File, 0, len(flag.Args())) @@ -241,7 +251,7 @@ func main() { for _, filename := range flag.Args() { // Parse the file. fset := token.NewFileSet() - f, err := parser.ParseFile(fset, filename, nil, 0) + f, err := parser.ParseFile(fset, filename, nil, parser.ParseComments) if err != nil { // Not a valid input file? fmt.Fprintf(os.Stderr, "Input %q can't be parsed: %v\n", filename, err) @@ -308,6 +318,26 @@ func main() { continue } + if *explicit { + // In explicit mode, only generate code for + // types explicitly marked + // "// +stateify savable" in one of the + // proceeding comment lines. + if d.Doc == nil { + continue + } + savable := false + for _, l := range d.Doc.List { + if l.Text == "// +stateify savable" { + savable = true + break + } + } + if !savable { + continue + } + } + for _, gs := range d.Specs { ts := gs.(*ast.TypeSpec) switch ts.Type.(type) { @@ -315,6 +345,8 @@ func main() { // Don't register. break case *ast.StructType: + maybeEmitImports() + ss := ts.Type.(*ast.StructType) // Define beforeSave if a definition was not found. This @@ -360,6 +392,8 @@ func main() { // Add to our registration. emitRegister(ts.Name.Name) case *ast.Ident, *ast.SelectorExpr, *ast.ArrayType: + maybeEmitImports() + _, val := resolveTypeName(ts.Name.Name, ts.Type) // Dispatch directly. @@ -377,10 +411,12 @@ func main() { } } - // Emit the init() function. - fmt.Fprintf(outputFile, "func init() {\n") - for _, ic := range initCalls { - fmt.Fprintf(outputFile, " %s\n", ic) + if len(initCalls) > 0 { + // Emit the init() function. + fmt.Fprintf(outputFile, "func init() {\n") + for _, ic := range initCalls { + fmt.Fprintf(outputFile, " %s\n", ic) + } + fmt.Fprintf(outputFile, "}\n") } - fmt.Fprintf(outputFile, "}\n") } -- cgit v1.2.3 From a6b00502b04ced2f12cfcf35c6f276cff349737b Mon Sep 17 00:00:00 2001 From: Zhaozhong Ni Date: Fri, 24 Aug 2018 14:52:23 -0700 Subject: compressio: support optional hashing and eliminate hashio. Compared to previous compressio / hashio nesting, there is up to 100% speedup. PiperOrigin-RevId: 210161269 Change-Id: I481aa9fe980bb817fe465fe34d32ea33fc8abf1c --- pkg/compressio/compressio.go | 223 ++++++++++++++++++++++--- pkg/compressio/compressio_test.go | 145 ++++++++++++----- pkg/hashio/BUILD | 19 --- pkg/hashio/hashio.go | 296 ---------------------------------- pkg/hashio/hashio_test.go | 142 ---------------- pkg/state/statefile/BUILD | 3 +- pkg/state/statefile/statefile.go | 11 +- pkg/state/statefile/statefile_test.go | 70 ++++---- 8 files changed, 339 insertions(+), 570 deletions(-) delete mode 100644 pkg/hashio/BUILD delete mode 100644 pkg/hashio/hashio.go delete mode 100644 pkg/hashio/hashio_test.go (limited to 'pkg/state') diff --git a/pkg/compressio/compressio.go b/pkg/compressio/compressio.go index ef8cbd2a5..591b37130 100644 --- a/pkg/compressio/compressio.go +++ b/pkg/compressio/compressio.go @@ -12,17 +12,48 @@ // See the License for the specific language governing permissions and // limitations under the License. -// Package compressio provides parallel compression and decompression. +// Package compressio provides parallel compression and decompression, as well +// as optional SHA-256 hashing. +// +// The stream format is defined as follows. +// +// /------------------------------------------------------\ +// | chunk size (4-bytes) | +// +------------------------------------------------------+ +// | (optional) hash (32-bytes) | +// +------------------------------------------------------+ +// | compressed data size (4-bytes) | +// +------------------------------------------------------+ +// | compressed data | +// +------------------------------------------------------+ +// | (optional) hash (32-bytes) | +// +------------------------------------------------------+ +// | compressed data size (4-bytes) | +// +------------------------------------------------------+ +// | ...... | +// \------------------------------------------------------/ +// +// where each subsequent hash is calculated from the following items in order +// +// compressed data +// compressed data size +// previous hash +// +// so the stream integrity cannot be compromised by switching and mixing +// compressed chunks. package compressio import ( "bytes" "compress/flate" "errors" + "hash" "io" "runtime" "sync" + "crypto/hmac" + "crypto/sha256" "gvisor.googlesource.com/gvisor/pkg/binary" ) @@ -51,12 +82,23 @@ type chunk struct { // This is not returned to the bufPool automatically, since it may // correspond to a inline slice (provided directly to Read or Write). uncompressed *bytes.Buffer + + // The current hash object. Only used in compress mode. + h hash.Hash + + // The hash from previous chunks. Only used in uncompress mode. + lastSum []byte + + // The expected hash after current chunk. Only used in uncompress mode. + sum []byte } // newChunk allocates a new chunk object (or pulls one from the pool). Buffers // will be allocated if nil is provided for compressed or uncompressed. -func newChunk(compressed *bytes.Buffer, uncompressed *bytes.Buffer) *chunk { +func newChunk(lastSum []byte, sum []byte, compressed *bytes.Buffer, uncompressed *bytes.Buffer) *chunk { c := chunkPool.Get().(*chunk) + c.lastSum = lastSum + c.sum = sum if compressed != nil { c.compressed = compressed } else { @@ -85,6 +127,7 @@ type result struct { // The goroutine will exit when input is closed, and the goroutine will close // output. type worker struct { + pool *pool input chan *chunk output chan result } @@ -93,17 +136,27 @@ type worker struct { func (w *worker) work(compress bool, level int) { defer close(w.output) + var h hash.Hash + for c := range w.input { + if h == nil && w.pool.key != nil { + h = w.pool.getHash() + } if compress { + mw := io.Writer(c.compressed) + if h != nil { + mw = io.MultiWriter(mw, h) + } + // Encode this slice. - fw, err := flate.NewWriter(c.compressed, level) + fw, err := flate.NewWriter(mw, level) if err != nil { w.output <- result{c, err} continue } // Encode the input. - if _, err := io.Copy(fw, c.uncompressed); err != nil { + if _, err := io.CopyN(fw, c.uncompressed, int64(c.uncompressed.Len())); err != nil { w.output <- result{c, err} continue } @@ -111,7 +164,28 @@ func (w *worker) work(compress bool, level int) { w.output <- result{c, err} continue } + + // Write the hash, if enabled. + if h != nil { + binary.WriteUint32(h, binary.BigEndian, uint32(c.compressed.Len())) + c.h = h + h = nil + } } else { + // Check the hash of the compressed contents. + if h != nil { + h.Write(c.compressed.Bytes()) + binary.WriteUint32(h, binary.BigEndian, uint32(c.compressed.Len())) + io.CopyN(h, bytes.NewReader(c.lastSum), int64(len(c.lastSum))) + + sum := h.Sum(nil) + h.Reset() + if !hmac.Equal(c.sum, sum) { + w.output <- result{c, ErrHashMismatch} + continue + } + } + // Decode this slice. fr := flate.NewReader(c.compressed) @@ -136,6 +210,16 @@ type pool struct { // stream and is shared across both the reader and writer. chunkSize uint32 + // key is the key used to create hash objects. + key []byte + + // hashMu protexts the hash list. + hashMu sync.Mutex + + // hashes is the hash object free list. Note that this cannot be + // globally shared across readers or writers, as it is key-specific. + hashes []hash.Hash + // mu protects below; it is generally the responsibility of users to // acquire this mutex before calling any methods on the pool. mu sync.Mutex @@ -149,15 +233,20 @@ type pool struct { // buf is the current active buffer; the exact semantics of this buffer // depending on whether this is a reader or a writer. buf *bytes.Buffer + + // lasSum records the hash of the last chunk processed. + lastSum []byte } // init initializes the worker pool. // // This should only be called once. -func (p *pool) init(compress bool, level int) { - p.workers = make([]worker, 1+runtime.GOMAXPROCS(0)) +func (p *pool) init(key []byte, workers int, compress bool, level int) { + p.key = key + p.workers = make([]worker, workers) for i := 0; i < len(p.workers); i++ { p.workers[i] = worker{ + pool: p, input: make(chan *chunk, 1), output: make(chan result, 1), } @@ -174,6 +263,30 @@ func (p *pool) stop() { p.workers = nil } +// getHash gets a hash object for the pool. It should only be called when the +// pool key is non-nil. +func (p *pool) getHash() hash.Hash { + p.hashMu.Lock() + defer p.hashMu.Unlock() + + if len(p.hashes) == 0 { + return hmac.New(sha256.New, p.key) + } + + h := p.hashes[len(p.hashes)-1] + p.hashes = p.hashes[:len(p.hashes)-1] + return h +} + +func (p *pool) putHash(h hash.Hash) { + h.Reset() + + p.hashMu.Lock() + defer p.hashMu.Unlock() + + p.hashes = append(p.hashes, h) +} + // handleResult calls the callback. func handleResult(r result, callback func(*chunk) error) error { defer func() { @@ -231,22 +344,46 @@ type reader struct { in io.Reader } -// NewReader returns a new compressed reader. -func NewReader(in io.Reader) (io.Reader, error) { +// NewReader returns a new compressed reader. If key is non-nil, the data stream +// is assumed to contain expected hash values, which will be compared against +// hash values computed from the compressed bytes. See package comments for +// details. +func NewReader(in io.Reader, key []byte) (io.Reader, error) { r := &reader{ in: in, } - r.init(false, 0) + + // Use double buffering for read. + r.init(key, 2*runtime.GOMAXPROCS(0), false, 0) + var err error - if r.chunkSize, err = binary.ReadUint32(r.in, binary.BigEndian); err != nil { + if r.chunkSize, err = binary.ReadUint32(in, binary.BigEndian); err != nil { return nil, err } + + if r.key != nil { + h := r.getHash() + binary.WriteUint32(h, binary.BigEndian, r.chunkSize) + r.lastSum = h.Sum(nil) + r.putHash(h) + sum := make([]byte, len(r.lastSum)) + if _, err := io.ReadFull(r.in, sum); err != nil { + return nil, err + } + if !hmac.Equal(r.lastSum, sum) { + return nil, ErrHashMismatch + } + } + return r, nil } // errNewBuffer is returned when a new buffer is completed. var errNewBuffer = errors.New("buffer ready") +// ErrHashMismatch is returned if the hash does not match. +var ErrHashMismatch = errors.New("hash mismatch") + // Read implements io.Reader.Read. func (r *reader) Read(p []byte) (int, error) { r.mu.Lock() @@ -331,14 +468,25 @@ func (r *reader) Read(p []byte) (int, error) { // Read this chunk and schedule decompression. compressed := bufPool.Get().(*bytes.Buffer) - if _, err := io.Copy(compressed, &io.LimitedReader{ - R: r.in, - N: int64(l), - }); err != nil { + if _, err := io.CopyN(compressed, r.in, int64(l)); err != nil { // Some other error occurred; see above. + if err == io.EOF { + err = io.ErrUnexpectedEOF + } return done, err } + var sum []byte + if r.key != nil { + sum = make([]byte, len(r.lastSum)) + if _, err := io.ReadFull(r.in, sum); err != nil { + if err == io.EOF { + err = io.ErrUnexpectedEOF + } + return done, err + } + } + // Are we doing inline decoding? // // Note that we need to check the length here against @@ -349,11 +497,12 @@ func (r *reader) Read(p []byte) (int, error) { var c *chunk start := done + ((pendingPre + pendingInline) * int(r.chunkSize)) if len(p) >= start+int(r.chunkSize) && len(p) >= start+bytes.MinRead { - c = newChunk(compressed, bytes.NewBuffer(p[start:start])) + c = newChunk(r.lastSum, sum, compressed, bytes.NewBuffer(p[start:start])) pendingInline++ } else { - c = newChunk(compressed, nil) + c = newChunk(r.lastSum, sum, compressed, nil) } + r.lastSum = sum if err := r.schedule(c, callback); err == errNewBuffer { // A new buffer was completed while we were reading. // That's great, but we need to force schedule the @@ -403,12 +552,14 @@ type writer struct { closed bool } -// NewWriter returns a new compressed writer. +// NewWriter returns a new compressed writer. If key is non-nil, hash values are +// generated and written out for compressed bytes. See package comments for +// details. // // The recommended chunkSize is on the order of 1M. Extra memory may be // buffered (in the form of read-ahead, or buffered writes), and is limited to // O(chunkSize * [1+GOMAXPROCS]). -func NewWriter(out io.Writer, chunkSize uint32, level int) (io.WriteCloser, error) { +func NewWriter(out io.Writer, key []byte, chunkSize uint32, level int) (io.WriteCloser, error) { w := &writer{ pool: pool{ chunkSize: chunkSize, @@ -416,10 +567,22 @@ func NewWriter(out io.Writer, chunkSize uint32, level int) (io.WriteCloser, erro }, out: out, } - w.init(true, level) + w.init(key, 1+runtime.GOMAXPROCS(0), true, level) + if err := binary.WriteUint32(w.out, binary.BigEndian, chunkSize); err != nil { return nil, err } + + if w.key != nil { + h := w.getHash() + binary.WriteUint32(h, binary.BigEndian, chunkSize) + w.lastSum = h.Sum(nil) + w.putHash(h) + if _, err := io.CopyN(w.out, bytes.NewReader(w.lastSum), int64(len(w.lastSum))); err != nil { + return nil, err + } + } + return w, nil } @@ -433,8 +596,22 @@ func (w *writer) flush(c *chunk) error { } // Write out to the stream. - _, err := io.Copy(w.out, c.compressed) - return err + if _, err := io.CopyN(w.out, c.compressed, int64(c.compressed.Len())); err != nil { + return err + } + + if w.key != nil { + io.CopyN(c.h, bytes.NewReader(w.lastSum), int64(len(w.lastSum))) + sum := c.h.Sum(nil) + w.putHash(c.h) + c.h = nil + if _, err := io.CopyN(w.out, bytes.NewReader(sum), int64(len(sum))); err != nil { + return err + } + w.lastSum = sum + } + + return nil } // Write implements io.Writer.Write. @@ -480,7 +657,7 @@ func (w *writer) Write(p []byte) (int, error) { // immediately following the inline case above. left := int(w.chunkSize) - w.buf.Len() if left == 0 { - if err := w.schedule(newChunk(nil, w.buf), callback); err != nil { + if err := w.schedule(newChunk(nil, nil, nil, w.buf), callback); err != nil { return done, err } // Reset the buffer, since this has now been scheduled @@ -538,7 +715,7 @@ func (w *writer) Close() error { // Schedule any remaining partial buffer; we pass w.flush directly here // because the final buffer is guaranteed to not be an inline buffer. if w.buf.Len() > 0 { - if err := w.schedule(newChunk(nil, w.buf), w.flush); err != nil { + if err := w.schedule(newChunk(nil, nil, nil, w.buf), w.flush); err != nil { return err } } diff --git a/pkg/compressio/compressio_test.go b/pkg/compressio/compressio_test.go index d7911419d..7cb5f8dc4 100644 --- a/pkg/compressio/compressio_test.go +++ b/pkg/compressio/compressio_test.go @@ -59,6 +59,7 @@ type testOpts struct { PostDecompress func() CompressIters int DecompressIters int + CorruptData bool } func doTest(t harness, opts testOpts) { @@ -104,15 +105,22 @@ func doTest(t harness, opts testOpts) { if opts.DecompressIters <= 0 { opts.DecompressIters = 1 } + if opts.CorruptData { + b := compressed.Bytes() + b[rand.Intn(len(b))]++ + } for i := 0; i < opts.DecompressIters; i++ { decompressed.Reset() r, err := opts.NewReader(bytes.NewBuffer(compressed.Bytes())) if err != nil { + if opts.CorruptData { + continue + } t.Errorf("%s: NewReader got err %v, expected nil", opts.Name, err) return } - if _, err := io.Copy(&decompressed, r); err != nil { - t.Errorf("%s: decompress got err %v, expected nil", opts.Name, err) + if _, err := io.Copy(&decompressed, r); (err != nil) != opts.CorruptData { + t.Errorf("%s: decompress got err %v unexpectly", opts.Name, err) return } } @@ -121,6 +129,10 @@ func doTest(t harness, opts testOpts) { } decompressionTime := time.Since(decompressionStartTime) + if opts.CorruptData { + return + } + // Verify. if decompressed.Len() != len(opts.Data) { t.Errorf("%s: got %d bytes, expected %d", opts.Name, decompressed.Len(), len(opts.Data)) @@ -136,7 +148,11 @@ func doTest(t harness, opts testOpts) { opts.Name, compressionTime, compressionRatio, decompressionTime) } +var hashKey = []byte("01234567890123456789012345678901") + func TestCompress(t *testing.T) { + rand.Seed(time.Now().Unix()) + var ( data = initTest(t, 10*1024*1024) data0 = data[:0] @@ -153,17 +169,27 @@ func TestCompress(t *testing.T) { continue } - // Do the compress test. - doTest(t, testOpts{ - Name: fmt.Sprintf("len(data)=%d, blockSize=%d", len(data), blockSize), - Data: data, - NewWriter: func(b *bytes.Buffer) (io.Writer, error) { - return NewWriter(b, blockSize, flate.BestCompression) - }, - NewReader: func(b *bytes.Buffer) (io.Reader, error) { - return NewReader(b) - }, - }) + for _, key := range [][]byte{nil, hashKey} { + for _, corruptData := range []bool{false, true} { + if key == nil && corruptData { + // No need to test corrupt data + // case when not doing hashing. + continue + } + // Do the compress test. + doTest(t, testOpts{ + Name: fmt.Sprintf("len(data)=%d, blockSize=%d, key=%s, corruptData=%v", len(data), blockSize, string(key), corruptData), + Data: data, + NewWriter: func(b *bytes.Buffer) (io.Writer, error) { + return NewWriter(b, key, blockSize, flate.BestSpeed) + }, + NewReader: func(b *bytes.Buffer) (io.Reader, error) { + return NewReader(b, key) + }, + CorruptData: corruptData, + }) + } + } } // Do the vanilla test. @@ -171,7 +197,7 @@ func TestCompress(t *testing.T) { Name: fmt.Sprintf("len(data)=%d, vanilla flate", len(data)), Data: data, NewWriter: func(b *bytes.Buffer) (io.Writer, error) { - return flate.NewWriter(b, flate.BestCompression) + return flate.NewWriter(b, flate.BestSpeed) }, NewReader: func(b *bytes.Buffer) (io.Reader, error) { return flate.NewReader(b), nil @@ -181,47 +207,84 @@ func TestCompress(t *testing.T) { } const ( - // benchBlockSize is the blockSize for benchmarks. - benchBlockSize = 32 * 1024 - - // benchDataSize is the amount of data for benchmarks. - benchDataSize = 10 * 1024 * 1024 + benchDataSize = 600 * 1024 * 1024 ) -func BenchmarkCompress(b *testing.B) { +func benchmark(b *testing.B, compress bool, hash bool, blockSize uint32) { b.StopTimer() b.SetBytes(benchDataSize) data := initTest(b, benchDataSize) + compIters := b.N + decompIters := b.N + if compress { + decompIters = 0 + } else { + compIters = 0 + } + key := hashKey + if !hash { + key = nil + } doTest(b, testOpts{ - Name: fmt.Sprintf("len(data)=%d, blockSize=%d", len(data), benchBlockSize), + Name: fmt.Sprintf("compress=%t, hash=%t, len(data)=%d, blockSize=%d", compress, hash, len(data), blockSize), Data: data, PreCompress: b.StartTimer, PostCompress: b.StopTimer, NewWriter: func(b *bytes.Buffer) (io.Writer, error) { - return NewWriter(b, benchBlockSize, flate.BestCompression) + return NewWriter(b, key, blockSize, flate.BestSpeed) }, NewReader: func(b *bytes.Buffer) (io.Reader, error) { - return NewReader(b) + return NewReader(b, key) }, - CompressIters: b.N, + CompressIters: compIters, + DecompressIters: decompIters, }) } -func BenchmarkDecompress(b *testing.B) { - b.StopTimer() - b.SetBytes(benchDataSize) - data := initTest(b, benchDataSize) - doTest(b, testOpts{ - Name: fmt.Sprintf("len(data)=%d, blockSize=%d", len(data), benchBlockSize), - Data: data, - PreDecompress: b.StartTimer, - PostDecompress: b.StopTimer, - NewWriter: func(b *bytes.Buffer) (io.Writer, error) { - return NewWriter(b, benchBlockSize, flate.BestCompression) - }, - NewReader: func(b *bytes.Buffer) (io.Reader, error) { - return NewReader(b) - }, - DecompressIters: b.N, - }) +func BenchmarkCompressNoHash64K(b *testing.B) { + benchmark(b, true, false, 64*1024) +} + +func BenchmarkCompressHash64K(b *testing.B) { + benchmark(b, true, true, 64*1024) +} + +func BenchmarkDecompressNoHash64K(b *testing.B) { + benchmark(b, false, false, 64*1024) +} + +func BenchmarkDecompressHash64K(b *testing.B) { + benchmark(b, false, true, 64*1024) +} + +func BenchmarkCompressNoHash1M(b *testing.B) { + benchmark(b, true, false, 1024*1024) +} + +func BenchmarkCompressHash1M(b *testing.B) { + benchmark(b, true, true, 1024*1024) +} + +func BenchmarkDecompressNoHash1M(b *testing.B) { + benchmark(b, false, false, 1024*1024) +} + +func BenchmarkDecompressHash1M(b *testing.B) { + benchmark(b, false, true, 1024*1024) +} + +func BenchmarkCompressNoHash16M(b *testing.B) { + benchmark(b, true, false, 16*1024*1024) +} + +func BenchmarkCompressHash16M(b *testing.B) { + benchmark(b, true, true, 16*1024*1024) +} + +func BenchmarkDecompressNoHash16M(b *testing.B) { + benchmark(b, false, false, 16*1024*1024) +} + +func BenchmarkDecompressHash16M(b *testing.B) { + benchmark(b, false, true, 16*1024*1024) } diff --git a/pkg/hashio/BUILD b/pkg/hashio/BUILD deleted file mode 100644 index 5736e2e73..000000000 --- a/pkg/hashio/BUILD +++ /dev/null @@ -1,19 +0,0 @@ -package(licenses = ["notice"]) # Apache 2.0 - -load("//tools/go_stateify:defs.bzl", "go_library", "go_test") - -go_library( - name = "hashio", - srcs = [ - "hashio.go", - ], - importpath = "gvisor.googlesource.com/gvisor/pkg/hashio", - visibility = ["//:sandbox"], -) - -go_test( - name = "hashio_test", - size = "small", - srcs = ["hashio_test.go"], - embed = [":hashio"], -) diff --git a/pkg/hashio/hashio.go b/pkg/hashio/hashio.go deleted file mode 100644 index e0e8ef413..000000000 --- a/pkg/hashio/hashio.go +++ /dev/null @@ -1,296 +0,0 @@ -// Copyright 2018 Google Inc. -// -// 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 -// -// 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 hashio provides hash-verified I/O streams. - -The I/O stream format is defined as follows. - -/-----------------------------------------\ -| payload | -+-----------------------------------------+ -| hash | -+-----------------------------------------+ -| payload | -+-----------------------------------------+ -| hash | -+-----------------------------------------+ -| ...... | -\-----------------------------------------/ - -Payload bytes written to / read from the stream are automatically split -into segments, each followed by a hash. All data read out must have already -passed hash verification. Hence the client code can safely do any kind of -(stream) processing of these data. -*/ -package hashio - -import ( - "errors" - "hash" - "io" - "sync" - - "crypto/hmac" -) - -// SegmentSize is the unit we split payload data and insert hash at. -const SegmentSize = 8 * 1024 - -// ErrHashMismatch is returned if the ErrHashMismatch does not match. -var ErrHashMismatch = errors.New("hash mismatch") - -// writer computes hashs during writes. -type writer struct { - mu sync.Mutex - w io.Writer - h hash.Hash - written int - closed bool - hashv []byte -} - -// NewWriter creates a hash-verified IO stream writer. -func NewWriter(w io.Writer, h hash.Hash) io.WriteCloser { - return &writer{ - w: w, - h: h, - hashv: make([]byte, h.Size()), - } -} - -// Write writes the given data. -func (w *writer) Write(p []byte) (int, error) { - w.mu.Lock() - defer w.mu.Unlock() - - // Did we already close? - if w.closed { - return 0, io.ErrUnexpectedEOF - } - - for done := 0; done < len(p); { - // Slice the data at segment boundary. - left := SegmentSize - w.written - if left > len(p[done:]) { - left = len(p[done:]) - } - - // Write the rest of the segment and write to hash writer the - // same number of bytes. Hash.Write may never return an error. - n, err := w.w.Write(p[done : done+left]) - w.h.Write(p[done : done+left]) - w.written += n - done += n - - // And only check the actual write errors here. - if n == 0 && err != nil { - return done, err - } - - // Write hash if starting a new segment. - if w.written == SegmentSize { - if err := w.closeSegment(); err != nil { - return done, err - } - } - } - - return len(p), nil -} - -// closeSegment closes the current segment and writes out its hash. -func (w *writer) closeSegment() error { - // Serialize and write the current segment's hash. - hashv := w.h.Sum(w.hashv[:0]) - for done := 0; done < len(hashv); { - n, err := w.w.Write(hashv[done:]) - done += n - if n == 0 && err != nil { - return err - } - } - w.written = 0 // reset counter. - return nil -} - -// Close writes the final hash to the stream and closes the underlying Writer. -func (w *writer) Close() error { - w.mu.Lock() - defer w.mu.Unlock() - - // Did we already close? - if w.closed { - return io.ErrUnexpectedEOF - } - - // Always mark as closed, regardless of errors. - w.closed = true - - // Write the final segment. - if err := w.closeSegment(); err != nil { - return err - } - - // Call the underlying closer. - if c, ok := w.w.(io.Closer); ok { - return c.Close() - } - return nil -} - -// reader computes and verifies hashs during reads. -type reader struct { - mu sync.Mutex - r io.Reader - h hash.Hash - - // data is remaining verified but unused payload data. This is - // populated on short reads and may be consumed without any - // verification. - data [SegmentSize]byte - - // index is the index into data above. - index int - - // available is the amount of valid data above. - available int - - // hashv is the read hash for the current segment. - hashv []byte - - // computev is the computed hash for the current segment. - computev []byte -} - -// NewReader creates a hash-verified IO stream reader. -func NewReader(r io.Reader, h hash.Hash) io.Reader { - return &reader{ - r: r, - h: h, - hashv: make([]byte, h.Size()), - computev: make([]byte, h.Size()), - } -} - -// readSegment reads a segment and hash vector. -// -// Precondition: datav must have length SegmentSize. -func (r *reader) readSegment(datav []byte) (data []byte, err error) { - // Make two reads: the first is the segment, the second is the hash - // which needs verification. We may need to adjust the resulting slices - // in the case of short reads. - for done := 0; done < SegmentSize; { - n, err := r.r.Read(datav[done:]) - done += n - if n == 0 && err == io.EOF { - if done == 0 { - // No data at all. - return nil, io.EOF - } else if done < len(r.hashv) { - // Not enough for a hash. - return nil, ErrHashMismatch - } - // Truncate the data and copy to the hash. - copy(r.hashv, datav[done-len(r.hashv):]) - datav = datav[:done-len(r.hashv)] - return datav, nil - } else if n == 0 && err != nil { - return nil, err - } - } - for done := 0; done < len(r.hashv); { - n, err := r.r.Read(r.hashv[done:]) - done += n - if n == 0 && err == io.EOF { - // Copy over from the data. - missing := len(r.hashv) - done - copy(r.hashv[missing:], r.hashv[:done]) - copy(r.hashv[:missing], datav[len(datav)-missing:]) - datav = datav[:len(datav)-missing] - return datav, nil - } else if n == 0 && err != nil { - return nil, err - } - } - return datav, nil -} - -// verifyHash verifies the given hash. -// -// The passed hash will be returned to the pool. -func (r *reader) verifyHash(datav []byte) error { - for done := 0; done < len(datav); { - n, _ := r.h.Write(datav[done:]) - done += n - } - computev := r.h.Sum(r.computev[:0]) - if !hmac.Equal(r.hashv, computev) { - return ErrHashMismatch - } - return nil -} - -// Read reads the data. -func (r *reader) Read(p []byte) (int, error) { - r.mu.Lock() - defer r.mu.Unlock() - - for done := 0; done < len(p); { - // Check for pending data. - if r.index < r.available { - n := copy(p[done:], r.data[r.index:r.available]) - done += n - r.index += n - continue - } - - // Prepare the next read. - var ( - datav []byte - inline bool - ) - - // We need to read a new segment. Can we read directly? - if len(p[done:]) >= SegmentSize { - datav = p[done : done+SegmentSize] - inline = true - } else { - datav = r.data[:] - inline = false - } - - // Read the next segments. - datav, err := r.readSegment(datav) - if err != nil && err != io.EOF { - return 0, err - } else if err == io.EOF { - return done, io.EOF - } - if err := r.verifyHash(datav); err != nil { - return done, err - } - - if inline { - // Move the cursor. - done += len(datav) - } else { - // Reset index & available. - r.index = 0 - r.available = len(datav) - } - } - - return len(p), nil -} diff --git a/pkg/hashio/hashio_test.go b/pkg/hashio/hashio_test.go deleted file mode 100644 index 41dbdf860..000000000 --- a/pkg/hashio/hashio_test.go +++ /dev/null @@ -1,142 +0,0 @@ -// Copyright 2018 Google Inc. -// -// 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 -// -// 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 hashio - -import ( - "bytes" - "crypto/hmac" - "crypto/sha256" - "fmt" - "io" - "math/rand" - "testing" -) - -var testKey = []byte("01234567890123456789012345678901") - -func runTest(c []byte, fn func(enc *bytes.Buffer), iters int) error { - // Encoding happens via a buffer. - var ( - enc bytes.Buffer - dec bytes.Buffer - ) - - for i := 0; i < iters; i++ { - enc.Reset() - w := NewWriter(&enc, hmac.New(sha256.New, testKey)) - if _, err := io.Copy(w, bytes.NewBuffer(c)); err != nil { - return err - } - if err := w.Close(); err != nil { - return err - } - } - - fn(&enc) - - for i := 0; i < iters; i++ { - dec.Reset() - r := NewReader(bytes.NewReader(enc.Bytes()), hmac.New(sha256.New, testKey)) - if _, err := io.Copy(&dec, r); err != nil { - return err - } - } - - // Check that the data matches; this should never fail. - if !bytes.Equal(c, dec.Bytes()) { - panic(fmt.Sprintf("data didn't match: got %v, expected %v", dec.Bytes(), c)) - } - - return nil -} - -func TestTable(t *testing.T) { - cases := [][]byte{ - // Various data sizes. - nil, - []byte(""), - []byte("_"), - []byte("0"), - []byte("01"), - []byte("012"), - []byte("0123"), - []byte("01234"), - []byte("012356"), - []byte("0123567"), - []byte("01235678"), - - // Make sure we have one longer than the hash length. - []byte("012356asdjflkasjlk3jlk23j4lkjaso0d789f0aujw3lkjlkxsdf78asdful2kj3ljka78"), - - // Make sure we have one longer than the segment size. - make([]byte, 3*SegmentSize), - make([]byte, 3*SegmentSize-1), - make([]byte, 3*SegmentSize+1), - make([]byte, 3*SegmentSize-32), - make([]byte, 3*SegmentSize+32), - make([]byte, 30*SegmentSize), - } - - for _, c := range cases { - for _, flip := range []bool{false, true} { - if len(c) == 0 && flip == true { - continue - } - - // Log the case. - t.Logf("case: len=%d flip=%v", len(c), flip) - - if err := runTest(c, func(enc *bytes.Buffer) { - if flip { - corrupted := rand.Intn(enc.Len()) - enc.Bytes()[corrupted]++ - } - }, 1); err != nil { - if !flip || err != ErrHashMismatch { - t.Errorf("error during read: got %v, expected nil", err) - } - continue - } else if flip { - t.Errorf("failed to detect ErrHashMismatch on corrupted data!") - continue - } - } - } -} - -const benchBytes = 10 * 1024 * 1024 // 10 MB. - -func BenchmarkWrite(b *testing.B) { - b.StopTimer() - x := make([]byte, benchBytes) - b.SetBytes(benchBytes) - b.StartTimer() - if err := runTest(x, func(enc *bytes.Buffer) { - b.StopTimer() - }, b.N); err != nil { - b.Errorf("benchmark failed: %v", err) - } -} - -func BenchmarkRead(b *testing.B) { - b.StopTimer() - x := make([]byte, benchBytes) - b.SetBytes(benchBytes) - if err := runTest(x, func(enc *bytes.Buffer) { - b.StartTimer() - }, b.N); err != nil { - b.Errorf("benchmark failed: %v", err) - } -} diff --git a/pkg/state/statefile/BUILD b/pkg/state/statefile/BUILD index 16abe1930..6be78dc9b 100644 --- a/pkg/state/statefile/BUILD +++ b/pkg/state/statefile/BUILD @@ -10,7 +10,6 @@ go_library( deps = [ "//pkg/binary", "//pkg/compressio", - "//pkg/hashio", ], ) @@ -19,5 +18,5 @@ go_test( size = "small", srcs = ["statefile_test.go"], embed = [":statefile"], - deps = ["//pkg/hashio"], + deps = ["//pkg/compressio"], ) diff --git a/pkg/state/statefile/statefile.go b/pkg/state/statefile/statefile.go index 0b4eff8fa..9c86c1934 100644 --- a/pkg/state/statefile/statefile.go +++ b/pkg/state/statefile/statefile.go @@ -57,7 +57,6 @@ import ( "crypto/sha256" "gvisor.googlesource.com/gvisor/pkg/binary" "gvisor.googlesource.com/gvisor/pkg/compressio" - "gvisor.googlesource.com/gvisor/pkg/hashio" ) // keySize is the AES-256 key length. @@ -139,13 +138,11 @@ func NewWriter(w io.Writer, key []byte, metadata map[string]string) (io.WriteClo } } - w = hashio.NewWriter(w, h) - // Wrap in compression. We always use "best speed" mode here. When using // "best compression" mode, there is usually only a little gain in file // size reduction, which translate to even smaller gain in restore // latency reduction, while inccuring much more CPU usage at save time. - return compressio.NewWriter(w, compressionChunkSize, flate.BestSpeed) + return compressio.NewWriter(w, key, compressionChunkSize, flate.BestSpeed) } // MetadataUnsafe reads out the metadata from a state file without verifying any @@ -204,7 +201,7 @@ func metadata(r io.Reader, h hash.Hash) (map[string]string, error) { return nil, err } if !hmac.Equal(cur, buf) { - return nil, hashio.ErrHashMismatch + return nil, compressio.ErrHashMismatch } } @@ -226,10 +223,8 @@ func NewReader(r io.Reader, key []byte) (io.Reader, map[string]string, error) { return nil, nil, err } - r = hashio.NewReader(r, h) - // Wrap in compression. - rc, err := compressio.NewReader(r) + rc, err := compressio.NewReader(r, key) if err != nil { return nil, nil, err } diff --git a/pkg/state/statefile/statefile_test.go b/pkg/state/statefile/statefile_test.go index 66d9581ed..fa3fb9f2c 100644 --- a/pkg/state/statefile/statefile_test.go +++ b/pkg/state/statefile/statefile_test.go @@ -20,9 +20,11 @@ import ( "encoding/base64" "io" "math/rand" + "runtime" "testing" + "time" - "gvisor.googlesource.com/gvisor/pkg/hashio" + "gvisor.googlesource.com/gvisor/pkg/compressio" ) func randomKey() ([]byte, error) { @@ -42,6 +44,8 @@ type testCase struct { } func TestStatefile(t *testing.T) { + rand.Seed(time.Now().Unix()) + cases := []testCase{ // Various data sizes. {"nil", nil, nil}, @@ -59,13 +63,9 @@ func TestStatefile(t *testing.T) { // Make sure we have one longer than the hash length. {"longer than hash", []byte("012356asdjflkasjlk3jlk23j4lkjaso0d789f0aujw3lkjlkxsdf78asdful2kj3ljka78"), nil}, - // Make sure we have one longer than the segment size. - {"segments", make([]byte, 3*hashio.SegmentSize), nil}, - {"segments minus one", make([]byte, 3*hashio.SegmentSize-1), nil}, - {"segments plus one", make([]byte, 3*hashio.SegmentSize+1), nil}, - {"segments minus hash", make([]byte, 3*hashio.SegmentSize-32), nil}, - {"segments plus hash", make([]byte, 3*hashio.SegmentSize+32), nil}, - {"large", make([]byte, 30*hashio.SegmentSize), nil}, + // Make sure we have one longer than the chunk size. + {"chunks", make([]byte, 3*compressionChunkSize), nil}, + {"large", make([]byte, 30*compressionChunkSize), nil}, // Different metadata. {"one metadata", []byte("data"), map[string]string{"foo": "bar"}}, @@ -130,27 +130,31 @@ func TestStatefile(t *testing.T) { } // Change the data and verify that it fails. - b := append([]byte(nil), bufEncoded.Bytes()...) - b[rand.Intn(len(b))]++ - r, _, err = NewReader(bytes.NewReader(b), key) - if err == nil { - _, err = io.Copy(&bufDecoded, r) - } - if err == nil { - t.Error("got no error: expected error on data corruption") + if key != nil { + b := append([]byte(nil), bufEncoded.Bytes()...) + b[rand.Intn(len(b))]++ + bufDecoded.Reset() + r, _, err = NewReader(bytes.NewReader(b), key) + if err == nil { + _, err = io.Copy(&bufDecoded, r) + } + if err == nil { + t.Error("got no error: expected error on data corruption") + } } // Change the key and verify that it fails. - if key == nil { - key = integrityKey - } else { - key[rand.Intn(len(key))]++ + newKey := integrityKey + if len(key) > 0 { + newKey = append([]byte{}, key...) + newKey[rand.Intn(len(newKey))]++ } - r, _, err = NewReader(bytes.NewReader(bufEncoded.Bytes()), key) + bufDecoded.Reset() + r, _, err = NewReader(bytes.NewReader(bufEncoded.Bytes()), newKey) if err == nil { _, err = io.Copy(&bufDecoded, r) } - if err != hashio.ErrHashMismatch { + if err != compressio.ErrHashMismatch { t.Errorf("got error: %v, expected ErrHashMismatch on key mismatch", err) } }) @@ -159,7 +163,7 @@ func TestStatefile(t *testing.T) { } } -const benchmarkDataSize = 10 * 1024 * 1024 +const benchmarkDataSize = 100 * 1024 * 1024 func benchmark(b *testing.B, size int, write bool, compressible bool) { b.StopTimer() @@ -249,14 +253,6 @@ func benchmark(b *testing.B, size int, write bool, compressible bool) { } } -func BenchmarkWrite1BCompressible(b *testing.B) { - benchmark(b, 1, true, true) -} - -func BenchmarkWrite1BNoncompressible(b *testing.B) { - benchmark(b, 1, true, false) -} - func BenchmarkWrite4KCompressible(b *testing.B) { benchmark(b, 4096, true, true) } @@ -273,14 +269,6 @@ func BenchmarkWrite1MNoncompressible(b *testing.B) { benchmark(b, 1024*1024, true, false) } -func BenchmarkRead1BCompressible(b *testing.B) { - benchmark(b, 1, false, true) -} - -func BenchmarkRead1BNoncompressible(b *testing.B) { - benchmark(b, 1, false, false) -} - func BenchmarkRead4KCompressible(b *testing.B) { benchmark(b, 4096, false, true) } @@ -296,3 +284,7 @@ func BenchmarkRead1MCompressible(b *testing.B) { func BenchmarkRead1MNoncompressible(b *testing.B) { benchmark(b, 1024*1024, false, false) } + +func init() { + runtime.GOMAXPROCS(runtime.NumCPU()) +} -- cgit v1.2.3 From d08ccdaaad0ff00622321957aa01cac149741005 Mon Sep 17 00:00:00 2001 From: Zhaozhong Ni Date: Tue, 28 Aug 2018 09:20:05 -0700 Subject: sentry: avoid double counting map objects in save / restore stats. PiperOrigin-RevId: 210551929 Change-Id: Idd05935bffc63b39166cc3751139aff61b689faa --- pkg/state/decode.go | 6 ++++++ pkg/state/encode.go | 6 ++++++ pkg/state/stats.go | 11 +++++++++++ 3 files changed, 23 insertions(+) (limited to 'pkg/state') diff --git a/pkg/state/decode.go b/pkg/state/decode.go index b3fbc793c..3ef59610b 100644 --- a/pkg/state/decode.go +++ b/pkg/state/decode.go @@ -365,6 +365,12 @@ func (ds *decodeState) decodeObject(os *objectState, obj reflect.Value, object * // (For non-interfaces this is a no-op). dyntyp := reflect.TypeOf(obj.Interface()) if dyntyp.Kind() == reflect.Map { + // Remove the map object count here to avoid + // double counting, as this object will be + // counted again when it gets processed later. + // We do not add a reference count as the + // reference is artificial. + ds.stats.Remove(obj) obj.Set(ds.register(id, dyntyp).obj) } else if dyntyp.Kind() == reflect.Ptr { ds.push(true /* dereference */, "", nil) diff --git a/pkg/state/encode.go b/pkg/state/encode.go index 59914bef4..fd052db12 100644 --- a/pkg/state/encode.go +++ b/pkg/state/encode.go @@ -335,6 +335,12 @@ func (es *encodeState) encodeObject(obj reflect.Value, mapAsValue bool, format s object = &pb.Object{Value: &pb.Object_MapValue{es.encodeMap(obj)}} } else { // Encode a reference to the map. + // + // Remove the map object count here to avoid double + // counting, as this object will be counted again when + // it gets processed later. We do not add a reference + // count as the reference is artificial. + es.stats.Remove(obj) object = &pb.Object{Value: &pb.Object_RefValue{es.register(obj)}} } default: diff --git a/pkg/state/stats.go b/pkg/state/stats.go index c4135a889..ddcc49f78 100644 --- a/pkg/state/stats.go +++ b/pkg/state/stats.go @@ -68,6 +68,17 @@ func (s *Stats) Add(obj reflect.Value) { entry.count++ } +// Remove removes a sample count. It should only be called after a previous +// Add(). +func (s *Stats) Remove(obj reflect.Value) { + if s == nil { + return + } + typ := obj.Type() + entry := s.byType[typ] + entry.count-- +} + // Start starts a sample. func (s *Stats) Start(obj reflect.Value) { if s == nil { -- cgit v1.2.3 From dd05c96d99b6dc7a8503c82e10ee5caeb6930cf6 Mon Sep 17 00:00:00 2001 From: Michael Pratt Date: Tue, 18 Sep 2018 14:37:29 -0700 Subject: Increase state test timeout PiperOrigin-RevId: 213519378 Change-Id: Iffdb987da3a7209a297ea2df171d2ae5fa9b2b34 --- pkg/state/BUILD | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'pkg/state') diff --git a/pkg/state/BUILD b/pkg/state/BUILD index 012b0484e..6a5b2d4ff 100644 --- a/pkg/state/BUILD +++ b/pkg/state/BUILD @@ -71,7 +71,7 @@ go_proto_library( go_test( name = "state_test", - size = "small", + timeout = "long", srcs = ["state_test.go"], embed = [":state"], ) -- cgit v1.2.3 From 8fce67af24945f82378b4c2731cca1788936d074 Mon Sep 17 00:00:00 2001 From: Ian Gudger Date: Fri, 19 Oct 2018 16:34:09 -0700 Subject: Use correct company name in copyright header PiperOrigin-RevId: 217951017 Change-Id: Ie08bf6987f98467d07457bcf35b5f1ff6e43c035 --- kokoro/run_build.sh | 2 +- kokoro/run_tests.sh | 2 +- pkg/abi/abi.go | 2 +- pkg/abi/abi_linux.go | 2 +- pkg/abi/flag.go | 2 +- pkg/abi/linux/aio.go | 2 +- pkg/abi/linux/ashmem.go | 2 +- pkg/abi/linux/binder.go | 2 +- pkg/abi/linux/bpf.go | 2 +- pkg/abi/linux/capability.go | 2 +- pkg/abi/linux/dev.go | 2 +- pkg/abi/linux/elf.go | 2 +- pkg/abi/linux/errors.go | 2 +- pkg/abi/linux/eventfd.go | 2 +- pkg/abi/linux/exec.go | 2 +- pkg/abi/linux/fcntl.go | 2 +- pkg/abi/linux/file.go | 2 +- pkg/abi/linux/fs.go | 2 +- pkg/abi/linux/futex.go | 2 +- pkg/abi/linux/inotify.go | 2 +- pkg/abi/linux/ioctl.go | 2 +- pkg/abi/linux/ip.go | 2 +- pkg/abi/linux/ipc.go | 2 +- pkg/abi/linux/limits.go | 2 +- pkg/abi/linux/linux.go | 2 +- pkg/abi/linux/mm.go | 2 +- pkg/abi/linux/netdevice.go | 2 +- pkg/abi/linux/netlink.go | 2 +- pkg/abi/linux/netlink_route.go | 2 +- pkg/abi/linux/poll.go | 2 +- pkg/abi/linux/prctl.go | 2 +- pkg/abi/linux/ptrace.go | 2 +- pkg/abi/linux/rusage.go | 2 +- pkg/abi/linux/sched.go | 2 +- pkg/abi/linux/seccomp.go | 2 +- pkg/abi/linux/sem.go | 2 +- pkg/abi/linux/shm.go | 2 +- pkg/abi/linux/signal.go | 2 +- pkg/abi/linux/socket.go | 2 +- pkg/abi/linux/time.go | 2 +- pkg/abi/linux/timer.go | 2 +- pkg/abi/linux/tty.go | 2 +- pkg/abi/linux/uio.go | 2 +- pkg/abi/linux/utsname.go | 2 +- pkg/amutex/amutex.go | 2 +- pkg/amutex/amutex_test.go | 2 +- pkg/atomicbitops/atomic_bitops.go | 2 +- pkg/atomicbitops/atomic_bitops_amd64.s | 2 +- pkg/atomicbitops/atomic_bitops_common.go | 2 +- pkg/atomicbitops/atomic_bitops_test.go | 2 +- pkg/binary/binary.go | 2 +- pkg/binary/binary_test.go | 2 +- pkg/bits/bits.go | 2 +- pkg/bits/bits_template.go | 2 +- pkg/bits/uint64_arch_amd64.go | 2 +- pkg/bits/uint64_arch_amd64_asm.s | 2 +- pkg/bits/uint64_arch_generic.go | 2 +- pkg/bits/uint64_test.go | 2 +- pkg/bpf/bpf.go | 2 +- pkg/bpf/decoder.go | 2 +- pkg/bpf/decoder_test.go | 2 +- pkg/bpf/input_bytes.go | 2 +- pkg/bpf/interpreter.go | 2 +- pkg/bpf/interpreter_test.go | 2 +- pkg/bpf/program_builder.go | 2 +- pkg/bpf/program_builder_test.go | 2 +- pkg/compressio/compressio.go | 2 +- pkg/compressio/compressio_test.go | 2 +- pkg/control/client/client.go | 2 +- pkg/control/server/server.go | 2 +- pkg/cpuid/cpu_amd64.s | 2 +- pkg/cpuid/cpuid.go | 2 +- pkg/cpuid/cpuid_parse_test.go | 2 +- pkg/cpuid/cpuid_test.go | 2 +- pkg/dhcp/client.go | 2 +- pkg/dhcp/dhcp.go | 2 +- pkg/dhcp/dhcp_string.go | 2 +- pkg/dhcp/dhcp_test.go | 2 +- pkg/dhcp/server.go | 2 +- pkg/eventchannel/event.go | 2 +- pkg/eventchannel/event.proto | 2 +- pkg/fd/fd.go | 2 +- pkg/fd/fd_test.go | 2 +- pkg/gate/gate.go | 2 +- pkg/gate/gate_test.go | 2 +- pkg/ilist/list.go | 2 +- pkg/ilist/list_test.go | 2 +- pkg/linewriter/linewriter.go | 2 +- pkg/linewriter/linewriter_test.go | 2 +- pkg/log/glog.go | 2 +- pkg/log/glog_unsafe.go | 2 +- pkg/log/json.go | 2 +- pkg/log/json_test.go | 2 +- pkg/log/log.go | 2 +- pkg/log/log_test.go | 2 +- pkg/metric/metric.go | 2 +- pkg/metric/metric.proto | 2 +- pkg/metric/metric_test.go | 2 +- pkg/p9/buffer.go | 2 +- pkg/p9/client.go | 2 +- pkg/p9/client_file.go | 2 +- pkg/p9/client_test.go | 2 +- pkg/p9/file.go | 2 +- pkg/p9/handlers.go | 2 +- pkg/p9/local_server/local_server.go | 2 +- pkg/p9/messages.go | 2 +- pkg/p9/messages_test.go | 2 +- pkg/p9/p9.go | 2 +- pkg/p9/p9_test.go | 2 +- pkg/p9/p9test/client_test.go | 2 +- pkg/p9/p9test/mocks.go | 2 +- pkg/p9/pool.go | 2 +- pkg/p9/pool_test.go | 2 +- pkg/p9/server.go | 2 +- pkg/p9/transport.go | 2 +- pkg/p9/transport_test.go | 2 +- pkg/p9/version.go | 2 +- pkg/p9/version_test.go | 2 +- pkg/rand/rand.go | 2 +- pkg/rand/rand_linux.go | 2 +- pkg/refs/refcounter.go | 2 +- pkg/refs/refcounter_state.go | 2 +- pkg/refs/refcounter_test.go | 2 +- pkg/seccomp/seccomp.go | 2 +- pkg/seccomp/seccomp_rules.go | 2 +- pkg/seccomp/seccomp_test.go | 2 +- pkg/seccomp/seccomp_test_victim.go | 2 +- pkg/seccomp/seccomp_unsafe.go | 2 +- pkg/secio/full_reader.go | 2 +- pkg/secio/secio.go | 2 +- pkg/secio/secio_test.go | 2 +- pkg/segment/range.go | 2 +- pkg/segment/set.go | 2 +- pkg/segment/set_state.go | 2 +- pkg/segment/test/segment_test.go | 2 +- pkg/segment/test/set_functions.go | 2 +- pkg/sentry/arch/aligned.go | 2 +- pkg/sentry/arch/arch.go | 2 +- pkg/sentry/arch/arch_amd64.go | 2 +- pkg/sentry/arch/arch_amd64.s | 2 +- pkg/sentry/arch/arch_state_x86.go | 2 +- pkg/sentry/arch/arch_x86.go | 2 +- pkg/sentry/arch/auxv.go | 2 +- pkg/sentry/arch/registers.proto | 2 +- pkg/sentry/arch/signal_act.go | 2 +- pkg/sentry/arch/signal_amd64.go | 2 +- pkg/sentry/arch/signal_info.go | 2 +- pkg/sentry/arch/signal_stack.go | 2 +- pkg/sentry/arch/stack.go | 2 +- pkg/sentry/arch/syscalls_amd64.go | 2 +- pkg/sentry/context/context.go | 2 +- pkg/sentry/context/contexttest/contexttest.go | 2 +- pkg/sentry/control/control.go | 2 +- pkg/sentry/control/proc.go | 2 +- pkg/sentry/control/proc_test.go | 2 +- pkg/sentry/control/state.go | 2 +- pkg/sentry/device/device.go | 2 +- pkg/sentry/device/device_test.go | 2 +- pkg/sentry/fs/anon/anon.go | 2 +- pkg/sentry/fs/anon/device.go | 2 +- pkg/sentry/fs/ashmem/area.go | 2 +- pkg/sentry/fs/ashmem/device.go | 2 +- pkg/sentry/fs/ashmem/pin_board.go | 2 +- pkg/sentry/fs/ashmem/pin_board_test.go | 2 +- pkg/sentry/fs/attr.go | 2 +- pkg/sentry/fs/binder/binder.go | 2 +- pkg/sentry/fs/context.go | 2 +- pkg/sentry/fs/copy_up.go | 2 +- pkg/sentry/fs/copy_up_test.go | 2 +- pkg/sentry/fs/dentry.go | 2 +- pkg/sentry/fs/dev/dev.go | 2 +- pkg/sentry/fs/dev/device.go | 2 +- pkg/sentry/fs/dev/fs.go | 2 +- pkg/sentry/fs/dev/full.go | 2 +- pkg/sentry/fs/dev/null.go | 2 +- pkg/sentry/fs/dev/random.go | 2 +- pkg/sentry/fs/dirent.go | 2 +- pkg/sentry/fs/dirent_cache.go | 2 +- pkg/sentry/fs/dirent_cache_test.go | 2 +- pkg/sentry/fs/dirent_refs_test.go | 2 +- pkg/sentry/fs/dirent_state.go | 2 +- pkg/sentry/fs/fdpipe/pipe.go | 2 +- pkg/sentry/fs/fdpipe/pipe_opener.go | 2 +- pkg/sentry/fs/fdpipe/pipe_opener_test.go | 2 +- pkg/sentry/fs/fdpipe/pipe_state.go | 2 +- pkg/sentry/fs/fdpipe/pipe_test.go | 2 +- pkg/sentry/fs/file.go | 2 +- pkg/sentry/fs/file_operations.go | 2 +- pkg/sentry/fs/file_overlay.go | 2 +- pkg/sentry/fs/file_overlay_test.go | 2 +- pkg/sentry/fs/file_state.go | 2 +- pkg/sentry/fs/file_test.go | 2 +- pkg/sentry/fs/filesystems.go | 2 +- pkg/sentry/fs/filetest/filetest.go | 2 +- pkg/sentry/fs/flags.go | 2 +- pkg/sentry/fs/fs.go | 2 +- pkg/sentry/fs/fsutil/dirty_set.go | 2 +- pkg/sentry/fs/fsutil/dirty_set_test.go | 2 +- pkg/sentry/fs/fsutil/file.go | 2 +- pkg/sentry/fs/fsutil/file_range_set.go | 2 +- pkg/sentry/fs/fsutil/frame_ref_set.go | 2 +- pkg/sentry/fs/fsutil/fsutil.go | 2 +- pkg/sentry/fs/fsutil/handle.go | 2 +- pkg/sentry/fs/fsutil/handle_test.go | 2 +- pkg/sentry/fs/fsutil/host_file_mapper.go | 2 +- pkg/sentry/fs/fsutil/host_file_mapper_state.go | 2 +- pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go | 2 +- pkg/sentry/fs/fsutil/inode.go | 2 +- pkg/sentry/fs/fsutil/inode_cached.go | 2 +- pkg/sentry/fs/fsutil/inode_cached_test.go | 2 +- pkg/sentry/fs/gofer/attr.go | 2 +- pkg/sentry/fs/gofer/cache_policy.go | 2 +- pkg/sentry/fs/gofer/context_file.go | 2 +- pkg/sentry/fs/gofer/device.go | 2 +- pkg/sentry/fs/gofer/file.go | 2 +- pkg/sentry/fs/gofer/file_state.go | 2 +- pkg/sentry/fs/gofer/fs.go | 2 +- pkg/sentry/fs/gofer/gofer_test.go | 2 +- pkg/sentry/fs/gofer/handles.go | 2 +- pkg/sentry/fs/gofer/inode.go | 2 +- pkg/sentry/fs/gofer/inode_state.go | 2 +- pkg/sentry/fs/gofer/path.go | 2 +- pkg/sentry/fs/gofer/session.go | 2 +- pkg/sentry/fs/gofer/session_state.go | 2 +- pkg/sentry/fs/gofer/socket.go | 2 +- pkg/sentry/fs/gofer/util.go | 2 +- pkg/sentry/fs/host/control.go | 2 +- pkg/sentry/fs/host/descriptor.go | 2 +- pkg/sentry/fs/host/descriptor_state.go | 2 +- pkg/sentry/fs/host/descriptor_test.go | 2 +- pkg/sentry/fs/host/device.go | 2 +- pkg/sentry/fs/host/file.go | 2 +- pkg/sentry/fs/host/fs.go | 2 +- pkg/sentry/fs/host/fs_test.go | 2 +- pkg/sentry/fs/host/inode.go | 2 +- pkg/sentry/fs/host/inode_state.go | 2 +- pkg/sentry/fs/host/inode_test.go | 2 +- pkg/sentry/fs/host/ioctl_unsafe.go | 2 +- pkg/sentry/fs/host/socket.go | 2 +- pkg/sentry/fs/host/socket_iovec.go | 2 +- pkg/sentry/fs/host/socket_state.go | 2 +- pkg/sentry/fs/host/socket_test.go | 2 +- pkg/sentry/fs/host/socket_unsafe.go | 2 +- pkg/sentry/fs/host/tty.go | 2 +- pkg/sentry/fs/host/util.go | 2 +- pkg/sentry/fs/host/util_unsafe.go | 2 +- pkg/sentry/fs/host/wait_test.go | 2 +- pkg/sentry/fs/inode.go | 2 +- pkg/sentry/fs/inode_inotify.go | 2 +- pkg/sentry/fs/inode_operations.go | 2 +- pkg/sentry/fs/inode_overlay.go | 2 +- pkg/sentry/fs/inode_overlay_test.go | 2 +- pkg/sentry/fs/inotify.go | 2 +- pkg/sentry/fs/inotify_event.go | 2 +- pkg/sentry/fs/inotify_watch.go | 2 +- pkg/sentry/fs/lock/lock.go | 2 +- pkg/sentry/fs/lock/lock_range_test.go | 2 +- pkg/sentry/fs/lock/lock_set_functions.go | 2 +- pkg/sentry/fs/lock/lock_test.go | 2 +- pkg/sentry/fs/mock.go | 2 +- pkg/sentry/fs/mount.go | 2 +- pkg/sentry/fs/mount_overlay.go | 2 +- pkg/sentry/fs/mount_state.go | 2 +- pkg/sentry/fs/mount_test.go | 2 +- pkg/sentry/fs/mounts.go | 2 +- pkg/sentry/fs/mounts_test.go | 2 +- pkg/sentry/fs/offset.go | 2 +- pkg/sentry/fs/overlay.go | 2 +- pkg/sentry/fs/path.go | 2 +- pkg/sentry/fs/path_test.go | 2 +- pkg/sentry/fs/proc/cpuinfo.go | 2 +- pkg/sentry/fs/proc/device/device.go | 2 +- pkg/sentry/fs/proc/exec_args.go | 2 +- pkg/sentry/fs/proc/fds.go | 2 +- pkg/sentry/fs/proc/file.go | 2 +- pkg/sentry/fs/proc/filesystems.go | 2 +- pkg/sentry/fs/proc/fs.go | 2 +- pkg/sentry/fs/proc/loadavg.go | 2 +- pkg/sentry/fs/proc/meminfo.go | 2 +- pkg/sentry/fs/proc/mounts.go | 2 +- pkg/sentry/fs/proc/net.go | 2 +- pkg/sentry/fs/proc/net_test.go | 2 +- pkg/sentry/fs/proc/proc.go | 2 +- pkg/sentry/fs/proc/rpcinet_proc.go | 2 +- pkg/sentry/fs/proc/seqfile/seqfile.go | 2 +- pkg/sentry/fs/proc/seqfile/seqfile_test.go | 2 +- pkg/sentry/fs/proc/stat.go | 2 +- pkg/sentry/fs/proc/sys.go | 2 +- pkg/sentry/fs/proc/sys_net.go | 2 +- pkg/sentry/fs/proc/sys_net_test.go | 2 +- pkg/sentry/fs/proc/task.go | 2 +- pkg/sentry/fs/proc/uid_gid_map.go | 2 +- pkg/sentry/fs/proc/uptime.go | 2 +- pkg/sentry/fs/proc/version.go | 2 +- pkg/sentry/fs/ramfs/dir.go | 2 +- pkg/sentry/fs/ramfs/file.go | 2 +- pkg/sentry/fs/ramfs/ramfs.go | 2 +- pkg/sentry/fs/ramfs/socket.go | 2 +- pkg/sentry/fs/ramfs/symlink.go | 2 +- pkg/sentry/fs/ramfs/test/test.go | 2 +- pkg/sentry/fs/ramfs/tree.go | 2 +- pkg/sentry/fs/ramfs/tree_test.go | 2 +- pkg/sentry/fs/restore.go | 2 +- pkg/sentry/fs/save.go | 2 +- pkg/sentry/fs/seek.go | 2 +- pkg/sentry/fs/sync.go | 2 +- pkg/sentry/fs/sys/device.go | 2 +- pkg/sentry/fs/sys/devices.go | 2 +- pkg/sentry/fs/sys/fs.go | 2 +- pkg/sentry/fs/sys/sys.go | 2 +- pkg/sentry/fs/timerfd/timerfd.go | 2 +- pkg/sentry/fs/tmpfs/device.go | 2 +- pkg/sentry/fs/tmpfs/file_regular.go | 2 +- pkg/sentry/fs/tmpfs/file_test.go | 2 +- pkg/sentry/fs/tmpfs/fs.go | 2 +- pkg/sentry/fs/tmpfs/inode_file.go | 2 +- pkg/sentry/fs/tmpfs/tmpfs.go | 2 +- pkg/sentry/fs/tty/dir.go | 2 +- pkg/sentry/fs/tty/fs.go | 2 +- pkg/sentry/fs/tty/inode.go | 2 +- pkg/sentry/fs/tty/line_discipline.go | 2 +- pkg/sentry/fs/tty/master.go | 2 +- pkg/sentry/fs/tty/queue.go | 2 +- pkg/sentry/fs/tty/slave.go | 2 +- pkg/sentry/fs/tty/terminal.go | 2 +- pkg/sentry/fs/tty/tty_test.go | 2 +- pkg/sentry/hostcpu/getcpu_amd64.s | 2 +- pkg/sentry/hostcpu/hostcpu.go | 2 +- pkg/sentry/hostcpu/hostcpu_test.go | 2 +- pkg/sentry/inet/context.go | 2 +- pkg/sentry/inet/inet.go | 2 +- pkg/sentry/inet/test_stack.go | 2 +- pkg/sentry/kernel/abstract_socket_namespace.go | 2 +- pkg/sentry/kernel/auth/auth.go | 2 +- pkg/sentry/kernel/auth/capability_set.go | 2 +- pkg/sentry/kernel/auth/context.go | 2 +- pkg/sentry/kernel/auth/credentials.go | 2 +- pkg/sentry/kernel/auth/id.go | 2 +- pkg/sentry/kernel/auth/id_map.go | 2 +- pkg/sentry/kernel/auth/id_map_functions.go | 2 +- pkg/sentry/kernel/auth/user_namespace.go | 2 +- pkg/sentry/kernel/context.go | 2 +- pkg/sentry/kernel/epoll/epoll.go | 2 +- pkg/sentry/kernel/epoll/epoll_state.go | 2 +- pkg/sentry/kernel/epoll/epoll_test.go | 2 +- pkg/sentry/kernel/eventfd/eventfd.go | 2 +- pkg/sentry/kernel/eventfd/eventfd_test.go | 2 +- pkg/sentry/kernel/fasync/fasync.go | 2 +- pkg/sentry/kernel/fd_map.go | 2 +- pkg/sentry/kernel/fd_map_test.go | 2 +- pkg/sentry/kernel/fs_context.go | 2 +- pkg/sentry/kernel/futex/futex.go | 2 +- pkg/sentry/kernel/futex/futex_test.go | 2 +- pkg/sentry/kernel/ipc_namespace.go | 2 +- pkg/sentry/kernel/kdefs/kdefs.go | 2 +- pkg/sentry/kernel/kernel.go | 2 +- pkg/sentry/kernel/kernel_state.go | 2 +- pkg/sentry/kernel/memevent/memory_events.go | 2 +- pkg/sentry/kernel/memevent/memory_events.proto | 2 +- pkg/sentry/kernel/pending_signals.go | 2 +- pkg/sentry/kernel/pending_signals_state.go | 2 +- pkg/sentry/kernel/pipe/buffers.go | 2 +- pkg/sentry/kernel/pipe/device.go | 2 +- pkg/sentry/kernel/pipe/node.go | 2 +- pkg/sentry/kernel/pipe/node_test.go | 2 +- pkg/sentry/kernel/pipe/pipe.go | 2 +- pkg/sentry/kernel/pipe/pipe_test.go | 2 +- pkg/sentry/kernel/pipe/reader.go | 2 +- pkg/sentry/kernel/pipe/reader_writer.go | 2 +- pkg/sentry/kernel/pipe/writer.go | 2 +- pkg/sentry/kernel/posixtimer.go | 2 +- pkg/sentry/kernel/ptrace.go | 2 +- pkg/sentry/kernel/rseq.go | 2 +- pkg/sentry/kernel/sched/cpuset.go | 2 +- pkg/sentry/kernel/sched/cpuset_test.go | 2 +- pkg/sentry/kernel/sched/sched.go | 2 +- pkg/sentry/kernel/seccomp.go | 2 +- pkg/sentry/kernel/semaphore/semaphore.go | 2 +- pkg/sentry/kernel/semaphore/semaphore_test.go | 2 +- pkg/sentry/kernel/sessions.go | 2 +- pkg/sentry/kernel/shm/device.go | 2 +- pkg/sentry/kernel/shm/shm.go | 2 +- pkg/sentry/kernel/signal.go | 2 +- pkg/sentry/kernel/signal_handlers.go | 2 +- pkg/sentry/kernel/syscalls.go | 2 +- pkg/sentry/kernel/syscalls_state.go | 2 +- pkg/sentry/kernel/syslog.go | 2 +- pkg/sentry/kernel/table_test.go | 2 +- pkg/sentry/kernel/task.go | 2 +- pkg/sentry/kernel/task_acct.go | 2 +- pkg/sentry/kernel/task_block.go | 2 +- pkg/sentry/kernel/task_clone.go | 2 +- pkg/sentry/kernel/task_context.go | 2 +- pkg/sentry/kernel/task_exec.go | 2 +- pkg/sentry/kernel/task_exit.go | 2 +- pkg/sentry/kernel/task_futex.go | 2 +- pkg/sentry/kernel/task_identity.go | 2 +- pkg/sentry/kernel/task_log.go | 2 +- pkg/sentry/kernel/task_net.go | 2 +- pkg/sentry/kernel/task_run.go | 2 +- pkg/sentry/kernel/task_sched.go | 2 +- pkg/sentry/kernel/task_signals.go | 2 +- pkg/sentry/kernel/task_start.go | 2 +- pkg/sentry/kernel/task_stop.go | 2 +- pkg/sentry/kernel/task_syscall.go | 2 +- pkg/sentry/kernel/task_test.go | 2 +- pkg/sentry/kernel/task_usermem.go | 2 +- pkg/sentry/kernel/thread_group.go | 2 +- pkg/sentry/kernel/threads.go | 2 +- pkg/sentry/kernel/time/context.go | 2 +- pkg/sentry/kernel/time/time.go | 2 +- pkg/sentry/kernel/timekeeper.go | 2 +- pkg/sentry/kernel/timekeeper_state.go | 2 +- pkg/sentry/kernel/timekeeper_test.go | 2 +- pkg/sentry/kernel/uts_namespace.go | 2 +- pkg/sentry/kernel/vdso.go | 2 +- pkg/sentry/kernel/version.go | 2 +- pkg/sentry/limits/context.go | 2 +- pkg/sentry/limits/limits.go | 2 +- pkg/sentry/limits/limits_test.go | 2 +- pkg/sentry/limits/linux.go | 2 +- pkg/sentry/loader/elf.go | 2 +- pkg/sentry/loader/interpreter.go | 2 +- pkg/sentry/loader/loader.go | 2 +- pkg/sentry/loader/vdso.go | 2 +- pkg/sentry/loader/vdso_state.go | 2 +- pkg/sentry/memmap/mapping_set.go | 2 +- pkg/sentry/memmap/mapping_set_test.go | 2 +- pkg/sentry/memmap/memmap.go | 2 +- pkg/sentry/memutil/memutil.go | 2 +- pkg/sentry/memutil/memutil_unsafe.go | 2 +- pkg/sentry/mm/address_space.go | 2 +- pkg/sentry/mm/aio_context.go | 2 +- pkg/sentry/mm/aio_context_state.go | 2 +- pkg/sentry/mm/debug.go | 2 +- pkg/sentry/mm/io.go | 2 +- pkg/sentry/mm/lifecycle.go | 2 +- pkg/sentry/mm/metadata.go | 2 +- pkg/sentry/mm/mm.go | 2 +- pkg/sentry/mm/mm_test.go | 2 +- pkg/sentry/mm/pma.go | 2 +- pkg/sentry/mm/proc_pid_maps.go | 2 +- pkg/sentry/mm/save_restore.go | 2 +- pkg/sentry/mm/shm.go | 2 +- pkg/sentry/mm/special_mappable.go | 2 +- pkg/sentry/mm/syscalls.go | 2 +- pkg/sentry/mm/vma.go | 2 +- pkg/sentry/platform/context.go | 2 +- pkg/sentry/platform/filemem/filemem.go | 2 +- pkg/sentry/platform/filemem/filemem_state.go | 2 +- pkg/sentry/platform/filemem/filemem_test.go | 2 +- pkg/sentry/platform/filemem/filemem_unsafe.go | 2 +- pkg/sentry/platform/interrupt/interrupt.go | 2 +- pkg/sentry/platform/interrupt/interrupt_test.go | 2 +- pkg/sentry/platform/kvm/address_space.go | 2 +- pkg/sentry/platform/kvm/allocator.go | 2 +- pkg/sentry/platform/kvm/bluepill.go | 2 +- pkg/sentry/platform/kvm/bluepill_amd64.go | 2 +- pkg/sentry/platform/kvm/bluepill_amd64.s | 2 +- pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go | 2 +- pkg/sentry/platform/kvm/bluepill_fault.go | 2 +- pkg/sentry/platform/kvm/bluepill_unsafe.go | 2 +- pkg/sentry/platform/kvm/context.go | 2 +- pkg/sentry/platform/kvm/host_map.go | 2 +- pkg/sentry/platform/kvm/kvm.go | 2 +- pkg/sentry/platform/kvm/kvm_amd64.go | 2 +- pkg/sentry/platform/kvm/kvm_amd64_unsafe.go | 2 +- pkg/sentry/platform/kvm/kvm_const.go | 2 +- pkg/sentry/platform/kvm/kvm_test.go | 2 +- pkg/sentry/platform/kvm/machine.go | 2 +- pkg/sentry/platform/kvm/machine_amd64.go | 2 +- pkg/sentry/platform/kvm/machine_amd64_unsafe.go | 2 +- pkg/sentry/platform/kvm/machine_unsafe.go | 2 +- pkg/sentry/platform/kvm/physical_map.go | 2 +- pkg/sentry/platform/kvm/testutil/testutil.go | 2 +- pkg/sentry/platform/kvm/testutil/testutil_amd64.go | 2 +- pkg/sentry/platform/kvm/testutil/testutil_amd64.s | 2 +- pkg/sentry/platform/kvm/virtual_map.go | 2 +- pkg/sentry/platform/kvm/virtual_map_test.go | 2 +- pkg/sentry/platform/mmap_min_addr.go | 2 +- pkg/sentry/platform/platform.go | 2 +- pkg/sentry/platform/procid/procid.go | 2 +- pkg/sentry/platform/procid/procid_amd64.s | 2 +- pkg/sentry/platform/procid/procid_net_test.go | 2 +- pkg/sentry/platform/procid/procid_test.go | 2 +- pkg/sentry/platform/ptrace/ptrace.go | 2 +- pkg/sentry/platform/ptrace/ptrace_unsafe.go | 2 +- pkg/sentry/platform/ptrace/stub_amd64.s | 2 +- pkg/sentry/platform/ptrace/stub_unsafe.go | 2 +- pkg/sentry/platform/ptrace/subprocess.go | 2 +- pkg/sentry/platform/ptrace/subprocess_amd64.go | 2 +- pkg/sentry/platform/ptrace/subprocess_linux.go | 2 +- pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go | 2 +- pkg/sentry/platform/ptrace/subprocess_unsafe.go | 2 +- pkg/sentry/platform/ring0/defs.go | 2 +- pkg/sentry/platform/ring0/defs_amd64.go | 2 +- pkg/sentry/platform/ring0/entry_amd64.go | 2 +- pkg/sentry/platform/ring0/entry_amd64.s | 2 +- pkg/sentry/platform/ring0/gen_offsets/main.go | 2 +- pkg/sentry/platform/ring0/kernel.go | 2 +- pkg/sentry/platform/ring0/kernel_amd64.go | 2 +- pkg/sentry/platform/ring0/kernel_unsafe.go | 2 +- pkg/sentry/platform/ring0/lib_amd64.go | 2 +- pkg/sentry/platform/ring0/lib_amd64.s | 2 +- pkg/sentry/platform/ring0/offsets_amd64.go | 2 +- pkg/sentry/platform/ring0/pagetables/allocator.go | 2 +- pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables_test.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables_x86.go | 2 +- pkg/sentry/platform/ring0/pagetables/pcids_x86.go | 2 +- pkg/sentry/platform/ring0/pagetables/walker_amd64.go | 2 +- pkg/sentry/platform/ring0/ring0.go | 2 +- pkg/sentry/platform/ring0/x86.go | 2 +- pkg/sentry/platform/safecopy/atomic_amd64.s | 2 +- pkg/sentry/platform/safecopy/memclr_amd64.s | 2 +- pkg/sentry/platform/safecopy/memcpy_amd64.s | 2 +- pkg/sentry/platform/safecopy/safecopy.go | 2 +- pkg/sentry/platform/safecopy/safecopy_test.go | 2 +- pkg/sentry/platform/safecopy/safecopy_unsafe.go | 2 +- pkg/sentry/platform/safecopy/sighandler_amd64.s | 2 +- pkg/sentry/safemem/block_unsafe.go | 2 +- pkg/sentry/safemem/io.go | 2 +- pkg/sentry/safemem/io_test.go | 2 +- pkg/sentry/safemem/safemem.go | 2 +- pkg/sentry/safemem/seq_test.go | 2 +- pkg/sentry/safemem/seq_unsafe.go | 2 +- pkg/sentry/sighandling/sighandling.go | 2 +- pkg/sentry/sighandling/sighandling_unsafe.go | 2 +- pkg/sentry/socket/control/control.go | 2 +- pkg/sentry/socket/epsocket/device.go | 2 +- pkg/sentry/socket/epsocket/epsocket.go | 2 +- pkg/sentry/socket/epsocket/provider.go | 2 +- pkg/sentry/socket/epsocket/save_restore.go | 2 +- pkg/sentry/socket/epsocket/stack.go | 2 +- pkg/sentry/socket/hostinet/device.go | 2 +- pkg/sentry/socket/hostinet/hostinet.go | 2 +- pkg/sentry/socket/hostinet/save_restore.go | 2 +- pkg/sentry/socket/hostinet/socket.go | 2 +- pkg/sentry/socket/hostinet/socket_unsafe.go | 2 +- pkg/sentry/socket/hostinet/stack.go | 2 +- pkg/sentry/socket/netlink/message.go | 2 +- pkg/sentry/socket/netlink/port/port.go | 2 +- pkg/sentry/socket/netlink/port/port_test.go | 2 +- pkg/sentry/socket/netlink/provider.go | 2 +- pkg/sentry/socket/netlink/route/protocol.go | 2 +- pkg/sentry/socket/netlink/socket.go | 2 +- pkg/sentry/socket/rpcinet/conn/conn.go | 2 +- pkg/sentry/socket/rpcinet/device.go | 2 +- pkg/sentry/socket/rpcinet/notifier/notifier.go | 2 +- pkg/sentry/socket/rpcinet/rpcinet.go | 2 +- pkg/sentry/socket/rpcinet/socket.go | 2 +- pkg/sentry/socket/rpcinet/stack.go | 2 +- pkg/sentry/socket/rpcinet/stack_unsafe.go | 2 +- pkg/sentry/socket/socket.go | 2 +- pkg/sentry/socket/unix/device.go | 2 +- pkg/sentry/socket/unix/io.go | 2 +- pkg/sentry/socket/unix/transport/connectioned.go | 2 +- pkg/sentry/socket/unix/transport/connectioned_state.go | 2 +- pkg/sentry/socket/unix/transport/connectionless.go | 2 +- pkg/sentry/socket/unix/transport/queue.go | 2 +- pkg/sentry/socket/unix/transport/unix.go | 2 +- pkg/sentry/socket/unix/unix.go | 2 +- pkg/sentry/state/state.go | 2 +- pkg/sentry/state/state_metadata.go | 2 +- pkg/sentry/state/state_unsafe.go | 2 +- pkg/sentry/strace/clone.go | 2 +- pkg/sentry/strace/futex.go | 2 +- pkg/sentry/strace/linux64.go | 2 +- pkg/sentry/strace/open.go | 2 +- pkg/sentry/strace/ptrace.go | 2 +- pkg/sentry/strace/socket.go | 2 +- pkg/sentry/strace/strace.go | 2 +- pkg/sentry/strace/strace.proto | 2 +- pkg/sentry/strace/syscalls.go | 2 +- pkg/sentry/syscalls/epoll.go | 2 +- pkg/sentry/syscalls/linux/error.go | 2 +- pkg/sentry/syscalls/linux/flags.go | 2 +- pkg/sentry/syscalls/linux/linux64.go | 2 +- pkg/sentry/syscalls/linux/sigset.go | 2 +- pkg/sentry/syscalls/linux/sys_aio.go | 2 +- pkg/sentry/syscalls/linux/sys_capability.go | 2 +- pkg/sentry/syscalls/linux/sys_epoll.go | 2 +- pkg/sentry/syscalls/linux/sys_eventfd.go | 2 +- pkg/sentry/syscalls/linux/sys_file.go | 2 +- pkg/sentry/syscalls/linux/sys_futex.go | 2 +- pkg/sentry/syscalls/linux/sys_getdents.go | 2 +- pkg/sentry/syscalls/linux/sys_identity.go | 2 +- pkg/sentry/syscalls/linux/sys_inotify.go | 2 +- pkg/sentry/syscalls/linux/sys_lseek.go | 2 +- pkg/sentry/syscalls/linux/sys_mmap.go | 2 +- pkg/sentry/syscalls/linux/sys_mount.go | 2 +- pkg/sentry/syscalls/linux/sys_pipe.go | 2 +- pkg/sentry/syscalls/linux/sys_poll.go | 2 +- pkg/sentry/syscalls/linux/sys_prctl.go | 2 +- pkg/sentry/syscalls/linux/sys_random.go | 2 +- pkg/sentry/syscalls/linux/sys_read.go | 2 +- pkg/sentry/syscalls/linux/sys_rlimit.go | 2 +- pkg/sentry/syscalls/linux/sys_rusage.go | 2 +- pkg/sentry/syscalls/linux/sys_sched.go | 2 +- pkg/sentry/syscalls/linux/sys_seccomp.go | 2 +- pkg/sentry/syscalls/linux/sys_sem.go | 2 +- pkg/sentry/syscalls/linux/sys_shm.go | 2 +- pkg/sentry/syscalls/linux/sys_signal.go | 2 +- pkg/sentry/syscalls/linux/sys_socket.go | 2 +- pkg/sentry/syscalls/linux/sys_stat.go | 2 +- pkg/sentry/syscalls/linux/sys_sync.go | 2 +- pkg/sentry/syscalls/linux/sys_sysinfo.go | 2 +- pkg/sentry/syscalls/linux/sys_syslog.go | 2 +- pkg/sentry/syscalls/linux/sys_thread.go | 2 +- pkg/sentry/syscalls/linux/sys_time.go | 2 +- pkg/sentry/syscalls/linux/sys_timer.go | 2 +- pkg/sentry/syscalls/linux/sys_timerfd.go | 2 +- pkg/sentry/syscalls/linux/sys_tls.go | 2 +- pkg/sentry/syscalls/linux/sys_utsname.go | 2 +- pkg/sentry/syscalls/linux/sys_write.go | 2 +- pkg/sentry/syscalls/linux/timespec.go | 2 +- pkg/sentry/syscalls/polling.go | 2 +- pkg/sentry/syscalls/syscalls.go | 2 +- pkg/sentry/syscalls/unimplemented_syscall.proto | 2 +- pkg/sentry/time/calibrated_clock.go | 2 +- pkg/sentry/time/calibrated_clock_test.go | 2 +- pkg/sentry/time/clock_id.go | 2 +- pkg/sentry/time/clocks.go | 2 +- pkg/sentry/time/muldiv_amd64.s | 2 +- pkg/sentry/time/parameters.go | 2 +- pkg/sentry/time/parameters_test.go | 2 +- pkg/sentry/time/sampler.go | 2 +- pkg/sentry/time/sampler_test.go | 2 +- pkg/sentry/time/sampler_unsafe.go | 2 +- pkg/sentry/time/tsc_amd64.s | 2 +- pkg/sentry/uniqueid/context.go | 2 +- pkg/sentry/usage/cpu.go | 2 +- pkg/sentry/usage/io.go | 2 +- pkg/sentry/usage/memory.go | 2 +- pkg/sentry/usage/memory_unsafe.go | 2 +- pkg/sentry/usage/usage.go | 2 +- pkg/sentry/usermem/access_type.go | 2 +- pkg/sentry/usermem/addr.go | 2 +- pkg/sentry/usermem/addr_range_seq_test.go | 2 +- pkg/sentry/usermem/addr_range_seq_unsafe.go | 2 +- pkg/sentry/usermem/bytes_io.go | 2 +- pkg/sentry/usermem/bytes_io_unsafe.go | 2 +- pkg/sentry/usermem/usermem.go | 2 +- pkg/sentry/usermem/usermem_test.go | 2 +- pkg/sentry/usermem/usermem_x86.go | 2 +- pkg/sentry/watchdog/watchdog.go | 2 +- pkg/sleep/commit_amd64.s | 2 +- pkg/sleep/commit_asm.go | 2 +- pkg/sleep/commit_noasm.go | 2 +- pkg/sleep/empty.s | 2 +- pkg/sleep/sleep_test.go | 2 +- pkg/sleep/sleep_unsafe.go | 2 +- pkg/state/decode.go | 2 +- pkg/state/encode.go | 2 +- pkg/state/encode_unsafe.go | 2 +- pkg/state/map.go | 2 +- pkg/state/object.proto | 2 +- pkg/state/printer.go | 2 +- pkg/state/state.go | 2 +- pkg/state/state_test.go | 2 +- pkg/state/statefile/statefile.go | 2 +- pkg/state/statefile/statefile_test.go | 2 +- pkg/state/stats.go | 2 +- pkg/sync/atomicptr_unsafe.go | 2 +- pkg/sync/atomicptrtest/atomicptr_test.go | 2 +- pkg/sync/memmove_unsafe.go | 2 +- pkg/sync/norace_unsafe.go | 2 +- pkg/sync/race_unsafe.go | 2 +- pkg/sync/seqatomic_unsafe.go | 2 +- pkg/sync/seqatomictest/seqatomic_test.go | 2 +- pkg/sync/seqcount.go | 2 +- pkg/sync/seqcount_test.go | 2 +- pkg/sync/sync.go | 2 +- pkg/syserr/host_linux.go | 2 +- pkg/syserr/netstack.go | 2 +- pkg/syserr/syserr.go | 2 +- pkg/syserror/syserror.go | 2 +- pkg/syserror/syserror_test.go | 2 +- pkg/tcpip/adapters/gonet/gonet.go | 2 +- pkg/tcpip/adapters/gonet/gonet_test.go | 2 +- pkg/tcpip/buffer/prependable.go | 2 +- pkg/tcpip/buffer/view.go | 2 +- pkg/tcpip/buffer/view_test.go | 2 +- pkg/tcpip/checker/checker.go | 2 +- pkg/tcpip/header/arp.go | 2 +- pkg/tcpip/header/checksum.go | 2 +- pkg/tcpip/header/eth.go | 2 +- pkg/tcpip/header/gue.go | 2 +- pkg/tcpip/header/icmpv4.go | 2 +- pkg/tcpip/header/icmpv6.go | 2 +- pkg/tcpip/header/interfaces.go | 2 +- pkg/tcpip/header/ipv4.go | 2 +- pkg/tcpip/header/ipv6.go | 2 +- pkg/tcpip/header/ipv6_fragment.go | 2 +- pkg/tcpip/header/ipversion_test.go | 2 +- pkg/tcpip/header/tcp.go | 2 +- pkg/tcpip/header/tcp_test.go | 2 +- pkg/tcpip/header/udp.go | 2 +- pkg/tcpip/link/channel/channel.go | 2 +- pkg/tcpip/link/fdbased/endpoint.go | 2 +- pkg/tcpip/link/fdbased/endpoint_test.go | 2 +- pkg/tcpip/link/loopback/loopback.go | 2 +- pkg/tcpip/link/rawfile/blockingpoll_amd64.s | 2 +- pkg/tcpip/link/rawfile/blockingpoll_unsafe.go | 2 +- pkg/tcpip/link/rawfile/blockingpoll_unsafe_amd64.go | 2 +- pkg/tcpip/link/rawfile/errors.go | 2 +- pkg/tcpip/link/rawfile/rawfile_unsafe.go | 2 +- pkg/tcpip/link/sharedmem/pipe/pipe.go | 2 +- pkg/tcpip/link/sharedmem/pipe/pipe_test.go | 2 +- pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go | 2 +- pkg/tcpip/link/sharedmem/pipe/rx.go | 2 +- pkg/tcpip/link/sharedmem/pipe/tx.go | 2 +- pkg/tcpip/link/sharedmem/queue/queue_test.go | 2 +- pkg/tcpip/link/sharedmem/queue/rx.go | 2 +- pkg/tcpip/link/sharedmem/queue/tx.go | 2 +- pkg/tcpip/link/sharedmem/rx.go | 2 +- pkg/tcpip/link/sharedmem/sharedmem.go | 2 +- pkg/tcpip/link/sharedmem/sharedmem_test.go | 2 +- pkg/tcpip/link/sharedmem/sharedmem_unsafe.go | 2 +- pkg/tcpip/link/sharedmem/tx.go | 2 +- pkg/tcpip/link/sniffer/pcap.go | 2 +- pkg/tcpip/link/sniffer/sniffer.go | 2 +- pkg/tcpip/link/tun/tun_unsafe.go | 2 +- pkg/tcpip/link/waitable/waitable.go | 2 +- pkg/tcpip/link/waitable/waitable_test.go | 2 +- pkg/tcpip/network/arp/arp.go | 2 +- pkg/tcpip/network/arp/arp_test.go | 2 +- pkg/tcpip/network/fragmentation/frag_heap.go | 2 +- pkg/tcpip/network/fragmentation/frag_heap_test.go | 2 +- pkg/tcpip/network/fragmentation/fragmentation.go | 2 +- pkg/tcpip/network/fragmentation/fragmentation_test.go | 2 +- pkg/tcpip/network/fragmentation/reassembler.go | 2 +- pkg/tcpip/network/fragmentation/reassembler_test.go | 2 +- pkg/tcpip/network/hash/hash.go | 2 +- pkg/tcpip/network/ip_test.go | 2 +- pkg/tcpip/network/ipv4/icmp.go | 2 +- pkg/tcpip/network/ipv4/ipv4.go | 2 +- pkg/tcpip/network/ipv4/ipv4_test.go | 2 +- pkg/tcpip/network/ipv6/icmp.go | 2 +- pkg/tcpip/network/ipv6/icmp_test.go | 2 +- pkg/tcpip/network/ipv6/ipv6.go | 2 +- pkg/tcpip/ports/ports.go | 2 +- pkg/tcpip/ports/ports_test.go | 2 +- pkg/tcpip/sample/tun_tcp_connect/main.go | 2 +- pkg/tcpip/sample/tun_tcp_echo/main.go | 2 +- pkg/tcpip/seqnum/seqnum.go | 2 +- pkg/tcpip/stack/linkaddrcache.go | 2 +- pkg/tcpip/stack/linkaddrcache_test.go | 2 +- pkg/tcpip/stack/nic.go | 2 +- pkg/tcpip/stack/registration.go | 2 +- pkg/tcpip/stack/route.go | 2 +- pkg/tcpip/stack/stack.go | 2 +- pkg/tcpip/stack/stack_global_state.go | 2 +- pkg/tcpip/stack/stack_test.go | 2 +- pkg/tcpip/stack/transport_demuxer.go | 2 +- pkg/tcpip/stack/transport_test.go | 2 +- pkg/tcpip/tcpip.go | 2 +- pkg/tcpip/tcpip_test.go | 2 +- pkg/tcpip/time.s | 2 +- pkg/tcpip/time_unsafe.go | 2 +- pkg/tcpip/transport/ping/endpoint.go | 2 +- pkg/tcpip/transport/ping/endpoint_state.go | 2 +- pkg/tcpip/transport/ping/protocol.go | 2 +- pkg/tcpip/transport/tcp/accept.go | 2 +- pkg/tcpip/transport/tcp/connect.go | 2 +- pkg/tcpip/transport/tcp/cubic.go | 2 +- pkg/tcpip/transport/tcp/dual_stack_test.go | 2 +- pkg/tcpip/transport/tcp/endpoint.go | 2 +- pkg/tcpip/transport/tcp/endpoint_state.go | 2 +- pkg/tcpip/transport/tcp/forwarder.go | 2 +- pkg/tcpip/transport/tcp/protocol.go | 2 +- pkg/tcpip/transport/tcp/rcv.go | 2 +- pkg/tcpip/transport/tcp/reno.go | 2 +- pkg/tcpip/transport/tcp/sack.go | 2 +- pkg/tcpip/transport/tcp/segment.go | 2 +- pkg/tcpip/transport/tcp/segment_heap.go | 2 +- pkg/tcpip/transport/tcp/segment_queue.go | 2 +- pkg/tcpip/transport/tcp/segment_state.go | 2 +- pkg/tcpip/transport/tcp/snd.go | 2 +- pkg/tcpip/transport/tcp/snd_state.go | 2 +- pkg/tcpip/transport/tcp/tcp_sack_test.go | 2 +- pkg/tcpip/transport/tcp/tcp_test.go | 2 +- pkg/tcpip/transport/tcp/tcp_timestamp_test.go | 2 +- pkg/tcpip/transport/tcp/testing/context/context.go | 2 +- pkg/tcpip/transport/tcp/timer.go | 2 +- pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go | 2 +- pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go | 2 +- pkg/tcpip/transport/udp/endpoint.go | 2 +- pkg/tcpip/transport/udp/endpoint_state.go | 2 +- pkg/tcpip/transport/udp/protocol.go | 2 +- pkg/tcpip/transport/udp/udp_test.go | 2 +- pkg/tmutex/tmutex.go | 2 +- pkg/tmutex/tmutex_test.go | 2 +- pkg/unet/unet.go | 2 +- pkg/unet/unet_test.go | 2 +- pkg/unet/unet_unsafe.go | 2 +- pkg/urpc/urpc.go | 2 +- pkg/urpc/urpc_test.go | 2 +- pkg/waiter/fdnotifier/fdnotifier.go | 2 +- pkg/waiter/fdnotifier/poll_unsafe.go | 2 +- pkg/waiter/waiter.go | 2 +- pkg/waiter/waiter_test.go | 2 +- runsc/boot/compat.go | 2 +- runsc/boot/config.go | 2 +- runsc/boot/controller.go | 2 +- runsc/boot/debug.go | 2 +- runsc/boot/events.go | 2 +- runsc/boot/fds.go | 2 +- runsc/boot/filter/config.go | 2 +- runsc/boot/filter/extra_filters.go | 2 +- runsc/boot/filter/extra_filters_msan.go | 2 +- runsc/boot/filter/extra_filters_race.go | 2 +- runsc/boot/filter/filter.go | 2 +- runsc/boot/fs.go | 2 +- runsc/boot/limits.go | 2 +- runsc/boot/loader.go | 2 +- runsc/boot/loader_test.go | 2 +- runsc/boot/network.go | 2 +- runsc/boot/strace.go | 2 +- runsc/cgroup/cgroup.go | 2 +- runsc/cgroup/cgroup_test.go | 2 +- runsc/cmd/boot.go | 2 +- runsc/cmd/capability.go | 2 +- runsc/cmd/capability_test.go | 2 +- runsc/cmd/checkpoint.go | 2 +- runsc/cmd/cmd.go | 2 +- runsc/cmd/create.go | 2 +- runsc/cmd/debug.go | 2 +- runsc/cmd/delete.go | 2 +- runsc/cmd/delete_test.go | 2 +- runsc/cmd/events.go | 2 +- runsc/cmd/exec.go | 2 +- runsc/cmd/exec_test.go | 2 +- runsc/cmd/gofer.go | 2 +- runsc/cmd/kill.go | 2 +- runsc/cmd/list.go | 2 +- runsc/cmd/path.go | 2 +- runsc/cmd/pause.go | 2 +- runsc/cmd/ps.go | 2 +- runsc/cmd/restore.go | 2 +- runsc/cmd/resume.go | 2 +- runsc/cmd/run.go | 2 +- runsc/cmd/spec.go | 2 +- runsc/cmd/start.go | 2 +- runsc/cmd/state.go | 2 +- runsc/cmd/wait.go | 2 +- runsc/console/console.go | 2 +- runsc/container/console_test.go | 2 +- runsc/container/container.go | 2 +- runsc/container/container_test.go | 2 +- runsc/container/fs.go | 2 +- runsc/container/fs_test.go | 2 +- runsc/container/hook.go | 2 +- runsc/container/multi_container_test.go | 2 +- runsc/container/status.go | 2 +- runsc/container/test_app.go | 2 +- runsc/fsgofer/filter/config.go | 2 +- runsc/fsgofer/filter/extra_filters.go | 2 +- runsc/fsgofer/filter/extra_filters_msan.go | 2 +- runsc/fsgofer/filter/extra_filters_race.go | 2 +- runsc/fsgofer/filter/filter.go | 2 +- runsc/fsgofer/fsgofer.go | 2 +- runsc/fsgofer/fsgofer_test.go | 2 +- runsc/fsgofer/fsgofer_unsafe.go | 2 +- runsc/main.go | 2 +- runsc/sandbox/chroot.go | 2 +- runsc/sandbox/network.go | 2 +- runsc/sandbox/sandbox.go | 2 +- runsc/specutils/namespace.go | 2 +- runsc/specutils/specutils.go | 2 +- runsc/specutils/specutils_test.go | 2 +- runsc/test/image/image.go | 2 +- runsc/test/image/image_test.go | 2 +- runsc/test/image/mysql.sql | 2 +- runsc/test/image/ruby.rb | 2 +- runsc/test/image/ruby.sh | 2 +- runsc/test/install.sh | 2 +- runsc/test/integration/exec_test.go | 2 +- runsc/test/integration/integration.go | 2 +- runsc/test/integration/integration_test.go | 2 +- runsc/test/root/cgroup_test.go | 2 +- runsc/test/root/chroot_test.go | 2 +- runsc/test/root/root.go | 2 +- runsc/test/testutil/docker.go | 2 +- runsc/test/testutil/testutil.go | 2 +- runsc/test/testutil/testutil_race.go | 2 +- runsc/tools/dockercfg/dockercfg.go | 2 +- tools/go_generics/generics.go | 2 +- tools/go_generics/generics_tests/all_stmts/input.go | 2 +- tools/go_generics/generics_tests/all_stmts/output/output.go | 2 +- tools/go_generics/generics_tests/all_types/input.go | 2 +- tools/go_generics/generics_tests/all_types/lib/lib.go | 2 +- tools/go_generics/generics_tests/all_types/output/output.go | 2 +- tools/go_generics/generics_tests/consts/input.go | 2 +- tools/go_generics/generics_tests/consts/output/output.go | 2 +- tools/go_generics/generics_tests/imports/input.go | 2 +- tools/go_generics/generics_tests/imports/output/output.go | 2 +- tools/go_generics/generics_tests/remove_typedef/input.go | 2 +- tools/go_generics/generics_tests/remove_typedef/output/output.go | 2 +- tools/go_generics/generics_tests/simple/input.go | 2 +- tools/go_generics/generics_tests/simple/output/output.go | 2 +- tools/go_generics/globals/globals_visitor.go | 2 +- tools/go_generics/globals/scope.go | 2 +- tools/go_generics/go_generics_unittest.sh | 2 +- tools/go_generics/imports.go | 2 +- tools/go_generics/merge.go | 2 +- tools/go_generics/remove.go | 2 +- tools/go_generics/rules_tests/template.go | 2 +- tools/go_generics/rules_tests/template_test.go | 2 +- tools/go_stateify/main.go | 2 +- tools/workspace_status.sh | 2 +- vdso/barrier.h | 2 +- vdso/check_vdso.py | 2 +- vdso/compiler.h | 2 +- vdso/cycle_clock.h | 2 +- vdso/seqlock.h | 2 +- vdso/syscalls.h | 2 +- vdso/vdso.cc | 2 +- vdso/vdso_time.cc | 2 +- vdso/vdso_time.h | 2 +- 923 files changed, 923 insertions(+), 923 deletions(-) (limited to 'pkg/state') diff --git a/kokoro/run_build.sh b/kokoro/run_build.sh index f2b719f52..89e24b037 100755 --- a/kokoro/run_build.sh +++ b/kokoro/run_build.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/kokoro/run_tests.sh b/kokoro/run_tests.sh index 3f8841cee..0a0d73d29 100755 --- a/kokoro/run_tests.sh +++ b/kokoro/run_tests.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/pkg/abi/abi.go b/pkg/abi/abi.go index a53c2747b..7770f0405 100644 --- a/pkg/abi/abi.go +++ b/pkg/abi/abi.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/abi_linux.go b/pkg/abi/abi_linux.go index dd5d67b51..9d9f361a4 100644 --- a/pkg/abi/abi_linux.go +++ b/pkg/abi/abi_linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/flag.go b/pkg/abi/flag.go index 0391ccf37..0698e410f 100644 --- a/pkg/abi/flag.go +++ b/pkg/abi/flag.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/aio.go b/pkg/abi/linux/aio.go index 9c39ca2ef..1b7ca714a 100644 --- a/pkg/abi/linux/aio.go +++ b/pkg/abi/linux/aio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/ashmem.go b/pkg/abi/linux/ashmem.go index 7fbfd2e68..ced1e44d4 100644 --- a/pkg/abi/linux/ashmem.go +++ b/pkg/abi/linux/ashmem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/binder.go b/pkg/abi/linux/binder.go index b228898f9..522dc6f53 100644 --- a/pkg/abi/linux/binder.go +++ b/pkg/abi/linux/binder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/bpf.go b/pkg/abi/linux/bpf.go index 80e5b1af1..d9cd09948 100644 --- a/pkg/abi/linux/bpf.go +++ b/pkg/abi/linux/bpf.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/capability.go b/pkg/abi/linux/capability.go index b470ce0a5..7d96f013e 100644 --- a/pkg/abi/linux/capability.go +++ b/pkg/abi/linux/capability.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/dev.go b/pkg/abi/linux/dev.go index ea5b16b7b..5b1199aac 100644 --- a/pkg/abi/linux/dev.go +++ b/pkg/abi/linux/dev.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/elf.go b/pkg/abi/linux/elf.go index 76c13b677..928067c04 100644 --- a/pkg/abi/linux/elf.go +++ b/pkg/abi/linux/elf.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/errors.go b/pkg/abi/linux/errors.go index b5ddb2b2f..01e4095b8 100644 --- a/pkg/abi/linux/errors.go +++ b/pkg/abi/linux/errors.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/eventfd.go b/pkg/abi/linux/eventfd.go index bc0fb44d2..5614f5cf1 100644 --- a/pkg/abi/linux/eventfd.go +++ b/pkg/abi/linux/eventfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/exec.go b/pkg/abi/linux/exec.go index 4d81eca54..a07c29243 100644 --- a/pkg/abi/linux/exec.go +++ b/pkg/abi/linux/exec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/fcntl.go b/pkg/abi/linux/fcntl.go index 2a5ad6ed7..c8558933a 100644 --- a/pkg/abi/linux/fcntl.go +++ b/pkg/abi/linux/fcntl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/file.go b/pkg/abi/linux/file.go index 9bf229a57..72e5c6f83 100644 --- a/pkg/abi/linux/file.go +++ b/pkg/abi/linux/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/fs.go b/pkg/abi/linux/fs.go index 32a0812b4..7817bfb52 100644 --- a/pkg/abi/linux/fs.go +++ b/pkg/abi/linux/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/futex.go b/pkg/abi/linux/futex.go index f63f5200c..5dff01fba 100644 --- a/pkg/abi/linux/futex.go +++ b/pkg/abi/linux/futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/inotify.go b/pkg/abi/linux/inotify.go index 072a2d146..79c5d3593 100644 --- a/pkg/abi/linux/inotify.go +++ b/pkg/abi/linux/inotify.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/ioctl.go b/pkg/abi/linux/ioctl.go index afd9ee82b..9afc3d1ef 100644 --- a/pkg/abi/linux/ioctl.go +++ b/pkg/abi/linux/ioctl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/ip.go b/pkg/abi/linux/ip.go index 6b68999ab..fcec16965 100644 --- a/pkg/abi/linux/ip.go +++ b/pkg/abi/linux/ip.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/ipc.go b/pkg/abi/linux/ipc.go index 81e9904dd..10681768b 100644 --- a/pkg/abi/linux/ipc.go +++ b/pkg/abi/linux/ipc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/limits.go b/pkg/abi/linux/limits.go index e1f0932ec..b2e51b9bd 100644 --- a/pkg/abi/linux/limits.go +++ b/pkg/abi/linux/limits.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/linux.go b/pkg/abi/linux/linux.go index de2af80dc..d365f693d 100644 --- a/pkg/abi/linux/linux.go +++ b/pkg/abi/linux/linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/mm.go b/pkg/abi/linux/mm.go index b48e1d18a..3fcdf8235 100644 --- a/pkg/abi/linux/mm.go +++ b/pkg/abi/linux/mm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/netdevice.go b/pkg/abi/linux/netdevice.go index 88654a1b3..e3b6b1e40 100644 --- a/pkg/abi/linux/netdevice.go +++ b/pkg/abi/linux/netdevice.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/netlink.go b/pkg/abi/linux/netlink.go index e823ffa7e..10ceb5bf2 100644 --- a/pkg/abi/linux/netlink.go +++ b/pkg/abi/linux/netlink.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/netlink_route.go b/pkg/abi/linux/netlink_route.go index a5d778748..4200b6506 100644 --- a/pkg/abi/linux/netlink_route.go +++ b/pkg/abi/linux/netlink_route.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/poll.go b/pkg/abi/linux/poll.go index f373cfca1..9f0b15d1c 100644 --- a/pkg/abi/linux/poll.go +++ b/pkg/abi/linux/poll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/prctl.go b/pkg/abi/linux/prctl.go index 074ec03f0..e152c4c27 100644 --- a/pkg/abi/linux/prctl.go +++ b/pkg/abi/linux/prctl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/ptrace.go b/pkg/abi/linux/ptrace.go index ba48d4d6d..7db4f5464 100644 --- a/pkg/abi/linux/ptrace.go +++ b/pkg/abi/linux/ptrace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/rusage.go b/pkg/abi/linux/rusage.go index a4a89abda..7fea4b589 100644 --- a/pkg/abi/linux/rusage.go +++ b/pkg/abi/linux/rusage.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/sched.go b/pkg/abi/linux/sched.go index 05fda1604..ef96a3801 100644 --- a/pkg/abi/linux/sched.go +++ b/pkg/abi/linux/sched.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/seccomp.go b/pkg/abi/linux/seccomp.go index a8de9d3d0..9963ceeba 100644 --- a/pkg/abi/linux/seccomp.go +++ b/pkg/abi/linux/seccomp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/sem.go b/pkg/abi/linux/sem.go index 3495f5cd0..d1a0bdb32 100644 --- a/pkg/abi/linux/sem.go +++ b/pkg/abi/linux/sem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/shm.go b/pkg/abi/linux/shm.go index f50b3c2e2..82a80e609 100644 --- a/pkg/abi/linux/shm.go +++ b/pkg/abi/linux/shm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/signal.go b/pkg/abi/linux/signal.go index b2c7230c4..bf9bce6ed 100644 --- a/pkg/abi/linux/signal.go +++ b/pkg/abi/linux/signal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/socket.go b/pkg/abi/linux/socket.go index 19b5fa212..af0761a3b 100644 --- a/pkg/abi/linux/socket.go +++ b/pkg/abi/linux/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/time.go b/pkg/abi/linux/time.go index 4569f4208..bbd21e726 100644 --- a/pkg/abi/linux/time.go +++ b/pkg/abi/linux/time.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/timer.go b/pkg/abi/linux/timer.go index 6c4675c35..a6f420bdb 100644 --- a/pkg/abi/linux/timer.go +++ b/pkg/abi/linux/timer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/tty.go b/pkg/abi/linux/tty.go index f63dc52aa..e6f7c5b2a 100644 --- a/pkg/abi/linux/tty.go +++ b/pkg/abi/linux/tty.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/uio.go b/pkg/abi/linux/uio.go index 93c972774..7e00d9959 100644 --- a/pkg/abi/linux/uio.go +++ b/pkg/abi/linux/uio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/utsname.go b/pkg/abi/linux/utsname.go index 7d33d20de..f80ed7d4a 100644 --- a/pkg/abi/linux/utsname.go +++ b/pkg/abi/linux/utsname.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/amutex/amutex.go b/pkg/amutex/amutex.go index 1cb73359a..26b674435 100644 --- a/pkg/amutex/amutex.go +++ b/pkg/amutex/amutex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/amutex/amutex_test.go b/pkg/amutex/amutex_test.go index 876e47b19..104e0dab1 100644 --- a/pkg/amutex/amutex_test.go +++ b/pkg/amutex/amutex_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/atomicbitops/atomic_bitops.go b/pkg/atomicbitops/atomic_bitops.go index 6635ea0d2..9a57f9599 100644 --- a/pkg/atomicbitops/atomic_bitops.go +++ b/pkg/atomicbitops/atomic_bitops.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/atomicbitops/atomic_bitops_amd64.s b/pkg/atomicbitops/atomic_bitops_amd64.s index 542452bec..b37e3aad3 100644 --- a/pkg/atomicbitops/atomic_bitops_amd64.s +++ b/pkg/atomicbitops/atomic_bitops_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/atomicbitops/atomic_bitops_common.go b/pkg/atomicbitops/atomic_bitops_common.go index 542ff4e83..b03242baa 100644 --- a/pkg/atomicbitops/atomic_bitops_common.go +++ b/pkg/atomicbitops/atomic_bitops_common.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/atomicbitops/atomic_bitops_test.go b/pkg/atomicbitops/atomic_bitops_test.go index ec0c07ee2..ee6207cb3 100644 --- a/pkg/atomicbitops/atomic_bitops_test.go +++ b/pkg/atomicbitops/atomic_bitops_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/binary/binary.go b/pkg/binary/binary.go index 3b18a86ee..02f7e9fb8 100644 --- a/pkg/binary/binary.go +++ b/pkg/binary/binary.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/binary/binary_test.go b/pkg/binary/binary_test.go index 921a0369a..d8d481f32 100644 --- a/pkg/binary/binary_test.go +++ b/pkg/binary/binary_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/bits.go b/pkg/bits/bits.go index 50ca4bff7..eb3c80f49 100644 --- a/pkg/bits/bits.go +++ b/pkg/bits/bits.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/bits_template.go b/pkg/bits/bits_template.go index 0a01f29c2..8c578cca2 100644 --- a/pkg/bits/bits_template.go +++ b/pkg/bits/bits_template.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/uint64_arch_amd64.go b/pkg/bits/uint64_arch_amd64.go index 068597f68..1fef89394 100644 --- a/pkg/bits/uint64_arch_amd64.go +++ b/pkg/bits/uint64_arch_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/uint64_arch_amd64_asm.s b/pkg/bits/uint64_arch_amd64_asm.s index 33885641a..8c7322f0f 100644 --- a/pkg/bits/uint64_arch_amd64_asm.s +++ b/pkg/bits/uint64_arch_amd64_asm.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/uint64_arch_generic.go b/pkg/bits/uint64_arch_generic.go index 862033a4b..cfb47400b 100644 --- a/pkg/bits/uint64_arch_generic.go +++ b/pkg/bits/uint64_arch_generic.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/uint64_test.go b/pkg/bits/uint64_test.go index 906017e1a..d6dbaf602 100644 --- a/pkg/bits/uint64_test.go +++ b/pkg/bits/uint64_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/bpf.go b/pkg/bpf/bpf.go index 757744090..98d44d911 100644 --- a/pkg/bpf/bpf.go +++ b/pkg/bpf/bpf.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/decoder.go b/pkg/bpf/decoder.go index ef41e9edc..ae6b8839a 100644 --- a/pkg/bpf/decoder.go +++ b/pkg/bpf/decoder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/decoder_test.go b/pkg/bpf/decoder_test.go index 18709b944..f093e1e41 100644 --- a/pkg/bpf/decoder_test.go +++ b/pkg/bpf/decoder_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/input_bytes.go b/pkg/bpf/input_bytes.go index 74af038eb..745c0749b 100644 --- a/pkg/bpf/input_bytes.go +++ b/pkg/bpf/input_bytes.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/interpreter.go b/pkg/bpf/interpreter.go index 111ada9d1..86c7add4d 100644 --- a/pkg/bpf/interpreter.go +++ b/pkg/bpf/interpreter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/interpreter_test.go b/pkg/bpf/interpreter_test.go index 9e5e33228..c46a43991 100644 --- a/pkg/bpf/interpreter_test.go +++ b/pkg/bpf/interpreter_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/program_builder.go b/pkg/bpf/program_builder.go index bad56d7ac..b4ce228e1 100644 --- a/pkg/bpf/program_builder.go +++ b/pkg/bpf/program_builder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/program_builder_test.go b/pkg/bpf/program_builder_test.go index 7e4f06584..0e0b79d88 100644 --- a/pkg/bpf/program_builder_test.go +++ b/pkg/bpf/program_builder_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/compressio/compressio.go b/pkg/compressio/compressio.go index 667f17c5c..205536812 100644 --- a/pkg/compressio/compressio.go +++ b/pkg/compressio/compressio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/compressio/compressio_test.go b/pkg/compressio/compressio_test.go index 7cb5f8dc4..1bbabee79 100644 --- a/pkg/compressio/compressio_test.go +++ b/pkg/compressio/compressio_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/control/client/client.go b/pkg/control/client/client.go index f7c2e8776..0d0c9f148 100644 --- a/pkg/control/client/client.go +++ b/pkg/control/client/client.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/control/server/server.go b/pkg/control/server/server.go index d00061ce3..c46b5d70b 100644 --- a/pkg/control/server/server.go +++ b/pkg/control/server/server.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/cpuid/cpu_amd64.s b/pkg/cpuid/cpu_amd64.s index 48a13c6fd..905c1d12e 100644 --- a/pkg/cpuid/cpu_amd64.s +++ b/pkg/cpuid/cpu_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/cpuid/cpuid.go b/pkg/cpuid/cpuid.go index e91e34dc7..5b083a5fb 100644 --- a/pkg/cpuid/cpuid.go +++ b/pkg/cpuid/cpuid.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/cpuid/cpuid_parse_test.go b/pkg/cpuid/cpuid_parse_test.go index c4f52818c..81b06f48c 100644 --- a/pkg/cpuid/cpuid_parse_test.go +++ b/pkg/cpuid/cpuid_parse_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/cpuid/cpuid_test.go b/pkg/cpuid/cpuid_test.go index 02f732f85..0decd8f08 100644 --- a/pkg/cpuid/cpuid_test.go +++ b/pkg/cpuid/cpuid_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/dhcp/client.go b/pkg/dhcp/client.go index 92c634a14..3330c4998 100644 --- a/pkg/dhcp/client.go +++ b/pkg/dhcp/client.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/dhcp/dhcp.go b/pkg/dhcp/dhcp.go index ceaba34c3..ad11e178a 100644 --- a/pkg/dhcp/dhcp.go +++ b/pkg/dhcp/dhcp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/dhcp/dhcp_string.go b/pkg/dhcp/dhcp_string.go index 7cabed29e..8533895bd 100644 --- a/pkg/dhcp/dhcp_string.go +++ b/pkg/dhcp/dhcp_string.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/dhcp/dhcp_test.go b/pkg/dhcp/dhcp_test.go index d60e3752b..a21dce6bc 100644 --- a/pkg/dhcp/dhcp_test.go +++ b/pkg/dhcp/dhcp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/dhcp/server.go b/pkg/dhcp/server.go index 26700bdbc..3e06ab4c7 100644 --- a/pkg/dhcp/server.go +++ b/pkg/dhcp/server.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/eventchannel/event.go b/pkg/eventchannel/event.go index bfd28256e..41a7b5ed3 100644 --- a/pkg/eventchannel/event.go +++ b/pkg/eventchannel/event.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/eventchannel/event.proto b/pkg/eventchannel/event.proto index 455f03658..c1679c7e7 100644 --- a/pkg/eventchannel/event.proto +++ b/pkg/eventchannel/event.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/fd/fd.go b/pkg/fd/fd.go index 32d24c41b..f6656ffa1 100644 --- a/pkg/fd/fd.go +++ b/pkg/fd/fd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/fd/fd_test.go b/pkg/fd/fd_test.go index 94b3eb7cc..42bb3ef6c 100644 --- a/pkg/fd/fd_test.go +++ b/pkg/fd/fd_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/gate/gate.go b/pkg/gate/gate.go index 93808c9dd..48122bf5a 100644 --- a/pkg/gate/gate.go +++ b/pkg/gate/gate.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/gate/gate_test.go b/pkg/gate/gate_test.go index 06587339b..95620fa8e 100644 --- a/pkg/gate/gate_test.go +++ b/pkg/gate/gate_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/ilist/list.go b/pkg/ilist/list.go index 4ae02eee9..51c9b6df3 100644 --- a/pkg/ilist/list.go +++ b/pkg/ilist/list.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/ilist/list_test.go b/pkg/ilist/list_test.go index 2c56280f6..4bda570b6 100644 --- a/pkg/ilist/list_test.go +++ b/pkg/ilist/list_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/linewriter/linewriter.go b/pkg/linewriter/linewriter.go index 98f974410..5fbd4e779 100644 --- a/pkg/linewriter/linewriter.go +++ b/pkg/linewriter/linewriter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/linewriter/linewriter_test.go b/pkg/linewriter/linewriter_test.go index ce97cca05..9140ee6af 100644 --- a/pkg/linewriter/linewriter_test.go +++ b/pkg/linewriter/linewriter_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/glog.go b/pkg/log/glog.go index 58b4052e6..fbb58501b 100644 --- a/pkg/log/glog.go +++ b/pkg/log/glog.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/glog_unsafe.go b/pkg/log/glog_unsafe.go index c320190b8..bb06aa7d3 100644 --- a/pkg/log/glog_unsafe.go +++ b/pkg/log/glog_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/json.go b/pkg/log/json.go index 3887f1cd5..96bd13d87 100644 --- a/pkg/log/json.go +++ b/pkg/log/json.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/json_test.go b/pkg/log/json_test.go index 3b167dab0..b8c7a795e 100644 --- a/pkg/log/json_test.go +++ b/pkg/log/json_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/log.go b/pkg/log/log.go index c496e86e4..b8d456aae 100644 --- a/pkg/log/log.go +++ b/pkg/log/log.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/log_test.go b/pkg/log/log_test.go index d93e989dc..a59d457dd 100644 --- a/pkg/log/log_test.go +++ b/pkg/log/log_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/metric/metric.go b/pkg/metric/metric.go index 763cd6bc2..02af75974 100644 --- a/pkg/metric/metric.go +++ b/pkg/metric/metric.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/metric/metric.proto b/pkg/metric/metric.proto index 6108cb7c0..917fda1ac 100644 --- a/pkg/metric/metric.proto +++ b/pkg/metric/metric.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/metric/metric_test.go b/pkg/metric/metric_test.go index 7d156e4a5..40034a589 100644 --- a/pkg/metric/metric_test.go +++ b/pkg/metric/metric_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/buffer.go b/pkg/p9/buffer.go index fc65d2c5f..9575ddf12 100644 --- a/pkg/p9/buffer.go +++ b/pkg/p9/buffer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/client.go b/pkg/p9/client.go index 5fa231bc5..3ebfab82a 100644 --- a/pkg/p9/client.go +++ b/pkg/p9/client.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/client_file.go b/pkg/p9/client_file.go index a46efd27f..066639fda 100644 --- a/pkg/p9/client_file.go +++ b/pkg/p9/client_file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/client_test.go b/pkg/p9/client_test.go index 06302a76a..f7145452d 100644 --- a/pkg/p9/client_test.go +++ b/pkg/p9/client_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/file.go b/pkg/p9/file.go index 9723fa24d..d2e89e373 100644 --- a/pkg/p9/file.go +++ b/pkg/p9/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/handlers.go b/pkg/p9/handlers.go index ea41f97c7..959dff31d 100644 --- a/pkg/p9/handlers.go +++ b/pkg/p9/handlers.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/local_server/local_server.go b/pkg/p9/local_server/local_server.go index cef3701a7..1e6aaa762 100644 --- a/pkg/p9/local_server/local_server.go +++ b/pkg/p9/local_server/local_server.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/messages.go b/pkg/p9/messages.go index b3d76801b..972c37344 100644 --- a/pkg/p9/messages.go +++ b/pkg/p9/messages.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/messages_test.go b/pkg/p9/messages_test.go index f353755f1..dfb41bb76 100644 --- a/pkg/p9/messages_test.go +++ b/pkg/p9/messages_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/p9.go b/pkg/p9/p9.go index c6899c3ce..3b0993ecd 100644 --- a/pkg/p9/p9.go +++ b/pkg/p9/p9.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/p9_test.go b/pkg/p9/p9_test.go index a50ac80a4..02498346c 100644 --- a/pkg/p9/p9_test.go +++ b/pkg/p9/p9_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/p9test/client_test.go b/pkg/p9/p9test/client_test.go index 34ddccd8b..db562b9ba 100644 --- a/pkg/p9/p9test/client_test.go +++ b/pkg/p9/p9test/client_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/p9test/mocks.go b/pkg/p9/p9test/mocks.go index 9d039ac63..9a8c14975 100644 --- a/pkg/p9/p9test/mocks.go +++ b/pkg/p9/p9test/mocks.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/pool.go b/pkg/p9/pool.go index 9a508b898..34ed898e8 100644 --- a/pkg/p9/pool.go +++ b/pkg/p9/pool.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/pool_test.go b/pkg/p9/pool_test.go index 96be2c8bd..71052d8c4 100644 --- a/pkg/p9/pool_test.go +++ b/pkg/p9/pool_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/server.go b/pkg/p9/server.go index 28a273ac6..5c7cb18c8 100644 --- a/pkg/p9/server.go +++ b/pkg/p9/server.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/transport.go b/pkg/p9/transport.go index b5df29961..97396806c 100644 --- a/pkg/p9/transport.go +++ b/pkg/p9/transport.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/transport_test.go b/pkg/p9/transport_test.go index d6d4b6365..3352a5205 100644 --- a/pkg/p9/transport_test.go +++ b/pkg/p9/transport_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/version.go b/pkg/p9/version.go index 8783eaa7e..ceb6fabbf 100644 --- a/pkg/p9/version.go +++ b/pkg/p9/version.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/version_test.go b/pkg/p9/version_test.go index 634ac3ca5..c053614c9 100644 --- a/pkg/p9/version_test.go +++ b/pkg/p9/version_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/rand/rand.go b/pkg/rand/rand.go index e81f0f5db..593a14380 100644 --- a/pkg/rand/rand.go +++ b/pkg/rand/rand.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/rand/rand_linux.go b/pkg/rand/rand_linux.go index a2be66b3b..7ebe8f3b0 100644 --- a/pkg/rand/rand_linux.go +++ b/pkg/rand/rand_linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/refs/refcounter.go b/pkg/refs/refcounter.go index 638a93bab..8f08c74c7 100644 --- a/pkg/refs/refcounter.go +++ b/pkg/refs/refcounter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/refs/refcounter_state.go b/pkg/refs/refcounter_state.go index 093eae785..136f06fbf 100644 --- a/pkg/refs/refcounter_state.go +++ b/pkg/refs/refcounter_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/refs/refcounter_test.go b/pkg/refs/refcounter_test.go index cc11bcd71..abaa87453 100644 --- a/pkg/refs/refcounter_test.go +++ b/pkg/refs/refcounter_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/seccomp/seccomp.go b/pkg/seccomp/seccomp.go index a746dc9b3..1dfbf749e 100644 --- a/pkg/seccomp/seccomp.go +++ b/pkg/seccomp/seccomp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/seccomp/seccomp_rules.go b/pkg/seccomp/seccomp_rules.go index 6b707f195..a9278c64b 100644 --- a/pkg/seccomp/seccomp_rules.go +++ b/pkg/seccomp/seccomp_rules.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/seccomp/seccomp_test.go b/pkg/seccomp/seccomp_test.go index 0188ad4f3..226f30b7b 100644 --- a/pkg/seccomp/seccomp_test.go +++ b/pkg/seccomp/seccomp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/seccomp/seccomp_test_victim.go b/pkg/seccomp/seccomp_test_victim.go index 4f2ae4dac..007038273 100644 --- a/pkg/seccomp/seccomp_test_victim.go +++ b/pkg/seccomp/seccomp_test_victim.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/seccomp/seccomp_unsafe.go b/pkg/seccomp/seccomp_unsafe.go index ae18534bf..dd009221a 100644 --- a/pkg/seccomp/seccomp_unsafe.go +++ b/pkg/seccomp/seccomp_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/secio/full_reader.go b/pkg/secio/full_reader.go index b2dbb8615..90b1772a7 100644 --- a/pkg/secio/full_reader.go +++ b/pkg/secio/full_reader.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/secio/secio.go b/pkg/secio/secio.go index fc625efb8..e5f74a497 100644 --- a/pkg/secio/secio.go +++ b/pkg/secio/secio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/secio/secio_test.go b/pkg/secio/secio_test.go index 64b4cc17d..8304c4f74 100644 --- a/pkg/secio/secio_test.go +++ b/pkg/secio/secio_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/segment/range.go b/pkg/segment/range.go index 34c067265..057bcd7ff 100644 --- a/pkg/segment/range.go +++ b/pkg/segment/range.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/segment/set.go b/pkg/segment/set.go index cffec2a2c..a9a3b8875 100644 --- a/pkg/segment/set.go +++ b/pkg/segment/set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/segment/set_state.go b/pkg/segment/set_state.go index a763d1915..b86e1b75f 100644 --- a/pkg/segment/set_state.go +++ b/pkg/segment/set_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/segment/test/segment_test.go b/pkg/segment/test/segment_test.go index 7ea24b177..0825105db 100644 --- a/pkg/segment/test/segment_test.go +++ b/pkg/segment/test/segment_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/segment/test/set_functions.go b/pkg/segment/test/set_functions.go index 37c196ea1..05ba5fbb9 100644 --- a/pkg/segment/test/set_functions.go +++ b/pkg/segment/test/set_functions.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/aligned.go b/pkg/sentry/arch/aligned.go index 193232e27..c88c034f6 100644 --- a/pkg/sentry/arch/aligned.go +++ b/pkg/sentry/arch/aligned.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/arch.go b/pkg/sentry/arch/arch.go index 21cb84502..575b7ba66 100644 --- a/pkg/sentry/arch/arch.go +++ b/pkg/sentry/arch/arch.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/arch_amd64.go b/pkg/sentry/arch/arch_amd64.go index 5ba6c19ea..bb80a7bed 100644 --- a/pkg/sentry/arch/arch_amd64.go +++ b/pkg/sentry/arch/arch_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/arch_amd64.s b/pkg/sentry/arch/arch_amd64.s index 10d621b6d..fa9857df7 100644 --- a/pkg/sentry/arch/arch_amd64.s +++ b/pkg/sentry/arch/arch_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/arch_state_x86.go b/pkg/sentry/arch/arch_state_x86.go index e9c23a06b..604bd08a6 100644 --- a/pkg/sentry/arch/arch_state_x86.go +++ b/pkg/sentry/arch/arch_state_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/arch_x86.go b/pkg/sentry/arch/arch_x86.go index b35eec53c..59bf89d99 100644 --- a/pkg/sentry/arch/arch_x86.go +++ b/pkg/sentry/arch/arch_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/auxv.go b/pkg/sentry/arch/auxv.go index 81cfb4a01..5df65a691 100644 --- a/pkg/sentry/arch/auxv.go +++ b/pkg/sentry/arch/auxv.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/registers.proto b/pkg/sentry/arch/registers.proto index 437ff44ca..f4c2f7043 100644 --- a/pkg/sentry/arch/registers.proto +++ b/pkg/sentry/arch/registers.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/signal_act.go b/pkg/sentry/arch/signal_act.go index 36437b965..ad098c746 100644 --- a/pkg/sentry/arch/signal_act.go +++ b/pkg/sentry/arch/signal_act.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/signal_amd64.go b/pkg/sentry/arch/signal_amd64.go index 9ca4c8ed1..f7f054b0b 100644 --- a/pkg/sentry/arch/signal_amd64.go +++ b/pkg/sentry/arch/signal_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/signal_info.go b/pkg/sentry/arch/signal_info.go index ec004ae75..fa0ecbec5 100644 --- a/pkg/sentry/arch/signal_info.go +++ b/pkg/sentry/arch/signal_info.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/signal_stack.go b/pkg/sentry/arch/signal_stack.go index ba43dd1d4..c02ae3b7c 100644 --- a/pkg/sentry/arch/signal_stack.go +++ b/pkg/sentry/arch/signal_stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/stack.go b/pkg/sentry/arch/stack.go index 6c1b9be82..716a3574d 100644 --- a/pkg/sentry/arch/stack.go +++ b/pkg/sentry/arch/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/syscalls_amd64.go b/pkg/sentry/arch/syscalls_amd64.go index 41d8ba0d1..47c31d4b9 100644 --- a/pkg/sentry/arch/syscalls_amd64.go +++ b/pkg/sentry/arch/syscalls_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/context/context.go b/pkg/sentry/context/context.go index 598c5b4ff..12bdcef85 100644 --- a/pkg/sentry/context/context.go +++ b/pkg/sentry/context/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/context/contexttest/contexttest.go b/pkg/sentry/context/contexttest/contexttest.go index b3c6a566b..d2f084ed7 100644 --- a/pkg/sentry/context/contexttest/contexttest.go +++ b/pkg/sentry/context/contexttest/contexttest.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/control/control.go b/pkg/sentry/control/control.go index a6ee6e649..32d30b6ea 100644 --- a/pkg/sentry/control/control.go +++ b/pkg/sentry/control/control.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/control/proc.go b/pkg/sentry/control/proc.go index 0ba730c1e..b6ac2f312 100644 --- a/pkg/sentry/control/proc.go +++ b/pkg/sentry/control/proc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/control/proc_test.go b/pkg/sentry/control/proc_test.go index 22c826236..5d52cd829 100644 --- a/pkg/sentry/control/proc_test.go +++ b/pkg/sentry/control/proc_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/control/state.go b/pkg/sentry/control/state.go index cee4db636..0a480c84a 100644 --- a/pkg/sentry/control/state.go +++ b/pkg/sentry/control/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/device/device.go b/pkg/sentry/device/device.go index 21fee8f8a..27e4eb258 100644 --- a/pkg/sentry/device/device.go +++ b/pkg/sentry/device/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/device/device_test.go b/pkg/sentry/device/device_test.go index dfec45046..5d8805c2f 100644 --- a/pkg/sentry/device/device_test.go +++ b/pkg/sentry/device/device_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/anon/anon.go b/pkg/sentry/fs/anon/anon.go index ddc2c0985..743cf511f 100644 --- a/pkg/sentry/fs/anon/anon.go +++ b/pkg/sentry/fs/anon/anon.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/anon/device.go b/pkg/sentry/fs/anon/device.go index 1c666729c..2d1249299 100644 --- a/pkg/sentry/fs/anon/device.go +++ b/pkg/sentry/fs/anon/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ashmem/area.go b/pkg/sentry/fs/ashmem/area.go index bfd7f2762..5372875ac 100644 --- a/pkg/sentry/fs/ashmem/area.go +++ b/pkg/sentry/fs/ashmem/area.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ashmem/device.go b/pkg/sentry/fs/ashmem/device.go index d0986fa11..962da141b 100644 --- a/pkg/sentry/fs/ashmem/device.go +++ b/pkg/sentry/fs/ashmem/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ashmem/pin_board.go b/pkg/sentry/fs/ashmem/pin_board.go index ecba395a0..7c997f533 100644 --- a/pkg/sentry/fs/ashmem/pin_board.go +++ b/pkg/sentry/fs/ashmem/pin_board.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ashmem/pin_board_test.go b/pkg/sentry/fs/ashmem/pin_board_test.go index f4ea5de6d..736e628dc 100644 --- a/pkg/sentry/fs/ashmem/pin_board_test.go +++ b/pkg/sentry/fs/ashmem/pin_board_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/attr.go b/pkg/sentry/fs/attr.go index 091f4ac63..59e060e3c 100644 --- a/pkg/sentry/fs/attr.go +++ b/pkg/sentry/fs/attr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/binder/binder.go b/pkg/sentry/fs/binder/binder.go index 502a262dd..42b9e8b26 100644 --- a/pkg/sentry/fs/binder/binder.go +++ b/pkg/sentry/fs/binder/binder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/context.go b/pkg/sentry/fs/context.go index da46ad77f..1775d3486 100644 --- a/pkg/sentry/fs/context.go +++ b/pkg/sentry/fs/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/copy_up.go b/pkg/sentry/fs/copy_up.go index 8c949b176..d65dc74bf 100644 --- a/pkg/sentry/fs/copy_up.go +++ b/pkg/sentry/fs/copy_up.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/copy_up_test.go b/pkg/sentry/fs/copy_up_test.go index c3c9d963d..64f030f72 100644 --- a/pkg/sentry/fs/copy_up_test.go +++ b/pkg/sentry/fs/copy_up_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dentry.go b/pkg/sentry/fs/dentry.go index b347468ff..ef6d1a870 100644 --- a/pkg/sentry/fs/dentry.go +++ b/pkg/sentry/fs/dentry.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/dev.go b/pkg/sentry/fs/dev/dev.go index 3f4f2a40a..05a5005ad 100644 --- a/pkg/sentry/fs/dev/dev.go +++ b/pkg/sentry/fs/dev/dev.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/device.go b/pkg/sentry/fs/dev/device.go index 9d935e008..3cecdf6e2 100644 --- a/pkg/sentry/fs/dev/device.go +++ b/pkg/sentry/fs/dev/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/fs.go b/pkg/sentry/fs/dev/fs.go index 2ae49be4e..d96f4f423 100644 --- a/pkg/sentry/fs/dev/fs.go +++ b/pkg/sentry/fs/dev/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/full.go b/pkg/sentry/fs/dev/full.go index 492b8eb3a..eeda646ab 100644 --- a/pkg/sentry/fs/dev/full.go +++ b/pkg/sentry/fs/dev/full.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/null.go b/pkg/sentry/fs/dev/null.go index 2977c8670..68090f353 100644 --- a/pkg/sentry/fs/dev/null.go +++ b/pkg/sentry/fs/dev/null.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/random.go b/pkg/sentry/fs/dev/random.go index 47b76218f..33e4913e4 100644 --- a/pkg/sentry/fs/dev/random.go +++ b/pkg/sentry/fs/dev/random.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dirent.go b/pkg/sentry/fs/dirent.go index 27fea0019..2c01485a8 100644 --- a/pkg/sentry/fs/dirent.go +++ b/pkg/sentry/fs/dirent.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dirent_cache.go b/pkg/sentry/fs/dirent_cache.go index c680e4828..502b0a09b 100644 --- a/pkg/sentry/fs/dirent_cache.go +++ b/pkg/sentry/fs/dirent_cache.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dirent_cache_test.go b/pkg/sentry/fs/dirent_cache_test.go index 82b7f6bd5..5d0e9d91c 100644 --- a/pkg/sentry/fs/dirent_cache_test.go +++ b/pkg/sentry/fs/dirent_cache_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dirent_refs_test.go b/pkg/sentry/fs/dirent_refs_test.go index f9dcba316..325404e27 100644 --- a/pkg/sentry/fs/dirent_refs_test.go +++ b/pkg/sentry/fs/dirent_refs_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dirent_state.go b/pkg/sentry/fs/dirent_state.go index 04ab197b9..5cf151dab 100644 --- a/pkg/sentry/fs/dirent_state.go +++ b/pkg/sentry/fs/dirent_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fdpipe/pipe.go b/pkg/sentry/fs/fdpipe/pipe.go index 2e34604e6..bfafff5ec 100644 --- a/pkg/sentry/fs/fdpipe/pipe.go +++ b/pkg/sentry/fs/fdpipe/pipe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fdpipe/pipe_opener.go b/pkg/sentry/fs/fdpipe/pipe_opener.go index 945cfaf08..92ab6ff0e 100644 --- a/pkg/sentry/fs/fdpipe/pipe_opener.go +++ b/pkg/sentry/fs/fdpipe/pipe_opener.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fdpipe/pipe_opener_test.go b/pkg/sentry/fs/fdpipe/pipe_opener_test.go index 83f6c1986..69516e048 100644 --- a/pkg/sentry/fs/fdpipe/pipe_opener_test.go +++ b/pkg/sentry/fs/fdpipe/pipe_opener_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fdpipe/pipe_state.go b/pkg/sentry/fs/fdpipe/pipe_state.go index 99c40d8ed..4395666ad 100644 --- a/pkg/sentry/fs/fdpipe/pipe_state.go +++ b/pkg/sentry/fs/fdpipe/pipe_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fdpipe/pipe_test.go b/pkg/sentry/fs/fdpipe/pipe_test.go index 6cd314f5b..d3f15be6b 100644 --- a/pkg/sentry/fs/fdpipe/pipe_test.go +++ b/pkg/sentry/fs/fdpipe/pipe_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file.go b/pkg/sentry/fs/file.go index 36794d378..d6752ed1b 100644 --- a/pkg/sentry/fs/file.go +++ b/pkg/sentry/fs/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file_operations.go b/pkg/sentry/fs/file_operations.go index d223bb5c7..28e8e233d 100644 --- a/pkg/sentry/fs/file_operations.go +++ b/pkg/sentry/fs/file_operations.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file_overlay.go b/pkg/sentry/fs/file_overlay.go index 41e646ee8..9b958b64b 100644 --- a/pkg/sentry/fs/file_overlay.go +++ b/pkg/sentry/fs/file_overlay.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file_overlay_test.go b/pkg/sentry/fs/file_overlay_test.go index 830458ff9..11e4f7203 100644 --- a/pkg/sentry/fs/file_overlay_test.go +++ b/pkg/sentry/fs/file_overlay_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file_state.go b/pkg/sentry/fs/file_state.go index f848d1b79..1c3bae3e8 100644 --- a/pkg/sentry/fs/file_state.go +++ b/pkg/sentry/fs/file_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file_test.go b/pkg/sentry/fs/file_test.go index 18aee7101..f3ed9a70b 100644 --- a/pkg/sentry/fs/file_test.go +++ b/pkg/sentry/fs/file_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/filesystems.go b/pkg/sentry/fs/filesystems.go index 5a1e7a270..ba8be85e4 100644 --- a/pkg/sentry/fs/filesystems.go +++ b/pkg/sentry/fs/filesystems.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/filetest/filetest.go b/pkg/sentry/fs/filetest/filetest.go index 1831aa82f..65ca196d9 100644 --- a/pkg/sentry/fs/filetest/filetest.go +++ b/pkg/sentry/fs/filetest/filetest.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/flags.go b/pkg/sentry/fs/flags.go index 1aa271560..bf2a20b33 100644 --- a/pkg/sentry/fs/flags.go +++ b/pkg/sentry/fs/flags.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fs.go b/pkg/sentry/fs/fs.go index 6ec9ff446..b5c72990e 100644 --- a/pkg/sentry/fs/fs.go +++ b/pkg/sentry/fs/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/dirty_set.go b/pkg/sentry/fs/fsutil/dirty_set.go index 8e31e48fd..5add16ac4 100644 --- a/pkg/sentry/fs/fsutil/dirty_set.go +++ b/pkg/sentry/fs/fsutil/dirty_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/dirty_set_test.go b/pkg/sentry/fs/fsutil/dirty_set_test.go index f7693cb19..f5c9d9215 100644 --- a/pkg/sentry/fs/fsutil/dirty_set_test.go +++ b/pkg/sentry/fs/fsutil/dirty_set_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/file.go b/pkg/sentry/fs/fsutil/file.go index d5881613b..46db2e51c 100644 --- a/pkg/sentry/fs/fsutil/file.go +++ b/pkg/sentry/fs/fsutil/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/file_range_set.go b/pkg/sentry/fs/fsutil/file_range_set.go index da6949ccb..dd7ab4b4a 100644 --- a/pkg/sentry/fs/fsutil/file_range_set.go +++ b/pkg/sentry/fs/fsutil/file_range_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/frame_ref_set.go b/pkg/sentry/fs/fsutil/frame_ref_set.go index 14dece315..b6e783614 100644 --- a/pkg/sentry/fs/fsutil/frame_ref_set.go +++ b/pkg/sentry/fs/fsutil/frame_ref_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/fsutil.go b/pkg/sentry/fs/fsutil/fsutil.go index 6fe4ef13d..3d7f3732d 100644 --- a/pkg/sentry/fs/fsutil/fsutil.go +++ b/pkg/sentry/fs/fsutil/fsutil.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/handle.go b/pkg/sentry/fs/fsutil/handle.go index e7efd3c0f..8920b72ee 100644 --- a/pkg/sentry/fs/fsutil/handle.go +++ b/pkg/sentry/fs/fsutil/handle.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/handle_test.go b/pkg/sentry/fs/fsutil/handle_test.go index d94c3eb0d..43e1a3bdf 100644 --- a/pkg/sentry/fs/fsutil/handle_test.go +++ b/pkg/sentry/fs/fsutil/handle_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/host_file_mapper.go b/pkg/sentry/fs/fsutil/host_file_mapper.go index 9c1e2f76f..9599665f0 100644 --- a/pkg/sentry/fs/fsutil/host_file_mapper.go +++ b/pkg/sentry/fs/fsutil/host_file_mapper.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/host_file_mapper_state.go b/pkg/sentry/fs/fsutil/host_file_mapper_state.go index 57705decd..bbd15b30b 100644 --- a/pkg/sentry/fs/fsutil/host_file_mapper_state.go +++ b/pkg/sentry/fs/fsutil/host_file_mapper_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go b/pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go index 790f3a5a6..86df76822 100644 --- a/pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go +++ b/pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/inode.go b/pkg/sentry/fs/fsutil/inode.go index 3acc32752..d4db1c2de 100644 --- a/pkg/sentry/fs/fsutil/inode.go +++ b/pkg/sentry/fs/fsutil/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/inode_cached.go b/pkg/sentry/fs/fsutil/inode_cached.go index 6777c8bf7..b0af44ddd 100644 --- a/pkg/sentry/fs/fsutil/inode_cached.go +++ b/pkg/sentry/fs/fsutil/inode_cached.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/inode_cached_test.go b/pkg/sentry/fs/fsutil/inode_cached_test.go index 996c91849..e388ec3d7 100644 --- a/pkg/sentry/fs/fsutil/inode_cached_test.go +++ b/pkg/sentry/fs/fsutil/inode_cached_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/attr.go b/pkg/sentry/fs/gofer/attr.go index 5e24767f9..98700d014 100644 --- a/pkg/sentry/fs/gofer/attr.go +++ b/pkg/sentry/fs/gofer/attr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/cache_policy.go b/pkg/sentry/fs/gofer/cache_policy.go index 98f43c578..3d380f0e8 100644 --- a/pkg/sentry/fs/gofer/cache_policy.go +++ b/pkg/sentry/fs/gofer/cache_policy.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/context_file.go b/pkg/sentry/fs/gofer/context_file.go index d4b6f6eb7..a0265c2aa 100644 --- a/pkg/sentry/fs/gofer/context_file.go +++ b/pkg/sentry/fs/gofer/context_file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/device.go b/pkg/sentry/fs/gofer/device.go index fac7306d4..52c5acf48 100644 --- a/pkg/sentry/fs/gofer/device.go +++ b/pkg/sentry/fs/gofer/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/file.go b/pkg/sentry/fs/gofer/file.go index c4a210656..6d961813d 100644 --- a/pkg/sentry/fs/gofer/file.go +++ b/pkg/sentry/fs/gofer/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/file_state.go b/pkg/sentry/fs/gofer/file_state.go index 715af8f16..dd4f817bf 100644 --- a/pkg/sentry/fs/gofer/file_state.go +++ b/pkg/sentry/fs/gofer/file_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/fs.go b/pkg/sentry/fs/gofer/fs.go index 3ae93f059..ed30cb1f1 100644 --- a/pkg/sentry/fs/gofer/fs.go +++ b/pkg/sentry/fs/gofer/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/gofer_test.go b/pkg/sentry/fs/gofer/gofer_test.go index c8d7bd773..3190d1e18 100644 --- a/pkg/sentry/fs/gofer/gofer_test.go +++ b/pkg/sentry/fs/gofer/gofer_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/handles.go b/pkg/sentry/fs/gofer/handles.go index a3e52aad6..f32e99ce0 100644 --- a/pkg/sentry/fs/gofer/handles.go +++ b/pkg/sentry/fs/gofer/handles.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/inode.go b/pkg/sentry/fs/gofer/inode.go index 7fc8f77b0..5811b8b12 100644 --- a/pkg/sentry/fs/gofer/inode.go +++ b/pkg/sentry/fs/gofer/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/inode_state.go b/pkg/sentry/fs/gofer/inode_state.go index ad11034f9..ad4d3df58 100644 --- a/pkg/sentry/fs/gofer/inode_state.go +++ b/pkg/sentry/fs/gofer/inode_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/path.go b/pkg/sentry/fs/gofer/path.go index 0bf7881da..a324dc990 100644 --- a/pkg/sentry/fs/gofer/path.go +++ b/pkg/sentry/fs/gofer/path.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/session.go b/pkg/sentry/fs/gofer/session.go index 4e2293398..7552216f3 100644 --- a/pkg/sentry/fs/gofer/session.go +++ b/pkg/sentry/fs/gofer/session.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/session_state.go b/pkg/sentry/fs/gofer/session_state.go index 8e6424492..f657135fc 100644 --- a/pkg/sentry/fs/gofer/session_state.go +++ b/pkg/sentry/fs/gofer/session_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/socket.go b/pkg/sentry/fs/gofer/socket.go index d072da624..76ce58810 100644 --- a/pkg/sentry/fs/gofer/socket.go +++ b/pkg/sentry/fs/gofer/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/util.go b/pkg/sentry/fs/gofer/util.go index d9ed8c81e..1a759370d 100644 --- a/pkg/sentry/fs/gofer/util.go +++ b/pkg/sentry/fs/gofer/util.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/control.go b/pkg/sentry/fs/host/control.go index d2e34a69d..0753640a2 100644 --- a/pkg/sentry/fs/host/control.go +++ b/pkg/sentry/fs/host/control.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/descriptor.go b/pkg/sentry/fs/host/descriptor.go index 148291ba6..7c9d2b299 100644 --- a/pkg/sentry/fs/host/descriptor.go +++ b/pkg/sentry/fs/host/descriptor.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/descriptor_state.go b/pkg/sentry/fs/host/descriptor_state.go index 7fb274451..530c0109f 100644 --- a/pkg/sentry/fs/host/descriptor_state.go +++ b/pkg/sentry/fs/host/descriptor_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/descriptor_test.go b/pkg/sentry/fs/host/descriptor_test.go index f393a8b54..6bc1bd2ae 100644 --- a/pkg/sentry/fs/host/descriptor_test.go +++ b/pkg/sentry/fs/host/descriptor_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/device.go b/pkg/sentry/fs/host/device.go index f2a0b6b15..b5adedf44 100644 --- a/pkg/sentry/fs/host/device.go +++ b/pkg/sentry/fs/host/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/file.go b/pkg/sentry/fs/host/file.go index 22a5d9f12..975084c86 100644 --- a/pkg/sentry/fs/host/file.go +++ b/pkg/sentry/fs/host/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/fs.go b/pkg/sentry/fs/host/fs.go index e46ae433c..fec890964 100644 --- a/pkg/sentry/fs/host/fs.go +++ b/pkg/sentry/fs/host/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/fs_test.go b/pkg/sentry/fs/host/fs_test.go index b08125ca8..e69559aac 100644 --- a/pkg/sentry/fs/host/fs_test.go +++ b/pkg/sentry/fs/host/fs_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/inode.go b/pkg/sentry/fs/host/inode.go index e32497203..08754bd6b 100644 --- a/pkg/sentry/fs/host/inode.go +++ b/pkg/sentry/fs/host/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/inode_state.go b/pkg/sentry/fs/host/inode_state.go index 8bc99d94b..b7c1a9581 100644 --- a/pkg/sentry/fs/host/inode_state.go +++ b/pkg/sentry/fs/host/inode_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/inode_test.go b/pkg/sentry/fs/host/inode_test.go index 0ff87c418..9f1561bd5 100644 --- a/pkg/sentry/fs/host/inode_test.go +++ b/pkg/sentry/fs/host/inode_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/ioctl_unsafe.go b/pkg/sentry/fs/host/ioctl_unsafe.go index bc965a1c2..175dca613 100644 --- a/pkg/sentry/fs/host/ioctl_unsafe.go +++ b/pkg/sentry/fs/host/ioctl_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/socket.go b/pkg/sentry/fs/host/socket.go index 0eb267c00..af53bf533 100644 --- a/pkg/sentry/fs/host/socket.go +++ b/pkg/sentry/fs/host/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/socket_iovec.go b/pkg/sentry/fs/host/socket_iovec.go index 1a9587b90..d4ce4a8c1 100644 --- a/pkg/sentry/fs/host/socket_iovec.go +++ b/pkg/sentry/fs/host/socket_iovec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/socket_state.go b/pkg/sentry/fs/host/socket_state.go index 7fa500bfb..2932c1f16 100644 --- a/pkg/sentry/fs/host/socket_state.go +++ b/pkg/sentry/fs/host/socket_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/socket_test.go b/pkg/sentry/fs/host/socket_test.go index 483e99dd6..e9a88b124 100644 --- a/pkg/sentry/fs/host/socket_test.go +++ b/pkg/sentry/fs/host/socket_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/socket_unsafe.go b/pkg/sentry/fs/host/socket_unsafe.go index 5e4c5feed..f35e2492d 100644 --- a/pkg/sentry/fs/host/socket_unsafe.go +++ b/pkg/sentry/fs/host/socket_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/tty.go b/pkg/sentry/fs/host/tty.go index ad1323610..cf3639c46 100644 --- a/pkg/sentry/fs/host/tty.go +++ b/pkg/sentry/fs/host/tty.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/util.go b/pkg/sentry/fs/host/util.go index 74c703eb7..40c450660 100644 --- a/pkg/sentry/fs/host/util.go +++ b/pkg/sentry/fs/host/util.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/util_unsafe.go b/pkg/sentry/fs/host/util_unsafe.go index 2ecb54319..d00da89d6 100644 --- a/pkg/sentry/fs/host/util_unsafe.go +++ b/pkg/sentry/fs/host/util_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/wait_test.go b/pkg/sentry/fs/host/wait_test.go index c5f5c9c0d..9ca8c399f 100644 --- a/pkg/sentry/fs/host/wait_test.go +++ b/pkg/sentry/fs/host/wait_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inode.go b/pkg/sentry/fs/inode.go index 409c81a97..95769ccf8 100644 --- a/pkg/sentry/fs/inode.go +++ b/pkg/sentry/fs/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inode_inotify.go b/pkg/sentry/fs/inode_inotify.go index 683140afe..e213df924 100644 --- a/pkg/sentry/fs/inode_inotify.go +++ b/pkg/sentry/fs/inode_inotify.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inode_operations.go b/pkg/sentry/fs/inode_operations.go index 3ee3de10e..77973ce79 100644 --- a/pkg/sentry/fs/inode_operations.go +++ b/pkg/sentry/fs/inode_operations.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inode_overlay.go b/pkg/sentry/fs/inode_overlay.go index cf698a4da..78923fb5b 100644 --- a/pkg/sentry/fs/inode_overlay.go +++ b/pkg/sentry/fs/inode_overlay.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inode_overlay_test.go b/pkg/sentry/fs/inode_overlay_test.go index 23e5635a4..bba20da14 100644 --- a/pkg/sentry/fs/inode_overlay_test.go +++ b/pkg/sentry/fs/inode_overlay_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inotify.go b/pkg/sentry/fs/inotify.go index 2aabdded8..f251df0d1 100644 --- a/pkg/sentry/fs/inotify.go +++ b/pkg/sentry/fs/inotify.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inotify_event.go b/pkg/sentry/fs/inotify_event.go index e9b5e0f56..9e3e9d816 100644 --- a/pkg/sentry/fs/inotify_event.go +++ b/pkg/sentry/fs/inotify_event.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inotify_watch.go b/pkg/sentry/fs/inotify_watch.go index 3e1959e83..b83544c9f 100644 --- a/pkg/sentry/fs/inotify_watch.go +++ b/pkg/sentry/fs/inotify_watch.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/lock/lock.go b/pkg/sentry/fs/lock/lock.go index 439e645db..5ff800d2d 100644 --- a/pkg/sentry/fs/lock/lock.go +++ b/pkg/sentry/fs/lock/lock.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/lock/lock_range_test.go b/pkg/sentry/fs/lock/lock_range_test.go index 06a37c701..b0ab882b9 100644 --- a/pkg/sentry/fs/lock/lock_range_test.go +++ b/pkg/sentry/fs/lock/lock_range_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/lock/lock_set_functions.go b/pkg/sentry/fs/lock/lock_set_functions.go index e16f485be..395592a4b 100644 --- a/pkg/sentry/fs/lock/lock_set_functions.go +++ b/pkg/sentry/fs/lock/lock_set_functions.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/lock/lock_test.go b/pkg/sentry/fs/lock/lock_test.go index c60f5f7a2..67fa4b1dd 100644 --- a/pkg/sentry/fs/lock/lock_test.go +++ b/pkg/sentry/fs/lock/lock_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mock.go b/pkg/sentry/fs/mock.go index 846b6e8bb..6bfcda6bb 100644 --- a/pkg/sentry/fs/mock.go +++ b/pkg/sentry/fs/mock.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mount.go b/pkg/sentry/fs/mount.go index 8345876fc..24e28ddb2 100644 --- a/pkg/sentry/fs/mount.go +++ b/pkg/sentry/fs/mount.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mount_overlay.go b/pkg/sentry/fs/mount_overlay.go index dbc608c7e..fb91635bc 100644 --- a/pkg/sentry/fs/mount_overlay.go +++ b/pkg/sentry/fs/mount_overlay.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mount_state.go b/pkg/sentry/fs/mount_state.go index f5ed1dd8d..6344d5160 100644 --- a/pkg/sentry/fs/mount_state.go +++ b/pkg/sentry/fs/mount_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mount_test.go b/pkg/sentry/fs/mount_test.go index 968b435ab..a1c9f4f79 100644 --- a/pkg/sentry/fs/mount_test.go +++ b/pkg/sentry/fs/mount_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mounts.go b/pkg/sentry/fs/mounts.go index c0a803b2d..7c5348cce 100644 --- a/pkg/sentry/fs/mounts.go +++ b/pkg/sentry/fs/mounts.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mounts_test.go b/pkg/sentry/fs/mounts_test.go index 8669f3a38..cc7c32c9b 100644 --- a/pkg/sentry/fs/mounts_test.go +++ b/pkg/sentry/fs/mounts_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/offset.go b/pkg/sentry/fs/offset.go index 7cc8398e6..38aee765a 100644 --- a/pkg/sentry/fs/offset.go +++ b/pkg/sentry/fs/offset.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/overlay.go b/pkg/sentry/fs/overlay.go index 5a30af419..036c0f733 100644 --- a/pkg/sentry/fs/overlay.go +++ b/pkg/sentry/fs/overlay.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/path.go b/pkg/sentry/fs/path.go index b74f6ed8c..91a9a8ffd 100644 --- a/pkg/sentry/fs/path.go +++ b/pkg/sentry/fs/path.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/path_test.go b/pkg/sentry/fs/path_test.go index 7ab070855..391b010a7 100644 --- a/pkg/sentry/fs/path_test.go +++ b/pkg/sentry/fs/path_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/cpuinfo.go b/pkg/sentry/fs/proc/cpuinfo.go index 4dfec03a4..f8be06dc3 100644 --- a/pkg/sentry/fs/proc/cpuinfo.go +++ b/pkg/sentry/fs/proc/cpuinfo.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/device/device.go b/pkg/sentry/fs/proc/device/device.go index 6194afe88..04b687bcf 100644 --- a/pkg/sentry/fs/proc/device/device.go +++ b/pkg/sentry/fs/proc/device/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/exec_args.go b/pkg/sentry/fs/proc/exec_args.go index a69cbaa0e..b4896053f 100644 --- a/pkg/sentry/fs/proc/exec_args.go +++ b/pkg/sentry/fs/proc/exec_args.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/fds.go b/pkg/sentry/fs/proc/fds.go index dada8f982..5ebb33703 100644 --- a/pkg/sentry/fs/proc/fds.go +++ b/pkg/sentry/fs/proc/fds.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/file.go b/pkg/sentry/fs/proc/file.go index 4b3448245..f659e590a 100644 --- a/pkg/sentry/fs/proc/file.go +++ b/pkg/sentry/fs/proc/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/filesystems.go b/pkg/sentry/fs/proc/filesystems.go index 49b92fd8a..c050a00be 100644 --- a/pkg/sentry/fs/proc/filesystems.go +++ b/pkg/sentry/fs/proc/filesystems.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/fs.go b/pkg/sentry/fs/proc/fs.go index 061824b8c..63f737ff4 100644 --- a/pkg/sentry/fs/proc/fs.go +++ b/pkg/sentry/fs/proc/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/loadavg.go b/pkg/sentry/fs/proc/loadavg.go index 6fac251d2..78f3a1dc0 100644 --- a/pkg/sentry/fs/proc/loadavg.go +++ b/pkg/sentry/fs/proc/loadavg.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/meminfo.go b/pkg/sentry/fs/proc/meminfo.go index 53dfd59ef..b31258eed 100644 --- a/pkg/sentry/fs/proc/meminfo.go +++ b/pkg/sentry/fs/proc/meminfo.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/mounts.go b/pkg/sentry/fs/proc/mounts.go index 81dcc153a..0b0e87528 100644 --- a/pkg/sentry/fs/proc/mounts.go +++ b/pkg/sentry/fs/proc/mounts.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/net.go b/pkg/sentry/fs/proc/net.go index 8cd6fe9d3..45f2a1211 100644 --- a/pkg/sentry/fs/proc/net.go +++ b/pkg/sentry/fs/proc/net.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/net_test.go b/pkg/sentry/fs/proc/net_test.go index a31a20494..94677cc1d 100644 --- a/pkg/sentry/fs/proc/net_test.go +++ b/pkg/sentry/fs/proc/net_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/proc.go b/pkg/sentry/fs/proc/proc.go index 07029a7bb..33030bebf 100644 --- a/pkg/sentry/fs/proc/proc.go +++ b/pkg/sentry/fs/proc/proc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/rpcinet_proc.go b/pkg/sentry/fs/proc/rpcinet_proc.go index 50d0271f9..d025069df 100644 --- a/pkg/sentry/fs/proc/rpcinet_proc.go +++ b/pkg/sentry/fs/proc/rpcinet_proc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/seqfile/seqfile.go b/pkg/sentry/fs/proc/seqfile/seqfile.go index 51cae5e37..0499ba65b 100644 --- a/pkg/sentry/fs/proc/seqfile/seqfile.go +++ b/pkg/sentry/fs/proc/seqfile/seqfile.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/seqfile/seqfile_test.go b/pkg/sentry/fs/proc/seqfile/seqfile_test.go index d90e3e736..f9a2ca38e 100644 --- a/pkg/sentry/fs/proc/seqfile/seqfile_test.go +++ b/pkg/sentry/fs/proc/seqfile/seqfile_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/stat.go b/pkg/sentry/fs/proc/stat.go index bf7650211..f2bbef375 100644 --- a/pkg/sentry/fs/proc/stat.go +++ b/pkg/sentry/fs/proc/stat.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/sys.go b/pkg/sentry/fs/proc/sys.go index 384b4ffe1..54562508d 100644 --- a/pkg/sentry/fs/proc/sys.go +++ b/pkg/sentry/fs/proc/sys.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/sys_net.go b/pkg/sentry/fs/proc/sys_net.go index beb25be20..801eb6a1e 100644 --- a/pkg/sentry/fs/proc/sys_net.go +++ b/pkg/sentry/fs/proc/sys_net.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/sys_net_test.go b/pkg/sentry/fs/proc/sys_net_test.go index 7ba392346..0ce9d30f1 100644 --- a/pkg/sentry/fs/proc/sys_net_test.go +++ b/pkg/sentry/fs/proc/sys_net_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/task.go b/pkg/sentry/fs/proc/task.go index 748ca4320..404faea0a 100644 --- a/pkg/sentry/fs/proc/task.go +++ b/pkg/sentry/fs/proc/task.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/uid_gid_map.go b/pkg/sentry/fs/proc/uid_gid_map.go index a7e4cf0a6..f70399686 100644 --- a/pkg/sentry/fs/proc/uid_gid_map.go +++ b/pkg/sentry/fs/proc/uid_gid_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/uptime.go b/pkg/sentry/fs/proc/uptime.go index f3a9b81df..80c7ce0b4 100644 --- a/pkg/sentry/fs/proc/uptime.go +++ b/pkg/sentry/fs/proc/uptime.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/version.go b/pkg/sentry/fs/proc/version.go index 00f6a2afd..b6d49d5e9 100644 --- a/pkg/sentry/fs/proc/version.go +++ b/pkg/sentry/fs/proc/version.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/dir.go b/pkg/sentry/fs/ramfs/dir.go index 075e13b01..0a911b155 100644 --- a/pkg/sentry/fs/ramfs/dir.go +++ b/pkg/sentry/fs/ramfs/dir.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/file.go b/pkg/sentry/fs/ramfs/file.go index 0b94d92a1..b7fc98ffc 100644 --- a/pkg/sentry/fs/ramfs/file.go +++ b/pkg/sentry/fs/ramfs/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/ramfs.go b/pkg/sentry/fs/ramfs/ramfs.go index 83cbcab23..d77688a34 100644 --- a/pkg/sentry/fs/ramfs/ramfs.go +++ b/pkg/sentry/fs/ramfs/ramfs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/socket.go b/pkg/sentry/fs/ramfs/socket.go index 9ac00eb18..8c81478c8 100644 --- a/pkg/sentry/fs/ramfs/socket.go +++ b/pkg/sentry/fs/ramfs/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/symlink.go b/pkg/sentry/fs/ramfs/symlink.go index 1c54d9991..a21fac2c7 100644 --- a/pkg/sentry/fs/ramfs/symlink.go +++ b/pkg/sentry/fs/ramfs/symlink.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/test/test.go b/pkg/sentry/fs/ramfs/test/test.go index fb669558f..11bff7729 100644 --- a/pkg/sentry/fs/ramfs/test/test.go +++ b/pkg/sentry/fs/ramfs/test/test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/tree.go b/pkg/sentry/fs/ramfs/tree.go index 1fb335f74..29a70f698 100644 --- a/pkg/sentry/fs/ramfs/tree.go +++ b/pkg/sentry/fs/ramfs/tree.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/tree_test.go b/pkg/sentry/fs/ramfs/tree_test.go index 68e2929d5..d5567d9e1 100644 --- a/pkg/sentry/fs/ramfs/tree_test.go +++ b/pkg/sentry/fs/ramfs/tree_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/restore.go b/pkg/sentry/fs/restore.go index b4ac85a27..da2df7e1d 100644 --- a/pkg/sentry/fs/restore.go +++ b/pkg/sentry/fs/restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/save.go b/pkg/sentry/fs/save.go index bf2a85143..90988d385 100644 --- a/pkg/sentry/fs/save.go +++ b/pkg/sentry/fs/save.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/seek.go b/pkg/sentry/fs/seek.go index 1268726c2..72f3fb632 100644 --- a/pkg/sentry/fs/seek.go +++ b/pkg/sentry/fs/seek.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/sync.go b/pkg/sentry/fs/sync.go index 9738a8f22..6dcc2fe8d 100644 --- a/pkg/sentry/fs/sync.go +++ b/pkg/sentry/fs/sync.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/sys/device.go b/pkg/sentry/fs/sys/device.go index 54e414d1b..38ecd0c18 100644 --- a/pkg/sentry/fs/sys/device.go +++ b/pkg/sentry/fs/sys/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/sys/devices.go b/pkg/sentry/fs/sys/devices.go index 2cf3a6f98..e64aa0edc 100644 --- a/pkg/sentry/fs/sys/devices.go +++ b/pkg/sentry/fs/sys/devices.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/sys/fs.go b/pkg/sentry/fs/sys/fs.go index 625525540..5ce33f87f 100644 --- a/pkg/sentry/fs/sys/fs.go +++ b/pkg/sentry/fs/sys/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/sys/sys.go b/pkg/sentry/fs/sys/sys.go index 7b9697668..7cc1942c7 100644 --- a/pkg/sentry/fs/sys/sys.go +++ b/pkg/sentry/fs/sys/sys.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/timerfd/timerfd.go b/pkg/sentry/fs/timerfd/timerfd.go index 767db95a0..7423e816c 100644 --- a/pkg/sentry/fs/timerfd/timerfd.go +++ b/pkg/sentry/fs/timerfd/timerfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/device.go b/pkg/sentry/fs/tmpfs/device.go index e588b3440..aade93c26 100644 --- a/pkg/sentry/fs/tmpfs/device.go +++ b/pkg/sentry/fs/tmpfs/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/file_regular.go b/pkg/sentry/fs/tmpfs/file_regular.go index 342688f81..1f9d69909 100644 --- a/pkg/sentry/fs/tmpfs/file_regular.go +++ b/pkg/sentry/fs/tmpfs/file_regular.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/file_test.go b/pkg/sentry/fs/tmpfs/file_test.go index f064eb1ac..b5830d3df 100644 --- a/pkg/sentry/fs/tmpfs/file_test.go +++ b/pkg/sentry/fs/tmpfs/file_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/fs.go b/pkg/sentry/fs/tmpfs/fs.go index ca620e65e..7c91e248b 100644 --- a/pkg/sentry/fs/tmpfs/fs.go +++ b/pkg/sentry/fs/tmpfs/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/inode_file.go b/pkg/sentry/fs/tmpfs/inode_file.go index 1e4fe47d2..42a7d7b9c 100644 --- a/pkg/sentry/fs/tmpfs/inode_file.go +++ b/pkg/sentry/fs/tmpfs/inode_file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/tmpfs.go b/pkg/sentry/fs/tmpfs/tmpfs.go index 38be6db46..91b782540 100644 --- a/pkg/sentry/fs/tmpfs/tmpfs.go +++ b/pkg/sentry/fs/tmpfs/tmpfs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/dir.go b/pkg/sentry/fs/tty/dir.go index 7c0c0b0c1..e32b05c1d 100644 --- a/pkg/sentry/fs/tty/dir.go +++ b/pkg/sentry/fs/tty/dir.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/fs.go b/pkg/sentry/fs/tty/fs.go index d9f8f02f3..0c412eb21 100644 --- a/pkg/sentry/fs/tty/fs.go +++ b/pkg/sentry/fs/tty/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/inode.go b/pkg/sentry/fs/tty/inode.go index c0fa2b407..d5d1caafc 100644 --- a/pkg/sentry/fs/tty/inode.go +++ b/pkg/sentry/fs/tty/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/line_discipline.go b/pkg/sentry/fs/tty/line_discipline.go index 31804571e..484366f85 100644 --- a/pkg/sentry/fs/tty/line_discipline.go +++ b/pkg/sentry/fs/tty/line_discipline.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/master.go b/pkg/sentry/fs/tty/master.go index ae7540eff..dad0cad79 100644 --- a/pkg/sentry/fs/tty/master.go +++ b/pkg/sentry/fs/tty/master.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/queue.go b/pkg/sentry/fs/tty/queue.go index 01dc8d1ac..a09ca0119 100644 --- a/pkg/sentry/fs/tty/queue.go +++ b/pkg/sentry/fs/tty/queue.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/slave.go b/pkg/sentry/fs/tty/slave.go index 4a0d4fdb9..9de3168bf 100644 --- a/pkg/sentry/fs/tty/slave.go +++ b/pkg/sentry/fs/tty/slave.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/terminal.go b/pkg/sentry/fs/tty/terminal.go index 3cb135124..79f9d76d7 100644 --- a/pkg/sentry/fs/tty/terminal.go +++ b/pkg/sentry/fs/tty/terminal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/tty_test.go b/pkg/sentry/fs/tty/tty_test.go index 32e1b1556..ad535838f 100644 --- a/pkg/sentry/fs/tty/tty_test.go +++ b/pkg/sentry/fs/tty/tty_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/hostcpu/getcpu_amd64.s b/pkg/sentry/hostcpu/getcpu_amd64.s index 7f6247d81..409db1450 100644 --- a/pkg/sentry/hostcpu/getcpu_amd64.s +++ b/pkg/sentry/hostcpu/getcpu_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/hostcpu/hostcpu.go b/pkg/sentry/hostcpu/hostcpu.go index fa46499ad..3adc847bb 100644 --- a/pkg/sentry/hostcpu/hostcpu.go +++ b/pkg/sentry/hostcpu/hostcpu.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/hostcpu/hostcpu_test.go b/pkg/sentry/hostcpu/hostcpu_test.go index a82e1a271..38de0e1f6 100644 --- a/pkg/sentry/hostcpu/hostcpu_test.go +++ b/pkg/sentry/hostcpu/hostcpu_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/inet/context.go b/pkg/sentry/inet/context.go index 370381f41..d05e96f15 100644 --- a/pkg/sentry/inet/context.go +++ b/pkg/sentry/inet/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/inet/inet.go b/pkg/sentry/inet/inet.go index 30ca4e0c0..8206377cc 100644 --- a/pkg/sentry/inet/inet.go +++ b/pkg/sentry/inet/inet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/inet/test_stack.go b/pkg/sentry/inet/test_stack.go index bc10926ee..05c1a1792 100644 --- a/pkg/sentry/inet/test_stack.go +++ b/pkg/sentry/inet/test_stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/abstract_socket_namespace.go b/pkg/sentry/kernel/abstract_socket_namespace.go index 45088c988..1ea2cee36 100644 --- a/pkg/sentry/kernel/abstract_socket_namespace.go +++ b/pkg/sentry/kernel/abstract_socket_namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/auth.go b/pkg/sentry/kernel/auth/auth.go index c49a6b852..19f15fd36 100644 --- a/pkg/sentry/kernel/auth/auth.go +++ b/pkg/sentry/kernel/auth/auth.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/capability_set.go b/pkg/sentry/kernel/auth/capability_set.go index 5b8164c49..88d6243aa 100644 --- a/pkg/sentry/kernel/auth/capability_set.go +++ b/pkg/sentry/kernel/auth/capability_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/context.go b/pkg/sentry/kernel/auth/context.go index 914589b28..f7e945599 100644 --- a/pkg/sentry/kernel/auth/context.go +++ b/pkg/sentry/kernel/auth/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/credentials.go b/pkg/sentry/kernel/auth/credentials.go index f18f7dac9..de33f1953 100644 --- a/pkg/sentry/kernel/auth/credentials.go +++ b/pkg/sentry/kernel/auth/credentials.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/id.go b/pkg/sentry/kernel/auth/id.go index 37522b018..e5bed44d7 100644 --- a/pkg/sentry/kernel/auth/id.go +++ b/pkg/sentry/kernel/auth/id.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/id_map.go b/pkg/sentry/kernel/auth/id_map.go index bd0090e0f..43f439825 100644 --- a/pkg/sentry/kernel/auth/id_map.go +++ b/pkg/sentry/kernel/auth/id_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/id_map_functions.go b/pkg/sentry/kernel/auth/id_map_functions.go index 889291d96..8f1a189ec 100644 --- a/pkg/sentry/kernel/auth/id_map_functions.go +++ b/pkg/sentry/kernel/auth/id_map_functions.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/user_namespace.go b/pkg/sentry/kernel/auth/user_namespace.go index d359f3f31..5bb9c44c0 100644 --- a/pkg/sentry/kernel/auth/user_namespace.go +++ b/pkg/sentry/kernel/auth/user_namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/context.go b/pkg/sentry/kernel/context.go index 261ca6f7a..b629521eb 100644 --- a/pkg/sentry/kernel/context.go +++ b/pkg/sentry/kernel/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/epoll/epoll.go b/pkg/sentry/kernel/epoll/epoll.go index a8eb114c0..9c13ecfcc 100644 --- a/pkg/sentry/kernel/epoll/epoll.go +++ b/pkg/sentry/kernel/epoll/epoll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/epoll/epoll_state.go b/pkg/sentry/kernel/epoll/epoll_state.go index dabb32f49..7f3e2004a 100644 --- a/pkg/sentry/kernel/epoll/epoll_state.go +++ b/pkg/sentry/kernel/epoll/epoll_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/epoll/epoll_test.go b/pkg/sentry/kernel/epoll/epoll_test.go index bc869fc13..d89c1b745 100644 --- a/pkg/sentry/kernel/epoll/epoll_test.go +++ b/pkg/sentry/kernel/epoll/epoll_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/eventfd/eventfd.go b/pkg/sentry/kernel/eventfd/eventfd.go index a4ada0e78..26dc59a85 100644 --- a/pkg/sentry/kernel/eventfd/eventfd.go +++ b/pkg/sentry/kernel/eventfd/eventfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/eventfd/eventfd_test.go b/pkg/sentry/kernel/eventfd/eventfd_test.go index 71326b62f..14e8996d9 100644 --- a/pkg/sentry/kernel/eventfd/eventfd_test.go +++ b/pkg/sentry/kernel/eventfd/eventfd_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/fasync/fasync.go b/pkg/sentry/kernel/fasync/fasync.go index f77339cae..aa4aac109 100644 --- a/pkg/sentry/kernel/fasync/fasync.go +++ b/pkg/sentry/kernel/fasync/fasync.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/fd_map.go b/pkg/sentry/kernel/fd_map.go index cad0b0a20..715f4714d 100644 --- a/pkg/sentry/kernel/fd_map.go +++ b/pkg/sentry/kernel/fd_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/fd_map_test.go b/pkg/sentry/kernel/fd_map_test.go index 95123aef3..b49996137 100644 --- a/pkg/sentry/kernel/fd_map_test.go +++ b/pkg/sentry/kernel/fd_map_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/fs_context.go b/pkg/sentry/kernel/fs_context.go index f3f05e8f5..3cf0db280 100644 --- a/pkg/sentry/kernel/fs_context.go +++ b/pkg/sentry/kernel/fs_context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/futex/futex.go b/pkg/sentry/kernel/futex/futex.go index 54b1982a0..ea69d433b 100644 --- a/pkg/sentry/kernel/futex/futex.go +++ b/pkg/sentry/kernel/futex/futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/futex/futex_test.go b/pkg/sentry/kernel/futex/futex_test.go index 726c26990..ea506a29b 100644 --- a/pkg/sentry/kernel/futex/futex_test.go +++ b/pkg/sentry/kernel/futex/futex_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/ipc_namespace.go b/pkg/sentry/kernel/ipc_namespace.go index 5eef49f59..9ceb9bd92 100644 --- a/pkg/sentry/kernel/ipc_namespace.go +++ b/pkg/sentry/kernel/ipc_namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/kdefs/kdefs.go b/pkg/sentry/kernel/kdefs/kdefs.go index bbb476544..8eafe810b 100644 --- a/pkg/sentry/kernel/kdefs/kdefs.go +++ b/pkg/sentry/kernel/kdefs/kdefs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go index 5d6856f3c..bad558d48 100644 --- a/pkg/sentry/kernel/kernel.go +++ b/pkg/sentry/kernel/kernel.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/kernel_state.go b/pkg/sentry/kernel/kernel_state.go index bb2d5102d..a0a69b498 100644 --- a/pkg/sentry/kernel/kernel_state.go +++ b/pkg/sentry/kernel/kernel_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/memevent/memory_events.go b/pkg/sentry/kernel/memevent/memory_events.go index f7a183a1d..f05ef1b64 100644 --- a/pkg/sentry/kernel/memevent/memory_events.go +++ b/pkg/sentry/kernel/memevent/memory_events.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/memevent/memory_events.proto b/pkg/sentry/kernel/memevent/memory_events.proto index abc565054..43b8deb76 100644 --- a/pkg/sentry/kernel/memevent/memory_events.proto +++ b/pkg/sentry/kernel/memevent/memory_events.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pending_signals.go b/pkg/sentry/kernel/pending_signals.go index bb5db0309..373e11772 100644 --- a/pkg/sentry/kernel/pending_signals.go +++ b/pkg/sentry/kernel/pending_signals.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pending_signals_state.go b/pkg/sentry/kernel/pending_signals_state.go index 6d90ed033..72be6702f 100644 --- a/pkg/sentry/kernel/pending_signals_state.go +++ b/pkg/sentry/kernel/pending_signals_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/buffers.go b/pkg/sentry/kernel/pipe/buffers.go index a82e45c3f..fa8045910 100644 --- a/pkg/sentry/kernel/pipe/buffers.go +++ b/pkg/sentry/kernel/pipe/buffers.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/device.go b/pkg/sentry/kernel/pipe/device.go index 8d383577a..eec5c5de8 100644 --- a/pkg/sentry/kernel/pipe/device.go +++ b/pkg/sentry/kernel/pipe/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/node.go b/pkg/sentry/kernel/pipe/node.go index 23d692da1..4b0e00b85 100644 --- a/pkg/sentry/kernel/pipe/node.go +++ b/pkg/sentry/kernel/pipe/node.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/node_test.go b/pkg/sentry/kernel/pipe/node_test.go index cc1ebf4f6..eda551594 100644 --- a/pkg/sentry/kernel/pipe/node_test.go +++ b/pkg/sentry/kernel/pipe/node_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/pipe.go b/pkg/sentry/kernel/pipe/pipe.go index ced2559a7..126054826 100644 --- a/pkg/sentry/kernel/pipe/pipe.go +++ b/pkg/sentry/kernel/pipe/pipe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/pipe_test.go b/pkg/sentry/kernel/pipe/pipe_test.go index 49ef8c8ac..3b9895927 100644 --- a/pkg/sentry/kernel/pipe/pipe_test.go +++ b/pkg/sentry/kernel/pipe/pipe_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/reader.go b/pkg/sentry/kernel/pipe/reader.go index 1fa5e9a32..f27379969 100644 --- a/pkg/sentry/kernel/pipe/reader.go +++ b/pkg/sentry/kernel/pipe/reader.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/reader_writer.go b/pkg/sentry/kernel/pipe/reader_writer.go index 82607367b..63efc5bbe 100644 --- a/pkg/sentry/kernel/pipe/reader_writer.go +++ b/pkg/sentry/kernel/pipe/reader_writer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/writer.go b/pkg/sentry/kernel/pipe/writer.go index d93324b53..6fea9769c 100644 --- a/pkg/sentry/kernel/pipe/writer.go +++ b/pkg/sentry/kernel/pipe/writer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/posixtimer.go b/pkg/sentry/kernel/posixtimer.go index 0ab958529..40b5acca3 100644 --- a/pkg/sentry/kernel/posixtimer.go +++ b/pkg/sentry/kernel/posixtimer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/ptrace.go b/pkg/sentry/kernel/ptrace.go index 9fe28f435..20bac2b70 100644 --- a/pkg/sentry/kernel/ptrace.go +++ b/pkg/sentry/kernel/ptrace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/rseq.go b/pkg/sentry/kernel/rseq.go index 1f3de58e3..46b03c700 100644 --- a/pkg/sentry/kernel/rseq.go +++ b/pkg/sentry/kernel/rseq.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/sched/cpuset.go b/pkg/sentry/kernel/sched/cpuset.go index 0a97603f0..69aee9127 100644 --- a/pkg/sentry/kernel/sched/cpuset.go +++ b/pkg/sentry/kernel/sched/cpuset.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/sched/cpuset_test.go b/pkg/sentry/kernel/sched/cpuset_test.go index 8a6e12958..a036ed513 100644 --- a/pkg/sentry/kernel/sched/cpuset_test.go +++ b/pkg/sentry/kernel/sched/cpuset_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/sched/sched.go b/pkg/sentry/kernel/sched/sched.go index f1de1da60..e59909baf 100644 --- a/pkg/sentry/kernel/sched/sched.go +++ b/pkg/sentry/kernel/sched/sched.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/seccomp.go b/pkg/sentry/kernel/seccomp.go index d77c05e2f..37dd3e4c9 100644 --- a/pkg/sentry/kernel/seccomp.go +++ b/pkg/sentry/kernel/seccomp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/semaphore/semaphore.go b/pkg/sentry/kernel/semaphore/semaphore.go index aa07946cf..232a276dc 100644 --- a/pkg/sentry/kernel/semaphore/semaphore.go +++ b/pkg/sentry/kernel/semaphore/semaphore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/semaphore/semaphore_test.go b/pkg/sentry/kernel/semaphore/semaphore_test.go index f9eb382e9..5f886bf31 100644 --- a/pkg/sentry/kernel/semaphore/semaphore_test.go +++ b/pkg/sentry/kernel/semaphore/semaphore_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/sessions.go b/pkg/sentry/kernel/sessions.go index a9b4e7647..78a5b4063 100644 --- a/pkg/sentry/kernel/sessions.go +++ b/pkg/sentry/kernel/sessions.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/shm/device.go b/pkg/sentry/kernel/shm/device.go index b0dacdbe0..bbc653ed8 100644 --- a/pkg/sentry/kernel/shm/device.go +++ b/pkg/sentry/kernel/shm/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/shm/shm.go b/pkg/sentry/kernel/shm/shm.go index 77973951e..8d0d14e45 100644 --- a/pkg/sentry/kernel/shm/shm.go +++ b/pkg/sentry/kernel/shm/shm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/signal.go b/pkg/sentry/kernel/signal.go index e3a2a777a..b066df132 100644 --- a/pkg/sentry/kernel/signal.go +++ b/pkg/sentry/kernel/signal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/signal_handlers.go b/pkg/sentry/kernel/signal_handlers.go index 3649f5e4d..3f1ac9898 100644 --- a/pkg/sentry/kernel/signal_handlers.go +++ b/pkg/sentry/kernel/signal_handlers.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/syscalls.go b/pkg/sentry/kernel/syscalls.go index 4c7811b6c..19b711e9c 100644 --- a/pkg/sentry/kernel/syscalls.go +++ b/pkg/sentry/kernel/syscalls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/syscalls_state.go b/pkg/sentry/kernel/syscalls_state.go index 826809a70..981455d46 100644 --- a/pkg/sentry/kernel/syscalls_state.go +++ b/pkg/sentry/kernel/syscalls_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/syslog.go b/pkg/sentry/kernel/syslog.go index 6531bd5d2..2aecf3eea 100644 --- a/pkg/sentry/kernel/syslog.go +++ b/pkg/sentry/kernel/syslog.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/table_test.go b/pkg/sentry/kernel/table_test.go index 71ca75555..3b29d3c6a 100644 --- a/pkg/sentry/kernel/table_test.go +++ b/pkg/sentry/kernel/table_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task.go b/pkg/sentry/kernel/task.go index 4f0b7fe3f..e22ec768d 100644 --- a/pkg/sentry/kernel/task.go +++ b/pkg/sentry/kernel/task.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_acct.go b/pkg/sentry/kernel/task_acct.go index d2052921e..24230af89 100644 --- a/pkg/sentry/kernel/task_acct.go +++ b/pkg/sentry/kernel/task_acct.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_block.go b/pkg/sentry/kernel/task_block.go index 6dc7b938e..e5027e551 100644 --- a/pkg/sentry/kernel/task_block.go +++ b/pkg/sentry/kernel/task_block.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_clone.go b/pkg/sentry/kernel/task_clone.go index de3aef40d..755fe0370 100644 --- a/pkg/sentry/kernel/task_clone.go +++ b/pkg/sentry/kernel/task_clone.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_context.go b/pkg/sentry/kernel/task_context.go index d2df7e9d1..45b8d2b04 100644 --- a/pkg/sentry/kernel/task_context.go +++ b/pkg/sentry/kernel/task_context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_exec.go b/pkg/sentry/kernel/task_exec.go index 1b760aba4..a9b74da8e 100644 --- a/pkg/sentry/kernel/task_exec.go +++ b/pkg/sentry/kernel/task_exec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_exit.go b/pkg/sentry/kernel/task_exit.go index 65969ca9b..44fbb487c 100644 --- a/pkg/sentry/kernel/task_exit.go +++ b/pkg/sentry/kernel/task_exit.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_futex.go b/pkg/sentry/kernel/task_futex.go index 62ebbcb0d..5a11ca3df 100644 --- a/pkg/sentry/kernel/task_futex.go +++ b/pkg/sentry/kernel/task_futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_identity.go b/pkg/sentry/kernel/task_identity.go index b0921b2eb..8f90ed786 100644 --- a/pkg/sentry/kernel/task_identity.go +++ b/pkg/sentry/kernel/task_identity.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_log.go b/pkg/sentry/kernel/task_log.go index 1769da210..f4c881c2d 100644 --- a/pkg/sentry/kernel/task_log.go +++ b/pkg/sentry/kernel/task_log.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_net.go b/pkg/sentry/kernel/task_net.go index 4df2e53d3..fc7cefc1f 100644 --- a/pkg/sentry/kernel/task_net.go +++ b/pkg/sentry/kernel/task_net.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_run.go b/pkg/sentry/kernel/task_run.go index 49ac933b7..596b9aa16 100644 --- a/pkg/sentry/kernel/task_run.go +++ b/pkg/sentry/kernel/task_run.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_sched.go b/pkg/sentry/kernel/task_sched.go index 19dcc963a..3b3cdc24a 100644 --- a/pkg/sentry/kernel/task_sched.go +++ b/pkg/sentry/kernel/task_sched.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_signals.go b/pkg/sentry/kernel/task_signals.go index e2925a708..fe24f7542 100644 --- a/pkg/sentry/kernel/task_signals.go +++ b/pkg/sentry/kernel/task_signals.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_start.go b/pkg/sentry/kernel/task_start.go index 6c8d7d316..c82a32c78 100644 --- a/pkg/sentry/kernel/task_start.go +++ b/pkg/sentry/kernel/task_start.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_stop.go b/pkg/sentry/kernel/task_stop.go index feaf6cae4..36846484c 100644 --- a/pkg/sentry/kernel/task_stop.go +++ b/pkg/sentry/kernel/task_stop.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_syscall.go b/pkg/sentry/kernel/task_syscall.go index f0373c375..0318adb35 100644 --- a/pkg/sentry/kernel/task_syscall.go +++ b/pkg/sentry/kernel/task_syscall.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_test.go b/pkg/sentry/kernel/task_test.go index 82ef858a1..3f37f505d 100644 --- a/pkg/sentry/kernel/task_test.go +++ b/pkg/sentry/kernel/task_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_usermem.go b/pkg/sentry/kernel/task_usermem.go index 2b4954869..c8e973bd5 100644 --- a/pkg/sentry/kernel/task_usermem.go +++ b/pkg/sentry/kernel/task_usermem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/thread_group.go b/pkg/sentry/kernel/thread_group.go index dfff7b52d..d7652f57c 100644 --- a/pkg/sentry/kernel/thread_group.go +++ b/pkg/sentry/kernel/thread_group.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/threads.go b/pkg/sentry/kernel/threads.go index 4e3d19e97..bdb907905 100644 --- a/pkg/sentry/kernel/threads.go +++ b/pkg/sentry/kernel/threads.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/time/context.go b/pkg/sentry/kernel/time/context.go index ac4dc01d8..3675ea20d 100644 --- a/pkg/sentry/kernel/time/context.go +++ b/pkg/sentry/kernel/time/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/time/time.go b/pkg/sentry/kernel/time/time.go index 52e0dfba1..ca0f4ba2e 100644 --- a/pkg/sentry/kernel/time/time.go +++ b/pkg/sentry/kernel/time/time.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/timekeeper.go b/pkg/sentry/kernel/timekeeper.go index 2167f3efe..6bff80f13 100644 --- a/pkg/sentry/kernel/timekeeper.go +++ b/pkg/sentry/kernel/timekeeper.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/timekeeper_state.go b/pkg/sentry/kernel/timekeeper_state.go index 2e7fed4d8..f3a3ed543 100644 --- a/pkg/sentry/kernel/timekeeper_state.go +++ b/pkg/sentry/kernel/timekeeper_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/timekeeper_test.go b/pkg/sentry/kernel/timekeeper_test.go index 34a5cec27..71674c21c 100644 --- a/pkg/sentry/kernel/timekeeper_test.go +++ b/pkg/sentry/kernel/timekeeper_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/uts_namespace.go b/pkg/sentry/kernel/uts_namespace.go index 7e0fe0d21..ed5f0c031 100644 --- a/pkg/sentry/kernel/uts_namespace.go +++ b/pkg/sentry/kernel/uts_namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/vdso.go b/pkg/sentry/kernel/vdso.go index 971e8bc59..0ec858a4a 100644 --- a/pkg/sentry/kernel/vdso.go +++ b/pkg/sentry/kernel/vdso.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/version.go b/pkg/sentry/kernel/version.go index a9e84673f..72bb0f93c 100644 --- a/pkg/sentry/kernel/version.go +++ b/pkg/sentry/kernel/version.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/limits/context.go b/pkg/sentry/limits/context.go index 75e97bf92..bf413eb7d 100644 --- a/pkg/sentry/limits/context.go +++ b/pkg/sentry/limits/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/limits/limits.go b/pkg/sentry/limits/limits.go index 02c8b60e3..ba0b7d4fd 100644 --- a/pkg/sentry/limits/limits.go +++ b/pkg/sentry/limits/limits.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/limits/limits_test.go b/pkg/sentry/limits/limits_test.go index dd6f80750..d41f62554 100644 --- a/pkg/sentry/limits/limits_test.go +++ b/pkg/sentry/limits/limits_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/limits/linux.go b/pkg/sentry/limits/linux.go index 8e6a24341..511db6733 100644 --- a/pkg/sentry/limits/linux.go +++ b/pkg/sentry/limits/linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/loader/elf.go b/pkg/sentry/loader/elf.go index 849be5a3d..9b1e81dc9 100644 --- a/pkg/sentry/loader/elf.go +++ b/pkg/sentry/loader/elf.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/loader/interpreter.go b/pkg/sentry/loader/interpreter.go index 54534952b..06a3c7156 100644 --- a/pkg/sentry/loader/interpreter.go +++ b/pkg/sentry/loader/interpreter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/loader/loader.go b/pkg/sentry/loader/loader.go index 62b39e52b..d1417c4f1 100644 --- a/pkg/sentry/loader/loader.go +++ b/pkg/sentry/loader/loader.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/loader/vdso.go b/pkg/sentry/loader/vdso.go index a06e27ac9..437cc5da1 100644 --- a/pkg/sentry/loader/vdso.go +++ b/pkg/sentry/loader/vdso.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/loader/vdso_state.go b/pkg/sentry/loader/vdso_state.go index dc71e1c2d..b327f0e1e 100644 --- a/pkg/sentry/loader/vdso_state.go +++ b/pkg/sentry/loader/vdso_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/memmap/mapping_set.go b/pkg/sentry/memmap/mapping_set.go index c9483905d..33cf16f91 100644 --- a/pkg/sentry/memmap/mapping_set.go +++ b/pkg/sentry/memmap/mapping_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/memmap/mapping_set_test.go b/pkg/sentry/memmap/mapping_set_test.go index 10668d404..49ee34548 100644 --- a/pkg/sentry/memmap/mapping_set_test.go +++ b/pkg/sentry/memmap/mapping_set_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/memmap/memmap.go b/pkg/sentry/memmap/memmap.go index cdc5f2b27..05349a77f 100644 --- a/pkg/sentry/memmap/memmap.go +++ b/pkg/sentry/memmap/memmap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/memutil/memutil.go b/pkg/sentry/memutil/memutil.go index 4f245cf3c..286d50ca4 100644 --- a/pkg/sentry/memutil/memutil.go +++ b/pkg/sentry/memutil/memutil.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/memutil/memutil_unsafe.go b/pkg/sentry/memutil/memutil_unsafe.go index 32c27eb2f..8d9fc64fb 100644 --- a/pkg/sentry/memutil/memutil_unsafe.go +++ b/pkg/sentry/memutil/memutil_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/address_space.go b/pkg/sentry/mm/address_space.go index 27554f163..7488f7c4a 100644 --- a/pkg/sentry/mm/address_space.go +++ b/pkg/sentry/mm/address_space.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/aio_context.go b/pkg/sentry/mm/aio_context.go index b42156d45..87942af0e 100644 --- a/pkg/sentry/mm/aio_context.go +++ b/pkg/sentry/mm/aio_context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/aio_context_state.go b/pkg/sentry/mm/aio_context_state.go index 1a5e56f8e..192a6f744 100644 --- a/pkg/sentry/mm/aio_context_state.go +++ b/pkg/sentry/mm/aio_context_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/debug.go b/pkg/sentry/mm/debug.go index 56d0490f0..d341b9c07 100644 --- a/pkg/sentry/mm/debug.go +++ b/pkg/sentry/mm/debug.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/io.go b/pkg/sentry/mm/io.go index 6741db594..6600ddd78 100644 --- a/pkg/sentry/mm/io.go +++ b/pkg/sentry/mm/io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/lifecycle.go b/pkg/sentry/mm/lifecycle.go index a4b5cb443..b248b76e7 100644 --- a/pkg/sentry/mm/lifecycle.go +++ b/pkg/sentry/mm/lifecycle.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/metadata.go b/pkg/sentry/mm/metadata.go index 32d5e2ff6..5ef1ba0b1 100644 --- a/pkg/sentry/mm/metadata.go +++ b/pkg/sentry/mm/metadata.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/mm.go b/pkg/sentry/mm/mm.go index 3299ae164..aab697f9e 100644 --- a/pkg/sentry/mm/mm.go +++ b/pkg/sentry/mm/mm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/mm_test.go b/pkg/sentry/mm/mm_test.go index b47aa7263..f2db43196 100644 --- a/pkg/sentry/mm/mm_test.go +++ b/pkg/sentry/mm/mm_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/pma.go b/pkg/sentry/mm/pma.go index 9febb25ac..5690fe6b4 100644 --- a/pkg/sentry/mm/pma.go +++ b/pkg/sentry/mm/pma.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/proc_pid_maps.go b/pkg/sentry/mm/proc_pid_maps.go index 5840b257c..0bf1cdb51 100644 --- a/pkg/sentry/mm/proc_pid_maps.go +++ b/pkg/sentry/mm/proc_pid_maps.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/save_restore.go b/pkg/sentry/mm/save_restore.go index 36fed8f1c..6e7080a84 100644 --- a/pkg/sentry/mm/save_restore.go +++ b/pkg/sentry/mm/save_restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/shm.go b/pkg/sentry/mm/shm.go index bab137a5a..3bc48c7e7 100644 --- a/pkg/sentry/mm/shm.go +++ b/pkg/sentry/mm/shm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/special_mappable.go b/pkg/sentry/mm/special_mappable.go index 5d7bd33bd..e511472f4 100644 --- a/pkg/sentry/mm/special_mappable.go +++ b/pkg/sentry/mm/special_mappable.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/syscalls.go b/pkg/sentry/mm/syscalls.go index b0622b0c3..a721cc456 100644 --- a/pkg/sentry/mm/syscalls.go +++ b/pkg/sentry/mm/syscalls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/vma.go b/pkg/sentry/mm/vma.go index b81e861f1..dafdbd0e4 100644 --- a/pkg/sentry/mm/vma.go +++ b/pkg/sentry/mm/vma.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/context.go b/pkg/sentry/platform/context.go index 0d200a5e2..cca21a23e 100644 --- a/pkg/sentry/platform/context.go +++ b/pkg/sentry/platform/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/filemem/filemem.go b/pkg/sentry/platform/filemem/filemem.go index f278c8d63..97da31e70 100644 --- a/pkg/sentry/platform/filemem/filemem.go +++ b/pkg/sentry/platform/filemem/filemem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/filemem/filemem_state.go b/pkg/sentry/platform/filemem/filemem_state.go index e28e021c9..964e2aaaa 100644 --- a/pkg/sentry/platform/filemem/filemem_state.go +++ b/pkg/sentry/platform/filemem/filemem_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/filemem/filemem_test.go b/pkg/sentry/platform/filemem/filemem_test.go index 4b165dc48..9becec25f 100644 --- a/pkg/sentry/platform/filemem/filemem_test.go +++ b/pkg/sentry/platform/filemem/filemem_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/filemem/filemem_unsafe.go b/pkg/sentry/platform/filemem/filemem_unsafe.go index a23b9825a..776aed74d 100644 --- a/pkg/sentry/platform/filemem/filemem_unsafe.go +++ b/pkg/sentry/platform/filemem/filemem_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/interrupt/interrupt.go b/pkg/sentry/platform/interrupt/interrupt.go index ca4f42087..9c83f41eb 100644 --- a/pkg/sentry/platform/interrupt/interrupt.go +++ b/pkg/sentry/platform/interrupt/interrupt.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/interrupt/interrupt_test.go b/pkg/sentry/platform/interrupt/interrupt_test.go index 7c49eeea6..fb3284395 100644 --- a/pkg/sentry/platform/interrupt/interrupt_test.go +++ b/pkg/sentry/platform/interrupt/interrupt_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/address_space.go b/pkg/sentry/platform/kvm/address_space.go index c4293c517..72e897a9a 100644 --- a/pkg/sentry/platform/kvm/address_space.go +++ b/pkg/sentry/platform/kvm/address_space.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/allocator.go b/pkg/sentry/platform/kvm/allocator.go index f5cebd5b3..b25cad155 100644 --- a/pkg/sentry/platform/kvm/allocator.go +++ b/pkg/sentry/platform/kvm/allocator.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill.go b/pkg/sentry/platform/kvm/bluepill.go index ecc33d7dd..9f1c9510b 100644 --- a/pkg/sentry/platform/kvm/bluepill.go +++ b/pkg/sentry/platform/kvm/bluepill.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill_amd64.go b/pkg/sentry/platform/kvm/bluepill_amd64.go index b364e3ef7..f013d1dc9 100644 --- a/pkg/sentry/platform/kvm/bluepill_amd64.go +++ b/pkg/sentry/platform/kvm/bluepill_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill_amd64.s b/pkg/sentry/platform/kvm/bluepill_amd64.s index 0881bd5f5..ec017f6c2 100644 --- a/pkg/sentry/platform/kvm/bluepill_amd64.s +++ b/pkg/sentry/platform/kvm/bluepill_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go b/pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go index 61ca61dcb..cd00a47f2 100644 --- a/pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go +++ b/pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill_fault.go b/pkg/sentry/platform/kvm/bluepill_fault.go index 8650cd78f..e79a30ef2 100644 --- a/pkg/sentry/platform/kvm/bluepill_fault.go +++ b/pkg/sentry/platform/kvm/bluepill_fault.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill_unsafe.go b/pkg/sentry/platform/kvm/bluepill_unsafe.go index 216d4b4b6..747a95997 100644 --- a/pkg/sentry/platform/kvm/bluepill_unsafe.go +++ b/pkg/sentry/platform/kvm/bluepill_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/context.go b/pkg/sentry/platform/kvm/context.go index aac84febf..be902be88 100644 --- a/pkg/sentry/platform/kvm/context.go +++ b/pkg/sentry/platform/kvm/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/host_map.go b/pkg/sentry/platform/kvm/host_map.go index fc16ad2de..ee6a1a42d 100644 --- a/pkg/sentry/platform/kvm/host_map.go +++ b/pkg/sentry/platform/kvm/host_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/kvm.go b/pkg/sentry/platform/kvm/kvm.go index 0c4dff308..d4f50024d 100644 --- a/pkg/sentry/platform/kvm/kvm.go +++ b/pkg/sentry/platform/kvm/kvm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/kvm_amd64.go b/pkg/sentry/platform/kvm/kvm_amd64.go index 3d56ed895..70d0ac63b 100644 --- a/pkg/sentry/platform/kvm/kvm_amd64.go +++ b/pkg/sentry/platform/kvm/kvm_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go b/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go index 476e783a0..c0a0af92d 100644 --- a/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go +++ b/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/kvm_const.go b/pkg/sentry/platform/kvm/kvm_const.go index ca44c31b3..8c53c6f06 100644 --- a/pkg/sentry/platform/kvm/kvm_const.go +++ b/pkg/sentry/platform/kvm/kvm_const.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/kvm_test.go b/pkg/sentry/platform/kvm/kvm_test.go index 52448839f..45eeb96ff 100644 --- a/pkg/sentry/platform/kvm/kvm_test.go +++ b/pkg/sentry/platform/kvm/kvm_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/machine.go b/pkg/sentry/platform/kvm/machine.go index 9f60b6b31..fc7ad258f 100644 --- a/pkg/sentry/platform/kvm/machine.go +++ b/pkg/sentry/platform/kvm/machine.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/machine_amd64.go b/pkg/sentry/platform/kvm/machine_amd64.go index bcd29a947..e0aec42b8 100644 --- a/pkg/sentry/platform/kvm/machine_amd64.go +++ b/pkg/sentry/platform/kvm/machine_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/machine_amd64_unsafe.go b/pkg/sentry/platform/kvm/machine_amd64_unsafe.go index 8b9041f13..50e513f3b 100644 --- a/pkg/sentry/platform/kvm/machine_amd64_unsafe.go +++ b/pkg/sentry/platform/kvm/machine_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/machine_unsafe.go b/pkg/sentry/platform/kvm/machine_unsafe.go index 86323c891..4f5b01321 100644 --- a/pkg/sentry/platform/kvm/machine_unsafe.go +++ b/pkg/sentry/platform/kvm/machine_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/physical_map.go b/pkg/sentry/platform/kvm/physical_map.go index 81a98656d..b908cae6a 100644 --- a/pkg/sentry/platform/kvm/physical_map.go +++ b/pkg/sentry/platform/kvm/physical_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/testutil/testutil.go b/pkg/sentry/platform/kvm/testutil/testutil.go index 8a614e25d..0d496561d 100644 --- a/pkg/sentry/platform/kvm/testutil/testutil.go +++ b/pkg/sentry/platform/kvm/testutil/testutil.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/testutil/testutil_amd64.go b/pkg/sentry/platform/kvm/testutil/testutil_amd64.go index 39286a0af..fcba33813 100644 --- a/pkg/sentry/platform/kvm/testutil/testutil_amd64.go +++ b/pkg/sentry/platform/kvm/testutil/testutil_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/testutil/testutil_amd64.s b/pkg/sentry/platform/kvm/testutil/testutil_amd64.s index 3b5ad8817..f1da41a44 100644 --- a/pkg/sentry/platform/kvm/testutil/testutil_amd64.s +++ b/pkg/sentry/platform/kvm/testutil/testutil_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/virtual_map.go b/pkg/sentry/platform/kvm/virtual_map.go index 0d3fbe043..0343e9267 100644 --- a/pkg/sentry/platform/kvm/virtual_map.go +++ b/pkg/sentry/platform/kvm/virtual_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/virtual_map_test.go b/pkg/sentry/platform/kvm/virtual_map_test.go index 7875bd3e9..935e0eb93 100644 --- a/pkg/sentry/platform/kvm/virtual_map_test.go +++ b/pkg/sentry/platform/kvm/virtual_map_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/mmap_min_addr.go b/pkg/sentry/platform/mmap_min_addr.go index 6398e5e01..1bcc1f8e9 100644 --- a/pkg/sentry/platform/mmap_min_addr.go +++ b/pkg/sentry/platform/mmap_min_addr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/platform.go b/pkg/sentry/platform/platform.go index 8a1620d93..f16588e6e 100644 --- a/pkg/sentry/platform/platform.go +++ b/pkg/sentry/platform/platform.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/procid/procid.go b/pkg/sentry/platform/procid/procid.go index 5f861908f..3f49ab093 100644 --- a/pkg/sentry/platform/procid/procid.go +++ b/pkg/sentry/platform/procid/procid.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/procid/procid_amd64.s b/pkg/sentry/platform/procid/procid_amd64.s index 5b1ba1f24..fd88ce82e 100644 --- a/pkg/sentry/platform/procid/procid_amd64.s +++ b/pkg/sentry/platform/procid/procid_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/procid/procid_net_test.go b/pkg/sentry/platform/procid/procid_net_test.go index 2d1605a08..e8dcc479d 100644 --- a/pkg/sentry/platform/procid/procid_net_test.go +++ b/pkg/sentry/platform/procid/procid_net_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/procid/procid_test.go b/pkg/sentry/platform/procid/procid_test.go index 5e44da36f..7a57c7cdc 100644 --- a/pkg/sentry/platform/procid/procid_test.go +++ b/pkg/sentry/platform/procid/procid_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/ptrace.go b/pkg/sentry/platform/ptrace/ptrace.go index 4f20716f7..00d92b092 100644 --- a/pkg/sentry/platform/ptrace/ptrace.go +++ b/pkg/sentry/platform/ptrace/ptrace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/ptrace_unsafe.go b/pkg/sentry/platform/ptrace/ptrace_unsafe.go index 46a8bda8e..7a3cb8f49 100644 --- a/pkg/sentry/platform/ptrace/ptrace_unsafe.go +++ b/pkg/sentry/platform/ptrace/ptrace_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/stub_amd64.s b/pkg/sentry/platform/ptrace/stub_amd64.s index 9bf87b6f6..63f98e40d 100644 --- a/pkg/sentry/platform/ptrace/stub_amd64.s +++ b/pkg/sentry/platform/ptrace/stub_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/stub_unsafe.go b/pkg/sentry/platform/ptrace/stub_unsafe.go index c868a2d68..48c16c4a1 100644 --- a/pkg/sentry/platform/ptrace/stub_unsafe.go +++ b/pkg/sentry/platform/ptrace/stub_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/subprocess.go b/pkg/sentry/platform/ptrace/subprocess.go index 6d5ad6b71..6a9da5db8 100644 --- a/pkg/sentry/platform/ptrace/subprocess.go +++ b/pkg/sentry/platform/ptrace/subprocess.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/subprocess_amd64.go b/pkg/sentry/platform/ptrace/subprocess_amd64.go index c38dc1ff8..d23a1133e 100644 --- a/pkg/sentry/platform/ptrace/subprocess_amd64.go +++ b/pkg/sentry/platform/ptrace/subprocess_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/subprocess_linux.go b/pkg/sentry/platform/ptrace/subprocess_linux.go index 53adadadd..7523487e7 100644 --- a/pkg/sentry/platform/ptrace/subprocess_linux.go +++ b/pkg/sentry/platform/ptrace/subprocess_linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go b/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go index 697431472..0c9263060 100644 --- a/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go +++ b/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/subprocess_unsafe.go b/pkg/sentry/platform/ptrace/subprocess_unsafe.go index fe41641d3..ca6c4ac97 100644 --- a/pkg/sentry/platform/ptrace/subprocess_unsafe.go +++ b/pkg/sentry/platform/ptrace/subprocess_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/defs.go b/pkg/sentry/platform/ring0/defs.go index f09d045eb..18137e55d 100644 --- a/pkg/sentry/platform/ring0/defs.go +++ b/pkg/sentry/platform/ring0/defs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/defs_amd64.go b/pkg/sentry/platform/ring0/defs_amd64.go index 84819f132..67242b92b 100644 --- a/pkg/sentry/platform/ring0/defs_amd64.go +++ b/pkg/sentry/platform/ring0/defs_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/entry_amd64.go b/pkg/sentry/platform/ring0/entry_amd64.go index a3e992e0d..4a9affe64 100644 --- a/pkg/sentry/platform/ring0/entry_amd64.go +++ b/pkg/sentry/platform/ring0/entry_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/entry_amd64.s b/pkg/sentry/platform/ring0/entry_amd64.s index 08c15ad65..d48fbd2d1 100644 --- a/pkg/sentry/platform/ring0/entry_amd64.s +++ b/pkg/sentry/platform/ring0/entry_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/gen_offsets/main.go b/pkg/sentry/platform/ring0/gen_offsets/main.go index ffa7eaf77..11c49855f 100644 --- a/pkg/sentry/platform/ring0/gen_offsets/main.go +++ b/pkg/sentry/platform/ring0/gen_offsets/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/kernel.go b/pkg/sentry/platform/ring0/kernel.go index 62e67005e..e70eafde2 100644 --- a/pkg/sentry/platform/ring0/kernel.go +++ b/pkg/sentry/platform/ring0/kernel.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/kernel_amd64.go b/pkg/sentry/platform/ring0/kernel_amd64.go index 0d2b0f7dc..ab562bca7 100644 --- a/pkg/sentry/platform/ring0/kernel_amd64.go +++ b/pkg/sentry/platform/ring0/kernel_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/kernel_unsafe.go b/pkg/sentry/platform/ring0/kernel_unsafe.go index cfb3ad853..faf4240e5 100644 --- a/pkg/sentry/platform/ring0/kernel_unsafe.go +++ b/pkg/sentry/platform/ring0/kernel_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/lib_amd64.go b/pkg/sentry/platform/ring0/lib_amd64.go index 989e3e383..2b95a0141 100644 --- a/pkg/sentry/platform/ring0/lib_amd64.go +++ b/pkg/sentry/platform/ring0/lib_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/lib_amd64.s b/pkg/sentry/platform/ring0/lib_amd64.s index 6f143ea5a..98a130525 100644 --- a/pkg/sentry/platform/ring0/lib_amd64.s +++ b/pkg/sentry/platform/ring0/lib_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/offsets_amd64.go b/pkg/sentry/platform/ring0/offsets_amd64.go index ca5fd456b..753d31ef8 100644 --- a/pkg/sentry/platform/ring0/offsets_amd64.go +++ b/pkg/sentry/platform/ring0/offsets_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/allocator.go b/pkg/sentry/platform/ring0/pagetables/allocator.go index 049fd0247..ee6e90a11 100644 --- a/pkg/sentry/platform/ring0/pagetables/allocator.go +++ b/pkg/sentry/platform/ring0/pagetables/allocator.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go b/pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go index aca778913..f48647b3a 100644 --- a/pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go +++ b/pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables.go b/pkg/sentry/platform/ring0/pagetables/pagetables.go index ff5787f89..c7207ec18 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go b/pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go index 878463018..746f614e5 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go b/pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go index a7f2ad9a4..2f82c4353 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables_test.go b/pkg/sentry/platform/ring0/pagetables/pagetables_test.go index dca3f69ef..3e5dc7dc7 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables_test.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables_x86.go b/pkg/sentry/platform/ring0/pagetables/pagetables_x86.go index ca49d20f8..6bd8c3584 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables_x86.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pcids_x86.go b/pkg/sentry/platform/ring0/pagetables/pcids_x86.go index fa068e35e..0d9a51aa5 100644 --- a/pkg/sentry/platform/ring0/pagetables/pcids_x86.go +++ b/pkg/sentry/platform/ring0/pagetables/pcids_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/walker_amd64.go b/pkg/sentry/platform/ring0/pagetables/walker_amd64.go index afa4d473a..c4c71d23e 100644 --- a/pkg/sentry/platform/ring0/pagetables/walker_amd64.go +++ b/pkg/sentry/platform/ring0/pagetables/walker_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/ring0.go b/pkg/sentry/platform/ring0/ring0.go index 4991031c5..10c51e88d 100644 --- a/pkg/sentry/platform/ring0/ring0.go +++ b/pkg/sentry/platform/ring0/ring0.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/x86.go b/pkg/sentry/platform/ring0/x86.go index f489fcecb..7c88010d8 100644 --- a/pkg/sentry/platform/ring0/x86.go +++ b/pkg/sentry/platform/ring0/x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/atomic_amd64.s b/pkg/sentry/platform/safecopy/atomic_amd64.s index 69947dec3..873ffa046 100644 --- a/pkg/sentry/platform/safecopy/atomic_amd64.s +++ b/pkg/sentry/platform/safecopy/atomic_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/memclr_amd64.s b/pkg/sentry/platform/safecopy/memclr_amd64.s index 7d1019f60..488b6e666 100644 --- a/pkg/sentry/platform/safecopy/memclr_amd64.s +++ b/pkg/sentry/platform/safecopy/memclr_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/memcpy_amd64.s b/pkg/sentry/platform/safecopy/memcpy_amd64.s index 96ef2eefc..0bf26fd7b 100644 --- a/pkg/sentry/platform/safecopy/memcpy_amd64.s +++ b/pkg/sentry/platform/safecopy/memcpy_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/safecopy.go b/pkg/sentry/platform/safecopy/safecopy.go index 90a2aad7b..c60f73103 100644 --- a/pkg/sentry/platform/safecopy/safecopy.go +++ b/pkg/sentry/platform/safecopy/safecopy.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/safecopy_test.go b/pkg/sentry/platform/safecopy/safecopy_test.go index 67df36121..1a682d28a 100644 --- a/pkg/sentry/platform/safecopy/safecopy_test.go +++ b/pkg/sentry/platform/safecopy/safecopy_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/safecopy_unsafe.go b/pkg/sentry/platform/safecopy/safecopy_unsafe.go index 72f243f8d..df1c35b66 100644 --- a/pkg/sentry/platform/safecopy/safecopy_unsafe.go +++ b/pkg/sentry/platform/safecopy/safecopy_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/sighandler_amd64.s b/pkg/sentry/platform/safecopy/sighandler_amd64.s index a65cb0c26..06614f1b4 100644 --- a/pkg/sentry/platform/safecopy/sighandler_amd64.s +++ b/pkg/sentry/platform/safecopy/sighandler_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/block_unsafe.go b/pkg/sentry/safemem/block_unsafe.go index 0b58f6497..e91ff66ae 100644 --- a/pkg/sentry/safemem/block_unsafe.go +++ b/pkg/sentry/safemem/block_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/io.go b/pkg/sentry/safemem/io.go index fd917648b..6cb52439f 100644 --- a/pkg/sentry/safemem/io.go +++ b/pkg/sentry/safemem/io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/io_test.go b/pkg/sentry/safemem/io_test.go index edac4c1d7..2eda8c3bb 100644 --- a/pkg/sentry/safemem/io_test.go +++ b/pkg/sentry/safemem/io_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/safemem.go b/pkg/sentry/safemem/safemem.go index 2f8002004..090932d3e 100644 --- a/pkg/sentry/safemem/safemem.go +++ b/pkg/sentry/safemem/safemem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/seq_test.go b/pkg/sentry/safemem/seq_test.go index 3e83b3851..fddcaf714 100644 --- a/pkg/sentry/safemem/seq_test.go +++ b/pkg/sentry/safemem/seq_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/seq_unsafe.go b/pkg/sentry/safemem/seq_unsafe.go index e0d29a0b3..83a6b7183 100644 --- a/pkg/sentry/safemem/seq_unsafe.go +++ b/pkg/sentry/safemem/seq_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/sighandling/sighandling.go b/pkg/sentry/sighandling/sighandling.go index 29bcf55ab..6b5d5f993 100644 --- a/pkg/sentry/sighandling/sighandling.go +++ b/pkg/sentry/sighandling/sighandling.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/sighandling/sighandling_unsafe.go b/pkg/sentry/sighandling/sighandling_unsafe.go index a455b919f..5913d47a8 100644 --- a/pkg/sentry/sighandling/sighandling_unsafe.go +++ b/pkg/sentry/sighandling/sighandling_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/control/control.go b/pkg/sentry/socket/control/control.go index db97e95f2..d44f5e88a 100644 --- a/pkg/sentry/socket/control/control.go +++ b/pkg/sentry/socket/control/control.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/epsocket/device.go b/pkg/sentry/socket/epsocket/device.go index 17f2c9559..3cc138eb0 100644 --- a/pkg/sentry/socket/epsocket/device.go +++ b/pkg/sentry/socket/epsocket/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/epsocket/epsocket.go b/pkg/sentry/socket/epsocket/epsocket.go index 47c575e7b..e90ef4835 100644 --- a/pkg/sentry/socket/epsocket/epsocket.go +++ b/pkg/sentry/socket/epsocket/epsocket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/epsocket/provider.go b/pkg/sentry/socket/epsocket/provider.go index dbc232d26..686554437 100644 --- a/pkg/sentry/socket/epsocket/provider.go +++ b/pkg/sentry/socket/epsocket/provider.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/epsocket/save_restore.go b/pkg/sentry/socket/epsocket/save_restore.go index 2613f90de..34d9a7cf0 100644 --- a/pkg/sentry/socket/epsocket/save_restore.go +++ b/pkg/sentry/socket/epsocket/save_restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/epsocket/stack.go b/pkg/sentry/socket/epsocket/stack.go index e4ed52fc8..c0081c819 100644 --- a/pkg/sentry/socket/epsocket/stack.go +++ b/pkg/sentry/socket/epsocket/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/device.go b/pkg/sentry/socket/hostinet/device.go index a9a673316..c5133f3bb 100644 --- a/pkg/sentry/socket/hostinet/device.go +++ b/pkg/sentry/socket/hostinet/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/hostinet.go b/pkg/sentry/socket/hostinet/hostinet.go index 67c6c8066..7858892ab 100644 --- a/pkg/sentry/socket/hostinet/hostinet.go +++ b/pkg/sentry/socket/hostinet/hostinet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/save_restore.go b/pkg/sentry/socket/hostinet/save_restore.go index 0821a794a..3827f082a 100644 --- a/pkg/sentry/socket/hostinet/save_restore.go +++ b/pkg/sentry/socket/hostinet/save_restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/socket.go b/pkg/sentry/socket/hostinet/socket.go index e82624b44..e4e950fbb 100644 --- a/pkg/sentry/socket/hostinet/socket.go +++ b/pkg/sentry/socket/hostinet/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/socket_unsafe.go b/pkg/sentry/socket/hostinet/socket_unsafe.go index f8bb75636..59c8910ca 100644 --- a/pkg/sentry/socket/hostinet/socket_unsafe.go +++ b/pkg/sentry/socket/hostinet/socket_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/stack.go b/pkg/sentry/socket/hostinet/stack.go index f64809d39..4ce73c1f1 100644 --- a/pkg/sentry/socket/hostinet/stack.go +++ b/pkg/sentry/socket/hostinet/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/message.go b/pkg/sentry/socket/netlink/message.go index b902d7ec9..a95172cba 100644 --- a/pkg/sentry/socket/netlink/message.go +++ b/pkg/sentry/socket/netlink/message.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/port/port.go b/pkg/sentry/socket/netlink/port/port.go index 1c5d4c3a5..20b9a6e37 100644 --- a/pkg/sentry/socket/netlink/port/port.go +++ b/pkg/sentry/socket/netlink/port/port.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/port/port_test.go b/pkg/sentry/socket/netlink/port/port_test.go index 34565e2f9..49b3b48ab 100644 --- a/pkg/sentry/socket/netlink/port/port_test.go +++ b/pkg/sentry/socket/netlink/port/port_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/provider.go b/pkg/sentry/socket/netlink/provider.go index 5d0a04a07..06786bd50 100644 --- a/pkg/sentry/socket/netlink/provider.go +++ b/pkg/sentry/socket/netlink/provider.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/route/protocol.go b/pkg/sentry/socket/netlink/route/protocol.go index 70322b9ed..7e70b09b2 100644 --- a/pkg/sentry/socket/netlink/route/protocol.go +++ b/pkg/sentry/socket/netlink/route/protocol.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/socket.go b/pkg/sentry/socket/netlink/socket.go index 0c03997f2..4d4130a4c 100644 --- a/pkg/sentry/socket/netlink/socket.go +++ b/pkg/sentry/socket/netlink/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/conn/conn.go b/pkg/sentry/socket/rpcinet/conn/conn.go index f4c8489b1..9c749b888 100644 --- a/pkg/sentry/socket/rpcinet/conn/conn.go +++ b/pkg/sentry/socket/rpcinet/conn/conn.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/device.go b/pkg/sentry/socket/rpcinet/device.go index f7b63436e..d2b9f9222 100644 --- a/pkg/sentry/socket/rpcinet/device.go +++ b/pkg/sentry/socket/rpcinet/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/notifier/notifier.go b/pkg/sentry/socket/rpcinet/notifier/notifier.go index f88a908ed..73c255c33 100644 --- a/pkg/sentry/socket/rpcinet/notifier/notifier.go +++ b/pkg/sentry/socket/rpcinet/notifier/notifier.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/rpcinet.go b/pkg/sentry/socket/rpcinet/rpcinet.go index 10b0dedc2..6c98e6acb 100644 --- a/pkg/sentry/socket/rpcinet/rpcinet.go +++ b/pkg/sentry/socket/rpcinet/rpcinet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index c7e761d54..44fa5c620 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/stack.go b/pkg/sentry/socket/rpcinet/stack.go index bcb89fb34..cb8344ec6 100644 --- a/pkg/sentry/socket/rpcinet/stack.go +++ b/pkg/sentry/socket/rpcinet/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/stack_unsafe.go b/pkg/sentry/socket/rpcinet/stack_unsafe.go index 9a896c623..d04fb2069 100644 --- a/pkg/sentry/socket/rpcinet/stack_unsafe.go +++ b/pkg/sentry/socket/rpcinet/stack_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/socket.go b/pkg/sentry/socket/socket.go index 31f8d42d7..a235c5249 100644 --- a/pkg/sentry/socket/socket.go +++ b/pkg/sentry/socket/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/device.go b/pkg/sentry/socket/unix/device.go index e8bcc7a9f..41820dbb3 100644 --- a/pkg/sentry/socket/unix/device.go +++ b/pkg/sentry/socket/unix/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/io.go b/pkg/sentry/socket/unix/io.go index 06333e14b..7d6434696 100644 --- a/pkg/sentry/socket/unix/io.go +++ b/pkg/sentry/socket/unix/io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/transport/connectioned.go b/pkg/sentry/socket/unix/transport/connectioned.go index 566e3d57b..4c913effc 100644 --- a/pkg/sentry/socket/unix/transport/connectioned.go +++ b/pkg/sentry/socket/unix/transport/connectioned.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/transport/connectioned_state.go b/pkg/sentry/socket/unix/transport/connectioned_state.go index 7e6c73dcc..608a6a97a 100644 --- a/pkg/sentry/socket/unix/transport/connectioned_state.go +++ b/pkg/sentry/socket/unix/transport/connectioned_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/transport/connectionless.go b/pkg/sentry/socket/unix/transport/connectionless.go index 86cd05199..cd4633106 100644 --- a/pkg/sentry/socket/unix/transport/connectionless.go +++ b/pkg/sentry/socket/unix/transport/connectionless.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/transport/queue.go b/pkg/sentry/socket/unix/transport/queue.go index c4d7d863c..5b4dfab68 100644 --- a/pkg/sentry/socket/unix/transport/queue.go +++ b/pkg/sentry/socket/unix/transport/queue.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/transport/unix.go b/pkg/sentry/socket/unix/transport/unix.go index 2934101a2..157133b65 100644 --- a/pkg/sentry/socket/unix/transport/unix.go +++ b/pkg/sentry/socket/unix/transport/unix.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/unix.go b/pkg/sentry/socket/unix/unix.go index 668363864..3543dd81f 100644 --- a/pkg/sentry/socket/unix/unix.go +++ b/pkg/sentry/socket/unix/unix.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/state/state.go b/pkg/sentry/state/state.go index 43e88a713..70b33f190 100644 --- a/pkg/sentry/state/state.go +++ b/pkg/sentry/state/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/state/state_metadata.go b/pkg/sentry/state/state_metadata.go index afa21672a..7f047b808 100644 --- a/pkg/sentry/state/state_metadata.go +++ b/pkg/sentry/state/state_metadata.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/state/state_unsafe.go b/pkg/sentry/state/state_unsafe.go index 3ff7d24c8..f02e12b2a 100644 --- a/pkg/sentry/state/state_unsafe.go +++ b/pkg/sentry/state/state_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/clone.go b/pkg/sentry/strace/clone.go index b82ca1ad1..e18ce84dc 100644 --- a/pkg/sentry/strace/clone.go +++ b/pkg/sentry/strace/clone.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/futex.go b/pkg/sentry/strace/futex.go index 3da108cb7..ceb3dc21d 100644 --- a/pkg/sentry/strace/futex.go +++ b/pkg/sentry/strace/futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/linux64.go b/pkg/sentry/strace/linux64.go index 1df148e7d..99714f12c 100644 --- a/pkg/sentry/strace/linux64.go +++ b/pkg/sentry/strace/linux64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/open.go b/pkg/sentry/strace/open.go index 839d5eda7..5a72a940c 100644 --- a/pkg/sentry/strace/open.go +++ b/pkg/sentry/strace/open.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/ptrace.go b/pkg/sentry/strace/ptrace.go index fcdb7e9f4..c572aafb4 100644 --- a/pkg/sentry/strace/ptrace.go +++ b/pkg/sentry/strace/ptrace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/socket.go b/pkg/sentry/strace/socket.go index 26831edd6..375418dc1 100644 --- a/pkg/sentry/strace/socket.go +++ b/pkg/sentry/strace/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/strace.go b/pkg/sentry/strace/strace.go index f7bfa3a1f..4286f0df7 100644 --- a/pkg/sentry/strace/strace.go +++ b/pkg/sentry/strace/strace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/strace.proto b/pkg/sentry/strace/strace.proto index 914e8c7b0..f1fc539d6 100644 --- a/pkg/sentry/strace/strace.proto +++ b/pkg/sentry/strace/strace.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/syscalls.go b/pkg/sentry/strace/syscalls.go index 8be4fa318..9eeb18a03 100644 --- a/pkg/sentry/strace/syscalls.go +++ b/pkg/sentry/strace/syscalls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/epoll.go b/pkg/sentry/syscalls/epoll.go index 01dd6fa71..b90d191b7 100644 --- a/pkg/sentry/syscalls/epoll.go +++ b/pkg/sentry/syscalls/epoll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/error.go b/pkg/sentry/syscalls/linux/error.go index 013b385bc..9fd002955 100644 --- a/pkg/sentry/syscalls/linux/error.go +++ b/pkg/sentry/syscalls/linux/error.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/flags.go b/pkg/sentry/syscalls/linux/flags.go index f01483cd3..d1e0833fc 100644 --- a/pkg/sentry/syscalls/linux/flags.go +++ b/pkg/sentry/syscalls/linux/flags.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/linux64.go b/pkg/sentry/syscalls/linux/linux64.go index 4465549ad..75e87f5ec 100644 --- a/pkg/sentry/syscalls/linux/linux64.go +++ b/pkg/sentry/syscalls/linux/linux64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sigset.go b/pkg/sentry/syscalls/linux/sigset.go index bfb541634..a033b7c70 100644 --- a/pkg/sentry/syscalls/linux/sigset.go +++ b/pkg/sentry/syscalls/linux/sigset.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_aio.go b/pkg/sentry/syscalls/linux/sys_aio.go index 54e4afa9e..355071131 100644 --- a/pkg/sentry/syscalls/linux/sys_aio.go +++ b/pkg/sentry/syscalls/linux/sys_aio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_capability.go b/pkg/sentry/syscalls/linux/sys_capability.go index 89c81ac90..cf972dc28 100644 --- a/pkg/sentry/syscalls/linux/sys_capability.go +++ b/pkg/sentry/syscalls/linux/sys_capability.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_epoll.go b/pkg/sentry/syscalls/linux/sys_epoll.go index e69dfc77a..62272efcd 100644 --- a/pkg/sentry/syscalls/linux/sys_epoll.go +++ b/pkg/sentry/syscalls/linux/sys_epoll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_eventfd.go b/pkg/sentry/syscalls/linux/sys_eventfd.go index 60fe5a133..903172890 100644 --- a/pkg/sentry/syscalls/linux/sys_eventfd.go +++ b/pkg/sentry/syscalls/linux/sys_eventfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_file.go b/pkg/sentry/syscalls/linux/sys_file.go index 64704bb88..a70f35be0 100644 --- a/pkg/sentry/syscalls/linux/sys_file.go +++ b/pkg/sentry/syscalls/linux/sys_file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_futex.go b/pkg/sentry/syscalls/linux/sys_futex.go index d35dcecbe..cf04428bc 100644 --- a/pkg/sentry/syscalls/linux/sys_futex.go +++ b/pkg/sentry/syscalls/linux/sys_futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_getdents.go b/pkg/sentry/syscalls/linux/sys_getdents.go index 29c0d7a39..4b441b31b 100644 --- a/pkg/sentry/syscalls/linux/sys_getdents.go +++ b/pkg/sentry/syscalls/linux/sys_getdents.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_identity.go b/pkg/sentry/syscalls/linux/sys_identity.go index 4fd0ed794..8d594aa83 100644 --- a/pkg/sentry/syscalls/linux/sys_identity.go +++ b/pkg/sentry/syscalls/linux/sys_identity.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_inotify.go b/pkg/sentry/syscalls/linux/sys_inotify.go index 725204dff..26a505782 100644 --- a/pkg/sentry/syscalls/linux/sys_inotify.go +++ b/pkg/sentry/syscalls/linux/sys_inotify.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_lseek.go b/pkg/sentry/syscalls/linux/sys_lseek.go index 97b51ba7c..ad3bfd761 100644 --- a/pkg/sentry/syscalls/linux/sys_lseek.go +++ b/pkg/sentry/syscalls/linux/sys_lseek.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_mmap.go b/pkg/sentry/syscalls/linux/sys_mmap.go index 1a98328dc..f8d9c43fd 100644 --- a/pkg/sentry/syscalls/linux/sys_mmap.go +++ b/pkg/sentry/syscalls/linux/sys_mmap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_mount.go b/pkg/sentry/syscalls/linux/sys_mount.go index 57cedccc1..bf0df7302 100644 --- a/pkg/sentry/syscalls/linux/sys_mount.go +++ b/pkg/sentry/syscalls/linux/sys_mount.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_pipe.go b/pkg/sentry/syscalls/linux/sys_pipe.go index 2b544f145..3652c429e 100644 --- a/pkg/sentry/syscalls/linux/sys_pipe.go +++ b/pkg/sentry/syscalls/linux/sys_pipe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_poll.go b/pkg/sentry/syscalls/linux/sys_poll.go index b9bdefadb..bf0958435 100644 --- a/pkg/sentry/syscalls/linux/sys_poll.go +++ b/pkg/sentry/syscalls/linux/sys_poll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_prctl.go b/pkg/sentry/syscalls/linux/sys_prctl.go index a1242acd3..c7b39ede8 100644 --- a/pkg/sentry/syscalls/linux/sys_prctl.go +++ b/pkg/sentry/syscalls/linux/sys_prctl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_random.go b/pkg/sentry/syscalls/linux/sys_random.go index be31e6b17..452dff058 100644 --- a/pkg/sentry/syscalls/linux/sys_random.go +++ b/pkg/sentry/syscalls/linux/sys_random.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_read.go b/pkg/sentry/syscalls/linux/sys_read.go index 0be2d195a..b2e5a5449 100644 --- a/pkg/sentry/syscalls/linux/sys_read.go +++ b/pkg/sentry/syscalls/linux/sys_read.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_rlimit.go b/pkg/sentry/syscalls/linux/sys_rlimit.go index d806b58ab..2f16e1791 100644 --- a/pkg/sentry/syscalls/linux/sys_rlimit.go +++ b/pkg/sentry/syscalls/linux/sys_rlimit.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_rusage.go b/pkg/sentry/syscalls/linux/sys_rusage.go index 82e42b589..ab07c77f9 100644 --- a/pkg/sentry/syscalls/linux/sys_rusage.go +++ b/pkg/sentry/syscalls/linux/sys_rusage.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_sched.go b/pkg/sentry/syscalls/linux/sys_sched.go index ff9e46077..e679a6694 100644 --- a/pkg/sentry/syscalls/linux/sys_sched.go +++ b/pkg/sentry/syscalls/linux/sys_sched.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_seccomp.go b/pkg/sentry/syscalls/linux/sys_seccomp.go index 4323a4df4..969acaa36 100644 --- a/pkg/sentry/syscalls/linux/sys_seccomp.go +++ b/pkg/sentry/syscalls/linux/sys_seccomp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_sem.go b/pkg/sentry/syscalls/linux/sys_sem.go index a8983705b..4ed52c4a7 100644 --- a/pkg/sentry/syscalls/linux/sys_sem.go +++ b/pkg/sentry/syscalls/linux/sys_sem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_shm.go b/pkg/sentry/syscalls/linux/sys_shm.go index 48ff1d5f0..b13d48b98 100644 --- a/pkg/sentry/syscalls/linux/sys_shm.go +++ b/pkg/sentry/syscalls/linux/sys_shm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_signal.go b/pkg/sentry/syscalls/linux/sys_signal.go index ecdec5d3a..a539354c5 100644 --- a/pkg/sentry/syscalls/linux/sys_signal.go +++ b/pkg/sentry/syscalls/linux/sys_signal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_socket.go b/pkg/sentry/syscalls/linux/sys_socket.go index 5fa5ddce6..0a7551742 100644 --- a/pkg/sentry/syscalls/linux/sys_socket.go +++ b/pkg/sentry/syscalls/linux/sys_socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_stat.go b/pkg/sentry/syscalls/linux/sys_stat.go index 619a14d7c..9c433c45d 100644 --- a/pkg/sentry/syscalls/linux/sys_stat.go +++ b/pkg/sentry/syscalls/linux/sys_stat.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_sync.go b/pkg/sentry/syscalls/linux/sys_sync.go index 902d210db..826c6869d 100644 --- a/pkg/sentry/syscalls/linux/sys_sync.go +++ b/pkg/sentry/syscalls/linux/sys_sync.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_sysinfo.go b/pkg/sentry/syscalls/linux/sys_sysinfo.go index 6560bac57..5eeb3ba58 100644 --- a/pkg/sentry/syscalls/linux/sys_sysinfo.go +++ b/pkg/sentry/syscalls/linux/sys_sysinfo.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_syslog.go b/pkg/sentry/syscalls/linux/sys_syslog.go index 792040c81..7193b7aed 100644 --- a/pkg/sentry/syscalls/linux/sys_syslog.go +++ b/pkg/sentry/syscalls/linux/sys_syslog.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_thread.go b/pkg/sentry/syscalls/linux/sys_thread.go index 550f63a43..820ca680e 100644 --- a/pkg/sentry/syscalls/linux/sys_thread.go +++ b/pkg/sentry/syscalls/linux/sys_thread.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_time.go b/pkg/sentry/syscalls/linux/sys_time.go index 8e6683444..063fbb106 100644 --- a/pkg/sentry/syscalls/linux/sys_time.go +++ b/pkg/sentry/syscalls/linux/sys_time.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_timer.go b/pkg/sentry/syscalls/linux/sys_timer.go index c41074d54..6baf4599b 100644 --- a/pkg/sentry/syscalls/linux/sys_timer.go +++ b/pkg/sentry/syscalls/linux/sys_timer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_timerfd.go b/pkg/sentry/syscalls/linux/sys_timerfd.go index 92c6a3d60..f70d13682 100644 --- a/pkg/sentry/syscalls/linux/sys_timerfd.go +++ b/pkg/sentry/syscalls/linux/sys_timerfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_tls.go b/pkg/sentry/syscalls/linux/sys_tls.go index b95d62320..27ddb3808 100644 --- a/pkg/sentry/syscalls/linux/sys_tls.go +++ b/pkg/sentry/syscalls/linux/sys_tls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_utsname.go b/pkg/sentry/syscalls/linux/sys_utsname.go index 899116374..689f2f838 100644 --- a/pkg/sentry/syscalls/linux/sys_utsname.go +++ b/pkg/sentry/syscalls/linux/sys_utsname.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_write.go b/pkg/sentry/syscalls/linux/sys_write.go index caa7b01ea..08e263112 100644 --- a/pkg/sentry/syscalls/linux/sys_write.go +++ b/pkg/sentry/syscalls/linux/sys_write.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/timespec.go b/pkg/sentry/syscalls/linux/timespec.go index e865c6fc0..752ec326d 100644 --- a/pkg/sentry/syscalls/linux/timespec.go +++ b/pkg/sentry/syscalls/linux/timespec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/polling.go b/pkg/sentry/syscalls/polling.go index fd90184ef..2b33d6c19 100644 --- a/pkg/sentry/syscalls/polling.go +++ b/pkg/sentry/syscalls/polling.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/syscalls.go b/pkg/sentry/syscalls/syscalls.go index 1176f858d..bae32d727 100644 --- a/pkg/sentry/syscalls/syscalls.go +++ b/pkg/sentry/syscalls/syscalls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/unimplemented_syscall.proto b/pkg/sentry/syscalls/unimplemented_syscall.proto index d6febf5b1..41579b016 100644 --- a/pkg/sentry/syscalls/unimplemented_syscall.proto +++ b/pkg/sentry/syscalls/unimplemented_syscall.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/calibrated_clock.go b/pkg/sentry/time/calibrated_clock.go index cbb95e2d7..c8cf4eca4 100644 --- a/pkg/sentry/time/calibrated_clock.go +++ b/pkg/sentry/time/calibrated_clock.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/calibrated_clock_test.go b/pkg/sentry/time/calibrated_clock_test.go index 8b6dd5592..a9237630e 100644 --- a/pkg/sentry/time/calibrated_clock_test.go +++ b/pkg/sentry/time/calibrated_clock_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/clock_id.go b/pkg/sentry/time/clock_id.go index 500102e58..1317a5dad 100644 --- a/pkg/sentry/time/clock_id.go +++ b/pkg/sentry/time/clock_id.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/clocks.go b/pkg/sentry/time/clocks.go index 9925b407d..e26386520 100644 --- a/pkg/sentry/time/clocks.go +++ b/pkg/sentry/time/clocks.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/muldiv_amd64.s b/pkg/sentry/time/muldiv_amd64.s index 291940b1d..bfcb8c724 100644 --- a/pkg/sentry/time/muldiv_amd64.s +++ b/pkg/sentry/time/muldiv_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/parameters.go b/pkg/sentry/time/parameters.go index 594b4874b..f3ad58454 100644 --- a/pkg/sentry/time/parameters.go +++ b/pkg/sentry/time/parameters.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/parameters_test.go b/pkg/sentry/time/parameters_test.go index 7394fc5ee..4a0c4e880 100644 --- a/pkg/sentry/time/parameters_test.go +++ b/pkg/sentry/time/parameters_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/sampler.go b/pkg/sentry/time/sampler.go index cf581b5fa..445690d49 100644 --- a/pkg/sentry/time/sampler.go +++ b/pkg/sentry/time/sampler.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/sampler_test.go b/pkg/sentry/time/sampler_test.go index caf7e5c53..ec0e442b6 100644 --- a/pkg/sentry/time/sampler_test.go +++ b/pkg/sentry/time/sampler_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/sampler_unsafe.go b/pkg/sentry/time/sampler_unsafe.go index 7ea19d387..0f8eb4fc8 100644 --- a/pkg/sentry/time/sampler_unsafe.go +++ b/pkg/sentry/time/sampler_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/tsc_amd64.s b/pkg/sentry/time/tsc_amd64.s index 4cc604392..e53d477f7 100644 --- a/pkg/sentry/time/tsc_amd64.s +++ b/pkg/sentry/time/tsc_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/uniqueid/context.go b/pkg/sentry/uniqueid/context.go index e48fabc2d..399d98c29 100644 --- a/pkg/sentry/uniqueid/context.go +++ b/pkg/sentry/uniqueid/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usage/cpu.go b/pkg/sentry/usage/cpu.go index ed7b04b9e..cbd7cfe19 100644 --- a/pkg/sentry/usage/cpu.go +++ b/pkg/sentry/usage/cpu.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usage/io.go b/pkg/sentry/usage/io.go index 49faa507d..8e27a0a88 100644 --- a/pkg/sentry/usage/io.go +++ b/pkg/sentry/usage/io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usage/memory.go b/pkg/sentry/usage/memory.go index 92a478d85..7e065cb76 100644 --- a/pkg/sentry/usage/memory.go +++ b/pkg/sentry/usage/memory.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usage/memory_unsafe.go b/pkg/sentry/usage/memory_unsafe.go index f990a7750..a3ae668a5 100644 --- a/pkg/sentry/usage/memory_unsafe.go +++ b/pkg/sentry/usage/memory_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usage/usage.go b/pkg/sentry/usage/usage.go index 3b3118659..ab327f8e2 100644 --- a/pkg/sentry/usage/usage.go +++ b/pkg/sentry/usage/usage.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/access_type.go b/pkg/sentry/usermem/access_type.go index 75346d854..c71d05afe 100644 --- a/pkg/sentry/usermem/access_type.go +++ b/pkg/sentry/usermem/access_type.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/addr.go b/pkg/sentry/usermem/addr.go index fc94bee80..2a75aa60c 100644 --- a/pkg/sentry/usermem/addr.go +++ b/pkg/sentry/usermem/addr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/addr_range_seq_test.go b/pkg/sentry/usermem/addr_range_seq_test.go index cf9d785ed..bd6a1ec8a 100644 --- a/pkg/sentry/usermem/addr_range_seq_test.go +++ b/pkg/sentry/usermem/addr_range_seq_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/addr_range_seq_unsafe.go b/pkg/sentry/usermem/addr_range_seq_unsafe.go index 13b2998b3..f5fd446fa 100644 --- a/pkg/sentry/usermem/addr_range_seq_unsafe.go +++ b/pkg/sentry/usermem/addr_range_seq_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/bytes_io.go b/pkg/sentry/usermem/bytes_io.go index 01a746404..274f568d0 100644 --- a/pkg/sentry/usermem/bytes_io.go +++ b/pkg/sentry/usermem/bytes_io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/bytes_io_unsafe.go b/pkg/sentry/usermem/bytes_io_unsafe.go index efd71fcbc..8bdf3a508 100644 --- a/pkg/sentry/usermem/bytes_io_unsafe.go +++ b/pkg/sentry/usermem/bytes_io_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/usermem.go b/pkg/sentry/usermem/usermem.go index 5d8a1c558..1d6c0b4d6 100644 --- a/pkg/sentry/usermem/usermem.go +++ b/pkg/sentry/usermem/usermem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/usermem_test.go b/pkg/sentry/usermem/usermem_test.go index 563560da8..1991a9641 100644 --- a/pkg/sentry/usermem/usermem_test.go +++ b/pkg/sentry/usermem/usermem_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/usermem_x86.go b/pkg/sentry/usermem/usermem_x86.go index 2484b0d82..9ec90f9ff 100644 --- a/pkg/sentry/usermem/usermem_x86.go +++ b/pkg/sentry/usermem/usermem_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/watchdog/watchdog.go b/pkg/sentry/watchdog/watchdog.go index 5b620693d..75b11237f 100644 --- a/pkg/sentry/watchdog/watchdog.go +++ b/pkg/sentry/watchdog/watchdog.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/commit_amd64.s b/pkg/sleep/commit_amd64.s index d525e5b79..d08df7f37 100644 --- a/pkg/sleep/commit_amd64.s +++ b/pkg/sleep/commit_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/commit_asm.go b/pkg/sleep/commit_asm.go index 39a55df7e..90eef4cbc 100644 --- a/pkg/sleep/commit_asm.go +++ b/pkg/sleep/commit_asm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/commit_noasm.go b/pkg/sleep/commit_noasm.go index 584866cd8..967d22e24 100644 --- a/pkg/sleep/commit_noasm.go +++ b/pkg/sleep/commit_noasm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/empty.s b/pkg/sleep/empty.s index 8aca31bee..85d52cd9c 100644 --- a/pkg/sleep/empty.s +++ b/pkg/sleep/empty.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/sleep_test.go b/pkg/sleep/sleep_test.go index bc1738371..8feb9ffc2 100644 --- a/pkg/sleep/sleep_test.go +++ b/pkg/sleep/sleep_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/sleep_unsafe.go b/pkg/sleep/sleep_unsafe.go index b12cce681..45fb6f0ea 100644 --- a/pkg/sleep/sleep_unsafe.go +++ b/pkg/sleep/sleep_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/decode.go b/pkg/state/decode.go index 3ef59610b..54b5ad8b8 100644 --- a/pkg/state/decode.go +++ b/pkg/state/decode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/encode.go b/pkg/state/encode.go index fd052db12..577aaf051 100644 --- a/pkg/state/encode.go +++ b/pkg/state/encode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/encode_unsafe.go b/pkg/state/encode_unsafe.go index d96ba56d4..be94742a8 100644 --- a/pkg/state/encode_unsafe.go +++ b/pkg/state/encode_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/map.go b/pkg/state/map.go index c3d165501..0035d7250 100644 --- a/pkg/state/map.go +++ b/pkg/state/map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/object.proto b/pkg/state/object.proto index c78efed2a..d3b46ea97 100644 --- a/pkg/state/object.proto +++ b/pkg/state/object.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/printer.go b/pkg/state/printer.go index 2c8ce60a5..aee4b69fb 100644 --- a/pkg/state/printer.go +++ b/pkg/state/printer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/state.go b/pkg/state/state.go index 23a0b5922..4b141777e 100644 --- a/pkg/state/state.go +++ b/pkg/state/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/state_test.go b/pkg/state/state_test.go index 38ad9da9c..22bcad9e1 100644 --- a/pkg/state/state_test.go +++ b/pkg/state/state_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/statefile/statefile.go b/pkg/state/statefile/statefile.go index 9c86c1934..99158fd02 100644 --- a/pkg/state/statefile/statefile.go +++ b/pkg/state/statefile/statefile.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/statefile/statefile_test.go b/pkg/state/statefile/statefile_test.go index fa3fb9f2c..b4f400e01 100644 --- a/pkg/state/statefile/statefile_test.go +++ b/pkg/state/statefile/statefile_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/stats.go b/pkg/state/stats.go index ddcc49f78..17ca258fc 100644 --- a/pkg/state/stats.go +++ b/pkg/state/stats.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/atomicptr_unsafe.go b/pkg/sync/atomicptr_unsafe.go index f12e9cb67..d943b7ff4 100644 --- a/pkg/sync/atomicptr_unsafe.go +++ b/pkg/sync/atomicptr_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/atomicptrtest/atomicptr_test.go b/pkg/sync/atomicptrtest/atomicptr_test.go index b458382b1..3262785ce 100644 --- a/pkg/sync/atomicptrtest/atomicptr_test.go +++ b/pkg/sync/atomicptrtest/atomicptr_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/memmove_unsafe.go b/pkg/sync/memmove_unsafe.go index 0c992d5a4..cd7a02dca 100644 --- a/pkg/sync/memmove_unsafe.go +++ b/pkg/sync/memmove_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/norace_unsafe.go b/pkg/sync/norace_unsafe.go index 968665078..1593b9e5d 100644 --- a/pkg/sync/norace_unsafe.go +++ b/pkg/sync/norace_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/race_unsafe.go b/pkg/sync/race_unsafe.go index d143a21c7..473eaddc6 100644 --- a/pkg/sync/race_unsafe.go +++ b/pkg/sync/race_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/seqatomic_unsafe.go b/pkg/sync/seqatomic_unsafe.go index a18e1229a..bea31adc5 100644 --- a/pkg/sync/seqatomic_unsafe.go +++ b/pkg/sync/seqatomic_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/seqatomictest/seqatomic_test.go b/pkg/sync/seqatomictest/seqatomic_test.go index b785d2344..f5e1fbfff 100644 --- a/pkg/sync/seqatomictest/seqatomic_test.go +++ b/pkg/sync/seqatomictest/seqatomic_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/seqcount.go b/pkg/sync/seqcount.go index 8e3304d69..732e856a4 100644 --- a/pkg/sync/seqcount.go +++ b/pkg/sync/seqcount.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/seqcount_test.go b/pkg/sync/seqcount_test.go index fa4abed1d..b14a8878e 100644 --- a/pkg/sync/seqcount_test.go +++ b/pkg/sync/seqcount_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/sync.go b/pkg/sync/sync.go index 36d4c4dee..22c5348d7 100644 --- a/pkg/sync/sync.go +++ b/pkg/sync/sync.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/syserr/host_linux.go b/pkg/syserr/host_linux.go index 22009a799..74bbe9f5b 100644 --- a/pkg/syserr/host_linux.go +++ b/pkg/syserr/host_linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/syserr/netstack.go b/pkg/syserr/netstack.go index b9786b48f..20e756edb 100644 --- a/pkg/syserr/netstack.go +++ b/pkg/syserr/netstack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/syserr/syserr.go b/pkg/syserr/syserr.go index dba6cb7de..6a66e23a2 100644 --- a/pkg/syserr/syserr.go +++ b/pkg/syserr/syserr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/syserror/syserror.go b/pkg/syserror/syserror.go index 5bc74e65e..4228707f4 100644 --- a/pkg/syserror/syserror.go +++ b/pkg/syserror/syserror.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/syserror/syserror_test.go b/pkg/syserror/syserror_test.go index fb7d8d5ee..0f0da5781 100644 --- a/pkg/syserror/syserror_test.go +++ b/pkg/syserror/syserror_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/adapters/gonet/gonet.go b/pkg/tcpip/adapters/gonet/gonet.go index b64dce720..81428770b 100644 --- a/pkg/tcpip/adapters/gonet/gonet.go +++ b/pkg/tcpip/adapters/gonet/gonet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/adapters/gonet/gonet_test.go b/pkg/tcpip/adapters/gonet/gonet_test.go index 79b7c77ee..05a730a05 100644 --- a/pkg/tcpip/adapters/gonet/gonet_test.go +++ b/pkg/tcpip/adapters/gonet/gonet_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/buffer/prependable.go b/pkg/tcpip/buffer/prependable.go index c5dd2819f..d3a9a0f88 100644 --- a/pkg/tcpip/buffer/prependable.go +++ b/pkg/tcpip/buffer/prependable.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/buffer/view.go b/pkg/tcpip/buffer/view.go index cea4e3657..24479ea40 100644 --- a/pkg/tcpip/buffer/view.go +++ b/pkg/tcpip/buffer/view.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/buffer/view_test.go b/pkg/tcpip/buffer/view_test.go index 02c264593..74a0a96fc 100644 --- a/pkg/tcpip/buffer/view_test.go +++ b/pkg/tcpip/buffer/view_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/checker/checker.go b/pkg/tcpip/checker/checker.go index 206531f20..5dfb3ca1d 100644 --- a/pkg/tcpip/checker/checker.go +++ b/pkg/tcpip/checker/checker.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/arp.go b/pkg/tcpip/header/arp.go index ae373f112..22b259ccb 100644 --- a/pkg/tcpip/header/arp.go +++ b/pkg/tcpip/header/arp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/checksum.go b/pkg/tcpip/header/checksum.go index e67c50f50..12f208fde 100644 --- a/pkg/tcpip/header/checksum.go +++ b/pkg/tcpip/header/checksum.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/eth.go b/pkg/tcpip/header/eth.go index 99c29b750..77365bc41 100644 --- a/pkg/tcpip/header/eth.go +++ b/pkg/tcpip/header/eth.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/gue.go b/pkg/tcpip/header/gue.go index aac4593c5..2ad13955a 100644 --- a/pkg/tcpip/header/gue.go +++ b/pkg/tcpip/header/gue.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/icmpv4.go b/pkg/tcpip/header/icmpv4.go index af1e94b7f..3ac89cdae 100644 --- a/pkg/tcpip/header/icmpv4.go +++ b/pkg/tcpip/header/icmpv4.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/icmpv6.go b/pkg/tcpip/header/icmpv6.go index 7d35caff7..e317975e8 100644 --- a/pkg/tcpip/header/icmpv6.go +++ b/pkg/tcpip/header/icmpv6.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/interfaces.go b/pkg/tcpip/header/interfaces.go index 042006983..ac327d8a5 100644 --- a/pkg/tcpip/header/interfaces.go +++ b/pkg/tcpip/header/interfaces.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/ipv4.go b/pkg/tcpip/header/ipv4.go index 29570cc34..1b882d3d8 100644 --- a/pkg/tcpip/header/ipv4.go +++ b/pkg/tcpip/header/ipv4.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/ipv6.go b/pkg/tcpip/header/ipv6.go index 66c778fe1..d985b745d 100644 --- a/pkg/tcpip/header/ipv6.go +++ b/pkg/tcpip/header/ipv6.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/ipv6_fragment.go b/pkg/tcpip/header/ipv6_fragment.go index 44b28b326..e36d5177b 100644 --- a/pkg/tcpip/header/ipv6_fragment.go +++ b/pkg/tcpip/header/ipv6_fragment.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/ipversion_test.go b/pkg/tcpip/header/ipversion_test.go index 3ae9b7e4a..8301ba5cf 100644 --- a/pkg/tcpip/header/ipversion_test.go +++ b/pkg/tcpip/header/ipversion_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/tcp.go b/pkg/tcpip/header/tcp.go index 6689a6dc5..567a21167 100644 --- a/pkg/tcpip/header/tcp.go +++ b/pkg/tcpip/header/tcp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/tcp_test.go b/pkg/tcpip/header/tcp_test.go index 7854d3523..7cd98df3b 100644 --- a/pkg/tcpip/header/tcp_test.go +++ b/pkg/tcpip/header/tcp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/udp.go b/pkg/tcpip/header/udp.go index cf2602e50..31c8ef456 100644 --- a/pkg/tcpip/header/udp.go +++ b/pkg/tcpip/header/udp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/channel/channel.go b/pkg/tcpip/link/channel/channel.go index 113cbbf5e..da34032cc 100644 --- a/pkg/tcpip/link/channel/channel.go +++ b/pkg/tcpip/link/channel/channel.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/fdbased/endpoint.go b/pkg/tcpip/link/fdbased/endpoint.go index ee99ada07..24af428dd 100644 --- a/pkg/tcpip/link/fdbased/endpoint.go +++ b/pkg/tcpip/link/fdbased/endpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/fdbased/endpoint_test.go b/pkg/tcpip/link/fdbased/endpoint_test.go index 52e532ebb..19b007a9e 100644 --- a/pkg/tcpip/link/fdbased/endpoint_test.go +++ b/pkg/tcpip/link/fdbased/endpoint_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/loopback/loopback.go b/pkg/tcpip/link/loopback/loopback.go index fc3f80c01..e6585be66 100644 --- a/pkg/tcpip/link/loopback/loopback.go +++ b/pkg/tcpip/link/loopback/loopback.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/rawfile/blockingpoll_amd64.s b/pkg/tcpip/link/rawfile/blockingpoll_amd64.s index fc5231831..63b8c4451 100644 --- a/pkg/tcpip/link/rawfile/blockingpoll_amd64.s +++ b/pkg/tcpip/link/rawfile/blockingpoll_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/rawfile/blockingpoll_unsafe.go b/pkg/tcpip/link/rawfile/blockingpoll_unsafe.go index a0a9d4acd..6a3e956ad 100644 --- a/pkg/tcpip/link/rawfile/blockingpoll_unsafe.go +++ b/pkg/tcpip/link/rawfile/blockingpoll_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/rawfile/blockingpoll_unsafe_amd64.go b/pkg/tcpip/link/rawfile/blockingpoll_unsafe_amd64.go index 1f143c0db..89a8a9954 100644 --- a/pkg/tcpip/link/rawfile/blockingpoll_unsafe_amd64.go +++ b/pkg/tcpip/link/rawfile/blockingpoll_unsafe_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/rawfile/errors.go b/pkg/tcpip/link/rawfile/errors.go index de7593d9c..f42ff98db 100644 --- a/pkg/tcpip/link/rawfile/errors.go +++ b/pkg/tcpip/link/rawfile/errors.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/rawfile/rawfile_unsafe.go b/pkg/tcpip/link/rawfile/rawfile_unsafe.go index cea3cd6a1..be4a4fa9c 100644 --- a/pkg/tcpip/link/rawfile/rawfile_unsafe.go +++ b/pkg/tcpip/link/rawfile/rawfile_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/pipe/pipe.go b/pkg/tcpip/link/sharedmem/pipe/pipe.go index 1a0edbaba..e014324cc 100644 --- a/pkg/tcpip/link/sharedmem/pipe/pipe.go +++ b/pkg/tcpip/link/sharedmem/pipe/pipe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/pipe/pipe_test.go b/pkg/tcpip/link/sharedmem/pipe/pipe_test.go index db0737c98..30742ccb1 100644 --- a/pkg/tcpip/link/sharedmem/pipe/pipe_test.go +++ b/pkg/tcpip/link/sharedmem/pipe/pipe_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go b/pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go index 480dc4a23..f491d74a2 100644 --- a/pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go +++ b/pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/pipe/rx.go b/pkg/tcpip/link/sharedmem/pipe/rx.go index ff778cecd..8d641c76f 100644 --- a/pkg/tcpip/link/sharedmem/pipe/rx.go +++ b/pkg/tcpip/link/sharedmem/pipe/rx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/pipe/tx.go b/pkg/tcpip/link/sharedmem/pipe/tx.go index 717f5a4b1..e75175d98 100644 --- a/pkg/tcpip/link/sharedmem/pipe/tx.go +++ b/pkg/tcpip/link/sharedmem/pipe/tx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/queue/queue_test.go b/pkg/tcpip/link/sharedmem/queue/queue_test.go index 3d5909cef..391165bc3 100644 --- a/pkg/tcpip/link/sharedmem/queue/queue_test.go +++ b/pkg/tcpip/link/sharedmem/queue/queue_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/queue/rx.go b/pkg/tcpip/link/sharedmem/queue/rx.go index c40d62c33..d3a5da08a 100644 --- a/pkg/tcpip/link/sharedmem/queue/rx.go +++ b/pkg/tcpip/link/sharedmem/queue/rx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/queue/tx.go b/pkg/tcpip/link/sharedmem/queue/tx.go index 39b595e56..845108db1 100644 --- a/pkg/tcpip/link/sharedmem/queue/tx.go +++ b/pkg/tcpip/link/sharedmem/queue/tx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/rx.go b/pkg/tcpip/link/sharedmem/rx.go index b8e39eca1..3eeab769e 100644 --- a/pkg/tcpip/link/sharedmem/rx.go +++ b/pkg/tcpip/link/sharedmem/rx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/sharedmem.go b/pkg/tcpip/link/sharedmem/sharedmem.go index ce6e86767..27d7eb3b9 100644 --- a/pkg/tcpip/link/sharedmem/sharedmem.go +++ b/pkg/tcpip/link/sharedmem/sharedmem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/sharedmem_test.go b/pkg/tcpip/link/sharedmem/sharedmem_test.go index ad987d382..4b8061b13 100644 --- a/pkg/tcpip/link/sharedmem/sharedmem_test.go +++ b/pkg/tcpip/link/sharedmem/sharedmem_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/sharedmem_unsafe.go b/pkg/tcpip/link/sharedmem/sharedmem_unsafe.go index f0be2dc73..b91adbaf7 100644 --- a/pkg/tcpip/link/sharedmem/sharedmem_unsafe.go +++ b/pkg/tcpip/link/sharedmem/sharedmem_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/tx.go b/pkg/tcpip/link/sharedmem/tx.go index 42a21cb43..37da34831 100644 --- a/pkg/tcpip/link/sharedmem/tx.go +++ b/pkg/tcpip/link/sharedmem/tx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sniffer/pcap.go b/pkg/tcpip/link/sniffer/pcap.go index 04f3d494e..3d0d8d852 100644 --- a/pkg/tcpip/link/sniffer/pcap.go +++ b/pkg/tcpip/link/sniffer/pcap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sniffer/sniffer.go b/pkg/tcpip/link/sniffer/sniffer.go index a30e57a32..1bd174bc3 100644 --- a/pkg/tcpip/link/sniffer/sniffer.go +++ b/pkg/tcpip/link/sniffer/sniffer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/tun/tun_unsafe.go b/pkg/tcpip/link/tun/tun_unsafe.go index 1dec41982..e4c589dda 100644 --- a/pkg/tcpip/link/tun/tun_unsafe.go +++ b/pkg/tcpip/link/tun/tun_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/waitable/waitable.go b/pkg/tcpip/link/waitable/waitable.go index ef8c88561..9ffb7b7e9 100644 --- a/pkg/tcpip/link/waitable/waitable.go +++ b/pkg/tcpip/link/waitable/waitable.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/waitable/waitable_test.go b/pkg/tcpip/link/waitable/waitable_test.go index 0a15c40de..5ebe09664 100644 --- a/pkg/tcpip/link/waitable/waitable_test.go +++ b/pkg/tcpip/link/waitable/waitable_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/arp/arp.go b/pkg/tcpip/network/arp/arp.go index 9d0881e11..2e0024925 100644 --- a/pkg/tcpip/network/arp/arp.go +++ b/pkg/tcpip/network/arp/arp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/arp/arp_test.go b/pkg/tcpip/network/arp/arp_test.go index 50628e4a2..5894f9114 100644 --- a/pkg/tcpip/network/arp/arp_test.go +++ b/pkg/tcpip/network/arp/arp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/frag_heap.go b/pkg/tcpip/network/fragmentation/frag_heap.go index 6c7faafe4..55615c8e6 100644 --- a/pkg/tcpip/network/fragmentation/frag_heap.go +++ b/pkg/tcpip/network/fragmentation/frag_heap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/frag_heap_test.go b/pkg/tcpip/network/fragmentation/frag_heap_test.go index a15540634..1b1b72e88 100644 --- a/pkg/tcpip/network/fragmentation/frag_heap_test.go +++ b/pkg/tcpip/network/fragmentation/frag_heap_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/fragmentation.go b/pkg/tcpip/network/fragmentation/fragmentation.go index 885e3cca2..a5dda0398 100644 --- a/pkg/tcpip/network/fragmentation/fragmentation.go +++ b/pkg/tcpip/network/fragmentation/fragmentation.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/fragmentation_test.go b/pkg/tcpip/network/fragmentation/fragmentation_test.go index fc62a15dd..5bf3463a9 100644 --- a/pkg/tcpip/network/fragmentation/fragmentation_test.go +++ b/pkg/tcpip/network/fragmentation/fragmentation_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/reassembler.go b/pkg/tcpip/network/fragmentation/reassembler.go index b57fe82ec..c9ad2bef6 100644 --- a/pkg/tcpip/network/fragmentation/reassembler.go +++ b/pkg/tcpip/network/fragmentation/reassembler.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/reassembler_test.go b/pkg/tcpip/network/fragmentation/reassembler_test.go index 4c137828f..a2bc9707a 100644 --- a/pkg/tcpip/network/fragmentation/reassembler_test.go +++ b/pkg/tcpip/network/fragmentation/reassembler_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/hash/hash.go b/pkg/tcpip/network/hash/hash.go index eddf7ca4d..07960ddf0 100644 --- a/pkg/tcpip/network/hash/hash.go +++ b/pkg/tcpip/network/hash/hash.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ip_test.go b/pkg/tcpip/network/ip_test.go index e3c7af1f9..5c1e88e56 100644 --- a/pkg/tcpip/network/ip_test.go +++ b/pkg/tcpip/network/ip_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv4/icmp.go b/pkg/tcpip/network/ipv4/icmp.go index ee8172ac8..f82dc098f 100644 --- a/pkg/tcpip/network/ipv4/icmp.go +++ b/pkg/tcpip/network/ipv4/icmp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv4/ipv4.go b/pkg/tcpip/network/ipv4/ipv4.go index d4eeeb5d9..d7801ec19 100644 --- a/pkg/tcpip/network/ipv4/ipv4.go +++ b/pkg/tcpip/network/ipv4/ipv4.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv4/ipv4_test.go b/pkg/tcpip/network/ipv4/ipv4_test.go index 2b7067a50..190d548eb 100644 --- a/pkg/tcpip/network/ipv4/ipv4_test.go +++ b/pkg/tcpip/network/ipv4/ipv4_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv6/icmp.go b/pkg/tcpip/network/ipv6/icmp.go index 81aba0923..14107443b 100644 --- a/pkg/tcpip/network/ipv6/icmp.go +++ b/pkg/tcpip/network/ipv6/icmp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv6/icmp_test.go b/pkg/tcpip/network/ipv6/icmp_test.go index fabbdc8c7..12c818b48 100644 --- a/pkg/tcpip/network/ipv6/icmp_test.go +++ b/pkg/tcpip/network/ipv6/icmp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv6/ipv6.go b/pkg/tcpip/network/ipv6/ipv6.go index 25bd998e5..4d0b6ee9c 100644 --- a/pkg/tcpip/network/ipv6/ipv6.go +++ b/pkg/tcpip/network/ipv6/ipv6.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/ports/ports.go b/pkg/tcpip/ports/ports.go index 4e24efddb..41ef32921 100644 --- a/pkg/tcpip/ports/ports.go +++ b/pkg/tcpip/ports/ports.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/ports/ports_test.go b/pkg/tcpip/ports/ports_test.go index 4ab6a1fa2..72577dfcb 100644 --- a/pkg/tcpip/ports/ports_test.go +++ b/pkg/tcpip/ports/ports_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/sample/tun_tcp_connect/main.go b/pkg/tcpip/sample/tun_tcp_connect/main.go index c4707736e..67e8f0b9e 100644 --- a/pkg/tcpip/sample/tun_tcp_connect/main.go +++ b/pkg/tcpip/sample/tun_tcp_connect/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/sample/tun_tcp_echo/main.go b/pkg/tcpip/sample/tun_tcp_echo/main.go index 910d1257f..ab40e9e0b 100644 --- a/pkg/tcpip/sample/tun_tcp_echo/main.go +++ b/pkg/tcpip/sample/tun_tcp_echo/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/seqnum/seqnum.go b/pkg/tcpip/seqnum/seqnum.go index e507d02f7..f2b988839 100644 --- a/pkg/tcpip/seqnum/seqnum.go +++ b/pkg/tcpip/seqnum/seqnum.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/linkaddrcache.go b/pkg/tcpip/stack/linkaddrcache.go index 3a147a75f..cb7b7116b 100644 --- a/pkg/tcpip/stack/linkaddrcache.go +++ b/pkg/tcpip/stack/linkaddrcache.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/linkaddrcache_test.go b/pkg/tcpip/stack/linkaddrcache_test.go index e46267f12..651fa17ac 100644 --- a/pkg/tcpip/stack/linkaddrcache_test.go +++ b/pkg/tcpip/stack/linkaddrcache_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/nic.go b/pkg/tcpip/stack/nic.go index dba95369c..3da99ac67 100644 --- a/pkg/tcpip/stack/nic.go +++ b/pkg/tcpip/stack/nic.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/registration.go b/pkg/tcpip/stack/registration.go index 0acec2984..b6266eb55 100644 --- a/pkg/tcpip/stack/registration.go +++ b/pkg/tcpip/stack/registration.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/route.go b/pkg/tcpip/stack/route.go index 6c6400c33..2b4185014 100644 --- a/pkg/tcpip/stack/route.go +++ b/pkg/tcpip/stack/route.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/stack.go b/pkg/tcpip/stack/stack.go index d1ec6a660..d4da980a9 100644 --- a/pkg/tcpip/stack/stack.go +++ b/pkg/tcpip/stack/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/stack_global_state.go b/pkg/tcpip/stack/stack_global_state.go index b6c095efb..f2c6c9a8d 100644 --- a/pkg/tcpip/stack/stack_global_state.go +++ b/pkg/tcpip/stack/stack_global_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/stack_test.go b/pkg/tcpip/stack/stack_test.go index a0b3399a8..74bf2c99e 100644 --- a/pkg/tcpip/stack/stack_test.go +++ b/pkg/tcpip/stack/stack_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/transport_demuxer.go b/pkg/tcpip/stack/transport_demuxer.go index a7470d606..c8522ad9e 100644 --- a/pkg/tcpip/stack/transport_demuxer.go +++ b/pkg/tcpip/stack/transport_demuxer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/transport_test.go b/pkg/tcpip/stack/transport_test.go index 98cc3b120..f09760180 100644 --- a/pkg/tcpip/stack/transport_test.go +++ b/pkg/tcpip/stack/transport_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/tcpip.go b/pkg/tcpip/tcpip.go index bf11c2175..413aee6c6 100644 --- a/pkg/tcpip/tcpip.go +++ b/pkg/tcpip/tcpip.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/tcpip_test.go b/pkg/tcpip/tcpip_test.go index d283f71c7..361e359d4 100644 --- a/pkg/tcpip/tcpip_test.go +++ b/pkg/tcpip/tcpip_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/time.s b/pkg/tcpip/time.s index 8aca31bee..85d52cd9c 100644 --- a/pkg/tcpip/time.s +++ b/pkg/tcpip/time.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/time_unsafe.go b/pkg/tcpip/time_unsafe.go index 2102e9633..231151bf3 100644 --- a/pkg/tcpip/time_unsafe.go +++ b/pkg/tcpip/time_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/ping/endpoint.go b/pkg/tcpip/transport/ping/endpoint.go index 055daa918..b3f54cfe0 100644 --- a/pkg/tcpip/transport/ping/endpoint.go +++ b/pkg/tcpip/transport/ping/endpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/ping/endpoint_state.go b/pkg/tcpip/transport/ping/endpoint_state.go index a16087304..80721d227 100644 --- a/pkg/tcpip/transport/ping/endpoint_state.go +++ b/pkg/tcpip/transport/ping/endpoint_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/ping/protocol.go b/pkg/tcpip/transport/ping/protocol.go index 549b1b2d3..1d504773b 100644 --- a/pkg/tcpip/transport/ping/protocol.go +++ b/pkg/tcpip/transport/ping/protocol.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/accept.go b/pkg/tcpip/transport/tcp/accept.go index c22ed5ea7..5a88d25d0 100644 --- a/pkg/tcpip/transport/tcp/accept.go +++ b/pkg/tcpip/transport/tcp/accept.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/connect.go b/pkg/tcpip/transport/tcp/connect.go index 27dbcace2..800d2409e 100644 --- a/pkg/tcpip/transport/tcp/connect.go +++ b/pkg/tcpip/transport/tcp/connect.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/cubic.go b/pkg/tcpip/transport/tcp/cubic.go index 8cea416d2..003525d86 100644 --- a/pkg/tcpip/transport/tcp/cubic.go +++ b/pkg/tcpip/transport/tcp/cubic.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/dual_stack_test.go b/pkg/tcpip/transport/tcp/dual_stack_test.go index c88e98977..d3120c1d8 100644 --- a/pkg/tcpip/transport/tcp/dual_stack_test.go +++ b/pkg/tcpip/transport/tcp/dual_stack_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/endpoint.go b/pkg/tcpip/transport/tcp/endpoint.go index 707d6be96..673a65c31 100644 --- a/pkg/tcpip/transport/tcp/endpoint.go +++ b/pkg/tcpip/transport/tcp/endpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/endpoint_state.go b/pkg/tcpip/transport/tcp/endpoint_state.go index bed7ec6a6..e32c73aae 100644 --- a/pkg/tcpip/transport/tcp/endpoint_state.go +++ b/pkg/tcpip/transport/tcp/endpoint_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/forwarder.go b/pkg/tcpip/transport/tcp/forwarder.go index c80f3c7d6..2f90839e9 100644 --- a/pkg/tcpip/transport/tcp/forwarder.go +++ b/pkg/tcpip/transport/tcp/forwarder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/protocol.go b/pkg/tcpip/transport/tcp/protocol.go index abdc825cd..753e1419e 100644 --- a/pkg/tcpip/transport/tcp/protocol.go +++ b/pkg/tcpip/transport/tcp/protocol.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/rcv.go b/pkg/tcpip/transport/tcp/rcv.go index 92ef9c6f7..05ff9e0d7 100644 --- a/pkg/tcpip/transport/tcp/rcv.go +++ b/pkg/tcpip/transport/tcp/rcv.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/reno.go b/pkg/tcpip/transport/tcp/reno.go index feb593234..e4f8b7d5a 100644 --- a/pkg/tcpip/transport/tcp/reno.go +++ b/pkg/tcpip/transport/tcp/reno.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/sack.go b/pkg/tcpip/transport/tcp/sack.go index 05bac08cb..24e48fe7b 100644 --- a/pkg/tcpip/transport/tcp/sack.go +++ b/pkg/tcpip/transport/tcp/sack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/segment.go b/pkg/tcpip/transport/tcp/segment.go index 51a3d6aba..fc87a05fd 100644 --- a/pkg/tcpip/transport/tcp/segment.go +++ b/pkg/tcpip/transport/tcp/segment.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/segment_heap.go b/pkg/tcpip/transport/tcp/segment_heap.go index e3a3405ef..98422fadf 100644 --- a/pkg/tcpip/transport/tcp/segment_heap.go +++ b/pkg/tcpip/transport/tcp/segment_heap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/segment_queue.go b/pkg/tcpip/transport/tcp/segment_queue.go index 6a2d7bc0b..0c637d7ad 100644 --- a/pkg/tcpip/transport/tcp/segment_queue.go +++ b/pkg/tcpip/transport/tcp/segment_queue.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/segment_state.go b/pkg/tcpip/transport/tcp/segment_state.go index 22f0bbf18..46b6d85a6 100644 --- a/pkg/tcpip/transport/tcp/segment_state.go +++ b/pkg/tcpip/transport/tcp/segment_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/snd.go b/pkg/tcpip/transport/tcp/snd.go index 0bd421ff4..eefe93d48 100644 --- a/pkg/tcpip/transport/tcp/snd.go +++ b/pkg/tcpip/transport/tcp/snd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/snd_state.go b/pkg/tcpip/transport/tcp/snd_state.go index d536839af..86bbd643f 100644 --- a/pkg/tcpip/transport/tcp/snd_state.go +++ b/pkg/tcpip/transport/tcp/snd_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/tcp_sack_test.go b/pkg/tcpip/transport/tcp/tcp_sack_test.go index a61d0ca64..06b0702c5 100644 --- a/pkg/tcpip/transport/tcp/tcp_sack_test.go +++ b/pkg/tcpip/transport/tcp/tcp_sack_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/tcp_test.go b/pkg/tcpip/transport/tcp/tcp_test.go index 48852ea47..04e046257 100644 --- a/pkg/tcpip/transport/tcp/tcp_test.go +++ b/pkg/tcpip/transport/tcp/tcp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/tcp_timestamp_test.go b/pkg/tcpip/transport/tcp/tcp_timestamp_test.go index ca16fc8fa..b08df0fec 100644 --- a/pkg/tcpip/transport/tcp/tcp_timestamp_test.go +++ b/pkg/tcpip/transport/tcp/tcp_timestamp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/testing/context/context.go b/pkg/tcpip/transport/tcp/testing/context/context.go index 5b25534f4..0695e8150 100644 --- a/pkg/tcpip/transport/tcp/testing/context/context.go +++ b/pkg/tcpip/transport/tcp/testing/context/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/timer.go b/pkg/tcpip/transport/tcp/timer.go index 938c0bcef..38240d2d5 100644 --- a/pkg/tcpip/transport/tcp/timer.go +++ b/pkg/tcpip/transport/tcp/timer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go b/pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go index 5f8f1a64d..f7b2900de 100644 --- a/pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go +++ b/pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go b/pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go index 514722ab7..aaeae9b18 100644 --- a/pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go +++ b/pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/udp/endpoint.go b/pkg/tcpip/transport/udp/endpoint.go index 840e95302..d777a80d0 100644 --- a/pkg/tcpip/transport/udp/endpoint.go +++ b/pkg/tcpip/transport/udp/endpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/udp/endpoint_state.go b/pkg/tcpip/transport/udp/endpoint_state.go index 70a37c7f2..db1e281ad 100644 --- a/pkg/tcpip/transport/udp/endpoint_state.go +++ b/pkg/tcpip/transport/udp/endpoint_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/udp/protocol.go b/pkg/tcpip/transport/udp/protocol.go index 1334fec8a..b3fbed6e4 100644 --- a/pkg/tcpip/transport/udp/protocol.go +++ b/pkg/tcpip/transport/udp/protocol.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/udp/udp_test.go b/pkg/tcpip/transport/udp/udp_test.go index c3f592bd4..58a346cd9 100644 --- a/pkg/tcpip/transport/udp/udp_test.go +++ b/pkg/tcpip/transport/udp/udp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tmutex/tmutex.go b/pkg/tmutex/tmutex.go index bd5c681dd..df61d89f5 100644 --- a/pkg/tmutex/tmutex.go +++ b/pkg/tmutex/tmutex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tmutex/tmutex_test.go b/pkg/tmutex/tmutex_test.go index a9dc9972f..a4537cb3b 100644 --- a/pkg/tmutex/tmutex_test.go +++ b/pkg/tmutex/tmutex_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/unet/unet.go b/pkg/unet/unet.go index f4800e0d9..deeea078d 100644 --- a/pkg/unet/unet.go +++ b/pkg/unet/unet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/unet/unet_test.go b/pkg/unet/unet_test.go index 6c546825f..ecc670925 100644 --- a/pkg/unet/unet_test.go +++ b/pkg/unet/unet_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/unet/unet_unsafe.go b/pkg/unet/unet_unsafe.go index fa15cf744..1d69de542 100644 --- a/pkg/unet/unet_unsafe.go +++ b/pkg/unet/unet_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/urpc/urpc.go b/pkg/urpc/urpc.go index 1ec06dd4c..753366be2 100644 --- a/pkg/urpc/urpc.go +++ b/pkg/urpc/urpc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/urpc/urpc_test.go b/pkg/urpc/urpc_test.go index d9cfc512e..f1b9a85ca 100644 --- a/pkg/urpc/urpc_test.go +++ b/pkg/urpc/urpc_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/waiter/fdnotifier/fdnotifier.go b/pkg/waiter/fdnotifier/fdnotifier.go index 8bb93e39b..624b1a0c5 100644 --- a/pkg/waiter/fdnotifier/fdnotifier.go +++ b/pkg/waiter/fdnotifier/fdnotifier.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/waiter/fdnotifier/poll_unsafe.go b/pkg/waiter/fdnotifier/poll_unsafe.go index 26bca2b53..8459d4c74 100644 --- a/pkg/waiter/fdnotifier/poll_unsafe.go +++ b/pkg/waiter/fdnotifier/poll_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/waiter/waiter.go b/pkg/waiter/waiter.go index 832b6a5a9..93390b299 100644 --- a/pkg/waiter/waiter.go +++ b/pkg/waiter/waiter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/waiter/waiter_test.go b/pkg/waiter/waiter_test.go index c45f22889..60853f9c1 100644 --- a/pkg/waiter/waiter_test.go +++ b/pkg/waiter/waiter_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/compat.go b/runsc/boot/compat.go index 3250cdcdc..6766953b3 100644 --- a/runsc/boot/compat.go +++ b/runsc/boot/compat.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/config.go b/runsc/boot/config.go index 51d20d06d..9ebbde424 100644 --- a/runsc/boot/config.go +++ b/runsc/boot/config.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/controller.go b/runsc/boot/controller.go index bee82f344..6dd7fadd9 100644 --- a/runsc/boot/controller.go +++ b/runsc/boot/controller.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/debug.go b/runsc/boot/debug.go index 971962c91..d224d08b7 100644 --- a/runsc/boot/debug.go +++ b/runsc/boot/debug.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/events.go b/runsc/boot/events.go index 595846b10..f954b8c0b 100644 --- a/runsc/boot/events.go +++ b/runsc/boot/events.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/fds.go b/runsc/boot/fds.go index 9416e3a5c..a3d21d963 100644 --- a/runsc/boot/fds.go +++ b/runsc/boot/fds.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/filter/config.go b/runsc/boot/filter/config.go index 92a73db9a..378396b9b 100644 --- a/runsc/boot/filter/config.go +++ b/runsc/boot/filter/config.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/filter/extra_filters.go b/runsc/boot/filter/extra_filters.go index 82cf00dfb..67f3101fe 100644 --- a/runsc/boot/filter/extra_filters.go +++ b/runsc/boot/filter/extra_filters.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/filter/extra_filters_msan.go b/runsc/boot/filter/extra_filters_msan.go index 76f3f6865..fb95283ab 100644 --- a/runsc/boot/filter/extra_filters_msan.go +++ b/runsc/boot/filter/extra_filters_msan.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/filter/extra_filters_race.go b/runsc/boot/filter/extra_filters_race.go index ebd56c553..02a122c95 100644 --- a/runsc/boot/filter/extra_filters_race.go +++ b/runsc/boot/filter/extra_filters_race.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/filter/filter.go b/runsc/boot/filter/filter.go index b656883ad..dc7294b1d 100644 --- a/runsc/boot/filter/filter.go +++ b/runsc/boot/filter/filter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/fs.go b/runsc/boot/fs.go index ea825e571..e52c89fe4 100644 --- a/runsc/boot/fs.go +++ b/runsc/boot/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/limits.go b/runsc/boot/limits.go index 510497eba..8ecda6d0e 100644 --- a/runsc/boot/limits.go +++ b/runsc/boot/limits.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/loader.go b/runsc/boot/loader.go index c79b95bde..fa3de0133 100644 --- a/runsc/boot/loader.go +++ b/runsc/boot/loader.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/loader_test.go b/runsc/boot/loader_test.go index 41ff3681b..c342ee005 100644 --- a/runsc/boot/loader_test.go +++ b/runsc/boot/loader_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/network.go b/runsc/boot/network.go index 6a2678ac9..89f186139 100644 --- a/runsc/boot/network.go +++ b/runsc/boot/network.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/strace.go b/runsc/boot/strace.go index 1e898672b..028bcc1f4 100644 --- a/runsc/boot/strace.go +++ b/runsc/boot/strace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cgroup/cgroup.go b/runsc/cgroup/cgroup.go index 7a75a189a..d6058a8a2 100644 --- a/runsc/cgroup/cgroup.go +++ b/runsc/cgroup/cgroup.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cgroup/cgroup_test.go b/runsc/cgroup/cgroup_test.go index cde915329..4a4713d4f 100644 --- a/runsc/cgroup/cgroup_test.go +++ b/runsc/cgroup/cgroup_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/boot.go b/runsc/cmd/boot.go index 023b63dc0..7c14857ba 100644 --- a/runsc/cmd/boot.go +++ b/runsc/cmd/boot.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/capability.go b/runsc/cmd/capability.go index 0b18c5481..e5da021e5 100644 --- a/runsc/cmd/capability.go +++ b/runsc/cmd/capability.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/capability_test.go b/runsc/cmd/capability_test.go index 3329b308d..dd278b32d 100644 --- a/runsc/cmd/capability_test.go +++ b/runsc/cmd/capability_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/checkpoint.go b/runsc/cmd/checkpoint.go index 023ab2455..d49d0169b 100644 --- a/runsc/cmd/checkpoint.go +++ b/runsc/cmd/checkpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/cmd.go b/runsc/cmd/cmd.go index 2937ae1c4..a1c3491a3 100644 --- a/runsc/cmd/cmd.go +++ b/runsc/cmd/cmd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/create.go b/runsc/cmd/create.go index 275a96f57..b84185b43 100644 --- a/runsc/cmd/create.go +++ b/runsc/cmd/create.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/debug.go b/runsc/cmd/debug.go index cb7d81057..288cbe435 100644 --- a/runsc/cmd/debug.go +++ b/runsc/cmd/debug.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/delete.go b/runsc/cmd/delete.go index 92b609c3c..ea1ca1278 100644 --- a/runsc/cmd/delete.go +++ b/runsc/cmd/delete.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/delete_test.go b/runsc/cmd/delete_test.go index f6d164394..4a5b4774a 100644 --- a/runsc/cmd/delete_test.go +++ b/runsc/cmd/delete_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/events.go b/runsc/cmd/events.go index df65ea31d..df03415ec 100644 --- a/runsc/cmd/events.go +++ b/runsc/cmd/events.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/exec.go b/runsc/cmd/exec.go index 336edf3f6..9a395e6f1 100644 --- a/runsc/cmd/exec.go +++ b/runsc/cmd/exec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/exec_test.go b/runsc/cmd/exec_test.go index 623461e78..686c5e150 100644 --- a/runsc/cmd/exec_test.go +++ b/runsc/cmd/exec_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/gofer.go b/runsc/cmd/gofer.go index fd4eee546..3842fdf64 100644 --- a/runsc/cmd/gofer.go +++ b/runsc/cmd/gofer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/kill.go b/runsc/cmd/kill.go index 7a98d10a2..1f1086250 100644 --- a/runsc/cmd/kill.go +++ b/runsc/cmd/kill.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/list.go b/runsc/cmd/list.go index 4d4a5cb0b..fd59b73e6 100644 --- a/runsc/cmd/list.go +++ b/runsc/cmd/list.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/path.go b/runsc/cmd/path.go index c207b80da..baba937a8 100644 --- a/runsc/cmd/path.go +++ b/runsc/cmd/path.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/pause.go b/runsc/cmd/pause.go index ac393b48e..5ff6f059c 100644 --- a/runsc/cmd/pause.go +++ b/runsc/cmd/pause.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/ps.go b/runsc/cmd/ps.go index 5d219bfdc..fd76cf975 100644 --- a/runsc/cmd/ps.go +++ b/runsc/cmd/ps.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/restore.go b/runsc/cmd/restore.go index 6dc044672..cc99b3503 100644 --- a/runsc/cmd/restore.go +++ b/runsc/cmd/restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/resume.go b/runsc/cmd/resume.go index a12adf1a3..274b5d084 100644 --- a/runsc/cmd/resume.go +++ b/runsc/cmd/resume.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/run.go b/runsc/cmd/run.go index 9a87cf240..b6a12f5d6 100644 --- a/runsc/cmd/run.go +++ b/runsc/cmd/run.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/spec.go b/runsc/cmd/spec.go index 6281fc49d..57ee37c86 100644 --- a/runsc/cmd/spec.go +++ b/runsc/cmd/spec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/start.go b/runsc/cmd/start.go index 97ea91fff..48bd4c401 100644 --- a/runsc/cmd/start.go +++ b/runsc/cmd/start.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/state.go b/runsc/cmd/state.go index 265014e1b..f8ce8c3d8 100644 --- a/runsc/cmd/state.go +++ b/runsc/cmd/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/wait.go b/runsc/cmd/wait.go index 956349140..121c54554 100644 --- a/runsc/cmd/wait.go +++ b/runsc/cmd/wait.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/console/console.go b/runsc/console/console.go index 3df184742..9f4f9214d 100644 --- a/runsc/console/console.go +++ b/runsc/console/console.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/console_test.go b/runsc/container/console_test.go index 8f019b54a..0b0dfb4cb 100644 --- a/runsc/container/console_test.go +++ b/runsc/container/console_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/container.go b/runsc/container/container.go index f76bad1aa..cb4c9b5c1 100644 --- a/runsc/container/container.go +++ b/runsc/container/container.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/container_test.go b/runsc/container/container_test.go index 662591b3b..243528d35 100644 --- a/runsc/container/container_test.go +++ b/runsc/container/container_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/fs.go b/runsc/container/fs.go index 2ed42fd93..41022686b 100644 --- a/runsc/container/fs.go +++ b/runsc/container/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/fs_test.go b/runsc/container/fs_test.go index 84bde18fb..87cdb078e 100644 --- a/runsc/container/fs_test.go +++ b/runsc/container/fs_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/hook.go b/runsc/container/hook.go index 3d93ca0be..6b9e5550a 100644 --- a/runsc/container/hook.go +++ b/runsc/container/hook.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/multi_container_test.go b/runsc/container/multi_container_test.go index 1781a4602..4548eb106 100644 --- a/runsc/container/multi_container_test.go +++ b/runsc/container/multi_container_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/status.go b/runsc/container/status.go index bf177e78a..234ffb0dd 100644 --- a/runsc/container/status.go +++ b/runsc/container/status.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/test_app.go b/runsc/container/test_app.go index cc3b087e1..b5071ada6 100644 --- a/runsc/container/test_app.go +++ b/runsc/container/test_app.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/filter/config.go b/runsc/fsgofer/filter/config.go index 35698f21f..75a087848 100644 --- a/runsc/fsgofer/filter/config.go +++ b/runsc/fsgofer/filter/config.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/filter/extra_filters.go b/runsc/fsgofer/filter/extra_filters.go index 82cf00dfb..67f3101fe 100644 --- a/runsc/fsgofer/filter/extra_filters.go +++ b/runsc/fsgofer/filter/extra_filters.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/filter/extra_filters_msan.go b/runsc/fsgofer/filter/extra_filters_msan.go index 169a79ed8..7e142b790 100644 --- a/runsc/fsgofer/filter/extra_filters_msan.go +++ b/runsc/fsgofer/filter/extra_filters_msan.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/filter/extra_filters_race.go b/runsc/fsgofer/filter/extra_filters_race.go index 9e6512d8c..3cd29472a 100644 --- a/runsc/fsgofer/filter/extra_filters_race.go +++ b/runsc/fsgofer/filter/extra_filters_race.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/filter/filter.go b/runsc/fsgofer/filter/filter.go index 6f341f688..f50b6bc87 100644 --- a/runsc/fsgofer/filter/filter.go +++ b/runsc/fsgofer/filter/filter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/fsgofer.go b/runsc/fsgofer/fsgofer.go index 9c4864cf1..e03bb7752 100644 --- a/runsc/fsgofer/fsgofer.go +++ b/runsc/fsgofer/fsgofer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/fsgofer_test.go b/runsc/fsgofer/fsgofer_test.go index a500a2976..48860f952 100644 --- a/runsc/fsgofer/fsgofer_test.go +++ b/runsc/fsgofer/fsgofer_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/fsgofer_unsafe.go b/runsc/fsgofer/fsgofer_unsafe.go index e676809ac..99bc25ec1 100644 --- a/runsc/fsgofer/fsgofer_unsafe.go +++ b/runsc/fsgofer/fsgofer_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/main.go b/runsc/main.go index 62b1f01b3..4a92db7c0 100644 --- a/runsc/main.go +++ b/runsc/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/sandbox/chroot.go b/runsc/sandbox/chroot.go index 35b19a0b1..354049871 100644 --- a/runsc/sandbox/chroot.go +++ b/runsc/sandbox/chroot.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/sandbox/network.go b/runsc/sandbox/network.go index 86a52c6ae..52fe8fc0f 100644 --- a/runsc/sandbox/network.go +++ b/runsc/sandbox/network.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/sandbox/sandbox.go b/runsc/sandbox/sandbox.go index 923a52f7f..0fe85cfe1 100644 --- a/runsc/sandbox/sandbox.go +++ b/runsc/sandbox/sandbox.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/specutils/namespace.go b/runsc/specutils/namespace.go index 00293d45b..73fab13e1 100644 --- a/runsc/specutils/namespace.go +++ b/runsc/specutils/namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/specutils/specutils.go b/runsc/specutils/specutils.go index b29802fde..ab14ed1fc 100644 --- a/runsc/specutils/specutils.go +++ b/runsc/specutils/specutils.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/specutils/specutils_test.go b/runsc/specutils/specutils_test.go index 64e2172c8..b61f1ca62 100644 --- a/runsc/specutils/specutils_test.go +++ b/runsc/specutils/specutils_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/image/image.go b/runsc/test/image/image.go index 069d08013..bcb6f876f 100644 --- a/runsc/test/image/image.go +++ b/runsc/test/image/image.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/image/image_test.go b/runsc/test/image/image_test.go index d89d80a86..763152b47 100644 --- a/runsc/test/image/image_test.go +++ b/runsc/test/image/image_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/image/mysql.sql b/runsc/test/image/mysql.sql index dd5bfaa4e..c1271e719 100644 --- a/runsc/test/image/mysql.sql +++ b/runsc/test/image/mysql.sql @@ -1,4 +1,4 @@ -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/runsc/test/image/ruby.rb b/runsc/test/image/ruby.rb index ae5de3419..25d1ac129 100644 --- a/runsc/test/image/ruby.rb +++ b/runsc/test/image/ruby.rb @@ -1,4 +1,4 @@ -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/runsc/test/image/ruby.sh b/runsc/test/image/ruby.sh index 54be2c931..d3a9b5656 100644 --- a/runsc/test/image/ruby.sh +++ b/runsc/test/image/ruby.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/runsc/test/install.sh b/runsc/test/install.sh index c239588d4..32e1e884e 100755 --- a/runsc/test/install.sh +++ b/runsc/test/install.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/runsc/test/integration/exec_test.go b/runsc/test/integration/exec_test.go index 3cac674d0..fac8337f4 100644 --- a/runsc/test/integration/exec_test.go +++ b/runsc/test/integration/exec_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/integration/integration.go b/runsc/test/integration/integration.go index 49c3c893a..e15321c87 100644 --- a/runsc/test/integration/integration.go +++ b/runsc/test/integration/integration.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/integration/integration_test.go b/runsc/test/integration/integration_test.go index 536bb17e0..526b3a7a1 100644 --- a/runsc/test/integration/integration_test.go +++ b/runsc/test/integration/integration_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/root/cgroup_test.go b/runsc/test/root/cgroup_test.go index 5cb4b794f..fdb94ff64 100644 --- a/runsc/test/root/cgroup_test.go +++ b/runsc/test/root/cgroup_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/root/chroot_test.go b/runsc/test/root/chroot_test.go index 8831e6a78..0ffaaf87b 100644 --- a/runsc/test/root/chroot_test.go +++ b/runsc/test/root/chroot_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/root/root.go b/runsc/test/root/root.go index 790f62c29..586ea0fe3 100644 --- a/runsc/test/root/root.go +++ b/runsc/test/root/root.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/testutil/docker.go b/runsc/test/testutil/docker.go index 7d6a72e5f..3f74e0770 100644 --- a/runsc/test/testutil/docker.go +++ b/runsc/test/testutil/docker.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/testutil/testutil.go b/runsc/test/testutil/testutil.go index 4d7ac3bc9..1b5a02c0f 100644 --- a/runsc/test/testutil/testutil.go +++ b/runsc/test/testutil/testutil.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/testutil/testutil_race.go b/runsc/test/testutil/testutil_race.go index 59cfdaa7b..9267af150 100644 --- a/runsc/test/testutil/testutil_race.go +++ b/runsc/test/testutil/testutil_race.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/tools/dockercfg/dockercfg.go b/runsc/tools/dockercfg/dockercfg.go index 0bd6cad93..110a581ff 100644 --- a/runsc/tools/dockercfg/dockercfg.go +++ b/runsc/tools/dockercfg/dockercfg.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics.go b/tools/go_generics/generics.go index cc61a7537..eaf5c4970 100644 --- a/tools/go_generics/generics.go +++ b/tools/go_generics/generics.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/all_stmts/input.go b/tools/go_generics/generics_tests/all_stmts/input.go index 870af3b6c..19184a3fe 100644 --- a/tools/go_generics/generics_tests/all_stmts/input.go +++ b/tools/go_generics/generics_tests/all_stmts/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/all_stmts/output/output.go b/tools/go_generics/generics_tests/all_stmts/output/output.go index e4e670bf1..51582346c 100644 --- a/tools/go_generics/generics_tests/all_stmts/output/output.go +++ b/tools/go_generics/generics_tests/all_stmts/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/all_types/input.go b/tools/go_generics/generics_tests/all_types/input.go index 3a8643e3d..ed6e97c29 100644 --- a/tools/go_generics/generics_tests/all_types/input.go +++ b/tools/go_generics/generics_tests/all_types/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/all_types/lib/lib.go b/tools/go_generics/generics_tests/all_types/lib/lib.go index d3911d12d..7e73e678e 100644 --- a/tools/go_generics/generics_tests/all_types/lib/lib.go +++ b/tools/go_generics/generics_tests/all_types/lib/lib.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/all_types/output/output.go b/tools/go_generics/generics_tests/all_types/output/output.go index b89840936..ec09a6be4 100644 --- a/tools/go_generics/generics_tests/all_types/output/output.go +++ b/tools/go_generics/generics_tests/all_types/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/consts/input.go b/tools/go_generics/generics_tests/consts/input.go index dabf76e1e..394bcc262 100644 --- a/tools/go_generics/generics_tests/consts/input.go +++ b/tools/go_generics/generics_tests/consts/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/consts/output/output.go b/tools/go_generics/generics_tests/consts/output/output.go index 72865607e..91a07fdc2 100644 --- a/tools/go_generics/generics_tests/consts/output/output.go +++ b/tools/go_generics/generics_tests/consts/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/imports/input.go b/tools/go_generics/generics_tests/imports/input.go index 66b43fee5..22e6641a6 100644 --- a/tools/go_generics/generics_tests/imports/input.go +++ b/tools/go_generics/generics_tests/imports/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/imports/output/output.go b/tools/go_generics/generics_tests/imports/output/output.go index 5f20d43ce..2555c0004 100644 --- a/tools/go_generics/generics_tests/imports/output/output.go +++ b/tools/go_generics/generics_tests/imports/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/remove_typedef/input.go b/tools/go_generics/generics_tests/remove_typedef/input.go index c02307d32..d9c9b8530 100644 --- a/tools/go_generics/generics_tests/remove_typedef/input.go +++ b/tools/go_generics/generics_tests/remove_typedef/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/remove_typedef/output/output.go b/tools/go_generics/generics_tests/remove_typedef/output/output.go index d20a89abd..f111a9426 100644 --- a/tools/go_generics/generics_tests/remove_typedef/output/output.go +++ b/tools/go_generics/generics_tests/remove_typedef/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/simple/input.go b/tools/go_generics/generics_tests/simple/input.go index 670161d6e..711687cf5 100644 --- a/tools/go_generics/generics_tests/simple/input.go +++ b/tools/go_generics/generics_tests/simple/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/simple/output/output.go b/tools/go_generics/generics_tests/simple/output/output.go index 75b5467cd..139c9bf9d 100644 --- a/tools/go_generics/generics_tests/simple/output/output.go +++ b/tools/go_generics/generics_tests/simple/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/globals/globals_visitor.go b/tools/go_generics/globals/globals_visitor.go index fc0de4381..daaa17b1d 100644 --- a/tools/go_generics/globals/globals_visitor.go +++ b/tools/go_generics/globals/globals_visitor.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/globals/scope.go b/tools/go_generics/globals/scope.go index 18743bdee..b75a91689 100644 --- a/tools/go_generics/globals/scope.go +++ b/tools/go_generics/globals/scope.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/go_generics_unittest.sh b/tools/go_generics/go_generics_unittest.sh index 699e1f631..e7553a071 100755 --- a/tools/go_generics/go_generics_unittest.sh +++ b/tools/go_generics/go_generics_unittest.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/tools/go_generics/imports.go b/tools/go_generics/imports.go index 97267098b..57f7c3dce 100644 --- a/tools/go_generics/imports.go +++ b/tools/go_generics/imports.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/merge.go b/tools/go_generics/merge.go index ebe7cf4e4..2f83facf8 100644 --- a/tools/go_generics/merge.go +++ b/tools/go_generics/merge.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/remove.go b/tools/go_generics/remove.go index 2a66de762..139d03955 100644 --- a/tools/go_generics/remove.go +++ b/tools/go_generics/remove.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/rules_tests/template.go b/tools/go_generics/rules_tests/template.go index 73c024f0e..f3f31ae8e 100644 --- a/tools/go_generics/rules_tests/template.go +++ b/tools/go_generics/rules_tests/template.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/rules_tests/template_test.go b/tools/go_generics/rules_tests/template_test.go index 76c4cdb64..3a38c8629 100644 --- a/tools/go_generics/rules_tests/template_test.go +++ b/tools/go_generics/rules_tests/template_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_stateify/main.go b/tools/go_stateify/main.go index 5646b879a..9e2c8e106 100644 --- a/tools/go_stateify/main.go +++ b/tools/go_stateify/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/workspace_status.sh b/tools/workspace_status.sh index d89db1f99..7d44dad37 100755 --- a/tools/workspace_status.sh +++ b/tools/workspace_status.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/vdso/barrier.h b/vdso/barrier.h index db8185b2e..7866af414 100644 --- a/vdso/barrier.h +++ b/vdso/barrier.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/check_vdso.py b/vdso/check_vdso.py index 9a3142ab8..6f7d7e7ec 100644 --- a/vdso/check_vdso.py +++ b/vdso/check_vdso.py @@ -1,4 +1,4 @@ -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/vdso/compiler.h b/vdso/compiler.h index a661516c3..d65f148fb 100644 --- a/vdso/compiler.h +++ b/vdso/compiler.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/cycle_clock.h b/vdso/cycle_clock.h index 93c5f2c0d..dfb5b427d 100644 --- a/vdso/cycle_clock.h +++ b/vdso/cycle_clock.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/seqlock.h b/vdso/seqlock.h index b527bdbca..ab2f3fda3 100644 --- a/vdso/seqlock.h +++ b/vdso/seqlock.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/syscalls.h b/vdso/syscalls.h index fd79c4642..0be8a7f9b 100644 --- a/vdso/syscalls.h +++ b/vdso/syscalls.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/vdso.cc b/vdso/vdso.cc index db3bdef01..f30dc26a2 100644 --- a/vdso/vdso.cc +++ b/vdso/vdso.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/vdso_time.cc b/vdso/vdso_time.cc index 5d5c8de65..a59771bff 100644 --- a/vdso/vdso_time.cc +++ b/vdso/vdso_time.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/vdso_time.h b/vdso/vdso_time.h index 71d6e2f64..464dadff2 100644 --- a/vdso/vdso_time.h +++ b/vdso/vdso_time.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. -- cgit v1.2.3 From 75cd70ecc9abfd5daaefea04da5070a0e0d620dd Mon Sep 17 00:00:00 2001 From: Adin Scannell Date: Tue, 23 Oct 2018 00:19:11 -0700 Subject: Track paths and provide a rename hook. This change also adds extensive testing to the p9 package via mocks. The sanity checks and type checks are moved from the gofer into the core package, where they can be more easily validated. PiperOrigin-RevId: 218296768 Change-Id: I4fc3c326e7bf1e0e140a454cbacbcc6fd617ab55 --- WORKSPACE | 20 +- pkg/amutex/BUILD | 4 +- pkg/atomicbitops/BUILD | 4 +- pkg/binary/BUILD | 4 +- pkg/bits/BUILD | 3 +- pkg/compressio/BUILD | 4 +- pkg/control/client/BUILD | 4 +- pkg/control/server/BUILD | 4 +- pkg/dhcp/BUILD | 4 +- pkg/eventchannel/BUILD | 4 +- pkg/fd/BUILD | 4 +- pkg/gate/BUILD | 4 +- pkg/ilist/BUILD | 4 +- pkg/linewriter/BUILD | 4 +- pkg/log/BUILD | 4 +- pkg/metric/BUILD | 4 +- pkg/p9/BUILD | 2 + pkg/p9/buffer_test.go | 31 + pkg/p9/client.go | 6 + pkg/p9/client_file.go | 4 +- pkg/p9/file.go | 151 +- pkg/p9/handlers.go | 697 ++++++-- pkg/p9/local_server/BUILD | 4 +- pkg/p9/local_server/local_server.go | 5 + pkg/p9/messages_test.go | 37 + pkg/p9/p9.go | 24 + pkg/p9/p9test/BUILD | 76 +- pkg/p9/p9test/client_test.go | 2263 ++++++++++++++++++++++--- pkg/p9/p9test/mocks.go | 489 ------ pkg/p9/p9test/p9test.go | 329 ++++ pkg/p9/path_tree.go | 109 ++ pkg/p9/server.go | 228 ++- pkg/p9/transport.go | 10 +- pkg/rand/BUILD | 4 +- pkg/seccomp/BUILD | 4 +- pkg/secio/BUILD | 4 +- pkg/sentry/arch/BUILD | 3 +- pkg/sentry/context/BUILD | 4 +- pkg/sentry/control/BUILD | 4 +- pkg/sentry/device/BUILD | 4 +- pkg/sentry/fs/anon/BUILD | 4 +- pkg/sentry/fs/gofer/BUILD | 4 - pkg/sentry/fs/gofer/context_file.go | 7 - pkg/sentry/fs/gofer/gofer_test.go | 894 ++-------- pkg/sentry/fs/gofer/session.go | 9 +- pkg/sentry/fs/gofer/session_state.go | 4 +- pkg/sentry/fs/proc/device/BUILD | 4 +- pkg/sentry/hostcpu/BUILD | 4 +- pkg/sentry/kernel/kdefs/BUILD | 4 +- pkg/sentry/kernel/memevent/BUILD | 4 +- pkg/sentry/kernel/sched/BUILD | 4 +- pkg/sentry/loader/BUILD | 3 +- pkg/sentry/memutil/BUILD | 4 +- pkg/sentry/platform/interrupt/BUILD | 4 +- pkg/sentry/platform/kvm/BUILD | 3 +- pkg/sentry/platform/kvm/testutil/BUILD | 4 +- pkg/sentry/platform/procid/BUILD | 4 +- pkg/sentry/platform/ptrace/BUILD | 4 +- pkg/sentry/platform/ring0/BUILD | 3 +- pkg/sentry/platform/ring0/gen_offsets/BUILD | 3 +- pkg/sentry/platform/ring0/pagetables/BUILD | 3 +- pkg/sentry/platform/safecopy/BUILD | 4 +- pkg/sentry/safemem/BUILD | 4 +- pkg/sentry/sighandling/BUILD | 4 +- pkg/sentry/socket/rpcinet/BUILD | 4 +- pkg/sentry/socket/rpcinet/conn/BUILD | 4 +- pkg/sentry/socket/rpcinet/notifier/BUILD | 4 +- pkg/sentry/state/BUILD | 4 +- pkg/sentry/strace/BUILD | 4 +- pkg/sentry/syscalls/BUILD | 4 +- pkg/sentry/time/BUILD | 3 +- pkg/sentry/unimpl/BUILD | 4 +- pkg/sentry/uniqueid/BUILD | 4 +- pkg/sentry/watchdog/BUILD | 4 +- pkg/sleep/BUILD | 4 +- pkg/state/BUILD | 5 +- pkg/state/statefile/BUILD | 4 +- pkg/sync/atomicptrtest/BUILD | 3 +- pkg/sync/seqatomictest/BUILD | 3 +- pkg/syserr/BUILD | 4 +- pkg/syserror/BUILD | 4 +- pkg/tcpip/adapters/gonet/BUILD | 4 +- pkg/tcpip/checker/BUILD | 4 +- pkg/tcpip/link/channel/BUILD | 4 +- pkg/tcpip/link/fdbased/BUILD | 4 +- pkg/tcpip/link/loopback/BUILD | 4 +- pkg/tcpip/link/rawfile/BUILD | 4 +- pkg/tcpip/link/sharedmem/BUILD | 4 +- pkg/tcpip/link/sharedmem/pipe/BUILD | 4 +- pkg/tcpip/link/sharedmem/queue/BUILD | 4 +- pkg/tcpip/link/sniffer/BUILD | 4 +- pkg/tcpip/link/tun/BUILD | 4 +- pkg/tcpip/link/waitable/BUILD | 4 +- pkg/tcpip/network/BUILD | 4 +- pkg/tcpip/network/arp/BUILD | 4 +- pkg/tcpip/network/hash/BUILD | 4 +- pkg/tcpip/network/ipv4/BUILD | 4 +- pkg/tcpip/network/ipv6/BUILD | 4 +- pkg/tcpip/ports/BUILD | 4 +- pkg/tcpip/sample/tun_tcp_connect/BUILD | 4 +- pkg/tcpip/sample/tun_tcp_echo/BUILD | 4 +- pkg/tcpip/transport/tcp/testing/context/BUILD | 4 +- pkg/tcpip/transport/tcpconntrack/BUILD | 4 +- pkg/tmutex/BUILD | 4 +- pkg/unet/BUILD | 4 +- pkg/urpc/BUILD | 4 +- pkg/waiter/fdnotifier/BUILD | 4 +- runsc/boot/BUILD | 4 +- runsc/boot/filter/BUILD | 4 +- runsc/cgroup/BUILD | 4 +- runsc/cmd/BUILD | 4 +- runsc/console/BUILD | 4 +- runsc/container/BUILD | 4 +- runsc/fsgofer/BUILD | 4 +- runsc/fsgofer/filter/BUILD | 4 +- runsc/fsgofer/fsgofer.go | 98 +- runsc/fsgofer/fsgofer_test.go | 78 +- runsc/sandbox/BUILD | 4 +- runsc/specutils/BUILD | 4 +- runsc/test/image/BUILD | 4 +- runsc/test/integration/BUILD | 4 +- runsc/test/root/BUILD | 4 +- runsc/test/testutil/BUILD | 4 +- runsc/tools/dockercfg/BUILD | 4 +- tools/go_generics/BUILD | 4 +- tools/go_generics/globals/BUILD | 4 +- tools/go_generics/rules_tests/BUILD | 3 +- tools/go_stateify/BUILD | 4 +- 128 files changed, 3834 insertions(+), 2147 deletions(-) create mode 100644 pkg/p9/buffer_test.go delete mode 100644 pkg/p9/p9test/mocks.go create mode 100644 pkg/p9/p9test/p9test.go create mode 100644 pkg/p9/path_tree.go (limited to 'pkg/state') diff --git a/WORKSPACE b/WORKSPACE index 48e0d3436..841a23e06 100644 --- a/WORKSPACE +++ b/WORKSPACE @@ -15,7 +15,7 @@ go_register_toolchains(go_version="1.11.1") load("@bazel_gazelle//:deps.bzl", "gazelle_dependencies", "go_repository") gazelle_dependencies() -# Add dependencies on external repositories. +# External repositories, in sorted order. go_repository( name = "com_github_cenkalti_backoff", importpath = "github.com/cenkalti/backoff", @@ -28,6 +28,12 @@ go_repository( commit = "886344bea0798d02ff3fae16a922be5f6b26cee0" ) +go_repository( + name = "com_github_golang_mock", + importpath = "github.com/golang/mock", + commit = "600781dde9cca80734169b9e969d9054ccc57937", +) + go_repository( name = "com_github_google_go-cmp", importpath = "github.com/google/go-cmp", @@ -58,6 +64,12 @@ go_repository( commit = "b2d941ef6a780da2d9982c1fb28d77ad97f54fc7", ) +go_repository( + name = "com_github_syndtr_gocapability", + importpath = "github.com/syndtr/gocapability", + commit = "d98352740cb2c55f81556b63d4a1ec64c5a319c2", +) + go_repository( name = "com_github_vishvananda_netlink", importpath = "github.com/vishvananda/netlink", @@ -81,9 +93,3 @@ go_repository( importpath = "golang.org/x/sys", commit = "0dd5e194bbf5eb84a39666eb4c98a4d007e4203a", ) - -go_repository( - name = "com_github_syndtr_gocapability", - importpath = "github.com/syndtr/gocapability", - commit = "d98352740cb2c55f81556b63d4a1ec64c5a319c2", -) diff --git a/pkg/amutex/BUILD b/pkg/amutex/BUILD index 84e6b79a5..815ee3a69 100644 --- a/pkg/amutex/BUILD +++ b/pkg/amutex/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "amutex", srcs = ["amutex.go"], diff --git a/pkg/atomicbitops/BUILD b/pkg/atomicbitops/BUILD index a8dd17825..235188531 100644 --- a/pkg/atomicbitops/BUILD +++ b/pkg/atomicbitops/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "atomicbitops", srcs = [ diff --git a/pkg/binary/BUILD b/pkg/binary/BUILD index 586d05634..571151f72 100644 --- a/pkg/binary/BUILD +++ b/pkg/binary/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "binary", srcs = ["binary.go"], diff --git a/pkg/bits/BUILD b/pkg/bits/BUILD index 8c943b615..46794bdb8 100644 --- a/pkg/bits/BUILD +++ b/pkg/bits/BUILD @@ -1,6 +1,7 @@ +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") + package(licenses = ["notice"]) # Apache 2.0 -load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") go_library( diff --git a/pkg/compressio/BUILD b/pkg/compressio/BUILD index d70f982c1..72952d735 100644 --- a/pkg/compressio/BUILD +++ b/pkg/compressio/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "compressio", srcs = ["compressio.go"], diff --git a/pkg/control/client/BUILD b/pkg/control/client/BUILD index d58cd1b71..32853875d 100644 --- a/pkg/control/client/BUILD +++ b/pkg/control/client/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "client", srcs = [ diff --git a/pkg/control/server/BUILD b/pkg/control/server/BUILD index c3f74a532..ba2b1be9f 100644 --- a/pkg/control/server/BUILD +++ b/pkg/control/server/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "server", srcs = ["server.go"], diff --git a/pkg/dhcp/BUILD b/pkg/dhcp/BUILD index 711a72c99..c97dfc14b 100644 --- a/pkg/dhcp/BUILD +++ b/pkg/dhcp/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "dhcp", srcs = [ diff --git a/pkg/eventchannel/BUILD b/pkg/eventchannel/BUILD index 9d531ce12..18348ef54 100644 --- a/pkg/eventchannel/BUILD +++ b/pkg/eventchannel/BUILD @@ -1,8 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "eventchannel", srcs = [ diff --git a/pkg/fd/BUILD b/pkg/fd/BUILD index 435b6fa34..06cfd445e 100644 --- a/pkg/fd/BUILD +++ b/pkg/fd/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "fd", srcs = ["fd.go"], diff --git a/pkg/gate/BUILD b/pkg/gate/BUILD index 872eff531..9a87a3a31 100644 --- a/pkg/gate/BUILD +++ b/pkg/gate/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "gate", srcs = [ diff --git a/pkg/ilist/BUILD b/pkg/ilist/BUILD index 1bd71b800..a67aa2cff 100644 --- a/pkg/ilist/BUILD +++ b/pkg/ilist/BUILD @@ -1,8 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "ilist", srcs = [ diff --git a/pkg/linewriter/BUILD b/pkg/linewriter/BUILD index 6c3795432..3f28ba867 100644 --- a/pkg/linewriter/BUILD +++ b/pkg/linewriter/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "linewriter", srcs = ["linewriter.go"], diff --git a/pkg/log/BUILD b/pkg/log/BUILD index fc9281079..bf85b4494 100644 --- a/pkg/log/BUILD +++ b/pkg/log/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "log", srcs = [ diff --git a/pkg/metric/BUILD b/pkg/metric/BUILD index c0cd40c7b..d96e5563b 100644 --- a/pkg/metric/BUILD +++ b/pkg/metric/BUILD @@ -1,8 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "metric", srcs = ["metric.go"], diff --git a/pkg/p9/BUILD b/pkg/p9/BUILD index 1cf5c6458..2c224e65b 100644 --- a/pkg/p9/BUILD +++ b/pkg/p9/BUILD @@ -15,6 +15,7 @@ go_library( "handlers.go", "messages.go", "p9.go", + "path_tree.go", "pool.go", "server.go", "transport.go", @@ -32,6 +33,7 @@ go_test( name = "p9_test", size = "small", srcs = [ + "buffer_test.go", "client_test.go", "messages_test.go", "p9_test.go", diff --git a/pkg/p9/buffer_test.go b/pkg/p9/buffer_test.go new file mode 100644 index 000000000..97eceefa7 --- /dev/null +++ b/pkg/p9/buffer_test.go @@ -0,0 +1,31 @@ +// Copyright 2018 Google Inc. +// +// 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 +// +// 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 p9 + +import ( + "testing" +) + +func TestBufferOverrun(t *testing.T) { + buf := &buffer{ + // This header indicates that a large string should follow, but + // it is only two bytes. Reading a string should cause an + // overrun. + data: []byte{0x0, 0x16}, + } + if s := buf.ReadString(); s != "" { + t.Errorf("overrun read got %s, want empty", s) + } +} diff --git a/pkg/p9/client.go b/pkg/p9/client.go index 3ebfab82a..67887874a 100644 --- a/pkg/p9/client.go +++ b/pkg/p9/client.go @@ -116,6 +116,7 @@ func NewClient(socket *unet.Socket, messageSize uint32, version string) (*Client msize: largestFixedSize, } } + // Compute a payload size and round to 512 (normal block size) // if it's larger than a single block. payloadSize := messageSize - largestFixedSize @@ -299,3 +300,8 @@ func (c *Client) sendRecv(t message, r message) error { func (c *Client) Version() uint32 { return c.version } + +// Close closes the underlying socket. +func (c *Client) Close() error { + return c.socket.Close() +} diff --git a/pkg/p9/client_file.go b/pkg/p9/client_file.go index 066639fda..992d1daf7 100644 --- a/pkg/p9/client_file.go +++ b/pkg/p9/client_file.go @@ -172,6 +172,9 @@ func (c *clientFile) SetAttr(valid SetAttrMask, attr SetAttr) error { } // Remove implements File.Remove. +// +// N.B. This method is no longer part of the file interface and should be +// considered deprecated. func (c *clientFile) Remove() error { // Avoid double close. if !atomic.CompareAndSwapUint32(&c.closed, 0, 1) { @@ -181,7 +184,6 @@ func (c *clientFile) Remove() error { // Send the remove message. if err := c.client.sendRecv(&Tremove{FID: c.fid}, &Rremove{}); err != nil { - log.Warningf("Tremove failed, losing FID %v: %v", c.fid, err) return err } diff --git a/pkg/p9/file.go b/pkg/p9/file.go index d2e89e373..55ceb52e1 100644 --- a/pkg/p9/file.go +++ b/pkg/p9/file.go @@ -31,35 +31,63 @@ type Attacher interface { // File is a set of operations corresponding to a single node. // -// Functions below MUST return syscall.Errno values. -// TODO: Enforce that with the type. +// Note that on the server side, the server logic places constraints on +// concurrent operations to make things easier. This may reduce the need for +// complex, error-prone locking and logic in the backend. These are documented +// for each method. // -// These must be implemented in all circumstances. +// There are three different types of guarantees provided: +// +// none: There is no concurrency guarantee. The method may be invoked +// concurrently with any other method on any other file. +// +// read: The method is guaranteed to be exclusive of any write or global +// operation that is mutating the state of the directory tree starting at this +// node. For example, this means creating new files, symlinks, directories or +// renaming a directory entry (or renaming in to this target), but the method +// may be called concurrently with other read methods. +// +// write: The method is guaranteed to be exclusive of any read, write or global +// operation that is mutating the state of the directory tree starting at this +// node, as described in read above. There may however, be other write +// operations executing concurrently on other components in the directory tree. +// +// global: The method is guaranteed to be exclusive of any read, write or +// global operation. type File interface { // Walk walks to the path components given in names. // // Walk returns QIDs in the same order that the names were passed in. // // An empty list of arguments should return a copy of the current file. + // + // On the server, Walk has a read concurrency guarantee. Walk(names []string) ([]QID, File, error) + // WalkGetAttr walks to the next file and returns its maximal set of + // attributes. + // + // Server-side p9.Files may return syscall.ENOSYS to indicate that Walk + // and GetAttr should be used separately to satisfy this request. + // + // On the server, WalkGetAttr has a read concurrency guarantee. + WalkGetAttr([]string) ([]QID, File, AttrMask, Attr, error) + // StatFS returns information about the file system associated with // this file. + // + // On the server, StatFS has no concurrency guarantee. StatFS() (FSStat, error) // GetAttr returns attributes of this node. + // + // On the server, GetAttr has a read concurrency guarantee. GetAttr(req AttrMask) (QID, AttrMask, Attr, error) // SetAttr sets attributes on this node. - SetAttr(valid SetAttrMask, attr SetAttr) error - - // Remove removes the file. // - // This is deprecated in favor of UnlinkAt below. - Remove() error - - // Rename renames the file. - Rename(directory File, name string) error + // On the server, SetAttr has a write concurrency guarantee. + SetAttr(valid SetAttrMask, attr SetAttr) error // Close is called when all references are dropped on the server side, // and Close should be called by the client to drop all references. @@ -67,65 +95,93 @@ type File interface { // For server-side implementations of Close, the error is ignored. // // Close must be called even when Open has not been called. + // + // On the server, Close has no concurrency guarantee. Close() error - // Open is called prior to using read/write. + // Open must be called prior to using Read, Write or Readdir. Once Open + // is called, some operations, such as Walk, will no longer work. // - // The *fd.FD may be nil. If an *fd.FD is provided, ownership now - // belongs to the caller and the FD must be non-blocking. + // On the client, Open should be called only once. The fd return is + // optional, and may be nil. // - // If Open returns a non-nil *fd.FD, it should do so for all possible - // OpenFlags. If Open returns a nil *fd.FD, it should similarly return - // a nil *fd.FD for all possible OpenFlags. + // On the server, Open has a read concurrency guarantee. If an *fd.FD + // is provided, ownership now belongs to the caller. Open is guaranteed + // to be called only once. // - // This can be assumed to be one-shot only. + // N.B. The server must resolve any lazy paths when open is called. + // After this point, read and write may be called on files with no + // deletion check, so resolving in the data path is not viable. Open(mode OpenFlags) (*fd.FD, QID, uint32, error) - // Read reads from this file. + // Read reads from this file. Open must be called first. // // This may return io.EOF in addition to syscall.Errno values. // - // Preconditions: Open has been called and returned success. + // On the server, ReadAt has a read concurrency guarantee. See Open for + // additional requirements regarding lazy path resolution. ReadAt(p []byte, offset uint64) (int, error) - // Write writes to this file. + // Write writes to this file. Open must be called first. // // This may return io.EOF in addition to syscall.Errno values. // - // Preconditions: Open has been called and returned success. + // On the server, WriteAt has a read concurrency guarantee. See Open + // for additional requirements regarding lazy path resolution. WriteAt(p []byte, offset uint64) (int, error) - // FSync syncs this node. + // FSync syncs this node. Open must be called first. // - // Preconditions: Open has been called and returned success. + // On the server, FSync has a read concurrency guarantee. FSync() error // Create creates a new regular file and opens it according to the - // flags given. + // flags given. This file is already Open. + // + // N.B. On the client, the returned file is a reference to the current + // file, which now represents the created file. This is not the case on + // the server. These semantics are very subtle and can easily lead to + // bugs, but are a consequence of the 9P create operation. // // See p9.File.Open for a description of *fd.FD. + // + // On the server, Create has a write concurrency guarantee. Create(name string, flags OpenFlags, permissions FileMode, uid UID, gid GID) (*fd.FD, File, QID, uint32, error) // Mkdir creates a subdirectory. + // + // On the server, Mkdir has a write concurrency guarantee. Mkdir(name string, permissions FileMode, uid UID, gid GID) (QID, error) // Symlink makes a new symbolic link. - Symlink(oldname string, newname string, uid UID, gid GID) (QID, error) + // + // On the server, Symlink has a write concurrency guarantee. + Symlink(oldName string, newName string, uid UID, gid GID) (QID, error) // Link makes a new hard link. - Link(target File, newname string) error + // + // On the server, Link has a write concurrency guarantee. + Link(target File, newName string) error // Mknod makes a new device node. + // + // On the server, Mknod has a write concurrency guarantee. Mknod(name string, permissions FileMode, major uint32, minor uint32, uid UID, gid GID) (QID, error) + // Rename renames the file. + // + // Rename will never be called on the server, and RenameAt will always + // be used instead. + Rename(newDir File, newName string) error + // RenameAt renames a given file to a new name in a potentially new // directory. // - // oldname must be a name relative to this file, which must be a - // directory. newname is a name relative to newdir. + // oldName must be a name relative to this file, which must be a + // directory. newName is a name relative to newDir. // - // This is deprecated in favor of Rename. - RenameAt(oldname string, newdir File, newname string) error + // On the server, RenameAt has a global concurrency guarantee. + RenameAt(oldName string, newDir File, newName string) error // UnlinkAt the given named file. // @@ -133,16 +189,20 @@ type File interface { // // Flags are implementation-specific (e.g. O_DIRECTORY), but are // generally Linux unlinkat(2) flags. + // + // On the server, UnlinkAt has a write concurrency guarantee. UnlinkAt(name string, flags uint32) error // Readdir reads directory entries. // // This may return io.EOF in addition to syscall.Errno values. // - // Preconditions: Open has been called and returned success. + // On the server, Readdir has a read concurrency guarantee. Readdir(offset uint64, count uint32) ([]Dirent, error) // Readlink reads the link target. + // + // On the server, Readlink has a read concurrency guarantee. Readlink() (string, error) // Flush is called prior to Close. @@ -150,16 +210,11 @@ type File interface { // Whereas Close drops all references to the file, Flush cleans up the // file state. Behavior is implementation-specific. // - // Flush is not related to flush(9p). Flush is an extension to 9P2000.L, + // Flush is not related to flush(9p). Flush is an extension to 9P2000.L, // see version.go. - Flush() error - - // WalkGetAttr walks to the next file and returns its maximal set of - // attributes. // - // Server-side p9.Files may return syscall.ENOSYS to indicate that Walk - // and GetAttr should be used separately to satisfy this request. - WalkGetAttr([]string) ([]QID, File, AttrMask, Attr, error) + // On the server, Flush has a read concurrency guarantee. + Flush() error // Connect establishes a new host-socket backed connection with a // socket. A File does not need to be opened before it can be connected @@ -170,8 +225,22 @@ type File interface { // // The returned FD must be non-blocking. // - // flags indicates the requested type of socket. + // Flags indicates the requested type of socket. + // + // On the server, Connect has a read concurrency guarantee. Connect(flags ConnectFlags) (*fd.FD, error) + + // Renamed is called when this node is renamed. + // + // This may not fail. The file will hold a reference to its parent + // within the p9 package, and is therefore safe to use for the lifetime + // of this File (until Close is called). + // + // This method should not be called by clients, who should use the + // relevant Rename methods. (Although the method will be a no-op.) + // + // On the server, Renamed has a global concurrency guarantee. + Renamed(newDir File, newName string) } // DefaultWalkGetAttr implements File.WalkGetAttr to return ENOSYS for server-side Files. diff --git a/pkg/p9/handlers.go b/pkg/p9/handlers.go index 959dff31d..0d7a6138f 100644 --- a/pkg/p9/handlers.go +++ b/pkg/p9/handlers.go @@ -15,6 +15,7 @@ package p9 import ( + "fmt" "io" "os" "path" @@ -22,22 +23,43 @@ import ( "sync/atomic" "syscall" + "gvisor.googlesource.com/gvisor/pkg/fd" "gvisor.googlesource.com/gvisor/pkg/log" ) -// newErr returns a new error message from an error. -func newErr(err error) *Rlerror { +const maximumNameLength = 255 + +// ExtractErrno extracts a syscall.Errno from a error, best effort. +func ExtractErrno(err error) syscall.Errno { + switch err { + case os.ErrNotExist: + return syscall.ENOENT + case os.ErrExist: + return syscall.EEXIST + case os.ErrPermission: + return syscall.EACCES + case os.ErrInvalid: + return syscall.EINVAL + } + + // Attempt to unwrap. switch e := err.(type) { case syscall.Errno: - return &Rlerror{Error: uint32(e)} + return e case *os.PathError: - return newErr(e.Err) + return ExtractErrno(e.Err) case *os.SyscallError: - return newErr(e.Err) - default: - log.Warningf("unknown error: %v", err) - return &Rlerror{Error: uint32(syscall.EIO)} + return ExtractErrno(e.Err) } + + // Default case. + log.Warningf("unknown error: %v", err) + return syscall.EIO +} + +// newErr returns a new error message from an error. +func newErr(err error) *Rlerror { + return &Rlerror{Error: uint32(ExtractErrno(err))} } // handler is implemented for server-handled messages. @@ -85,13 +107,15 @@ func (t *Tflush) handle(cs *connState) message { return &Rflush{} } -// isSafeName returns true iff the name does not contain directory characters. -// -// We permit walks only on safe names and store the sequence of paths used for -// any given walk in each FID. (This is immutable.) We use this to mark -// relevant FIDs as moved when a successful rename occurs. -func isSafeName(name string) bool { - return name != "" && !strings.Contains(name, "/") && name != "." && name != ".." +// checkSafeName validates the name and returns nil or returns an error. +func checkSafeName(name string) error { + if name == "" || strings.Contains(name, "/") || name == "." || name == ".." { + return syscall.EINVAL + } + if len(name) > maximumNameLength { + return syscall.ENAMETOOLONG + } + return nil } // handle implements handler.handle. @@ -110,22 +134,54 @@ func (t *Tremove) handle(cs *connState) message { } defer ref.DecRef() + // Frustratingly, because we can't be guaranteed that a rename is not + // occurring simultaneously with this removal, we need to acquire the + // global rename lock for this kind of remove operation to ensure that + // ref.parent does not change out from underneath us. + // + // This is why Tremove is a bad idea, and clients should generally use + // Tunlinkat. All p9 clients will use Tunlinkat. + err := ref.safelyGlobal(func() error { + // Is this a root? Can't remove that. + if ref.isRoot() { + return syscall.EINVAL + } + + // N.B. this remove operation is permitted, even if the file is open. + // See also rename below for reasoning. + + // Is this file already deleted? + if ref.isDeleted() { + return syscall.EINVAL + } + + // Retrieve the file's proper name. + name := ref.parent.pathNode.nameFor(ref) + + // Attempt the removal. + if err := ref.parent.file.UnlinkAt(name, 0); err != nil { + return err + } + + // Mark all relevant fids as deleted. We don't need to lock any + // individual nodes because we already hold the global lock. + ref.parent.markChildDeleted(name) + return nil + }) + // "The remove request asks the file server both to remove the file // represented by fid and to clunk the fid, even if the remove fails." // // "It is correct to consider remove to be a clunk with the side effect // of removing the file if permissions allow." // https://swtch.com/plan9port/man/man9/remove.html - err := ref.file.Remove() - - // Clunk the FID regardless of Remove error. if !cs.DeleteFID(t.FID) { return newErr(syscall.EBADF) } - if err != nil { return newErr(err) } + return &Rremove{} } @@ -168,9 +224,12 @@ func (t *Tattach) handle(cs *connState) message { // Build a transient reference. root := &fidRef{ + server: cs.server, + parent: nil, file: sf, refs: 1, - walkable: attr.Mode.IsDir(), + mode: attr.Mode.FileType(), + pathNode: &cs.server.pathTree, } defer root.DecRef() @@ -183,20 +242,24 @@ func (t *Tattach) handle(cs *connState) message { // We want the same traversal checks to apply on attach, so always // attach at the root and use the regular walk paths. names := strings.Split(t.Auth.AttachName, "/") - _, target, _, attr, err := doWalk(cs, root, names) + _, newRef, _, attr, err := doWalk(cs, root, names) if err != nil { return newErr(err) } + defer newRef.DecRef() // Insert the FID. - cs.InsertFID(t.FID, &fidRef{ - file: target, - walkable: attr.Mode.IsDir(), - }) - + cs.InsertFID(t.FID, newRef) return &Rattach{} } +// CanOpen returns whether this file open can be opened, read and written to. +// +// This includes everything except symlinks and sockets. +func CanOpen(mode FileMode) bool { + return mode.IsRegular() || mode.IsDir() || mode.IsNamedPipe() || mode.IsBlockDevice() || mode.IsCharacterDevice() +} + // handle implements handler.handle. func (t *Tlopen) handle(cs *connState) message { // Lookup the FID. @@ -210,13 +273,35 @@ func (t *Tlopen) handle(cs *connState) message { defer ref.openedMu.Unlock() // Has it been opened already? - if ref.opened { + if ref.opened || !CanOpen(ref.mode) { return newErr(syscall.EINVAL) } - // Do the open. - osFile, qid, ioUnit, err := ref.file.Open(t.Flags) - if err != nil { + // Are flags valid? + if t.Flags&^OpenFlagsModeMask != 0 { + return newErr(syscall.EINVAL) + } + + // Is this an attempt to open a directory as writable? Don't accept. + if ref.mode.IsDir() && t.Flags != ReadOnly { + return newErr(syscall.EINVAL) + } + + var ( + qid QID + ioUnit uint32 + osFile *fd.FD + ) + if err := ref.safelyRead(func() (err error) { + // Has it been deleted already? + if ref.isDeleted() { + return syscall.EINVAL + } + + // Do the open. + osFile, qid, ioUnit, err = ref.file.Open(t.Flags) + return err + }); err != nil { return newErr(err) } @@ -229,8 +314,8 @@ func (t *Tlopen) handle(cs *connState) message { func (t *Tlcreate) do(cs *connState, uid UID) (*Rlcreate, error) { // Don't allow complex names. - if !isSafeName(t.Name) { - return nil, syscall.EINVAL + if err := checkSafeName(t.Name); err != nil { + return nil, err } // Lookup the FID. @@ -240,20 +325,48 @@ func (t *Tlcreate) do(cs *connState, uid UID) (*Rlcreate, error) { } defer ref.DecRef() - // Do the create. - osFile, nsf, qid, ioUnit, err := ref.file.Create(t.Name, t.OpenFlags, t.Permissions, uid, t.GID) - if err != nil { + var ( + osFile *fd.FD + nsf File + qid QID + ioUnit uint32 + newRef *fidRef + ) + if err := ref.safelyWrite(func() (err error) { + // Don't allow creation from non-directories or deleted directories. + if ref.isDeleted() || !ref.mode.IsDir() { + return syscall.EINVAL + } + + // Not allowed on open directories. + if _, opened := ref.OpenFlags(); opened { + return syscall.EINVAL + } + + // Do the create. + osFile, nsf, qid, ioUnit, err = ref.file.Create(t.Name, t.OpenFlags, t.Permissions, uid, t.GID) + if err != nil { + return err + } + + newRef = &fidRef{ + server: cs.server, + parent: ref, + file: nsf, + opened: true, + openFlags: t.OpenFlags, + mode: ModeRegular, + pathNode: ref.pathNode.pathNodeFor(t.Name), + } + ref.pathNode.addChild(newRef, t.Name) + ref.IncRef() // Acquire parent reference. + return nil + }); err != nil { return nil, err } // Replace the FID reference. - // - // The new file will be opened already. - cs.InsertFID(t.FID, &fidRef{ - file: nsf, - opened: true, - openFlags: t.OpenFlags, - }) + cs.InsertFID(t.FID, newRef) return &Rlcreate{Rlopen: Rlopen{QID: qid, IoUnit: ioUnit, File: osFile}}, nil } @@ -278,8 +391,8 @@ func (t *Tsymlink) handle(cs *connState) message { func (t *Tsymlink) do(cs *connState, uid UID) (*Rsymlink, error) { // Don't allow complex names. - if !isSafeName(t.Name) { - return nil, syscall.EINVAL + if err := checkSafeName(t.Name); err != nil { + return nil, err } // Lookup the FID. @@ -289,9 +402,22 @@ func (t *Tsymlink) do(cs *connState, uid UID) (*Rsymlink, error) { } defer ref.DecRef() - // Do the symlink. - qid, err := ref.file.Symlink(t.Target, t.Name, uid, t.GID) - if err != nil { + var qid QID + if err := ref.safelyWrite(func() (err error) { + // Don't allow symlinks from non-directories or deleted directories. + if ref.isDeleted() || !ref.mode.IsDir() { + return syscall.EINVAL + } + + // Not allowed on open directories. + if _, opened := ref.OpenFlags(); opened { + return syscall.EINVAL + } + + // Do the symlink. + qid, err = ref.file.Symlink(t.Target, t.Name, uid, t.GID) + return err + }); err != nil { return nil, err } @@ -301,8 +427,8 @@ func (t *Tsymlink) do(cs *connState, uid UID) (*Rsymlink, error) { // handle implements handler.handle. func (t *Tlink) handle(cs *connState) message { // Don't allow complex names. - if !isSafeName(t.Name) { - return newErr(syscall.EINVAL) + if err := checkSafeName(t.Name); err != nil { + return newErr(err) } // Lookup the FID. @@ -319,8 +445,20 @@ func (t *Tlink) handle(cs *connState) message { } defer refTarget.DecRef() - // Do the link. - if err := ref.file.Link(refTarget.file, t.Name); err != nil { + if err := ref.safelyWrite(func() (err error) { + // Don't allow create links from non-directories or deleted directories. + if ref.isDeleted() || !ref.mode.IsDir() { + return syscall.EINVAL + } + + // Not allowed on open directories. + if _, opened := ref.OpenFlags(); opened { + return syscall.EINVAL + } + + // Do the link. + return ref.file.Link(refTarget.file, t.Name) + }); err != nil { return newErr(err) } @@ -330,8 +468,11 @@ func (t *Tlink) handle(cs *connState) message { // handle implements handler.handle. func (t *Trenameat) handle(cs *connState) message { // Don't allow complex names. - if !isSafeName(t.OldName) || !isSafeName(t.NewName) { - return newErr(syscall.EINVAL) + if err := checkSafeName(t.OldName); err != nil { + return newErr(err) + } + if err := checkSafeName(t.NewName); err != nil { + return newErr(err) } // Lookup the FID. @@ -348,8 +489,32 @@ func (t *Trenameat) handle(cs *connState) message { } defer refTarget.DecRef() - // Do the rename. - if err := ref.file.RenameAt(t.OldName, refTarget.file, t.NewName); err != nil { + // Perform the rename holding the global lock. + if err := ref.safelyGlobal(func() (err error) { + // Don't allow renaming across deleted directories. + if ref.isDeleted() || !ref.mode.IsDir() || refTarget.isDeleted() || !refTarget.mode.IsDir() { + return syscall.EINVAL + } + + // Not allowed on open directories. + if _, opened := ref.OpenFlags(); opened { + return syscall.EINVAL + } + + // Is this the same file? If yes, short-circuit and return success. + if ref.pathNode == refTarget.pathNode && t.OldName == t.NewName { + return nil + } + + // Attempt the actual rename. + if err := ref.file.RenameAt(t.OldName, refTarget.file, t.NewName); err != nil { + return err + } + + // Update the path tree. + ref.renameChildTo(t.OldName, refTarget, t.NewName) + return nil + }); err != nil { return newErr(err) } @@ -359,8 +524,8 @@ func (t *Trenameat) handle(cs *connState) message { // handle implements handler.handle. func (t *Tunlinkat) handle(cs *connState) message { // Don't allow complex names. - if !isSafeName(t.Name) { - return newErr(syscall.EINVAL) + if err := checkSafeName(t.Name); err != nil { + return newErr(err) } // Lookup the FID. @@ -370,8 +535,40 @@ func (t *Tunlinkat) handle(cs *connState) message { } defer ref.DecRef() - // Do the unlink. - if err := ref.file.UnlinkAt(t.Name, t.Flags); err != nil { + if err := ref.safelyWrite(func() (err error) { + // Don't allow deletion from non-directories or deleted directories. + if ref.isDeleted() || !ref.mode.IsDir() { + return syscall.EINVAL + } + + // Not allowed on open directories. + if _, opened := ref.OpenFlags(); opened { + return syscall.EINVAL + } + + // Before we do the unlink itself, we need to ensure that there + // are no operations in flight on associated path node. The + // child's path node lock must be held to ensure that the + // unlink at marking the child deleted below is atomic with + // respect to any other read or write operations. + // + // This is one case where we have a lock ordering issue, but + // since we always acquire deeper in the hierarchy, we know + // that we are free of lock cycles. + childPathNode := ref.pathNode.pathNodeFor(t.Name) + childPathNode.mu.Lock() + defer childPathNode.mu.Unlock() + + // Do the unlink. + err = ref.file.UnlinkAt(t.Name, t.Flags) + if err != nil { + return err + } + + // Mark the path as deleted. + ref.markChildDeleted(t.Name) + return nil + }); err != nil { return newErr(err) } @@ -381,8 +578,8 @@ func (t *Tunlinkat) handle(cs *connState) message { // handle implements handler.handle. func (t *Trename) handle(cs *connState) message { // Don't allow complex names. - if !isSafeName(t.Name) { - return newErr(syscall.EINVAL) + if err := checkSafeName(t.Name); err != nil { + return newErr(err) } // Lookup the FID. @@ -399,8 +596,43 @@ func (t *Trename) handle(cs *connState) message { } defer refTarget.DecRef() - // Call the rename method. - if err := ref.file.Rename(refTarget.file, t.Name); err != nil { + if err := ref.safelyGlobal(func() (err error) { + // Don't allow a root rename. + if ref.isRoot() { + return syscall.EINVAL + } + + // Don't allow renaming deleting entries, or target non-directories. + if ref.isDeleted() || refTarget.isDeleted() || !refTarget.mode.IsDir() { + return syscall.EINVAL + } + + // If the parent is deleted, but we not, something is seriously wrong. + // It's fail to die at this point with an assertion failure. + if ref.parent.isDeleted() { + panic(fmt.Sprintf("parent %+v deleted, child %+v is not", ref.parent, ref)) + } + + // N.B. The rename operation is allowed to proceed on open files. It + // does impact the state of its parent, but this is merely a sanity + // check in any case, and the operation is safe. There may be other + // files corresponding to the same path that are renamed anyways. + + // Check for the exact same file and short-circuit. + oldName := ref.parent.pathNode.nameFor(ref) + if ref.parent.pathNode == refTarget.pathNode && oldName == t.Name { + return nil + } + + // Call the rename method on the parent. + if err := ref.parent.file.RenameAt(oldName, refTarget.file, t.Name); err != nil { + return err + } + + // Update the path tree. + ref.parent.renameChildTo(oldName, refTarget, t.Name) + return nil + }); err != nil { return newErr(err) } @@ -416,9 +648,19 @@ func (t *Treadlink) handle(cs *connState) message { } defer ref.DecRef() - // Do the read. - target, err := ref.file.Readlink() - if err != nil { + var target string + if err := ref.safelyRead(func() (err error) { + // Don't allow readlink on deleted files. There is no need to + // check if this file is opened because symlinks cannot be + // opened. + if ref.isDeleted() || !ref.mode.IsSymlink() { + return syscall.EINVAL + } + + // Do the read. + target, err = ref.file.Readlink() + return err + }); err != nil { return newErr(err) } @@ -434,26 +676,30 @@ func (t *Tread) handle(cs *connState) message { } defer ref.DecRef() - // Has it been opened already? - openFlags, opened := ref.OpenFlags() - if !opened { - return newErr(syscall.EINVAL) - } - - // Can it be read? Check permissions. - if openFlags&OpenFlagsModeMask == WriteOnly { - return newErr(syscall.EPERM) - } - // Constrain the size of the read buffer. if int(t.Count) > int(maximumLength) { return newErr(syscall.ENOBUFS) } - // Do the read. - data := make([]byte, t.Count) - n, err := ref.file.ReadAt(data, t.Offset) - if err != nil && err != io.EOF { + var ( + data = make([]byte, t.Count) + n int + ) + if err := ref.safelyRead(func() (err error) { + // Has it been opened already? + openFlags, opened := ref.OpenFlags() + if !opened { + return syscall.EINVAL + } + + // Can it be read? Check permissions. + if openFlags&OpenFlagsModeMask == WriteOnly { + return syscall.EPERM + } + + n, err = ref.file.ReadAt(data, t.Offset) + return err + }); err != nil && err != io.EOF { return newErr(err) } @@ -469,20 +715,22 @@ func (t *Twrite) handle(cs *connState) message { } defer ref.DecRef() - // Has it been opened already? - openFlags, opened := ref.OpenFlags() - if !opened { - return newErr(syscall.EINVAL) - } + var n int + if err := ref.safelyRead(func() (err error) { + // Has it been opened already? + openFlags, opened := ref.OpenFlags() + if !opened { + return syscall.EINVAL + } - // Can it be write? Check permissions. - if openFlags&OpenFlagsModeMask == ReadOnly { - return newErr(syscall.EPERM) - } + // Can it be write? Check permissions. + if openFlags&OpenFlagsModeMask == ReadOnly { + return syscall.EPERM + } - // Do the write. - n, err := ref.file.WriteAt(t.Data, t.Offset) - if err != nil { + n, err = ref.file.WriteAt(t.Data, t.Offset) + return err + }); err != nil { return newErr(err) } @@ -500,8 +748,8 @@ func (t *Tmknod) handle(cs *connState) message { func (t *Tmknod) do(cs *connState, uid UID) (*Rmknod, error) { // Don't allow complex names. - if !isSafeName(t.Name) { - return nil, syscall.EINVAL + if err := checkSafeName(t.Name); err != nil { + return nil, err } // Lookup the FID. @@ -511,9 +759,22 @@ func (t *Tmknod) do(cs *connState, uid UID) (*Rmknod, error) { } defer ref.DecRef() - // Do the mknod. - qid, err := ref.file.Mknod(t.Name, t.Permissions, t.Major, t.Minor, uid, t.GID) - if err != nil { + var qid QID + if err := ref.safelyWrite(func() (err error) { + // Don't allow mknod on deleted files. + if ref.isDeleted() || !ref.mode.IsDir() { + return syscall.EINVAL + } + + // Not allowed on open directories. + if _, opened := ref.OpenFlags(); opened { + return syscall.EINVAL + } + + // Do the mknod. + qid, err = ref.file.Mknod(t.Name, t.Permissions, t.Major, t.Minor, uid, t.GID) + return err + }); err != nil { return nil, err } @@ -531,8 +792,8 @@ func (t *Tmkdir) handle(cs *connState) message { func (t *Tmkdir) do(cs *connState, uid UID) (*Rmkdir, error) { // Don't allow complex names. - if !isSafeName(t.Name) { - return nil, syscall.EINVAL + if err := checkSafeName(t.Name); err != nil { + return nil, err } // Lookup the FID. @@ -542,9 +803,22 @@ func (t *Tmkdir) do(cs *connState, uid UID) (*Rmkdir, error) { } defer ref.DecRef() - // Do the mkdir. - qid, err := ref.file.Mkdir(t.Name, t.Permissions, uid, t.GID) - if err != nil { + var qid QID + if err := ref.safelyWrite(func() (err error) { + // Don't allow mkdir on deleted files. + if ref.isDeleted() || !ref.mode.IsDir() { + return syscall.EINVAL + } + + // Not allowed on open directories. + if _, opened := ref.OpenFlags(); opened { + return syscall.EINVAL + } + + // Do the mkdir. + qid, err = ref.file.Mkdir(t.Name, t.Permissions, uid, t.GID) + return err + }); err != nil { return nil, err } @@ -560,9 +834,20 @@ func (t *Tgetattr) handle(cs *connState) message { } defer ref.DecRef() - // Get attributes. - qid, valid, attr, err := ref.file.GetAttr(t.AttrMask) - if err != nil { + // We allow getattr on deleted files. Depending on the backing + // implementation, it's possible that races exist that might allow + // fetching attributes of other files. But we need to generally allow + // refreshing attributes and this is a minor leak, if at all. + + var ( + qid QID + valid AttrMask + attr Attr + ) + if err := ref.safelyRead(func() (err error) { + qid, valid, attr, err = ref.file.GetAttr(t.AttrMask) + return err + }); err != nil { return newErr(err) } @@ -578,8 +863,18 @@ func (t *Tsetattr) handle(cs *connState) message { } defer ref.DecRef() - // Set attributes. - if err := ref.file.SetAttr(t.Valid, t.SetAttr); err != nil { + if err := ref.safelyWrite(func() error { + // We don't allow setattr on files that have been deleted. + // This might be technically incorrect, as it's possible that + // there were multiple links and you can still change the + // corresponding inode information. + if ref.isDeleted() { + return syscall.EINVAL + } + + // Set the attributes. + return ref.file.SetAttr(t.Valid, t.SetAttr) + }); err != nil { return newErr(err) } @@ -621,14 +916,25 @@ func (t *Treaddir) handle(cs *connState) message { } defer ref.DecRef() - // Has it been opened already? - if _, opened := ref.OpenFlags(); !opened { - return newErr(syscall.EINVAL) - } + var entries []Dirent + if err := ref.safelyRead(func() (err error) { + // Don't allow reading deleted directories. + if ref.isDeleted() || !ref.mode.IsDir() { + return syscall.EINVAL + } + + // Has it been opened already? + if _, opened := ref.OpenFlags(); !opened { + return syscall.EINVAL + } - // Read the entries. - entries, err := ref.file.Readdir(t.Offset, t.Count) - if err != nil && err != io.EOF { + // Read the entries. + entries, err = ref.file.Readdir(t.Offset, t.Count) + if err != nil && err != io.EOF { + return err + } + return nil + }); err != nil { return newErr(err) } @@ -644,13 +950,15 @@ func (t *Tfsync) handle(cs *connState) message { } defer ref.DecRef() - // Has it been opened already? - if _, opened := ref.OpenFlags(); !opened { - return newErr(syscall.EINVAL) - } + if err := ref.safelyRead(func() (err error) { + // Has it been opened already? + if _, opened := ref.OpenFlags(); !opened { + return syscall.EINVAL + } - err := ref.file.FSync() - if err != nil { + // Perform the sync. + return ref.file.FSync() + }); err != nil { return newErr(err) } @@ -671,6 +979,11 @@ func (t *Tstatfs) handle(cs *connState) message { return newErr(err) } + // Constrain the name length. + if st.NameLength > maximumNameLength { + st.NameLength = maximumNameLength + } + return &Rstatfs{st} } @@ -682,7 +995,7 @@ func (t *Tflushf) handle(cs *connState) message { } defer ref.DecRef() - if err := ref.file.Flush(); err != nil { + if err := ref.safelyRead(ref.file.Flush); err != nil { return newErr(err) } @@ -726,12 +1039,14 @@ func walkOne(qids []QID, from File, names []string) ([]QID, File, AttrMask, Attr // doWalk walks from a given fidRef. // -// This enforces that all intermediate nodes are walkable (directories). -func doWalk(cs *connState, ref *fidRef, names []string) (qids []QID, sf File, valid AttrMask, attr Attr, err error) { +// This enforces that all intermediate nodes are walkable (directories). The +// fidRef returned (newRef) has a reference associated with it that is now +// owned by the caller and must be handled appropriately. +func doWalk(cs *connState, ref *fidRef, names []string) (qids []QID, newRef *fidRef, valid AttrMask, attr Attr, err error) { // Check the names. for _, name := range names { - if !isSafeName(name) { - err = syscall.EINVAL + err = checkSafeName(name) + if err != nil { return } } @@ -745,44 +1060,88 @@ func doWalk(cs *connState, ref *fidRef, names []string) (qids []QID, sf File, va // Is this an empty list? Handle specially. We don't actually need to // validate anything since this is always permitted. if len(names) == 0 { - return walkOne(nil, ref.file, nil) - } - - // Is it walkable? - if !ref.walkable { - err = syscall.EINVAL - return + var sf File // Temporary. + if err := ref.maybeParent().safelyRead(func() (err error) { + // Clone the single element. + qids, sf, valid, attr, err = walkOne(nil, ref.file, nil) + if err != nil { + return err + } + + newRef = &fidRef{ + server: cs.server, + parent: ref.parent, + file: sf, + mode: ref.mode, + pathNode: ref.pathNode, + + // For the clone case, the cloned fid must + // preserve the deleted property of the + // original FID. + deleted: ref.deleted, + } + if !ref.isRoot() { + if !newRef.isDeleted() { + // Add only if a non-root node; the same node. + ref.parent.pathNode.addChild(newRef, ref.parent.pathNode.nameFor(ref)) + } + ref.parent.IncRef() // Acquire parent reference. + } + // doWalk returns a reference. + newRef.IncRef() + return nil + }); err != nil { + return nil, nil, AttrMask{}, Attr{}, err + } + return qids, newRef, valid, attr, nil } - from := ref.file // Start at the passed ref. - // Do the walk, one element at a time. + walkRef := ref + walkRef.IncRef() for i := 0; i < len(names); i++ { - qids, sf, valid, attr, err = walkOne(qids, from, names[i:i+1]) - - // Close the intermediate file. Note that we don't close the - // first file because in that case we are walking from the - // existing reference. - if i > 0 { - from.Close() - } - from = sf // Use the new file. - - // Was there an error walking? - if err != nil { - return nil, nil, AttrMask{}, Attr{}, err - } - // We won't allow beyond past symlinks; stop here if this isn't // a proper directory and we have additional paths to walk. - if !valid.Mode || (!attr.Mode.IsDir() && i < len(names)-1) { - from.Close() // Not using the file object. + if !walkRef.mode.IsDir() { + walkRef.DecRef() // Drop walk reference; no lock required. return nil, nil, AttrMask{}, Attr{}, syscall.EINVAL } + + var sf File // Temporary. + if err := walkRef.safelyRead(func() (err error) { + qids, sf, valid, attr, err = walkOne(qids, walkRef.file, names[i:i+1]) + if err != nil { + return err + } + + // Note that we don't need to acquire a lock on any of + // these individual instances. That's because they are + // not actually addressable via a FID. They are + // anonymous. They exist in the tree for tracking + // purposes. + newRef := &fidRef{ + server: cs.server, + parent: walkRef, + file: sf, + mode: attr.Mode.FileType(), + pathNode: walkRef.pathNode.pathNodeFor(names[i]), + } + walkRef.pathNode.addChild(newRef, names[i]) + // We allow our walk reference to become the new parent + // reference here and so we don't IncRef. Instead, just + // set walkRef to the newRef above and acquire a new + // walk reference. + walkRef = newRef + walkRef.IncRef() + return nil + }); err != nil { + walkRef.DecRef() // Drop the old walkRef. + return nil, nil, AttrMask{}, Attr{}, err + } } // Success. - return qids, sf, valid, attr, nil + return qids, walkRef, valid, attr, nil } // handle implements handler.handle. @@ -795,17 +1154,14 @@ func (t *Twalk) handle(cs *connState) message { defer ref.DecRef() // Do the walk. - qids, sf, _, attr, err := doWalk(cs, ref, t.Names) + qids, newRef, _, _, err := doWalk(cs, ref, t.Names) if err != nil { return newErr(err) } + defer newRef.DecRef() // Install the new FID. - cs.InsertFID(t.NewFID, &fidRef{ - file: sf, - walkable: attr.Mode.IsDir(), - }) - + cs.InsertFID(t.NewFID, newRef) return &Rwalk{QIDs: qids} } @@ -819,17 +1175,14 @@ func (t *Twalkgetattr) handle(cs *connState) message { defer ref.DecRef() // Do the walk. - qids, sf, valid, attr, err := doWalk(cs, ref, t.Names) + qids, newRef, valid, attr, err := doWalk(cs, ref, t.Names) if err != nil { return newErr(err) } + defer newRef.DecRef() // Install the new FID. - cs.InsertFID(t.NewFID, &fidRef{ - file: sf, - walkable: attr.Mode.IsDir(), - }) - + cs.InsertFID(t.NewFID, newRef) return &Rwalkgetattr{QIDs: qids, Valid: valid, Attr: attr} } @@ -878,9 +1231,17 @@ func (t *Tlconnect) handle(cs *connState) message { } defer ref.DecRef() - // Do the connect. - osFile, err := ref.file.Connect(t.Flags) - if err != nil { + var osFile *fd.FD + if err := ref.safelyRead(func() (err error) { + // Don't allow connecting to deleted files. + if ref.isDeleted() || !ref.mode.IsSocket() { + return syscall.EINVAL + } + + // Do the connect. + osFile, err = ref.file.Connect(t.Flags) + return err + }); err != nil { return newErr(err) } diff --git a/pkg/p9/local_server/BUILD b/pkg/p9/local_server/BUILD index 8229e6308..b17ebb79d 100644 --- a/pkg/p9/local_server/BUILD +++ b/pkg/p9/local_server/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_binary") +package(licenses = ["notice"]) # Apache 2.0 + go_binary( name = "local_server", srcs = ["local_server.go"], diff --git a/pkg/p9/local_server/local_server.go b/pkg/p9/local_server/local_server.go index 1e6aaa762..69b90c6cd 100644 --- a/pkg/p9/local_server/local_server.go +++ b/pkg/p9/local_server/local_server.go @@ -318,6 +318,11 @@ func (l *local) Connect(p9.ConnectFlags) (*fd.FD, error) { return nil, syscall.ECONNREFUSED } +// Renamed implements p9.File.Renamed. +func (l *local) Renamed(parent p9.File, newName string) { + l.path = path.Join(parent.(*local).path, newName) +} + func main() { log.SetLevel(log.Debug) diff --git a/pkg/p9/messages_test.go b/pkg/p9/messages_test.go index dfb41bb76..c0d65d82c 100644 --- a/pkg/p9/messages_test.go +++ b/pkg/p9/messages_test.go @@ -15,6 +15,7 @@ package p9 import ( + "fmt" "reflect" "testing" ) @@ -186,6 +187,13 @@ func TestEncodeDecode(t *testing.T) { &Rxattrwalk{ Size: 1, }, + &Txattrcreate{ + FID: 1, + Name: "a", + AttrSize: 2, + Flags: 3, + }, + &Rxattrcreate{}, &Treaddir{ Directory: 1, Offset: 2, @@ -389,3 +397,32 @@ func TestEncodeDecode(t *testing.T) { } } } + +func TestMessageStrings(t *testing.T) { + for typ, fn := range messageRegistry { + name := fmt.Sprintf("%+v", typ) + t.Run(name, func(t *testing.T) { + defer func() { // Ensure no panic. + if r := recover(); r != nil { + t.Errorf("printing %s failed: %v", name, r) + } + }() + m := fn() + _ = fmt.Sprintf("%v", m) + err := ErrInvalidMsgType{typ} + _ = err.Error() + }) + } +} + +func TestRegisterDuplicate(t *testing.T) { + defer func() { + if r := recover(); r == nil { + // We expect a panic. + t.FailNow() + } + }() + + // Register a duplicate. + register(&Rlerror{}) +} diff --git a/pkg/p9/p9.go b/pkg/p9/p9.go index 3b0993ecd..be644e7bf 100644 --- a/pkg/p9/p9.go +++ b/pkg/p9/p9.go @@ -984,6 +984,30 @@ func (s *SetAttr) Encode(b *buffer) { b.Write64(s.MTimeNanoSeconds) } +// Apply applies this to the given Attr. +func (a *Attr) Apply(mask SetAttrMask, attr SetAttr) { + if mask.Permissions { + a.Mode = a.Mode&^PermissionsMask | (attr.Permissions & PermissionsMask) + } + if mask.UID { + a.UID = attr.UID + } + if mask.GID { + a.GID = attr.GID + } + if mask.Size { + a.Size = attr.Size + } + if mask.ATime { + a.ATimeSeconds = attr.ATimeSeconds + a.ATimeNanoSeconds = attr.ATimeNanoSeconds + } + if mask.MTime { + a.MTimeSeconds = attr.MTimeSeconds + a.MTimeNanoSeconds = attr.MTimeNanoSeconds + } +} + // Dirent is used for readdir. type Dirent struct { // QID is the entry QID. diff --git a/pkg/p9/p9test/BUILD b/pkg/p9/p9test/BUILD index d6f428e11..7c4b875ce 100644 --- a/pkg/p9/p9test/BUILD +++ b/pkg/p9/p9test/BUILD @@ -1,16 +1,60 @@ +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +load("@io_bazel_rules_go//go:def.bzl", "go_binary") + package(licenses = ["notice"]) # Apache 2.0 -load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +alias( + name = "mockgen", + actual = "@com_github_golang_mock//mockgen:mockgen", +) -go_test( - name = "p9test_test", - size = "small", - srcs = ["client_test.go"], - embed = [":p9test"], +MOCK_SRC_PACKAGE = "gvisor.googlesource.com/gvisor/pkg/p9" + +# mockgen_reflect is a source file that contains mock generation code that +# imports the p9 package and generates a specification via reflection. The +# usual generation path must be split into two distinct parts because the full +# source tree is not available to all build targets. Only declared depencies +# are available (and even then, not the Go source files). +genrule( + name = "mockgen_reflect", + testonly = 1, + outs = ["mockgen_reflect.go"], + cmd = ( + "$(location :mockgen) " + + "-package p9test " + + "-prog_only " + MOCK_SRC_PACKAGE + " " + + "Attacher,File > $@" + ), + tools = [":mockgen"], +) + +# mockgen_exec is the binary that includes the above reflection generator. +# Running this binary will emit an encoded version of the p9 Attacher and File +# structures. This is consumed by the mocks genrule, below. +go_binary( + name = "mockgen_exec", + testonly = 1, + srcs = ["mockgen_reflect.go"], deps = [ - "//pkg/fd", "//pkg/p9", - "//pkg/unet", + "@com_github_golang_mock//mockgen/model:go_default_library", + ], +) + +# mocks consumes the encoded output above, and generates the full source for a +# set of mocks. These are included directly in the p9test library. +genrule( + name = "mocks", + testonly = 1, + outs = ["mocks.go"], + cmd = ( + "$(location :mockgen) " + + "-package p9test " + + "-exec_only $(location :mockgen_exec) " + MOCK_SRC_PACKAGE + " File > $@" + ), + tools = [ + ":mockgen", + ":mockgen_exec", ], ) @@ -18,11 +62,27 @@ go_library( name = "p9test", srcs = [ "mocks.go", + "p9test.go", ], importpath = "gvisor.googlesource.com/gvisor/pkg/p9/p9test", visibility = ["//:sandbox"], + deps = [ + "//pkg/fd", + "//pkg/log", + "//pkg/p9", + "//pkg/unet", + "@com_github_golang_mock//gomock:go_default_library", + ], +) + +go_test( + name = "client_test", + size = "small", + srcs = ["client_test.go"], + embed = [":p9test"], deps = [ "//pkg/fd", "//pkg/p9", + "@com_github_golang_mock//gomock:go_default_library", ], ) diff --git a/pkg/p9/p9test/client_test.go b/pkg/p9/p9test/client_test.go index db562b9ba..242d81b95 100644 --- a/pkg/p9/p9test/client_test.go +++ b/pkg/p9/p9test/client_test.go @@ -15,360 +15,2059 @@ package p9test import ( - "io/ioutil" + "bytes" + "fmt" + "io" + "math/rand" "os" "reflect" + "strings" + "sync" "syscall" "testing" + "time" + "github.com/golang/mock/gomock" "gvisor.googlesource.com/gvisor/pkg/fd" "gvisor.googlesource.com/gvisor/pkg/p9" - "gvisor.googlesource.com/gvisor/pkg/unet" ) -func TestDonateFD(t *testing.T) { - // Temporary file. - osFile, err := ioutil.TempFile("", "p9") +func TestPanic(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + // Create a new root. + d := h.NewDirectory(nil)(nil) + defer d.Close() // Needed manually. + h.Attacher.EXPECT().Attach().Return(d, nil).Do(func() { + // Panic here, and ensure that we get back EFAULT. + panic("handler") + }) + + // Attach to the client. + if _, err := c.Attach("/"); err != syscall.EFAULT { + t.Fatalf("got attach err %v, want EFAULT", err) + } +} + +func TestAttachNoLeak(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + // Create a new root. + d := h.NewDirectory(nil)(nil) + h.Attacher.EXPECT().Attach().Return(d, nil).Times(1) + + // Attach to the client. + f, err := c.Attach("/") + if err != nil { + t.Fatalf("got attach err %v, want nil", err) + } + + // Don't close the file. This should be closed automatically when the + // client disconnects. The mock asserts that everything is closed + // exactly once. This statement just removes the unused variable error. + _ = f +} + +func TestBadAttach(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + // Return an error on attach. + h.Attacher.EXPECT().Attach().Return(nil, syscall.EINVAL).Times(1) + + // Attach to the client. + if _, err := c.Attach("/"); err != syscall.EINVAL { + t.Fatalf("got attach err %v, want syscall.EINVAL", err) + } +} + +func TestWalkAttach(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + // Create a new root. + d := h.NewDirectory(map[string]Generator{ + "a": h.NewDirectory(map[string]Generator{ + "b": h.NewFile(), + }), + })(nil) + h.Attacher.EXPECT().Attach().Return(d, nil).Times(1) + + // Attach to the client as a non-root, and ensure that the walk above + // occurs as expected. We should get back b, and all references should + // be dropped when the file is closed. + f, err := c.Attach("/a/b") + if err != nil { + t.Fatalf("got attach err %v, want nil", err) + } + defer f.Close() + + // Check that's a regular file. + if _, _, attr, err := f.GetAttr(p9.AttrMaskAll()); err != nil { + t.Errorf("got err %v, want nil", err) + } else if !attr.Mode.IsRegular() { + t.Errorf("got mode %v, want regular file", err) + } +} + +// newTypeMap returns a new type map dictionary. +func newTypeMap(h *Harness) map[string]Generator { + return map[string]Generator{ + "directory": h.NewDirectory(map[string]Generator{}), + "file": h.NewFile(), + "symlink": h.NewSymlink(), + "block-device": h.NewBlockDevice(), + "character-device": h.NewCharacterDevice(), + "named-pipe": h.NewNamedPipe(), + "socket": h.NewSocket(), + } +} + +// newRoot returns a new root filesystem. +// +// This is set up in a deterministic way for testing most operations. +// +// The represented file system looks like: +// - file +// - symlink +// - directory +// ... +// + one +// - file +// - symlink +// - directory +// ... +// + two +// - file +// - symlink +// - directory +// ... +// + three +// - file +// - symlink +// - directory +// ... +func newRoot(h *Harness, c *p9.Client) (*Mock, p9.File) { + root := newTypeMap(h) + one := newTypeMap(h) + two := newTypeMap(h) + three := newTypeMap(h) + one["two"] = h.NewDirectory(two) // Will be nested in one. + root["one"] = h.NewDirectory(one) // Top level. + root["three"] = h.NewDirectory(three) // Alternate top-level. + + // Create a new root. + rootBackend := h.NewDirectory(root)(nil) + h.Attacher.EXPECT().Attach().Return(rootBackend, nil) + + // Attach to the client. + r, err := c.Attach("/") + if err != nil { + h.t.Fatalf("got attach err %v, want nil", err) + } + + return rootBackend, r +} + +func allInvalidNames(from string) []string { + return []string{ + from + "/other", + from + "/..", + from + "/.", + from + "/", + "other/" + from, + "/" + from, + "./" + from, + "../" + from, + ".", + "..", + "/", + "", + } +} + +func TestWalkInvalid(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + // Run relevant tests. + for name := range newTypeMap(h) { + // These are all the various ways that one might attempt to + // construct compound paths. They should all be rejected, as + // any compound that contains a / is not allowed, as well as + // the singular paths of '.' and '..'. + if _, _, err := root.Walk([]string{".", name}); err != syscall.EINVAL { + t.Errorf("Walk through . %s wanted EINVAL, got %v", name, err) + } + if _, _, err := root.Walk([]string{"..", name}); err != syscall.EINVAL { + t.Errorf("Walk through . %s wanted EINVAL, got %v", name, err) + } + if _, _, err := root.Walk([]string{name, "."}); err != syscall.EINVAL { + t.Errorf("Walk through %s . wanted EINVAL, got %v", name, err) + } + if _, _, err := root.Walk([]string{name, ".."}); err != syscall.EINVAL { + t.Errorf("Walk through %s .. wanted EINVAL, got %v", name, err) + } + for _, invalidName := range allInvalidNames(name) { + if _, _, err := root.Walk([]string{invalidName}); err != syscall.EINVAL { + t.Errorf("Walk through %s wanted EINVAL, got %v", invalidName, err) + } + } + wantErr := syscall.EINVAL + if name == "directory" { + // We can attempt a walk through a directory. However, + // we should never see a file named "other", so we + // expect this to return ENOENT. + wantErr = syscall.ENOENT + } + if _, _, err := root.Walk([]string{name, "other"}); err != wantErr { + t.Errorf("Walk through %s/other wanted %v, got %v", name, wantErr, err) + } + + // Do a successful walk. + _, f, err := root.Walk([]string{name}) + if err != nil { + t.Errorf("Walk to %s wanted nil, got %v", name, err) + } + defer f.Close() + local := h.Pop(f) + + // Check that the file matches. + _, localMask, localAttr, localErr := local.GetAttr(p9.AttrMaskAll()) + if _, mask, attr, err := f.GetAttr(p9.AttrMaskAll()); mask != localMask || attr != localAttr || err != localErr { + t.Errorf("GetAttr got (%v, %v, %v), wanted (%v, %v, %v)", + mask, attr, err, localMask, localAttr, localErr) + } + + // Ensure we can't walk backwards. + if _, _, err := f.Walk([]string{"."}); err != syscall.EINVAL { + t.Errorf("Walk through %s/. wanted EINVAL, got %v", name, err) + } + if _, _, err := f.Walk([]string{".."}); err != syscall.EINVAL { + t.Errorf("Walk through %s/.. wanted EINVAL, got %v", name, err) + } + } +} + +// fileGenerator is a function to generate files via walk or create. +// +// Examples are: +// - walkHelper +// - walkAndOpenHelper +// - createHelper +type fileGenerator func(*Harness, string, p9.File) (*Mock, *Mock, p9.File) + +// walkHelper walks to the given file. +// +// The backends of the parent and walked file are returned, as well as the +// walked client file. +func walkHelper(h *Harness, name string, dir p9.File) (parentBackend *Mock, walkedBackend *Mock, walked p9.File) { + _, parent, err := dir.Walk(nil) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer parent.Close() + parentBackend = h.Pop(parent) + + _, walked, err = parent.Walk([]string{name}) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + walkedBackend = h.Pop(walked) + + return parentBackend, walkedBackend, walked +} + +// walkAndOpenHelper additionally opens the walked file, if possible. +func walkAndOpenHelper(h *Harness, name string, dir p9.File) (*Mock, *Mock, p9.File) { + parentBackend, walkedBackend, walked := walkHelper(h, name, dir) + if p9.CanOpen(walkedBackend.Attr.Mode) { + // Open for all file types that we can. We stick to a read-only + // open here because directories may not be opened otherwise. + walkedBackend.EXPECT().Open(p9.ReadOnly).Times(1) + if _, _, _, err := walked.Open(p9.ReadOnly); err != nil { + h.t.Errorf("got open err %v, want nil", err) + } + } else { + // ... or assert an error for others. + if _, _, _, err := walked.Open(p9.ReadOnly); err != syscall.EINVAL { + h.t.Errorf("got open err %v, want EINVAL", err) + } + } + return parentBackend, walkedBackend, walked +} + +// createHelper creates the given file and returns the parent directory, +// created file and client file, which must be closed when done. +func createHelper(h *Harness, name string, dir p9.File) (*Mock, *Mock, p9.File) { + // Clone the directory first, since Create replaces the existing file. + // We change the type after calling create. + _, dirThenFile, err := dir.Walk(nil) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + + // Create a new server-side file. On the server-side, the a new file is + // returned from a create call. The client will reuse the same file, + // but we still expect the normal chain of closes. This complicates + // things a bit because the "parent" will always chain to the cloned + // dir above. + dirBackend := h.Pop(dirThenFile) // New backend directory. + newFile := h.NewFile()(dirBackend) // New file with backend parent. + dirBackend.EXPECT().Create(name, gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, newFile, newFile.QID, uint32(0), nil) + + // Create via the client. + _, dirThenFile, _, _, err = dirThenFile.Create(name, p9.ReadOnly, 0, 0, 0) if err != nil { - t.Fatalf("could not create temporary file: %v", err) + h.t.Fatalf("got create err %v, want nil", err) + } + + // Ensure subsequent walks succeed. + dirBackend.AddChild(name, h.NewFile()) + return dirBackend, newFile, dirThenFile +} + +// deprecatedRemover allows us to access the deprecated Remove operation within +// the p9.File client object. +type deprecatedRemover interface { + Remove() error +} + +// checkDeleted asserts that relevant methods fail for an unlinked file. +// +// This function will close the file at the end. +func checkDeleted(h *Harness, file p9.File) { + defer file.Close() // See doc. + + if _, _, _, err := file.Open(p9.ReadOnly); err != syscall.EINVAL { + h.t.Errorf("open while deleted, got %v, want EINVAL", err) + } + if _, _, _, _, err := file.Create("created", p9.ReadOnly, 0, 0, 0); err != syscall.EINVAL { + h.t.Errorf("create while deleted, got %v, want EINVAL", err) + } + if _, err := file.Symlink("old", "new", 0, 0); err != syscall.EINVAL { + h.t.Errorf("symlink while deleted, got %v, want EINVAL", err) + } + // N.B. This link is technically invalid, but if a call to link is + // actually made in the backend then the mock will panic. + if err := file.Link(file, "new"); err != syscall.EINVAL { + h.t.Errorf("link while deleted, got %v, want EINVAL", err) + } + if err := file.RenameAt("src", file, "dst"); err != syscall.EINVAL { + h.t.Errorf("renameAt while deleted, got %v, want EINVAL", err) + } + if err := file.UnlinkAt("file", 0); err != syscall.EINVAL { + h.t.Errorf("unlinkAt while deleted, got %v, want EINVAL", err) + } + if err := file.Rename(file, "dst"); err != syscall.EINVAL { + h.t.Errorf("rename while deleted, got %v, want EINVAL", err) + } + if _, err := file.Readlink(); err != syscall.EINVAL { + h.t.Errorf("readlink while deleted, got %v, want EINVAL", err) + } + if _, err := file.Mkdir("dir", p9.ModeDirectory, 0, 0); err != syscall.EINVAL { + h.t.Errorf("mkdir while deleted, got %v, want EINVAL", err) + } + if _, err := file.Mknod("dir", p9.ModeDirectory, 0, 0, 0, 0); err != syscall.EINVAL { + h.t.Errorf("mknod while deleted, got %v, want EINVAL", err) + } + if _, err := file.Readdir(0, 1); err != syscall.EINVAL { + h.t.Errorf("readdir while deleted, got %v, want EINVAL", err) + } + if _, err := file.Connect(p9.ConnectFlags(0)); err != syscall.EINVAL { + h.t.Errorf("connect while deleted, got %v, want EINVAL", err) + } + + // The remove method is technically deprecated, but we want to ensure + // that it still checks for deleted appropriately. We must first clone + // the file because remove is equivalent to close. + _, newFile, err := file.Walk(nil) + if err == syscall.EBUSY { + // We can't walk from here because this reference is open + // aleady. Okay, we will also have unopened cases through + // TestUnlink, just skip the remove operation for now. + return + } else if err != nil { + h.t.Fatalf("clone failed, got %v, want nil", err) + } + if err := newFile.(deprecatedRemover).Remove(); err != syscall.EINVAL { + h.t.Errorf("remove while deleted, got %v, want EINVAL", err) + } +} + +// deleter is a function to remove a file. +type deleter func(parent p9.File, name string) error + +// unlinkAt is a deleter. +func unlinkAt(parent p9.File, name string) error { + // Call unlink. Note that a filesystem may normally impose additional + // constaints on unlinkat success, such as ensuring that a directory is + // empty, requiring AT_REMOVEDIR in flags to remove a directory, etc. + // None of that is required internally (entire trees can be marked + // deleted when this operation succeeds), so the mock will succeed. + return parent.UnlinkAt(name, 0) +} + +// remove is a deleter. +func remove(parent p9.File, name string) error { + // See notes above re: remove. + _, newFile, err := parent.Walk([]string{name}) + if err != nil { + // Should not be expected. + return err + } + + // Do the actual remove. + if err := newFile.(deprecatedRemover).Remove(); err != nil { + return err + } + + // Ensure that the remove closed the file. + if err := newFile.(deprecatedRemover).Remove(); err != syscall.EBADF { + return syscall.EBADF // Propagate this code. + } + + return nil +} + +// unlinkHelper unlinks the noted path, and ensures that all relevant +// operations on that path, acquired from multiple paths, start failing. +func unlinkHelper(h *Harness, root p9.File, targetNames []string, targetGen fileGenerator, deleteFn deleter) { + // name is the file to be unlinked. + name := targetNames[len(targetNames)-1] + + // Walk to the directory containing the target. + _, parent, err := root.Walk(targetNames[:len(targetNames)-1]) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer parent.Close() + parentBackend := h.Pop(parent) + + // Walk to or generate the target file. + _, _, target := targetGen(h, name, parent) + defer checkDeleted(h, target) + + // Walk to a second reference. + _, second, err := parent.Walk([]string{name}) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer checkDeleted(h, second) + + // Walk to a third reference, from the start. + _, third, err := root.Walk(targetNames) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer checkDeleted(h, third) + + // This will be translated in the backend to an unlinkat. + parentBackend.EXPECT().UnlinkAt(name, uint32(0)).Return(nil) + + // Actually perform the deletion. + if err := deleteFn(parent, name); err != nil { + h.t.Fatalf("got delete err %v, want nil", err) + } +} + +func unlinkTest(t *testing.T, targetNames []string, targetGen fileGenerator) { + t.Run(fmt.Sprintf("unlinkAt(%s)", strings.Join(targetNames, "/")), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + unlinkHelper(h, root, targetNames, targetGen, unlinkAt) + }) + t.Run(fmt.Sprintf("remove(%s)", strings.Join(targetNames, "/")), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + unlinkHelper(h, root, targetNames, targetGen, remove) + }) +} + +func TestUnlink(t *testing.T) { + // Unlink all files. + for name := range newTypeMap(nil) { + unlinkTest(t, []string{name}, walkHelper) + unlinkTest(t, []string{name}, walkAndOpenHelper) + unlinkTest(t, []string{"one", name}, walkHelper) + unlinkTest(t, []string{"one", name}, walkAndOpenHelper) + unlinkTest(t, []string{"one", "two", name}, walkHelper) + unlinkTest(t, []string{"one", "two", name}, walkAndOpenHelper) + } + + // Unlink a directory. + unlinkTest(t, []string{"one"}, walkHelper) + unlinkTest(t, []string{"one"}, walkAndOpenHelper) + unlinkTest(t, []string{"one", "two"}, walkHelper) + unlinkTest(t, []string{"one", "two"}, walkAndOpenHelper) + + // Unlink created files. + unlinkTest(t, []string{"created"}, createHelper) + unlinkTest(t, []string{"one", "created"}, createHelper) + unlinkTest(t, []string{"one", "two", "created"}, createHelper) +} + +func TestUnlinkAtInvalid(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + for name := range newTypeMap(nil) { + for _, invalidName := range allInvalidNames(name) { + if err := root.UnlinkAt(invalidName, 0); err != syscall.EINVAL { + t.Errorf("got %v for name %q, want EINVAL", err, invalidName) + } + } + } +} + +// expectRenamed asserts an ordered sequence of rename calls, based on all the +// elements in elements being the source, and the first element therein +// changing to dstName, parented at dstParent. +func expectRenamed(file *Mock, elements []string, dstParent *Mock, dstName string) *gomock.Call { + if len(elements) > 0 { + // Recurse to the parent, if necessary. + call := expectRenamed(file.parent, elements[:len(elements)-1], dstParent, dstName) + + // Recursive case: this element is unchanged, but should have + // it's hook called after the parent. + return file.EXPECT().Renamed(file.parent, elements[len(elements)-1]).Do(func(p p9.File, _ string) { + file.parent = p.(*Mock) + }).After(call) + } + + // Base case: this is the changed element. + return file.EXPECT().Renamed(dstParent, dstName).Do(func(p p9.File, name string) { + file.parent = p.(*Mock) + }) +} + +// renamer is a rename function. +type renamer func(h *Harness, srcParent, dstParent p9.File, origName, newName string, selfRename bool) error + +// renameAt is a renamer. +func renameAt(_ *Harness, srcParent, dstParent p9.File, srcName, dstName string, selfRename bool) error { + return srcParent.RenameAt(srcName, dstParent, dstName) +} + +// rename is a renamer. +func rename(h *Harness, srcParent, dstParent p9.File, srcName, dstName string, selfRename bool) error { + _, f, err := srcParent.Walk([]string{srcName}) + if err != nil { + return err + } + defer f.Close() + if !selfRename { + backend := h.Pop(f) + backend.EXPECT().Renamed(gomock.Any(), dstName).Do(func(p p9.File, name string) { + backend.parent = p.(*Mock) // Required for close ordering. + }) + } + return f.Rename(dstParent, dstName) +} + +// renameHelper executes a rename, and asserts that all relevant elements +// receive expected notifications. If overwriting a file, this includes +// ensuring that the target has been appropriately marked as unlinked. +func renameHelper(h *Harness, root p9.File, srcNames []string, dstNames []string, target fileGenerator, renameFn renamer) { + // Walk to the directory containing the target. + srcQID, targetParent, err := root.Walk(srcNames[:len(srcNames)-1]) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer targetParent.Close() + targetParentBackend := h.Pop(targetParent) + + // Walk to or generate the target file. + _, targetBackend, src := target(h, srcNames[len(srcNames)-1], targetParent) + defer src.Close() + + // Walk to a second reference. + _, second, err := targetParent.Walk([]string{srcNames[len(srcNames)-1]}) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer second.Close() + secondBackend := h.Pop(second) + + // Walk to a third reference, from the start. + _, third, err := root.Walk(srcNames) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer third.Close() + thirdBackend := h.Pop(third) + + // Find the common suffix to identify the rename parent. + var ( + renameDestPath []string + renameSrcPath []string + selfRename bool + ) + for i := 1; i <= len(srcNames) && i <= len(dstNames); i++ { + if srcNames[len(srcNames)-i] != dstNames[len(dstNames)-i] { + // Take the full prefix of dstNames up until this + // point, including the first mismatched name. The + // first mismatch must be the renamed entry. + renameDestPath = dstNames[:len(dstNames)-i+1] + renameSrcPath = srcNames[:len(srcNames)-i+1] + + // Does the renameDestPath fully contain the + // renameSrcPath here? If yes, then this is a mismatch. + // We can't rename the src to some subpath of itself. + if len(renameDestPath) > len(renameSrcPath) && + reflect.DeepEqual(renameDestPath[:len(renameSrcPath)], renameSrcPath) { + renameDestPath = nil + renameSrcPath = nil + continue + } + break + } + } + if len(renameSrcPath) == 0 || len(renameDestPath) == 0 { + // This must be a rename to self, or a tricky look-alike. This + // happens iff we fail to find a suitable divergence in the two + // paths. It's a true self move if the path length is the same. + renameDestPath = dstNames + renameSrcPath = srcNames + selfRename = len(srcNames) == len(dstNames) + } + + // Walk to the source parent. + _, srcParent, err := root.Walk(renameSrcPath[:len(renameSrcPath)-1]) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer srcParent.Close() + srcParentBackend := h.Pop(srcParent) + + // Walk to the destination parent. + _, dstParent, err := root.Walk(renameDestPath[:len(renameDestPath)-1]) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer dstParent.Close() + dstParentBackend := h.Pop(dstParent) + + // expectedErr is the result of the rename operation. + var expectedErr error + + // Walk to the target file, if one exists. + dstQID, dst, err := root.Walk(renameDestPath) + if err == nil { + if !selfRename && srcQID[0].Type == dstQID[0].Type { + // If there is a destination file, and is it of the + // same type as the source file, then we expect the + // rename to succeed. We expect the destination file to + // be deleted, so we run a deletion test on it in this + // case. + defer checkDeleted(h, dst) + } else { + if !selfRename { + // If the type is different than the + // destination, then we expect the rename to + // fail. We expect ensure that this is + // returned. + expectedErr = syscall.EINVAL + } else { + // This is the file being renamed to itself. + // This is technically allowed and a no-op, but + // all the triggers will fire. + } + dst.Close() + } + } + dstName := renameDestPath[len(renameDestPath)-1] // Renamed element. + srcName := renameSrcPath[len(renameSrcPath)-1] // Renamed element. + if expectedErr == nil && !selfRename { + // Expect all to be renamed appropriately. Note that if this is + // a final file being renamed, then we expect the file to be + // called with the new parent. If not, then we expect the + // rename hook to be called, but the parent will remain + // unchanged. + elements := srcNames[len(renameSrcPath):] + expectRenamed(targetBackend, elements, dstParentBackend, dstName) + expectRenamed(secondBackend, elements, dstParentBackend, dstName) + expectRenamed(thirdBackend, elements, dstParentBackend, dstName) + + // The target parent has also been opened, and may be moved + // directly or indirectly. + if len(elements) > 1 { + expectRenamed(targetParentBackend, elements[:len(elements)-1], dstParentBackend, dstName) + } + } + + // Expect the rename if it's not the same file. Note that like unlink, + // renames are always translated to the at variant in the backend. + if !selfRename { + srcParentBackend.EXPECT().RenameAt(srcName, dstParentBackend, dstName).Return(expectedErr) + } + + // Perform the actual rename; everything has been lined up. + if err := renameFn(h, srcParent, dstParent, srcName, dstName, selfRename); err != expectedErr { + h.t.Fatalf("got rename err %v, want %v", err, expectedErr) + } +} + +func renameTest(t *testing.T, srcNames []string, dstNames []string, target fileGenerator) { + t.Run(fmt.Sprintf("renameAt(%s->%s)", strings.Join(srcNames, "/"), strings.Join(dstNames, "/")), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + renameHelper(h, root, srcNames, dstNames, target, renameAt) + }) + t.Run(fmt.Sprintf("rename(%s->%s)", strings.Join(srcNames, "/"), strings.Join(dstNames, "/")), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + renameHelper(h, root, srcNames, dstNames, target, rename) + }) +} + +func TestRename(t *testing.T) { + // In-directory rename, simple case. + for name := range newTypeMap(nil) { + // Within the root. + renameTest(t, []string{name}, []string{"renamed"}, walkHelper) + renameTest(t, []string{name}, []string{"renamed"}, walkAndOpenHelper) + + // Within a subdirectory. + renameTest(t, []string{"one", name}, []string{"one", "renamed"}, walkHelper) + renameTest(t, []string{"one", name}, []string{"one", "renamed"}, walkAndOpenHelper) + } + + // ... with created files. + renameTest(t, []string{"created"}, []string{"renamed"}, createHelper) + renameTest(t, []string{"one", "created"}, []string{"one", "renamed"}, createHelper) + + // Across directories. + for name := range newTypeMap(nil) { + // Down one level. + renameTest(t, []string{"one", name}, []string{"one", "two", "renamed"}, walkHelper) + renameTest(t, []string{"one", name}, []string{"one", "two", "renamed"}, walkAndOpenHelper) + + // Up one level. + renameTest(t, []string{"one", "two", name}, []string{"one", "renamed"}, walkHelper) + renameTest(t, []string{"one", "two", name}, []string{"one", "renamed"}, walkAndOpenHelper) + + // Across at the same level. + renameTest(t, []string{"one", name}, []string{"three", "renamed"}, walkHelper) + renameTest(t, []string{"one", name}, []string{"three", "renamed"}, walkAndOpenHelper) + } + + // ... with created files. + renameTest(t, []string{"one", "created"}, []string{"one", "two", "renamed"}, createHelper) + renameTest(t, []string{"one", "two", "created"}, []string{"one", "renamed"}, createHelper) + renameTest(t, []string{"one", "created"}, []string{"three", "renamed"}, createHelper) + + // Renaming parents. + for name := range newTypeMap(nil) { + // Rename a parent. + renameTest(t, []string{"one", name}, []string{"renamed", name}, walkHelper) + renameTest(t, []string{"one", name}, []string{"renamed", name}, walkAndOpenHelper) + + // Rename a super parent. + renameTest(t, []string{"one", "two", name}, []string{"renamed", name}, walkHelper) + renameTest(t, []string{"one", "two", name}, []string{"renamed", name}, walkAndOpenHelper) + } + + // ... with created files. + renameTest(t, []string{"one", "created"}, []string{"renamed", "created"}, createHelper) + renameTest(t, []string{"one", "two", "created"}, []string{"renamed", "created"}, createHelper) + + // Over existing files, including itself. + for name := range newTypeMap(nil) { + for other := range newTypeMap(nil) { + // Overwrite the noted file (may be itself). + renameTest(t, []string{"one", name}, []string{"one", other}, walkHelper) + renameTest(t, []string{"one", name}, []string{"one", other}, walkAndOpenHelper) + + // Overwrite other files in another directory. + renameTest(t, []string{"one", name}, []string{"one", "two", other}, walkHelper) + renameTest(t, []string{"one", name}, []string{"one", "two", other}, walkAndOpenHelper) + } + + // Overwrite by moving the parent. + renameTest(t, []string{"three", name}, []string{"one", name}, walkHelper) + renameTest(t, []string{"three", name}, []string{"one", name}, walkAndOpenHelper) + + // Create over the types. + renameTest(t, []string{"one", "created"}, []string{"one", name}, createHelper) + renameTest(t, []string{"one", "created"}, []string{"one", "two", name}, createHelper) + renameTest(t, []string{"three", "created"}, []string{"one", name}, createHelper) + } +} + +func TestRenameInvalid(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + for name := range newTypeMap(nil) { + for _, invalidName := range allInvalidNames(name) { + if err := root.Rename(root, invalidName); err != syscall.EINVAL { + t.Errorf("got %v for name %q, want EINVAL", err, invalidName) + } + } + } +} + +func TestRenameAtInvalid(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + for name := range newTypeMap(nil) { + for _, invalidName := range allInvalidNames(name) { + if err := root.RenameAt(invalidName, root, "okay"); err != syscall.EINVAL { + t.Errorf("got %v for name %q, want EINVAL", err, invalidName) + } + if err := root.RenameAt("okay", root, invalidName); err != syscall.EINVAL { + t.Errorf("got %v for name %q, want EINVAL", err, invalidName) + } + } + } +} + +func TestReadlink(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + // Walk to the file normally. + _, f, err := root.Walk([]string{name}) + if err != nil { + t.Fatalf("walk failed: got %v, wanted nil", err) + } + defer f.Close() + backend := h.Pop(f) + + const symlinkTarget = "symlink-target" + + if backend.Attr.Mode.IsSymlink() { + // This should only go through on symlinks. + backend.EXPECT().Readlink().Return(symlinkTarget, nil) + } + + // Attempt a Readlink operation. + target, err := f.Readlink() + if err != nil && err != syscall.EINVAL { + t.Errorf("readlink got %v, wanted EINVAL", err) + } else if err == nil && target != symlinkTarget { + t.Errorf("readlink got %v, wanted %v", target, symlinkTarget) + } + }) + } +} + +// fdTest is a wrapper around operations that may send file descriptors. This +// asserts that the file descriptors are working as intended. +func fdTest(t *testing.T, sendFn func(*fd.FD) *fd.FD) { + // Create a pipe that we can read from. + r, w, err := os.Pipe() + if err != nil { + t.Fatalf("unable to create pipe: %v", err) + } + defer r.Close() + defer w.Close() + + // Attempt to send the write end. + wFD, err := fd.NewFromFile(w) + if err != nil { + t.Fatalf("unable to convert file: %v", err) + } + defer wFD.Close() // This is a copy. + + // Send wFD and receive newFD. + newFD := sendFn(wFD) + defer newFD.Close() + + // Attempt to write. + const message = "hello" + if _, err := newFD.Write([]byte(message)); err != nil { + t.Fatalf("write got %v, wanted nil", err) + } + + // Should see the message on our end. + buffer := []byte(message) + if _, err := io.ReadFull(r, buffer); err != nil { + t.Fatalf("read got %v, wanted nil", err) + } + if string(buffer) != message { + t.Errorf("got message %v, wanted %v", string(buffer), message) + } +} + +func TestConnect(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + // Walk to the file normally. + _, backend, f := walkHelper(h, name, root) + defer f.Close() + + // Catch all the non-socket cases. + if !backend.Attr.Mode.IsSocket() { + // This has been set up to fail if Connect is called. + if _, err := f.Connect(p9.ConnectFlags(0)); err != syscall.EINVAL { + t.Errorf("connect got %v, wanted EINVAL", err) + } + return + } + + // Ensure the fd exchange works. + fdTest(t, func(send *fd.FD) *fd.FD { + backend.EXPECT().Connect(p9.ConnectFlags(0)).Return(send, nil) + recv, err := backend.Connect(p9.ConnectFlags(0)) + if err != nil { + t.Fatalf("connect got %v, wanted nil", err) + } + return recv + }) + }) + } +} + +func TestReaddir(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + // Walk to the file normally. + _, backend, f := walkHelper(h, name, root) + defer f.Close() + + // Catch all the non-directory cases. + if !backend.Attr.Mode.IsDir() { + // This has also been set up to fail if Readdir is called. + if _, err := f.Readdir(0, 1); err != syscall.EINVAL { + t.Errorf("readdir got %v, wanted EINVAL", err) + } + return + } + + // Ensure that readdir works for directories. + if _, err := f.Readdir(0, 1); err != syscall.EINVAL { + t.Errorf("readdir got %v, wanted EINVAL", err) + } + if _, _, _, err := f.Open(p9.ReadWrite); err != syscall.EINVAL { + t.Errorf("readdir got %v, wanted EINVAL", err) + } + if _, _, _, err := f.Open(p9.WriteOnly); err != syscall.EINVAL { + t.Errorf("readdir got %v, wanted EINVAL", err) + } + backend.EXPECT().Open(p9.ReadOnly).Times(1) + if _, _, _, err := f.Open(p9.ReadOnly); err != nil { + t.Errorf("readdir got %v, wanted nil", err) + } + backend.EXPECT().Readdir(uint64(0), uint32(1)).Times(1) + if _, err := f.Readdir(0, 1); err != nil { + t.Errorf("readdir got %v, wanted nil", err) + } + }) + } +} + +func TestOpen(t *testing.T) { + type openTest struct { + name string + mode p9.OpenFlags + err error + match func(p9.FileMode) bool + } + + cases := []openTest{ + { + name: "invalid", + mode: ^p9.OpenFlagsModeMask, + err: syscall.EINVAL, + match: func(p9.FileMode) bool { return true }, + }, + { + name: "not-openable-read-only", + mode: p9.ReadOnly, + err: syscall.EINVAL, + match: func(mode p9.FileMode) bool { return !p9.CanOpen(mode) }, + }, + { + name: "not-openable-write-only", + mode: p9.WriteOnly, + err: syscall.EINVAL, + match: func(mode p9.FileMode) bool { return !p9.CanOpen(mode) }, + }, + { + name: "not-openable-read-write", + mode: p9.ReadWrite, + err: syscall.EINVAL, + match: func(mode p9.FileMode) bool { return !p9.CanOpen(mode) }, + }, + { + name: "directory-read-only", + mode: p9.ReadOnly, + err: nil, + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + }, + { + name: "directory-read-write", + mode: p9.ReadWrite, + err: syscall.EINVAL, + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + }, + { + name: "directory-write-only", + mode: p9.WriteOnly, + err: syscall.EINVAL, + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + }, + { + name: "read-only", + mode: p9.ReadOnly, + err: nil, + match: func(mode p9.FileMode) bool { return p9.CanOpen(mode) }, + }, + { + name: "write-only", + mode: p9.WriteOnly, + err: nil, + match: func(mode p9.FileMode) bool { return p9.CanOpen(mode) && !mode.IsDir() }, + }, + { + name: "read-write", + mode: p9.ReadWrite, + err: nil, + match: func(mode p9.FileMode) bool { return p9.CanOpen(mode) && !mode.IsDir() }, + }, + } + + // Open(mode OpenFlags) (*fd.FD, QID, uint32, error) + // - only works on Regular, NamedPipe, BLockDevice, CharacterDevice + // - returning a file works as expected + for name := range newTypeMap(nil) { + for _, tc := range cases { + t.Run(fmt.Sprintf("%s-%s", tc.name, name), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + // Walk to the file normally. + _, backend, f := walkHelper(h, name, root) + defer f.Close() + + // Does this match the case? + if !tc.match(backend.Attr.Mode) { + t.SkipNow() + } + + // Ensure open-required operations fail. + if _, err := f.ReadAt([]byte("hello"), 0); err != syscall.EINVAL { + t.Errorf("readAt got %v, wanted EINVAL", err) + } + if _, err := f.WriteAt(make([]byte, 6), 0); err != syscall.EINVAL { + t.Errorf("writeAt got %v, wanted EINVAL", err) + } + if err := f.FSync(); err != syscall.EINVAL { + t.Errorf("fsync got %v, wanted EINVAL", err) + } + if _, err := f.Readdir(0, 1); err != syscall.EINVAL { + t.Errorf("readdir got %v, wanted EINVAL", err) + } + + // Attempt the given open. + if tc.err != nil { + // We expect an error, just test and return. + if _, _, _, err := f.Open(tc.mode); err != tc.err { + t.Fatalf("open with mode %v got %v, want %v", tc.mode, err, tc.err) + } + return + } + + // Run an FD test, since we expect success. + fdTest(t, func(send *fd.FD) *fd.FD { + backend.EXPECT().Open(tc.mode).Return(send, p9.QID{}, uint32(0), nil).Times(1) + recv, _, _, err := f.Open(tc.mode) + if err != tc.err { + t.Fatalf("open with mode %v got %v, want %v", tc.mode, err, tc.err) + } + return recv + }) + + // If the open was successful, attempt another one. + if _, _, _, err := f.Open(tc.mode); err != syscall.EINVAL { + t.Errorf("second open with mode %v got %v, want EINVAL", tc.mode, err) + } + + // Ensure that all illegal operations fail. + if _, _, err := f.Walk(nil); err != syscall.EINVAL && err != syscall.EBUSY { + t.Errorf("walk got %v, wanted EINVAL or EBUSY", err) + } + if _, _, _, _, err := f.WalkGetAttr(nil); err != syscall.EINVAL && err != syscall.EBUSY { + t.Errorf("walkgetattr got %v, wanted EINVAL or EBUSY", err) + } + }) + } + } +} + +func TestClose(t *testing.T) { + type closeTest struct { + name string + closeFn func(backend *Mock, f p9.File) + } + + cases := []closeTest{ + { + name: "close", + closeFn: func(_ *Mock, f p9.File) { + f.Close() + }, + }, + { + name: "remove", + closeFn: func(backend *Mock, f p9.File) { + // Allow the rename call in the parent, automatically translated. + backend.parent.EXPECT().UnlinkAt(gomock.Any(), gomock.Any()).Times(1) + f.(deprecatedRemover).Remove() + }, + }, } - os.Remove(osFile.Name()) - hfi, err := osFile.Stat() - if err != nil { - osFile.Close() - t.Fatalf("stat failed: %v", err) + for name := range newTypeMap(nil) { + for _, tc := range cases { + t.Run(fmt.Sprintf("%s(%s)", tc.name, name), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + // Walk to the file normally. + _, backend, f := walkHelper(h, name, root) + + // Close via the prescribed method. + tc.closeFn(backend, f) + + // Everything should fail with EBADF. + if _, _, err := f.Walk(nil); err != syscall.EBADF { + t.Errorf("walk got %v, wanted EBADF", err) + } + if _, err := f.StatFS(); err != syscall.EBADF { + t.Errorf("statfs got %v, wanted EBADF", err) + } + if _, _, _, err := f.GetAttr(p9.AttrMaskAll()); err != syscall.EBADF { + t.Errorf("getattr got %v, wanted EBADF", err) + } + if err := f.SetAttr(p9.SetAttrMask{}, p9.SetAttr{}); err != syscall.EBADF { + t.Errorf("setattrk got %v, wanted EBADF", err) + } + if err := f.Rename(root, "new-name"); err != syscall.EBADF { + t.Errorf("rename got %v, wanted EBADF", err) + } + if err := f.Close(); err != syscall.EBADF { + t.Errorf("close got %v, wanted EBADF", err) + } + if _, _, _, err := f.Open(p9.ReadOnly); err != syscall.EBADF { + t.Errorf("open got %v, wanted EBADF", err) + } + if _, err := f.ReadAt([]byte("hello"), 0); err != syscall.EBADF { + t.Errorf("readAt got %v, wanted EBADF", err) + } + if _, err := f.WriteAt(make([]byte, 6), 0); err != syscall.EBADF { + t.Errorf("writeAt got %v, wanted EBADF", err) + } + if err := f.FSync(); err != syscall.EBADF { + t.Errorf("fsync got %v, wanted EBADF", err) + } + if _, _, _, _, err := f.Create("new-file", p9.ReadWrite, 0, 0, 0); err != syscall.EBADF { + t.Errorf("create got %v, wanted EBADF", err) + } + if _, err := f.Mkdir("new-directory", 0, 0, 0); err != syscall.EBADF { + t.Errorf("mkdir got %v, wanted EBADF", err) + } + if _, err := f.Symlink("old-name", "new-name", 0, 0); err != syscall.EBADF { + t.Errorf("symlink got %v, wanted EBADF", err) + } + if err := f.Link(root, "new-name"); err != syscall.EBADF { + t.Errorf("link got %v, wanted EBADF", err) + } + if _, err := f.Mknod("new-block-device", 0, 0, 0, 0, 0); err != syscall.EBADF { + t.Errorf("mknod got %v, wanted EBADF", err) + } + if err := f.RenameAt("old-name", root, "new-name"); err != syscall.EBADF { + t.Errorf("renameAt got %v, wanted EBADF", err) + } + if err := f.UnlinkAt("name", 0); err != syscall.EBADF { + t.Errorf("unlinkAt got %v, wanted EBADF", err) + } + if _, err := f.Readdir(0, 1); err != syscall.EBADF { + t.Errorf("readdir got %v, wanted EBADF", err) + } + if _, err := f.Readlink(); err != syscall.EBADF { + t.Errorf("readlink got %v, wanted EBADF", err) + } + if err := f.Flush(); err != syscall.EBADF { + t.Errorf("flush got %v, wanted EBADF", err) + } + if _, _, _, _, err := f.WalkGetAttr(nil); err != syscall.EBADF { + t.Errorf("walkgetattr got %v, wanted EBADF", err) + } + if _, err := f.Connect(p9.ConnectFlags(0)); err != syscall.EBADF { + t.Errorf("connect got %v, wanted EBADF", err) + } + }) + } + } +} + +// onlyWorksOnOpenThings is a helper test method for operations that should +// only work on files that have been explicitly opened. +func onlyWorksOnOpenThings(h *Harness, t *testing.T, name string, root p9.File, mode p9.OpenFlags, expectedErr error, fn func(backend *Mock, f p9.File, shouldSucceed bool) error) { + // Walk to the file normally. + _, backend, f := walkHelper(h, name, root) + defer f.Close() + + // Does it work before opening? + if err := fn(backend, f, false); err != syscall.EINVAL { + t.Errorf("operation got %v, wanted EINVAL", err) } - osFileStat := hfi.Sys().(*syscall.Stat_t) - f, err := fd.NewFromFile(osFile) - // osFile should always be closed. - osFile.Close() - if err != nil { - t.Fatalf("unable to create file: %v", err) + // Is this openable? + if !p9.CanOpen(backend.Attr.Mode) { + return // Nothing to do. + } + + // If this is a directory, we can't handle writing. + if backend.Attr.Mode.IsDir() && (mode == p9.ReadWrite || mode == p9.WriteOnly) { + return // Skip. + } + + // Open the file. + backend.EXPECT().Open(mode) + if _, _, _, err := f.Open(mode); err != nil { + t.Fatalf("open got %v, wanted nil", err) + } + + // Attempt the operation. + if err := fn(backend, f, expectedErr == nil); err != expectedErr { + t.Fatalf("operation got %v, wanted %v", err, expectedErr) + } +} + +func TestRead(t *testing.T) { + type readTest struct { + name string + mode p9.OpenFlags + err error + } + + cases := []readTest{ + { + name: "read-only", + mode: p9.ReadOnly, + err: nil, + }, + { + name: "read-write", + mode: p9.ReadWrite, + err: nil, + }, + { + name: "write-only", + mode: p9.WriteOnly, + err: syscall.EPERM, + }, + } + + for name := range newTypeMap(nil) { + for _, tc := range cases { + t.Run(fmt.Sprintf("%s-%s", tc.name, name), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + const message = "hello" + + onlyWorksOnOpenThings(h, t, name, root, tc.mode, tc.err, func(backend *Mock, f p9.File, shouldSucceed bool) error { + if !shouldSucceed { + _, err := f.ReadAt([]byte(message), 0) + return err + } + + // Prepare for the call to readAt in the backend. + backend.EXPECT().ReadAt(gomock.Any(), uint64(0)).Do(func(p []byte, offset uint64) { + copy(p, message) + }).Return(len(message), nil) + + // Make the client call. + p := make([]byte, 2*len(message)) // Double size. + n, err := f.ReadAt(p, 0) + + // Sanity check result. + if err != nil { + return err + } + if n != len(message) { + t.Fatalf("message length incorrect, got %d, want %d", n, len(message)) + } + if !bytes.Equal(p[:n], []byte(message)) { + t.Fatalf("message incorrect, got %v, want %v", p, []byte(message)) + } + return nil // Success. + }) + }) + } + } +} + +func TestWrite(t *testing.T) { + type writeTest struct { + name string + mode p9.OpenFlags + err error } - // Craft attacher to attach to the mocked file which will return our - // temporary file. - fileMock := &FileMock{ - OpenMock: OpenMock{File: f}, - GetAttrMock: GetAttrMock{ - // The mode must be valid always. - Valid: p9.AttrMask{Mode: true}, + cases := []writeTest{ + { + name: "read-only", + mode: p9.ReadOnly, + err: syscall.EPERM, + }, + { + name: "read-write", + mode: p9.ReadWrite, + err: nil, + }, + { + name: "write-only", + mode: p9.WriteOnly, + err: nil, }, } - attacher := &AttachMock{ - File: fileMock, + + for name := range newTypeMap(nil) { + for _, tc := range cases { + t.Run(fmt.Sprintf("%s-%s", tc.name, name), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + const message = "hello" + + onlyWorksOnOpenThings(h, t, name, root, tc.mode, tc.err, func(backend *Mock, f p9.File, shouldSucceed bool) error { + if !shouldSucceed { + _, err := f.WriteAt([]byte(message), 0) + return err + } + + // Prepare for the call to readAt in the backend. + var output []byte // Saved by Do below. + backend.EXPECT().WriteAt(gomock.Any(), uint64(0)).Do(func(p []byte, offset uint64) { + output = p + }).Return(len(message), nil) + + // Make the client call. + n, err := f.WriteAt([]byte(message), 0) + + // Sanity check result. + if err != nil { + return err + } + if n != len(message) { + t.Fatalf("message length incorrect, got %d, want %d", n, len(message)) + } + if !bytes.Equal(output, []byte(message)) { + t.Fatalf("message incorrect, got %v, want %v", output, []byte(message)) + } + return nil // Success. + }) + }) + } } +} - // Make socket pair. - serverSocket, clientSocket, err := unet.SocketPair(false) - if err != nil { - t.Fatalf("socketpair got err %v wanted nil", err) +func TestFSync(t *testing.T) { + for name := range newTypeMap(nil) { + for _, mode := range []p9.OpenFlags{p9.ReadOnly, p9.WriteOnly, p9.ReadWrite} { + t.Run(fmt.Sprintf("%s-%s", mode, name), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + onlyWorksOnOpenThings(h, t, name, root, mode, nil, func(backend *Mock, f p9.File, shouldSucceed bool) error { + if shouldSucceed { + backend.EXPECT().FSync().Times(1) + } + return f.FSync() + }) + }) + } } - defer clientSocket.Close() - server := p9.NewServer(attacher) - go server.Handle(serverSocket) - client, err := p9.NewClient(clientSocket, 1024*1024 /* 1M message size */, p9.HighestVersionString()) - if err != nil { - t.Fatalf("new client got %v, expected nil", err) +} + +func TestFlush(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + _, backend, f := walkHelper(h, name, root) + defer f.Close() + + backend.EXPECT().Flush() + f.Flush() + }) } +} - // Attach to the mocked file. - cFile, err := client.Attach("") - if err != nil { - t.Fatalf("attach failed: %v", err) +// onlyWorksOnDirectories is a helper test method for operations that should +// only work on unopened directories, such as create, mkdir and symlink. +func onlyWorksOnDirectories(h *Harness, t *testing.T, name string, root p9.File, fn func(backend *Mock, f p9.File, shouldSucceed bool) error) { + // Walk to the file normally. + _, backend, f := walkHelper(h, name, root) + defer f.Close() + + // Only directories support mknod. + if !backend.Attr.Mode.IsDir() { + if err := fn(backend, f, false); err != syscall.EINVAL { + t.Errorf("operation got %v, wanted EINVAL", err) + } + return // Nothing else to do. } - // Try to open the mocked file. - clientHostFile, _, _, err := cFile.Open(0) - if err != nil { - t.Fatalf("open failed: %v", err) + // Should succeed. + if err := fn(backend, f, true); err != nil { + t.Fatalf("operation got %v, wanted nil", err) } - var clientStat syscall.Stat_t - if err := syscall.Fstat(clientHostFile.FD(), &clientStat); err != nil { - t.Fatalf("stat failed: %v", err) + + // Open the directory. + backend.EXPECT().Open(p9.ReadOnly).Times(1) + if _, _, _, err := f.Open(p9.ReadOnly); err != nil { + t.Fatalf("open got %v, wanted nil", err) } - // Compare inode nums to make sure it's the same file. - if clientStat.Ino != osFileStat.Ino { - t.Errorf("fd donation failed") + // Should not work again. + if err := fn(backend, f, false); err != syscall.EINVAL { + t.Fatalf("operation got %v, wanted EINVAL", err) } } -// TestClient is a megatest. -// -// This allows us to probe various edge cases, while changing the state of the -// underlying server in expected ways. The test slowly builds server state and -// is documented inline. -// -// We wind up with the following, after probing edge cases: -// -// FID 1: ServerFile (sf). -// FID 2: Directory (d). -// FID 3: File (f). -// FID 4: Symlink (s). -// -// Although you should use the FID method on the individual files. -func TestClient(t *testing.T) { +func TestCreate(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + onlyWorksOnDirectories(h, t, name, root, func(backend *Mock, f p9.File, shouldSucceed bool) error { + if !shouldSucceed { + _, _, _, _, err := f.Create("new-file", p9.ReadWrite, 0, 1, 2) + return err + } + + // If the create is going to succeed, then we + // need to create a new backend file, and we + // clone to ensure that we don't close the + // original. + _, newF, err := f.Walk(nil) + if err != nil { + t.Fatalf("clone got %v, wanted nil", err) + } + defer newF.Close() + newBackend := h.Pop(newF) + + // Run a regular FD test to validate that path. + fdTest(t, func(send *fd.FD) *fd.FD { + // Return the send FD on success. + newFile := h.NewFile()(backend) // New file with the parent backend. + newBackend.EXPECT().Create("new-file", p9.ReadWrite, p9.FileMode(0), p9.UID(1), p9.GID(2)).Return(send, newFile, p9.QID{}, uint32(0), nil) + + // Receive the fd back. + recv, _, _, _, err := newF.Create("new-file", p9.ReadWrite, 0, 1, 2) + if err != nil { + t.Fatalf("create got %v, wanted nil", err) + } + return recv + }) + + // The above will fail via normal test flow, so + // we can assume that it passed. + return nil + }) + }) + } +} + +func TestCreateInvalid(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + for name := range newTypeMap(nil) { + for _, invalidName := range allInvalidNames(name) { + if _, _, _, _, err := root.Create(invalidName, p9.ReadWrite, 0, 0, 0); err != syscall.EINVAL { + t.Errorf("got %v for name %q, want EINVAL", err, invalidName) + } + } + } +} + +func TestMkdir(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + onlyWorksOnDirectories(h, t, name, root, func(backend *Mock, f p9.File, shouldSucceed bool) error { + if shouldSucceed { + backend.EXPECT().Mkdir("new-directory", p9.FileMode(0), p9.UID(1), p9.GID(2)) + } + _, err := f.Mkdir("new-directory", 0, 1, 2) + return err + }) + }) + } +} + +func TestMkdirInvalid(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + for name := range newTypeMap(nil) { + for _, invalidName := range allInvalidNames(name) { + if _, err := root.Mkdir(invalidName, 0, 0, 0); err != syscall.EINVAL { + t.Errorf("got %v for name %q, want EINVAL", err, invalidName) + } + } + } +} + +func TestSymlink(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + onlyWorksOnDirectories(h, t, name, root, func(backend *Mock, f p9.File, shouldSucceed bool) error { + if shouldSucceed { + backend.EXPECT().Symlink("old-name", "new-name", p9.UID(1), p9.GID(2)) + } + _, err := f.Symlink("old-name", "new-name", 1, 2) + return err + }) + }) + } +} + +func TestSyminkInvalid(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + for name := range newTypeMap(nil) { + for _, invalidName := range allInvalidNames(name) { + // We need only test for invalid names in the new name, + // the target can be an arbitrary string and we don't + // need to sanity check it. + if _, err := root.Symlink("old-name", invalidName, 0, 0); err != syscall.EINVAL { + t.Errorf("got %v for name %q, want EINVAL", err, invalidName) + } + } + } +} + +func TestLink(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + onlyWorksOnDirectories(h, t, name, root, func(backend *Mock, f p9.File, shouldSucceed bool) error { + if shouldSucceed { + backend.EXPECT().Link(gomock.Any(), "new-link") + } + return f.Link(f, "new-link") + }) + }) + } +} + +func TestLinkInvalid(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + for name := range newTypeMap(nil) { + for _, invalidName := range allInvalidNames(name) { + if err := root.Link(root, invalidName); err != syscall.EINVAL { + t.Errorf("got %v for name %q, want EINVAL", err, invalidName) + } + } + } +} + +func TestMknod(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + onlyWorksOnDirectories(h, t, name, root, func(backend *Mock, f p9.File, shouldSucceed bool) error { + if shouldSucceed { + backend.EXPECT().Mknod("new-block-device", p9.FileMode(0), uint32(1), uint32(2), p9.UID(3), p9.GID(4)).Times(1) + } + _, err := f.Mknod("new-block-device", 0, 1, 2, 3, 4) + return err + }) + }) + } +} + +// concurrentFn is a specification of a concurrent operation. This is used to +// drive the concurrency tests below. +type concurrentFn struct { + name string + match func(p9.FileMode) bool + op func(h *Harness, backend *Mock, f p9.File, callback func()) +} + +func concurrentTest(t *testing.T, name string, fn1, fn2 concurrentFn, sameDir, expectedOkay bool) { var ( - // Sentinel error. - sentinelErr = syscall.Errno(4383) - - // Backend mocks. - a = &AttachMock{} - sf = &FileMock{} - d = &FileMock{} - f = &FileMock{} - s = &FileMock{} - - // Client Files for the above. - sfFile p9.File + names1 []string + names2 []string ) + if sameDir { + // Use the same file one directory up. + names1, names2 = []string{"one", name}, []string{"one", name} + } else { + // For different directories, just use siblings. + names1, names2 = []string{"one", name}, []string{"three", name} + } - testSteps := []struct { - name string - fn func(*p9.Client) error - want error - }{ - { - name: "bad-attach", - want: sentinelErr, - fn: func(c *p9.Client) error { - a.File = nil - a.Err = sentinelErr - _, err := c.Attach("") - return err + t.Run(fmt.Sprintf("%s(%v)+%s(%v)", fn1.name, names1, fn2.name, names2), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + // Walk to both files as given. + _, f1, err := root.Walk(names1) + if err != nil { + t.Fatalf("error walking, got %v, want nil", err) + } + defer f1.Close() + b1 := h.Pop(f1) + _, f2, err := root.Walk(names2) + if err != nil { + t.Fatalf("error walking, got %v, want nil", err) + } + defer f2.Close() + b2 := h.Pop(f2) + + // Are these a good match for the current test case? + if !fn1.match(b1.Attr.Mode) { + t.SkipNow() + } + if !fn2.match(b2.Attr.Mode) { + t.SkipNow() + } + + // Construct our "concurrency creator". + in1 := make(chan struct{}, 1) + in2 := make(chan struct{}, 1) + var top sync.WaitGroup + var fns sync.WaitGroup + defer top.Wait() + top.Add(2) // Accounting for below. + defer fns.Done() + fns.Add(1) // See line above; released before top.Wait. + go func() { + defer top.Done() + fn1.op(h, b1, f1, func() { + in1 <- struct{}{} + fns.Wait() + }) + }() + go func() { + defer top.Done() + fn2.op(h, b2, f2, func() { + in2 <- struct{}{} + fns.Wait() + }) + }() + + // Compute a reasonable timeout. If we expect the operation to hang, + // give it 10 milliseconds before we assert that it's fine. After all, + // there will be a lot of these tests. If we don't expect it to hang, + // give it a full minute, since the machine could be slow. + timeout := 10 * time.Millisecond + if expectedOkay { + timeout = 1 * time.Minute + } + + // Read the first channel. + var second chan struct{} + select { + case <-in1: + second = in2 + case <-in2: + second = in1 + } + + // Catch concurrency. + select { + case <-second: + // We finished successful. Is this good? Depends on the + // expected result. + if !expectedOkay { + t.Errorf("%q and %q proceeded concurrently!", fn1.name, fn2.name) + } + case <-time.After(timeout): + // Great, things did not proceed concurrently. Is that what we + // expected? + if expectedOkay { + t.Errorf("%q and %q hung concurrently!", fn1.name, fn2.name) + } + } + }) +} + +func randomFileName() string { + return fmt.Sprintf("%x", rand.Int63()) +} + +func TestConcurrency(t *testing.T) { + readExclusive := []concurrentFn{ + { + // N.B. We can't explicitly check WalkGetAttr behavior, + // but we rely on the fact that the internal code paths + // are the same. + name: "walk", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + // See the documentation of WalkCallback. + // Because walk is actually implemented by the + // mock, we need a special place for this + // callback. + // + // Note that a clone actually locks the parent + // node. So we walk from this node to test + // concurrent operations appropriately. + backend.WalkCallback = func() error { + callback() + return nil + } + f.Walk([]string{randomFileName()}) // Won't exist. }, }, { - name: "attach", - fn: func(c *p9.Client) error { - a.Called = false - a.File = sf - a.Err = nil - // The attached root must have a valid mode. - sf.GetAttrMock.Attr = p9.Attr{Mode: p9.ModeDirectory} - sf.GetAttrMock.Valid = p9.AttrMask{Mode: true} - var err error - sfFile, err = c.Attach("") - if !a.Called { - t.Errorf("Attach never Called?") - } - return err + name: "fsync", + match: func(mode p9.FileMode) bool { return p9.CanOpen(mode) }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Open(gomock.Any()) + backend.EXPECT().FSync().Do(func() { + callback() + }) + f.Open(p9.ReadOnly) // Required. + f.FSync() }, }, { - name: "bad-walk", - want: sentinelErr, - fn: func(c *p9.Client) error { - // Walk only called when WalkGetAttr not available. - sf.WalkGetAttrMock.Err = syscall.ENOSYS - sf.WalkMock.File = d - sf.WalkMock.Err = sentinelErr - _, _, err := sfFile.Walk([]string{"foo", "bar"}) - return err + name: "readdir", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Open(gomock.Any()) + backend.EXPECT().Readdir(gomock.Any(), gomock.Any()).Do(func(uint64, uint32) { + callback() + }) + f.Open(p9.ReadOnly) // Required. + f.Readdir(0, 1) }, }, { - name: "walk-to-dir", - fn: func(c *p9.Client) error { - // Walk only called when WalkGetAttr not available. - sf.WalkGetAttrMock.Err = syscall.ENOSYS - sf.WalkMock.Called = false - sf.WalkMock.Names = nil - sf.WalkMock.File = d - sf.WalkMock.Err = nil - sf.WalkMock.QIDs = []p9.QID{{Type: 1}} - // All intermediate values must be directories. - d.WalkGetAttrMock.Err = syscall.ENOSYS - d.WalkMock.Called = false - d.WalkMock.Names = nil - d.WalkMock.File = d // Walk to self. - d.WalkMock.Err = nil - d.WalkMock.QIDs = []p9.QID{{Type: 1}} - d.GetAttrMock.Attr = p9.Attr{Mode: p9.ModeDirectory} - d.GetAttrMock.Valid = p9.AttrMask{Mode: true} - var qids []p9.QID - var err error - qids, _, err = sfFile.Walk([]string{"foo", "bar"}) - if !sf.WalkMock.Called { - t.Errorf("Walk never Called?") - } - if !d.GetAttrMock.Called { - t.Errorf("GetAttr never Called?") - } - if !reflect.DeepEqual(sf.WalkMock.Names, []string{"foo"}) { - t.Errorf("got names %v wanted []{foo}", sf.WalkMock.Names) - } - if !reflect.DeepEqual(d.WalkMock.Names, []string{"bar"}) { - t.Errorf("got names %v wanted []{bar}", d.WalkMock.Names) - } - if len(qids) != 2 || qids[len(qids)-1].Type != 1 { - t.Errorf("got qids %v wanted []{..., {Type: 1}}", qids) - } - return err + name: "readlink", + match: func(mode p9.FileMode) bool { return mode.IsSymlink() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Readlink().Do(func() { + callback() + }) + f.Readlink() }, }, { - name: "walkgetattr-to-dir", - fn: func(c *p9.Client) error { - sf.WalkGetAttrMock.Called = false - sf.WalkGetAttrMock.Names = nil - sf.WalkGetAttrMock.File = d - sf.WalkGetAttrMock.Err = nil - sf.WalkGetAttrMock.QIDs = []p9.QID{{Type: 1}} - sf.WalkGetAttrMock.Attr = p9.Attr{Mode: p9.ModeDirectory, UID: 1} - sf.WalkGetAttrMock.Valid = p9.AttrMask{Mode: true} - // See above. - d.WalkGetAttrMock.Called = false - d.WalkGetAttrMock.Names = nil - d.WalkGetAttrMock.File = d // Walk to self. - d.WalkGetAttrMock.Err = nil - d.WalkGetAttrMock.QIDs = []p9.QID{{Type: 1}} - d.WalkGetAttrMock.Attr = p9.Attr{Mode: p9.ModeDirectory, UID: 1} - d.WalkGetAttrMock.Valid = p9.AttrMask{Mode: true} - var qids []p9.QID - var err error - var mask p9.AttrMask - var attr p9.Attr - qids, _, mask, attr, err = sfFile.WalkGetAttr([]string{"foo", "bar"}) - if !sf.WalkGetAttrMock.Called { - t.Errorf("Walk never Called?") - } - if !reflect.DeepEqual(sf.WalkGetAttrMock.Names, []string{"foo"}) { - t.Errorf("got names %v wanted []{foo}", sf.WalkGetAttrMock.Names) - } - if !reflect.DeepEqual(d.WalkGetAttrMock.Names, []string{"bar"}) { - t.Errorf("got names %v wanted []{bar}", d.WalkGetAttrMock.Names) - } - if len(qids) != 2 || qids[len(qids)-1].Type != 1 { - t.Errorf("got qids %v wanted []{..., {Type: 1}}", qids) - } - if !reflect.DeepEqual(attr, sf.WalkGetAttrMock.Attr) { - t.Errorf("got attrs %s wanted %s", attr, sf.WalkGetAttrMock.Attr) - } - if !reflect.DeepEqual(mask, sf.WalkGetAttrMock.Valid) { - t.Errorf("got mask %s wanted %s", mask, sf.WalkGetAttrMock.Valid) - } - return err + name: "connect", + match: func(mode p9.FileMode) bool { return mode.IsSocket() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Connect(gomock.Any()).Do(func(p9.ConnectFlags) { + callback() + }) + f.Connect(0) }, }, { - name: "walk-to-file", - fn: func(c *p9.Client) error { - // Basic sanity check is done in walk-to-dir. - // - // Here we just create basic file FIDs to use. - sf.WalkMock.File = f - sf.WalkMock.Err = nil - var err error - _, _, err = sfFile.Walk(nil) - return err + name: "open", + match: func(mode p9.FileMode) bool { return p9.CanOpen(mode) }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Open(gomock.Any()).Do(func(p9.OpenFlags) { + callback() + }) + f.Open(p9.ReadOnly) }, }, { - name: "walk-to-symlink", - fn: func(c *p9.Client) error { - // See note in walk-to-file. - sf.WalkMock.File = s - sf.WalkMock.Err = nil - var err error - _, _, err = sfFile.Walk(nil) - return err + name: "flush", + match: func(mode p9.FileMode) bool { return true }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Flush().Do(func() { + callback() + }) + f.Flush() + }, + }, + } + writeExclusive := []concurrentFn{ + { + // N.B. We can't really check getattr. But this is an + // extremely low-risk function, it seems likely that + // this check is paranoid anyways. + name: "setattr", + match: func(mode p9.FileMode) bool { return true }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().SetAttr(gomock.Any(), gomock.Any()).Do(func(p9.SetAttrMask, p9.SetAttr) { + callback() + }) + f.SetAttr(p9.SetAttrMask{}, p9.SetAttr{}) }, }, { - name: "bad-statfs", - want: sentinelErr, - fn: func(c *p9.Client) error { - sf.StatFSMock.Err = sentinelErr - _, err := sfFile.StatFS() - return err + name: "unlinkAt", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().UnlinkAt(gomock.Any(), gomock.Any()).Do(func(string, uint32) { + callback() + }) + f.UnlinkAt(randomFileName(), 0) }, }, { - name: "statfs", - fn: func(c *p9.Client) error { - sf.StatFSMock.Called = false - sf.StatFSMock.Stat = p9.FSStat{Type: 1} - sf.StatFSMock.Err = nil - stat, err := sfFile.StatFS() - if !sf.StatFSMock.Called { - t.Errorf("StatfS never Called?") - } - if stat.Type != 1 { - t.Errorf("got stat %v wanted {Type: 1}", stat) - } - return err + name: "mknod", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Mknod(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Do(func(string, p9.FileMode, uint32, uint32, p9.UID, p9.GID) { + callback() + }) + f.Mknod(randomFileName(), 0, 0, 0, 0, 0) + }, + }, + { + name: "link", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Link(gomock.Any(), gomock.Any()).Do(func(p9.File, string) { + callback() + }) + f.Link(f, randomFileName()) + }, + }, + { + name: "symlink", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Symlink(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Do(func(string, string, p9.UID, p9.GID) { + callback() + }) + f.Symlink(randomFileName(), randomFileName(), 0, 0) + }, + }, + { + name: "mkdir", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Mkdir(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Do(func(string, p9.FileMode, p9.UID, p9.GID) { + callback() + }) + f.Mkdir(randomFileName(), 0, 0, 0) + }, + }, + { + name: "create", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + // Return an error for the creation operation, as this is the simplest. + backend.EXPECT().Create(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, nil, p9.QID{}, uint32(0), syscall.EINVAL).Do(func(string, p9.OpenFlags, p9.FileMode, p9.UID, p9.GID) { + callback() + }) + f.Create(randomFileName(), p9.ReadOnly, 0, 0, 0) }, }, } + globalExclusive := []concurrentFn{ + { + name: "remove", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + // Remove operates on a locked parent. So we + // add a child, walk to it and call remove. + // Note that because this operation can operate + // concurrently with itself, we need to + // generate a random file name. + randomFile := randomFileName() + backend.AddChild(randomFile, h.NewFile()) + defer backend.RemoveChild(randomFile) + _, file, err := f.Walk([]string{randomFile}) + if err != nil { + h.t.Fatalf("walk got %v, want nil", err) + } - // First, create a new server and connection. - serverSocket, clientSocket, err := unet.SocketPair(false) - if err != nil { - t.Fatalf("socketpair got err %v wanted nil", err) - } - defer clientSocket.Close() - server := p9.NewServer(a) - go server.Handle(serverSocket) - client, err := p9.NewClient(clientSocket, 1024*1024 /* 1M message size */, p9.HighestVersionString()) - if err != nil { - t.Fatalf("new client got err %v, wanted nil", err) - } + // Remove is automatically translated to the parent. + backend.EXPECT().UnlinkAt(gomock.Any(), gomock.Any()).Do(func(string, uint32) { + callback() + }) - // Now, run through each of the test steps. - for _, step := range testSteps { - err := step.fn(client) - if err != step.want { - // Don't fail, just note this one step failed. - t.Errorf("step %q got %v wanted %v", step.name, err, step.want) - } - } -} + // Remove is also a close. + file.(deprecatedRemover).Remove() + }, + }, + { + name: "rename", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + // Similarly to remove, because we need to + // operate on a child, we allow a walk. + randomFile := randomFileName() + backend.AddChild(randomFile, h.NewFile()) + defer backend.RemoveChild(randomFile) + _, file, err := f.Walk([]string{randomFile}) + if err != nil { + h.t.Fatalf("walk got %v, want nil", err) + } + defer file.Close() + fileBackend := h.Pop(file) -func BenchmarkClient(b *testing.B) { - // Backend mock. - a := &AttachMock{ - File: &FileMock{ - ReadAtMock: ReadAtMock{N: 1}, + // Rename is automatically translated to the parent. + backend.EXPECT().RenameAt(gomock.Any(), gomock.Any(), gomock.Any()).Do(func(string, p9.File, string) { + callback() + }) + + // Attempt the rename. + fileBackend.EXPECT().Renamed(gomock.Any(), gomock.Any()) + file.Rename(f, randomFileName()) + }, }, - } + { + name: "renameAt", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().RenameAt(gomock.Any(), gomock.Any(), gomock.Any()).Do(func(string, p9.File, string) { + callback() + }) - // First, create a new server and connection. - serverSocket, clientSocket, err := unet.SocketPair(false) - if err != nil { - b.Fatalf("socketpair got err %v wanted nil", err) - } - defer clientSocket.Close() - server := p9.NewServer(a) - go server.Handle(serverSocket) - client, err := p9.NewClient(clientSocket, 1024*1024 /* 1M message size */, p9.HighestVersionString()) - if err != nil { - b.Fatalf("new client got %v, expected nil", err) + // Attempt the rename. There are no active fids + // with this name, so we don't need to expect + // Renamed hooks on anything. + f.RenameAt(randomFileName(), f, randomFileName()) + }, + }, } - // Attach to the server. - f, err := client.Attach("") - if err != nil { - b.Fatalf("error during attach, got %v wanted nil", err) + for _, fn1 := range readExclusive { + for _, fn2 := range readExclusive { + for name := range newTypeMap(nil) { + // Everything should be able to proceed in parallel. + concurrentTest(t, name, fn1, fn2, true, true) + concurrentTest(t, name, fn1, fn2, false, true) + } + } } - // Open the file. - if _, _, _, err := f.Open(p9.ReadOnly); err != nil { - b.Fatalf("error during open, got %v wanted nil", err) + for _, fn1 := range append(readExclusive, writeExclusive...) { + for _, fn2 := range writeExclusive { + for name := range newTypeMap(nil) { + // Only cross-directory functions should proceed in parallel. + concurrentTest(t, name, fn1, fn2, true, false) + concurrentTest(t, name, fn1, fn2, false, true) + } + } } - // Reset the clock. - b.ResetTimer() - - // Do N reads. - var buf [1]byte - for i := 0; i < b.N; i++ { - _, err := f.ReadAt(buf[:], 0) - if err != nil { - b.Fatalf("error during read %d, got %v wanted nil", i, err) + for _, fn1 := range append(append(readExclusive, writeExclusive...), globalExclusive...) { + for _, fn2 := range globalExclusive { + for name := range newTypeMap(nil) { + // Nothing should be able to run in parallel. + concurrentTest(t, name, fn1, fn2, true, false) + concurrentTest(t, name, fn1, fn2, false, false) + } } } } diff --git a/pkg/p9/p9test/mocks.go b/pkg/p9/p9test/mocks.go deleted file mode 100644 index 9a8c14975..000000000 --- a/pkg/p9/p9test/mocks.go +++ /dev/null @@ -1,489 +0,0 @@ -// Copyright 2018 Google LLC -// -// 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 -// -// 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 p9test - -import ( - "gvisor.googlesource.com/gvisor/pkg/fd" - "gvisor.googlesource.com/gvisor/pkg/p9" -) - -// StatFSMock mocks p9.File.StatFS. -type StatFSMock struct { - Called bool - - // Return. - Stat p9.FSStat - Err error -} - -// StatFS implements p9.File.StatFS. -func (f *StatFSMock) StatFS() (p9.FSStat, error) { - f.Called = true - return f.Stat, f.Err -} - -// GetAttrMock mocks p9.File.GetAttr. -type GetAttrMock struct { - Called bool - - // Args. - Req p9.AttrMask - - // Return. - QID p9.QID - Valid p9.AttrMask - Attr p9.Attr - Err error -} - -// GetAttr implements p9.File.GetAttr. -func (g *GetAttrMock) GetAttr(req p9.AttrMask) (p9.QID, p9.AttrMask, p9.Attr, error) { - g.Called, g.Req = true, req - return g.QID, g.Valid, g.Attr, g.Err -} - -// WalkGetAttrMock mocks p9.File.WalkGetAttr. -type WalkGetAttrMock struct { - Called bool - - // Args. - Names []string - - // Return. - QIDs []p9.QID - File p9.File - Valid p9.AttrMask - Attr p9.Attr - Err error -} - -// WalkGetAttr implements p9.File.WalkGetAttr. -func (w *WalkGetAttrMock) WalkGetAttr(names []string) ([]p9.QID, p9.File, p9.AttrMask, p9.Attr, error) { - w.Called = true - w.Names = append(w.Names, names...) - return w.QIDs, w.File, w.Valid, w.Attr, w.Err -} - -// SetAttrMock mocks p9.File.SetAttr. -type SetAttrMock struct { - Called bool - - // Args. - Valid p9.SetAttrMask - Attr p9.SetAttr - - // Return. - Err error -} - -// SetAttr implements p9.File.SetAttr. -func (s *SetAttrMock) SetAttr(valid p9.SetAttrMask, attr p9.SetAttr) error { - s.Called, s.Valid, s.Attr = true, valid, attr - return s.Err -} - -// RemoveMock mocks p9.File.Remove. -type RemoveMock struct { - Called bool - - // Return. - Err error -} - -// Remove implements p9.File.Remove. -func (r *RemoveMock) Remove() error { - r.Called = true - return r.Err -} - -// OpenMock mocks p9.File.Open. -type OpenMock struct { - Called bool - - // Args. - Flags p9.OpenFlags - - // Return. - File *fd.FD - QID p9.QID - IOUnit uint32 - Err error -} - -// Open implements p9.File.Open. -func (o *OpenMock) Open(flags p9.OpenFlags) (*fd.FD, p9.QID, uint32, error) { - o.Called, o.Flags = true, flags - return o.File, o.QID, o.IOUnit, o.Err -} - -// ReadAtMock mocks p9.File.ReadAt. -type ReadAtMock struct { - Called bool - - // Args. - P []byte - Offset uint64 - - // Return. - N int - Err error -} - -// ReadAt implements p9.File.ReadAt. -func (r *ReadAtMock) ReadAt(p []byte, offset uint64) (int, error) { - r.Called, r.P, r.Offset = true, p, offset - return r.N, r.Err -} - -// WriteAtMock mocks p9.File.WriteAt. -type WriteAtMock struct { - Called bool - - // Args. - P []byte - Offset uint64 - - // Return. - N int - Err error -} - -// WriteAt implements p9.File.WriteAt. -func (w *WriteAtMock) WriteAt(p []byte, offset uint64) (int, error) { - w.Called, w.P, w.Offset = true, p, offset - return w.N, w.Err -} - -// FSyncMock mocks p9.File.FSync. -type FSyncMock struct { - Called bool - - // Return. - Err error -} - -// FSync implements p9.File.FSync. -func (f *FSyncMock) FSync() error { - f.Called = true - return f.Err -} - -// MkdirMock mocks p9.File.Mkdir. -type MkdirMock struct { - Called bool - - // Args. - Name string - Permissions p9.FileMode - UID p9.UID - GID p9.GID - - // Return. - QID p9.QID - Err error -} - -// Mkdir implements p9.File.Mkdir. -func (s *MkdirMock) Mkdir(name string, permissions p9.FileMode, uid p9.UID, gid p9.GID) (p9.QID, error) { - s.Called, s.Name, s.Permissions, s.UID, s.GID = true, name, permissions, uid, gid - return s.QID, s.Err -} - -// SymlinkMock mocks p9.File.Symlink. -type SymlinkMock struct { - Called bool - - // Args. - Oldname string - Newname string - UID p9.UID - GID p9.GID - - // Return. - QID p9.QID - Err error -} - -// Symlink implements p9.File.Symlink. -func (s *SymlinkMock) Symlink(oldname string, newname string, uid p9.UID, gid p9.GID) (p9.QID, error) { - s.Called, s.Oldname, s.Newname, s.UID, s.GID = true, oldname, newname, uid, gid - return s.QID, s.Err -} - -// MknodMock mocks p9.File.Mknod. -type MknodMock struct { - Called bool - - // Args. - Name string - Permissions p9.FileMode - Major uint32 - Minor uint32 - UID p9.UID - GID p9.GID - - // Return. - QID p9.QID - Err error -} - -// Mknod implements p9.File.Mknod. -func (m *MknodMock) Mknod(name string, permissions p9.FileMode, major uint32, minor uint32, uid p9.UID, gid p9.GID) (p9.QID, error) { - m.Called, m.Name, m.Permissions, m.Major, m.Minor, m.UID, m.GID = true, name, permissions, major, minor, uid, gid - return m.QID, m.Err -} - -// UnlinkAtMock mocks p9.File.UnlinkAt. -type UnlinkAtMock struct { - Called bool - - // Args. - Name string - Flags uint32 - - // Return. - Err error -} - -// UnlinkAt implements p9.File.UnlinkAt. -func (u *UnlinkAtMock) UnlinkAt(name string, flags uint32) error { - u.Called, u.Name, u.Flags = true, name, flags - return u.Err -} - -// ReaddirMock mocks p9.File.Readdir. -type ReaddirMock struct { - Called bool - - // Args. - Offset uint64 - Count uint32 - - // Return. - Dirents []p9.Dirent - Err error -} - -// Readdir implements p9.File.Readdir. -func (r *ReaddirMock) Readdir(offset uint64, count uint32) ([]p9.Dirent, error) { - r.Called, r.Offset, r.Count = true, offset, count - return r.Dirents, r.Err -} - -// ReadlinkMock mocks p9.File.Readlink. -type ReadlinkMock struct { - Called bool - - // Return. - Target string - Err error -} - -// Readlink implements p9.File.Readlink. -func (r *ReadlinkMock) Readlink() (string, error) { - r.Called = true - return r.Target, r.Err -} - -// AttachMock mocks p9.Attacher.Attach. -type AttachMock struct { - Called bool - - // Return. - File p9.File - Err error -} - -// Attach implements p9.Attacher.Attach. -func (a *AttachMock) Attach() (p9.File, error) { - a.Called = true - return a.File, a.Err -} - -// WalkMock mocks p9.File.Walk. -type WalkMock struct { - Called bool - - // Args. - Names []string - - // Return. - QIDs []p9.QID - File p9.File - Err error -} - -// Walk implements p9.File.Walk. -func (w *WalkMock) Walk(names []string) ([]p9.QID, p9.File, error) { - w.Called = true - w.Names = append(w.Names, names...) - return w.QIDs, w.File, w.Err -} - -// RenameMock mocks p9.File.Rename. -type RenameMock struct { - Called bool - - // Args. - Directory p9.File - Name string - - // Return. - Err error -} - -// Rename implements p9.File.Rename. -func (r *RenameMock) Rename(directory p9.File, name string) error { - r.Called, r.Directory, r.Name = true, directory, name - return r.Err -} - -// CloseMock mocks p9.File.Close. -type CloseMock struct { - Called bool - - // Return. - Err error -} - -// Close implements p9.File.Close. -func (d *CloseMock) Close() error { - d.Called = true - return d.Err -} - -// CreateMock mocks p9.File.Create. -type CreateMock struct { - Called bool - - // Args. - Name string - Flags p9.OpenFlags - Permissions p9.FileMode - UID p9.UID - GID p9.GID - - // Return. - HostFile *fd.FD - File p9.File - QID p9.QID - IOUnit uint32 - Err error -} - -// Create implements p9.File.Create. -func (c *CreateMock) Create(name string, flags p9.OpenFlags, permissions p9.FileMode, uid p9.UID, gid p9.GID) (*fd.FD, p9.File, p9.QID, uint32, error) { - c.Called, c.Name, c.Flags, c.Permissions, c.UID, c.GID = true, name, flags, permissions, uid, gid - return c.HostFile, c.File, c.QID, c.IOUnit, c.Err -} - -// LinkMock mocks p9.File.Link. -type LinkMock struct { - Called bool - - // Args. - Target p9.File - Newname string - - // Return. - Err error -} - -// Link implements p9.File.Link. -func (l *LinkMock) Link(target p9.File, newname string) error { - l.Called, l.Target, l.Newname = true, target, newname - return l.Err -} - -// RenameAtMock mocks p9.File.RenameAt. -type RenameAtMock struct { - Called bool - - // Args. - Oldname string - Newdir p9.File - Newname string - - // Return. - Err error -} - -// RenameAt implements p9.File.RenameAt. -func (r *RenameAtMock) RenameAt(oldname string, newdir p9.File, newname string) error { - r.Called, r.Oldname, r.Newdir, r.Newname = true, oldname, newdir, newname - return r.Err -} - -// FlushMock mocks p9.File.Flush. -type FlushMock struct { - Called bool - - // Return. - Err error -} - -// Flush implements p9.File.Flush. -func (f *FlushMock) Flush() error { - return f.Err -} - -// ConnectMock mocks p9.File.Connect. -type ConnectMock struct { - Called bool - - // Args. - Flags p9.ConnectFlags - - // Return. - File *fd.FD - Err error -} - -// Connect implements p9.File.Connect. -func (o *ConnectMock) Connect(flags p9.ConnectFlags) (*fd.FD, error) { - o.Called, o.Flags = true, flags - return o.File, o.Err -} - -// FileMock mocks p9.File. -type FileMock struct { - WalkMock - WalkGetAttrMock - StatFSMock - GetAttrMock - SetAttrMock - RemoveMock - RenameMock - CloseMock - OpenMock - ReadAtMock - WriteAtMock - FSyncMock - CreateMock - MkdirMock - SymlinkMock - LinkMock - MknodMock - RenameAtMock - UnlinkAtMock - ReaddirMock - ReadlinkMock - FlushMock - ConnectMock -} - -var ( - _ p9.File = &FileMock{} -) diff --git a/pkg/p9/p9test/p9test.go b/pkg/p9/p9test/p9test.go new file mode 100644 index 000000000..417b55950 --- /dev/null +++ b/pkg/p9/p9test/p9test.go @@ -0,0 +1,329 @@ +// Copyright 2018 Google Inc. +// +// 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 +// +// 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 p9test provides standard mocks for p9. +package p9test + +import ( + "fmt" + "sync" + "sync/atomic" + "syscall" + "testing" + + "github.com/golang/mock/gomock" + "gvisor.googlesource.com/gvisor/pkg/p9" + "gvisor.googlesource.com/gvisor/pkg/unet" +) + +// Harness is an attacher mock. +type Harness struct { + t *testing.T + mockCtrl *gomock.Controller + Attacher *MockAttacher + wg sync.WaitGroup + clientSocket *unet.Socket + mu sync.Mutex + created []*Mock +} + +// globalPath is a QID.Path Generator. +var globalPath uint64 + +// MakePath returns a globally unique path. +func MakePath() uint64 { + return atomic.AddUint64(&globalPath, 1) +} + +// Generator is a function that generates a new file. +type Generator func(parent *Mock) *Mock + +// Mock is a common mock element. +type Mock struct { + p9.DefaultWalkGetAttr + *MockFile + parent *Mock + closed bool + harness *Harness + QID p9.QID + Attr p9.Attr + children map[string]Generator + + // WalkCallback is a special function that will be called from within + // the walk context. This is needed for the concurrent tests within + // this package. + WalkCallback func() error +} + +// globalMu protects the children maps in all mocks. Note that this is not a +// particularly elegant solution, but because the test has walks from the root +// through to final nodes, we must share maps below, and it's easiest to simply +// protect against concurrent access globally. +var globalMu sync.RWMutex + +// AddChild adds a new child to the Mock. +func (m *Mock) AddChild(name string, generator Generator) { + globalMu.Lock() + defer globalMu.Unlock() + m.children[name] = generator +} + +// RemoveChild removes the child with the given name. +func (m *Mock) RemoveChild(name string) { + globalMu.Lock() + defer globalMu.Unlock() + delete(m.children, name) +} + +// Matches implements gomock.Matcher.Matches. +func (m *Mock) Matches(x interface{}) bool { + if om, ok := x.(*Mock); ok { + return m.QID.Path == om.QID.Path + } + return false +} + +// String implements gomock.Matcher.String. +func (m *Mock) String() string { + return fmt.Sprintf("Mock{Mode: 0x%x, QID.Path: %d}", m.Attr.Mode, m.QID.Path) +} + +// GetAttr returns the current attributes. +func (m *Mock) GetAttr(mask p9.AttrMask) (p9.QID, p9.AttrMask, p9.Attr, error) { + return m.QID, p9.AttrMaskAll(), m.Attr, nil +} + +// Walk supports clone and walking in directories. +func (m *Mock) Walk(names []string) ([]p9.QID, p9.File, error) { + if m.WalkCallback != nil { + if err := m.WalkCallback(); err != nil { + return nil, nil, err + } + } + if len(names) == 0 { + // Clone the file appropriately. + nm := m.harness.NewMock(m.parent, m.QID.Path, m.Attr) + nm.children = m.children // Inherit children. + return []p9.QID{nm.QID}, nm, nil + } else if len(names) != 1 { + m.harness.t.Fail() // Should not happen. + return nil, nil, syscall.EINVAL + } + + if m.Attr.Mode.IsDir() { + globalMu.RLock() + defer globalMu.RUnlock() + if fn, ok := m.children[names[0]]; ok { + // Generate the child. + nm := fn(m) + return []p9.QID{nm.QID}, nm, nil + } + // No child found. + return nil, nil, syscall.ENOENT + } + + // Call the underlying mock. + return m.MockFile.Walk(names) +} + +// WalkGetAttr calls the default implementation; this is a client-side optimization. +func (m *Mock) WalkGetAttr(names []string) ([]p9.QID, p9.File, p9.AttrMask, p9.Attr, error) { + return m.DefaultWalkGetAttr.WalkGetAttr(names) +} + +// Pop pops off the most recently created Mock and assert that this mock +// represents the same file passed in. If nil is passed in, no check is +// performed. +// +// Precondition: there must be at least one Mock or this will panic. +func (h *Harness) Pop(clientFile p9.File) *Mock { + h.mu.Lock() + defer h.mu.Unlock() + + if clientFile == nil { + // If no clientFile is provided, then we always return the last + // created file. The caller can safely use this as long as + // there is no concurrency. + m := h.created[len(h.created)-1] + h.created = h.created[:len(h.created)-1] + return m + } + + qid, _, _, err := clientFile.GetAttr(p9.AttrMaskAll()) + if err != nil { + // We do not expect this to happen. + panic(fmt.Sprintf("err during Pop: %v", err)) + } + + // Find the relevant file in our created list. We must scan the last + // from back to front to ensure that we favor the most recently + // generated file. + for i := len(h.created) - 1; i >= 0; i-- { + m := h.created[i] + if qid.Path == m.QID.Path { + // Copy and truncate. + copy(h.created[i:], h.created[i+1:]) + h.created = h.created[:len(h.created)-1] + return m + } + } + + // Unable to find relevant file. + panic(fmt.Sprintf("unable to locate file with QID %+v", qid.Path)) +} + +// NewMock returns a new base file. +func (h *Harness) NewMock(parent *Mock, path uint64, attr p9.Attr) *Mock { + m := &Mock{ + MockFile: NewMockFile(h.mockCtrl), + parent: parent, + harness: h, + QID: p9.QID{ + Type: p9.QIDType((attr.Mode & p9.FileModeMask) >> 12), + Path: path, + }, + Attr: attr, + } + + // Always ensure Close is after the parent's close. Note that this + // can't be done via a straight-forward After call, because the parent + // might change after initial creation. We ensure that this is true at + // close time. + m.EXPECT().Close().Return(nil).Times(1).Do(func() { + if m.parent != nil && m.parent.closed { + h.t.FailNow() + } + // Note that this should not be racy, as this operation should + // be protected by the Times(1) above first. + m.closed = true + }) + + // Remember what was created. + h.mu.Lock() + defer h.mu.Unlock() + h.created = append(h.created, m) + + return m +} + +// NewFile returns a new file mock. +// +// Note that ReadAt and WriteAt must be mocked separately. +func (h *Harness) NewFile() Generator { + return func(parent *Mock) *Mock { + return h.NewMock(parent, MakePath(), p9.Attr{Mode: p9.ModeRegular}) + } +} + +// NewDirectory returns a new mock directory. +// +// Note that Mkdir, Link, Mknod, RenameAt, UnlinkAt and Readdir must be mocked +// separately. Walk is provided and children may be manipulated via AddChild +// and RemoveChild. After calling Walk remotely, one can use Pop to find the +// corresponding backend mock on the server side. +func (h *Harness) NewDirectory(contents map[string]Generator) Generator { + return func(parent *Mock) *Mock { + m := h.NewMock(parent, MakePath(), p9.Attr{Mode: p9.ModeDirectory}) + m.children = contents // Save contents. + return m + } +} + +// NewSymlink returns a new mock directory. +// +// Note that Readlink must be mocked separately. +func (h *Harness) NewSymlink() Generator { + return func(parent *Mock) *Mock { + return h.NewMock(parent, MakePath(), p9.Attr{Mode: p9.ModeSymlink}) + } +} + +// NewBlockDevice returns a new mock block device. +func (h *Harness) NewBlockDevice() Generator { + return func(parent *Mock) *Mock { + return h.NewMock(parent, MakePath(), p9.Attr{Mode: p9.ModeBlockDevice}) + } +} + +// NewCharacterDevice returns a new mock character device. +func (h *Harness) NewCharacterDevice() Generator { + return func(parent *Mock) *Mock { + return h.NewMock(parent, MakePath(), p9.Attr{Mode: p9.ModeCharacterDevice}) + } +} + +// NewNamedPipe returns a new mock named pipe. +func (h *Harness) NewNamedPipe() Generator { + return func(parent *Mock) *Mock { + return h.NewMock(parent, MakePath(), p9.Attr{Mode: p9.ModeNamedPipe}) + } +} + +// NewSocket returns a new mock socket. +func (h *Harness) NewSocket() Generator { + return func(parent *Mock) *Mock { + return h.NewMock(parent, MakePath(), p9.Attr{Mode: p9.ModeSocket}) + } +} + +// Finish completes all checks and shuts down the server. +func (h *Harness) Finish() { + h.clientSocket.Close() + h.wg.Wait() + h.mockCtrl.Finish() +} + +// NewHarness creates and returns a new test server. +// +// It should always be used as: +// +// h, c := NewHarness(t) +// defer h.Finish() +// +func NewHarness(t *testing.T) (*Harness, *p9.Client) { + // Create the mock. + mockCtrl := gomock.NewController(t) + h := &Harness{ + t: t, + mockCtrl: mockCtrl, + Attacher: NewMockAttacher(mockCtrl), + } + + // Make socket pair. + serverSocket, clientSocket, err := unet.SocketPair(false) + if err != nil { + t.Fatalf("socketpair got err %v wanted nil", err) + } + + // Start the server, synchronized on exit. + server := p9.NewServer(h.Attacher) + h.wg.Add(1) + go func() { + defer h.wg.Done() + server.Handle(serverSocket) + }() + + // Create the client. + client, err := p9.NewClient(clientSocket, 1024, p9.HighestVersionString()) + if err != nil { + serverSocket.Close() + clientSocket.Close() + t.Fatalf("new client got %v, expected nil", err) + return nil, nil // Never hit. + } + + // Capture the client socket. + h.clientSocket = clientSocket + return h, client +} diff --git a/pkg/p9/path_tree.go b/pkg/p9/path_tree.go new file mode 100644 index 000000000..97f90bcd5 --- /dev/null +++ b/pkg/p9/path_tree.go @@ -0,0 +1,109 @@ +// Copyright 2018 Google Inc. +// +// 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 +// +// 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 p9 + +import ( + "fmt" + "sync" +) + +// pathNode is a single node in a path traversal. +// +// These are shared by all fidRefs that point to the same path. +// +// These are not synchronized because we allow certain operations (file walk) +// to proceed without having to acquire a write lock. The lock in this +// structure exists to synchronize high-level, semantic operations, such as the +// simultaneous creation and deletion of a file. +// +// (+) below is the path component string. +type pathNode struct { + mu sync.RWMutex // See above. + fidRefs sync.Map // => map[*fidRef]string(+) + children sync.Map // => map[string(+)]*pathNode + count int64 +} + +// pathNodeFor returns the path node for the given name, or a new one. +// +// Precondition: mu must be held in a readable fashion. +func (p *pathNode) pathNodeFor(name string) *pathNode { + // Load the existing path node. + if pn, ok := p.children.Load(name); ok { + return pn.(*pathNode) + } + + // Create a new pathNode for shared use. + pn, _ := p.children.LoadOrStore(name, new(pathNode)) + return pn.(*pathNode) +} + +// nameFor returns the name for the given fidRef. +// +// Precondition: mu must be held in a readable fashion. +func (p *pathNode) nameFor(ref *fidRef) string { + if s, ok := p.fidRefs.Load(ref); ok { + return s.(string) + } + + // This should not happen, don't proceed. + panic(fmt.Sprintf("expected name for %+v, none found", ref)) +} + +// addChild adds a child to the given pathNode. +// +// This applies only to an individual fidRef. +// +// Precondition: mu must be held in a writable fashion. +func (p *pathNode) addChild(ref *fidRef, name string) { + if s, ok := p.fidRefs.Load(ref); ok { + // This should not happen, don't proceed. + panic(fmt.Sprintf("unexpected fidRef %+v with path %q, wanted %q", ref, s, name)) + } + + p.fidRefs.Store(ref, name) +} + +// removeChild removes the given child. +// +// This applies only to an individual fidRef. +// +// Precondition: mu must be held in a writable fashion. +func (p *pathNode) removeChild(ref *fidRef) { + p.fidRefs.Delete(ref) +} + +// removeWithName removes all references with the given name. +// +// The original pathNode is returned by this function, and removed from this +// pathNode. Any operations on the removed tree must use this value. +// +// The provided function is executed after removal. +// +// Precondition: mu must be held in a writable fashion. +func (p *pathNode) removeWithName(name string, fn func(ref *fidRef)) *pathNode { + p.fidRefs.Range(func(key, value interface{}) bool { + if value.(string) == name { + p.fidRefs.Delete(key) + fn(key.(*fidRef)) + } + return true + }) + + // Return the original path node. + origPathNode := p.pathNodeFor(name) + p.children.Delete(name) + return origPathNode +} diff --git a/pkg/p9/server.go b/pkg/p9/server.go index 5c7cb18c8..3ef151595 100644 --- a/pkg/p9/server.go +++ b/pkg/p9/server.go @@ -15,6 +15,8 @@ package p9 import ( + "io" + "runtime/debug" "sync" "sync/atomic" "syscall" @@ -27,6 +29,19 @@ import ( type Server struct { // attacher provides the attach function. attacher Attacher + + // pathTree is the full set of paths opened on this server. + // + // These may be across different connections, but rename operations + // must be serialized globally for safely. There is a single pathTree + // for the entire server, and not per connection. + pathTree pathNode + + // renameMu is a global lock protecting rename operations. With this + // lock, we can be certain that any given rename operation can safely + // acquire two path nodes in any order, as all other concurrent + // operations acquire at most a single node. + renameMu sync.RWMutex } // NewServer returns a new server. @@ -81,6 +96,9 @@ type connState struct { // fidRef wraps a node and tracks references. type fidRef struct { + // server is the associated server. + server *Server + // file is the associated File. file File @@ -97,13 +115,39 @@ type fidRef struct { // This is updated in handlers.go. opened bool - // walkable indicates this fidRef may be walked. - walkable bool + // mode is the fidRef's mode from the walk. Only the type bits are + // valid, the permissions may change. This is used to sanity check + // operations on this element, and prevent walks across + // non-directories. + mode FileMode // openFlags is the mode used in the open. // // This is updated in handlers.go. openFlags OpenFlags + + // pathNode is the current pathNode for this FID. + pathNode *pathNode + + // parent is the parent fidRef. We hold on to a parent reference to + // ensure that hooks, such as Renamed, can be executed safely by the + // server code. + // + // Note that parent cannot be changed without holding both the global + // rename lock and a writable lock on the associated pathNode for this + // fidRef. Holding either of these locks is sufficient to examine + // parent safely. + // + // The parent will be nil for root fidRefs, and non-nil otherwise. The + // method maybeParent can be used to return a cyclical reference, and + // isRoot should be used to check for root over looking at parent + // directly. + parent *fidRef + + // deleted indicates that the backing file has been deleted. We stop + // many operations at the API level if they are incompatible with a + // file that has already been unlinked. + deleted uint32 } // OpenFlags returns the flags the file was opened with and true iff the fid was opened previously. @@ -113,13 +157,146 @@ func (f *fidRef) OpenFlags() (OpenFlags, bool) { return f.openFlags, f.opened } +// IncRef increases the references on a fid. +func (f *fidRef) IncRef() { + atomic.AddInt64(&f.refs, 1) +} + // DecRef should be called when you're finished with a fid. func (f *fidRef) DecRef() { if atomic.AddInt64(&f.refs, -1) == 0 { f.file.Close() + + // Drop the parent reference. + // + // Since this fidRef is guaranteed to be non-discoverable when + // the references reach zero, we don't need to worry about + // clearing the parent. + if f.parent != nil { + // If we've been previously deleted, this removing this + // ref is a no-op. That's expected. + f.parent.pathNode.removeChild(f) + f.parent.DecRef() + } } } +// isDeleted returns true if this fidRef has been deleted. +func (f *fidRef) isDeleted() bool { + return atomic.LoadUint32(&f.deleted) != 0 +} + +// isRoot indicates whether this is a root fid. +func (f *fidRef) isRoot() bool { + return f.parent == nil +} + +// maybeParent returns a cyclic reference for roots, and the parent otherwise. +func (f *fidRef) maybeParent() *fidRef { + if f.parent != nil { + return f.parent + } + return f // Root has itself. +} + +// notifyDelete marks all fidRefs as deleted. +// +// Precondition: the write lock must be held on the given pathNode. +func notifyDelete(pn *pathNode) { + // Call on all local references. + pn.fidRefs.Range(func(key, _ interface{}) bool { + ref := key.(*fidRef) + atomic.StoreUint32(&ref.deleted, 1) + return true + }) + + // Call on all subtrees. + pn.children.Range(func(_, value interface{}) bool { + notifyDelete(value.(*pathNode)) + return true + }) +} + +// markChildDeleted marks all children below the given name as deleted. +// +// Precondition: this must be called via safelyWrite or safelyGlobal. +func (f *fidRef) markChildDeleted(name string) { + origPathNode := f.pathNode.removeWithName(name, func(ref *fidRef) { + atomic.StoreUint32(&ref.deleted, 1) + }) + + // Mark everything below as deleted. + notifyDelete(origPathNode) +} + +// notifyNameChange calls the relevant Renamed method on all nodes in the path, +// recursively. Note that this applies only for subtrees, as these +// notifications do not apply to the actual file whose name has changed. +// +// Precondition: the write lock must be held on the given pathNode. +func notifyNameChange(pn *pathNode) { + // Call on all local references. + pn.fidRefs.Range(func(key, value interface{}) bool { + ref := key.(*fidRef) + name := value.(string) + ref.file.Renamed(ref.parent.file, name) + return true + }) + + // Call on all subtrees. + pn.children.Range(func(_, value interface{}) bool { + notifyNameChange(value.(*pathNode)) + return true + }) +} + +// renameChildTo renames the given child to the target. +// +// Precondition: this must be called via safelyGlobal. +func (f *fidRef) renameChildTo(oldName string, target *fidRef, newName string) { + target.markChildDeleted(newName) + origPathNode := f.pathNode.removeWithName(oldName, func(ref *fidRef) { + ref.parent.DecRef() // Drop original reference. + ref.parent = target // Change parent. + ref.parent.IncRef() // Acquire new one. + target.pathNode.addChild(ref, newName) + ref.file.Renamed(target.file, newName) + }) + + // Replace the previous (now deleted) path node. + f.pathNode.children.Store(newName, origPathNode) + + // Call Renamed on everything above. + notifyNameChange(origPathNode) +} + +// safelyRead executes the given operation with the local path node locked. +// This implies that paths will not change during the operation. +func (f *fidRef) safelyRead(fn func() error) (err error) { + f.server.renameMu.RLock() + defer f.server.renameMu.RUnlock() + f.pathNode.mu.RLock() + defer f.pathNode.mu.RUnlock() + return fn() +} + +// safelyWrite executes the given operation with the local path node locked in +// a writable fashion. This implies some paths may change. +func (f *fidRef) safelyWrite(fn func() error) (err error) { + f.server.renameMu.RLock() + defer f.server.renameMu.RUnlock() + f.pathNode.mu.Lock() + defer f.pathNode.mu.Unlock() + return fn() +} + +// safelyGlobal executes the given operation with the global path lock held. +func (f *fidRef) safelyGlobal(fn func() error) (err error) { + f.server.renameMu.Lock() + defer f.server.renameMu.Unlock() + return fn() +} + // LookupFID finds the given FID. // // You should call fid.DecRef when you are finished using the fid. @@ -128,7 +305,7 @@ func (cs *connState) LookupFID(fid FID) (*fidRef, bool) { defer cs.fidMu.Unlock() fidRef, ok := cs.fids[fid] if ok { - atomic.AddInt64(&fidRef.refs, 1) + fidRef.IncRef() return fidRef, true } return nil, false @@ -145,7 +322,7 @@ func (cs *connState) InsertFID(fid FID, newRef *fidRef) { if ok { defer origRef.DecRef() } - atomic.AddInt64(&newRef.refs, 1) + newRef.IncRef() cs.fids[fid] = newRef } @@ -229,10 +406,9 @@ func (cs *connState) handleRequest() { cs.recvDone <- nil // Deal with other errors. - if err != nil { + if err != nil && err != io.EOF { // If it's not a connection error, but some other protocol error, // we can send a response immediately. - log.Debugf("err [%05d] %v", tag, err) cs.sendMu.Lock() err := send(cs.conn, tag, newErr(err)) cs.sendMu.Unlock() @@ -243,12 +419,38 @@ func (cs *connState) handleRequest() { // Try to start the tag. if !cs.StartTag(tag) { // Nothing we can do at this point; client is bogus. + log.Debugf("no valid tag [%05d]", tag) cs.sendDone <- ErrNoValidMessage return } // Handle the message. - var r message + var r message // r is the response. + defer func() { + if r == nil { + // Don't allow a panic to propagate. + recover() + + // Include a useful log message. + log.Warningf("panic in handler: %s", debug.Stack()) + + // Wrap in an EFAULT error; we don't really have a + // better way to describe this kind of error. It will + // usually manifest as a result of the test framework. + r = newErr(syscall.EFAULT) + } + + // Clear the tag before sending. That's because as soon as this + // hits the wire, the client can legally send another message + // with the same tag. + cs.ClearTag(tag) + + // Send back the result. + cs.sendMu.Lock() + err = send(cs.conn, tag, r) + cs.sendMu.Unlock() + cs.sendDone <- err + }() if handler, ok := m.(handler); ok { // Call the message handler. r = handler.handle(cs) @@ -256,18 +458,6 @@ func (cs *connState) handleRequest() { // Produce an ENOSYS error. r = newErr(syscall.ENOSYS) } - - // Clear the tag before sending. That's because as soon - // as this hits the wire, the client can legally send - // another message with the same tag. - cs.ClearTag(tag) - - // Send back the result. - cs.sendMu.Lock() - err = send(cs.conn, tag, r) - cs.sendMu.Unlock() - cs.sendDone <- err - return } func (cs *connState) handleRequests() { diff --git a/pkg/p9/transport.go b/pkg/p9/transport.go index 97396806c..bafb377de 100644 --- a/pkg/p9/transport.go +++ b/pkg/p9/transport.go @@ -167,7 +167,7 @@ func recv(s *unet.Socket, msize uint32, lookup lookupTagAndType) (Tag, message, r.EnableFDs(1) n, err := r.ReadVec([][]byte{hdr[:]}) - if err != nil { + if err != nil && (n == 0 || err != io.EOF) { r.CloseFDs() return NoTag, nil, ErrSocket{err} } @@ -189,10 +189,8 @@ func recv(s *unet.Socket, msize uint32, lookup lookupTagAndType) (Tag, message, // Continuing reading for a short header. for n < int(headerLength) { cur, err := r.ReadVec([][]byte{hdr[n:]}) - if err != nil { + if err != nil && (cur == 0 || err != io.EOF) { return NoTag, nil, ErrSocket{err} - } else if cur == 0 { - return NoTag, nil, ErrSocket{io.EOF} } n += cur } @@ -296,10 +294,8 @@ func recv(s *unet.Socket, msize uint32, lookup lookupTagAndType) (Tag, message, r := s.Reader(true) for n := 0; n < int(remaining); { cur, err := r.ReadVec(vecs) - if err != nil { + if err != nil && (cur == 0 || err != io.EOF) { return NoTag, nil, ErrSocket{err} - } else if cur == 0 { - return NoTag, nil, ErrSocket{io.EOF} } n += cur diff --git a/pkg/rand/BUILD b/pkg/rand/BUILD index 97b9ba3ff..0c9efc709 100644 --- a/pkg/rand/BUILD +++ b/pkg/rand/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "rand", srcs = [ diff --git a/pkg/seccomp/BUILD b/pkg/seccomp/BUILD index 1975d17a6..657f923ed 100644 --- a/pkg/seccomp/BUILD +++ b/pkg/seccomp/BUILD @@ -1,8 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_embed_data") +package(licenses = ["notice"]) # Apache 2.0 + go_binary( name = "victim", testonly = 1, diff --git a/pkg/secio/BUILD b/pkg/secio/BUILD index 0ed38c64a..29f751725 100644 --- a/pkg/secio/BUILD +++ b/pkg/secio/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "secio", srcs = [ diff --git a/pkg/sentry/arch/BUILD b/pkg/sentry/arch/BUILD index 314b3e962..9bf04360a 100644 --- a/pkg/sentry/arch/BUILD +++ b/pkg/sentry/arch/BUILD @@ -1,6 +1,7 @@ +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") + package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") load("//tools/go_stateify:defs.bzl", "go_library") go_library( diff --git a/pkg/sentry/context/BUILD b/pkg/sentry/context/BUILD index 2a7a6df23..02d24defd 100644 --- a/pkg/sentry/context/BUILD +++ b/pkg/sentry/context/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "context", srcs = ["context.go"], diff --git a/pkg/sentry/control/BUILD b/pkg/sentry/control/BUILD index fbdde0721..c3b682d6f 100644 --- a/pkg/sentry/control/BUILD +++ b/pkg/sentry/control/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "control", srcs = [ diff --git a/pkg/sentry/device/BUILD b/pkg/sentry/device/BUILD index 69c99b0b3..bebdb2939 100644 --- a/pkg/sentry/device/BUILD +++ b/pkg/sentry/device/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "device", srcs = ["device.go"], diff --git a/pkg/sentry/fs/anon/BUILD b/pkg/sentry/fs/anon/BUILD index ff4ab850a..4bd912e95 100644 --- a/pkg/sentry/fs/anon/BUILD +++ b/pkg/sentry/fs/anon/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "anon", srcs = [ diff --git a/pkg/sentry/fs/gofer/BUILD b/pkg/sentry/fs/gofer/BUILD index cef01829a..c9e531e40 100644 --- a/pkg/sentry/fs/gofer/BUILD +++ b/pkg/sentry/fs/gofer/BUILD @@ -56,14 +56,10 @@ go_test( srcs = ["gofer_test.go"], embed = [":gofer"], deps = [ - "//pkg/log", "//pkg/p9", "//pkg/p9/p9test", "//pkg/sentry/context", "//pkg/sentry/context/contexttest", "//pkg/sentry/fs", - "//pkg/sentry/kernel/time", - "//pkg/sentry/usermem", - "//pkg/unet", ], ) diff --git a/pkg/sentry/fs/gofer/context_file.go b/pkg/sentry/fs/gofer/context_file.go index a0265c2aa..455953237 100644 --- a/pkg/sentry/fs/gofer/context_file.go +++ b/pkg/sentry/fs/gofer/context_file.go @@ -58,13 +58,6 @@ func (c *contextFile) setAttr(ctx context.Context, valid p9.SetAttrMask, attr p9 return c.file.SetAttr(valid, attr) } -func (c *contextFile) remove(ctx context.Context) error { - ctx.UninterruptibleSleepStart(false) - defer ctx.UninterruptibleSleepFinish(false) - - return c.file.Remove() -} - func (c *contextFile) rename(ctx context.Context, directory contextFile, name string) error { ctx.UninterruptibleSleepStart(false) defer ctx.UninterruptibleSleepFinish(false) diff --git a/pkg/sentry/fs/gofer/gofer_test.go b/pkg/sentry/fs/gofer/gofer_test.go index 3190d1e18..b450778ca 100644 --- a/pkg/sentry/fs/gofer/gofer_test.go +++ b/pkg/sentry/fs/gofer/gofer_test.go @@ -16,110 +16,102 @@ package gofer import ( "fmt" - "io" "syscall" "testing" "time" - "gvisor.googlesource.com/gvisor/pkg/log" "gvisor.googlesource.com/gvisor/pkg/p9" "gvisor.googlesource.com/gvisor/pkg/p9/p9test" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/context/contexttest" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" - "gvisor.googlesource.com/gvisor/pkg/unet" ) -// goodMockFile returns a file that can be Walk'ed to and created. -func goodMockFile(mode p9.FileMode, size uint64) *p9test.FileMock { - return &p9test.FileMock{ - GetAttrMock: p9test.GetAttrMock{ - Attr: p9.Attr{Mode: mode, Size: size, RDev: 0}, - Valid: p9.AttrMaskAll(), - }, - } -} - -func newClosedSocket() (*unet.Socket, error) { - fd, err := syscall.Socket(syscall.AF_UNIX, syscall.SOCK_STREAM, 0) - if err != nil { - return nil, err - } - - s, err := unet.NewSocket(fd) - if err != nil { - syscall.Close(fd) - return nil, err - } - - return s, s.Close() -} - -// root returns a p9 file mock and an fs.InodeOperations created from that file. Any -// functions performed on fs.InodeOperations will use the p9 file mock. -func root(ctx context.Context, cp cachePolicy, mode p9.FileMode, size uint64) (*p9test.FileMock, *fs.Inode, error) { - sock, err := newClosedSocket() - if err != nil { - return nil, nil, err - } - - // Construct a dummy session that we can destruct. - s := &session{ - conn: sock, - mounter: fs.RootOwner, - cachePolicy: cp, - client: &p9.Client{}, - } - - rootFile := goodMockFile(mode, size) - sattr, rootInodeOperations := newInodeOperations(ctx, s, contextFile{file: rootFile}, p9.QID{}, rootFile.GetAttrMock.Valid, rootFile.GetAttrMock.Attr, false /* socket */) - m := fs.NewMountSource(s, &filesystem{}, fs.MountSourceFlags{}) - return rootFile, fs.NewInode(rootInodeOperations, m, sattr), nil +// rootTest runs a test with a p9 mock and an fs.InodeOperations created from +// the attached root directory. The root file will be closed and client +// disconnected, but additional files must be closed manually. +func rootTest(t *testing.T, name string, cp cachePolicy, fn func(context.Context, *p9test.Harness, *p9test.Mock, *fs.Inode)) { + t.Run(name, func(t *testing.T) { + h, c := p9test.NewHarness(t) + defer h.Finish() + + // Create a new root. Note that we pass an empty, but non-nil + // map here. This allows tests to extend the root children + // dynamically. + root := h.NewDirectory(map[string]p9test.Generator{})(nil) + + // Return this as the root. + h.Attacher.EXPECT().Attach().Return(root, nil).Times(1) + + // ... and open via the client. + rootFile, err := c.Attach("/") + if err != nil { + t.Fatalf("unable to attach: %v", err) + } + defer rootFile.Close() + + // Wrap an a session. + s := &session{ + mounter: fs.RootOwner, + cachePolicy: cp, + client: c, + } + + // ... and an INode, with only the mode being explicitly valid for now. + ctx := contexttest.Context(t) + sattr, rootInodeOperations := newInodeOperations(ctx, s, contextFile{ + file: rootFile, + }, root.QID, p9.AttrMaskAll(), root.Attr, false /* socket */) + m := fs.NewMountSource(s, &filesystem{}, fs.MountSourceFlags{}) + rootInode := fs.NewInode(rootInodeOperations, m, sattr) + + // Ensure that the cache is fully invalidated, so that any + // close actions actually take place before the full harness is + // torn down. + defer m.FlushDirentRefs() + + // Execute the test. + fn(ctx, h, root, rootInode) + }) } func TestLookup(t *testing.T) { - // Test parameters. type lookupTest struct { // Name of the test. name string - // Function input parameters. - fileName string - // Expected return value. want error } tests := []lookupTest{ { - name: "mock Walk passes (function succeeds)", - fileName: "ppp", - want: nil, + name: "mock Walk passes (function succeeds)", + want: nil, }, { - name: "mock Walk fails (function fails)", - fileName: "ppp", - want: syscall.ENOENT, + name: "mock Walk fails (function fails)", + want: syscall.ENOENT, }, } - ctx := contexttest.Context(t) + const file = "file" // The walked target file. + for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - // Set up mock. - rootFile, rootInode, err := root(ctx, cacheNone, p9.PermissionsMask, 0) - if err != nil { - t.Fatalf("error creating root: %v", err) + rootTest(t, test.name, cacheNone, func(ctx context.Context, h *p9test.Harness, rootFile *p9test.Mock, rootInode *fs.Inode) { + // Setup the appropriate result. + rootFile.WalkCallback = func() error { + return test.want + } + if test.want == nil { + // Set the contents of the root. We expect a + // normal file generator for ppp above. This is + // overriden by setting WalkErr in the mock. + rootFile.AddChild(file, h.NewFile()) } - - rootFile.WalkGetAttrMock.QIDs = []p9.QID{{}} - rootFile.WalkGetAttrMock.Err = test.want - rootFile.WalkGetAttrMock.File = goodMockFile(p9.PermissionsMask, 0) // Call function. - dirent, err := rootInode.Lookup(ctx, test.fileName) + dirent, err := rootInode.Lookup(ctx, file) // Unwrap the InodeOperations. var newInodeOperations fs.InodeOperations @@ -138,19 +130,12 @@ func TestLookup(t *testing.T) { if err == nil && newInodeOperations == nil { t.Errorf("Lookup got non-nil err and non-nil node, wanted at least one non-nil") } - - // Check mock parameters. - if !rootFile.WalkGetAttrMock.Called { - t.Errorf("GetAttr not called; error: %v", err) - } else if rootFile.WalkGetAttrMock.Names[0] != test.fileName { - t.Errorf("file name not set") - } }) } } func TestRevalidation(t *testing.T) { - tests := []struct { + type revalidationTest struct { cachePolicy cachePolicy // Whether dirent should be reloaded before any modifications. @@ -167,7 +152,9 @@ func TestRevalidation(t *testing.T) { // Whether dirent should be reloaded after the remote has // removed the file. postRemovalWantReload bool - }{ + } + + tests := []revalidationTest{ { // Policy cacheNone causes Revalidate to always return // true. @@ -208,67 +195,83 @@ func TestRevalidation(t *testing.T) { }, } - ctx := contexttest.Context(t) + const file = "file" // The file walked below. + for _, test := range tests { name := fmt.Sprintf("cachepolicy=%s", test.cachePolicy) - t.Run(name, func(t *testing.T) { - // Set up mock. - rootFile, rootInode, err := root(ctx, test.cachePolicy, p9.ModeDirectory|p9.PermissionsMask, 0) - if err != nil { - t.Fatalf("error creating root: %v", err) - } - + rootTest(t, name, test.cachePolicy, func(ctx context.Context, h *p9test.Harness, rootFile *p9test.Mock, rootInode *fs.Inode) { + // Wrap in a dirent object. rootDir := fs.NewDirent(rootInode, "root") - // Create a mock file that we will walk to from the root. - const ( - name = "foo" - mode = p9.PermissionsMask - ) - file := goodMockFile(mode, 0) - file.GetAttrMock.Valid = p9.AttrMaskAll() - - // Tell the root mock how to walk to this file. - rootFile.WalkGetAttrMock.QIDs = []p9.QID{{}} - rootFile.WalkGetAttrMock.File = file - rootFile.WalkGetAttrMock.Attr = file.GetAttrMock.Attr - rootFile.WalkGetAttrMock.Valid = file.GetAttrMock.Valid + // Create a mock file a child of the root. We save when + // this is generated, so that when the time changed, we + // can update the original entry. + var origMocks []*p9test.Mock + rootFile.AddChild(file, func(parent *p9test.Mock) *p9test.Mock { + // Regular a regular file that has a consistent + // path number. This might be used by + // validation so we don't change it. + m := h.NewMock(parent, 0, p9.Attr{ + Mode: p9.ModeRegular, + }) + origMocks = append(origMocks, m) + return m + }) // Do the walk. - dirent, err := rootDir.Walk(ctx, rootDir, name) + dirent, err := rootDir.Walk(ctx, rootDir, file) if err != nil { - t.Fatalf("Lookup(%q) failed: %v", name, err) + t.Fatalf("Lookup failed: %v", err) } - // Walk again. Depending on the cache policy, we may get a new - // dirent. - newDirent, err := rootDir.Walk(ctx, rootDir, name) + // We must release the dirent, of the test will fail + // with a reference leak. This is tracked by p9test. + defer dirent.DecRef() + + // Walk again. Depending on the cache policy, we may + // get a new dirent. + newDirent, err := rootDir.Walk(ctx, rootDir, file) if err != nil { - t.Fatalf("Lookup(%q) failed: %v", name, err) + t.Fatalf("Lookup failed: %v", err) } if test.preModificationWantReload && dirent == newDirent { - t.Errorf("Lookup(%q) with cachePolicy=%s got old dirent %v, wanted a new dirent", name, test.cachePolicy, dirent) + t.Errorf("Lookup with cachePolicy=%s got old dirent %+v, wanted a new dirent", test.cachePolicy, dirent) } if !test.preModificationWantReload && dirent != newDirent { - t.Errorf("Lookup(%q) with cachePolicy=%s got new dirent %v, wanted old dirent %v", name, test.cachePolicy, newDirent, dirent) + t.Errorf("Lookup with cachePolicy=%s got new dirent %+v, wanted old dirent %+v", test.cachePolicy, newDirent, dirent) } + newDirent.DecRef() // See above. - // Modify the underlying mocked file's modification time. + // Modify the underlying mocked file's modification + // time for the next walk that occurs. nowSeconds := time.Now().Unix() - rootFile.WalkGetAttrMock.Attr.MTimeSeconds = uint64(nowSeconds) - file.GetAttrMock.Attr.MTimeSeconds = uint64(nowSeconds) + rootFile.AddChild(file, func(parent *p9test.Mock) *p9test.Mock { + // Ensure that the path is the same as above, + // but we change only the modification time of + // the file. + return h.NewMock(parent, 0, p9.Attr{ + Mode: p9.ModeRegular, + MTimeSeconds: uint64(nowSeconds), + }) + }) + + // We also modify the original time, so that GetAttr + // behaves as expected for the caching case. + for _, m := range origMocks { + m.Attr.MTimeSeconds = uint64(nowSeconds) + } - // Walk again. Depending on the cache policy, we may get a new - // dirent. - newDirent, err = rootDir.Walk(ctx, rootDir, name) + // Walk again. Depending on the cache policy, we may + // get a new dirent. + newDirent, err = rootDir.Walk(ctx, rootDir, file) if err != nil { - t.Fatalf("Lookup(%q) failed: %v", name, err) + t.Fatalf("Lookup failed: %v", err) } if test.postModificationWantReload && dirent == newDirent { - t.Errorf("Lookup(%q) with cachePolicy=%s got old dirent %v, wanted a new dirent", name, test.cachePolicy, dirent) + t.Errorf("Lookup with cachePolicy=%s got old dirent, wanted a new dirent", test.cachePolicy) } if !test.postModificationWantReload && dirent != newDirent { - t.Errorf("Lookup(%q) with cachePolicy=%s got new dirent %v, wanted old dirent %v", name, test.cachePolicy, newDirent, dirent) + t.Errorf("Lookup with cachePolicy=%s got new dirent, wanted old dirent", test.cachePolicy) } uattrs, err := newDirent.Inode.UnstableAttr(ctx) if err != nil { @@ -276,660 +279,25 @@ func TestRevalidation(t *testing.T) { } gotModTimeSeconds := uattrs.ModificationTime.Seconds() if test.postModificationWantUpdatedAttrs && gotModTimeSeconds != nowSeconds { - t.Fatalf("Lookup(%q) with cachePolicy=%s got new modification time %v, wanted %v", name, test.cachePolicy, gotModTimeSeconds, nowSeconds) + t.Fatalf("Lookup with cachePolicy=%s got new modification time %v, wanted %v", test.cachePolicy, gotModTimeSeconds, nowSeconds) } + newDirent.DecRef() // See above. - // Make WalkGetAttr return ENOENT. This simulates - // removing the file from the remote fs. - rootFile.WalkGetAttrMock = p9test.WalkGetAttrMock{ - Err: syscall.ENOENT, - } + // Remove the file from the remote fs, subsequent walks + // should now fail to find anything. + rootFile.RemoveChild(file) // Walk again. Depending on the cache policy, we may // get ENOENT. - newDirent, err = rootDir.Walk(ctx, rootDir, name) + newDirent, err = rootDir.Walk(ctx, rootDir, file) if test.postRemovalWantReload && err == nil { - t.Errorf("Lookup(%q) with cachePolicy=%s got nil error, wanted ENOENT", name, test.cachePolicy) + t.Errorf("Lookup with cachePolicy=%s got nil error, wanted ENOENT", test.cachePolicy) } if !test.postRemovalWantReload && (err != nil || dirent != newDirent) { - t.Errorf("Lookup(%q) with cachePolicy=%s got new dirent %v and error %v, wanted old dirent %v and nil error", name, test.cachePolicy, newDirent, err, dirent) - } - }) - } -} - -func TestSetTimestamps(t *testing.T) { - // Test parameters. - type setTimestampsTest struct { - // Name of the test. - name string - - // Function input parameters. - ts fs.TimeSpec - } - - ctx := contexttest.Context(t) - now := ktime.NowFromContext(ctx) - tests := []setTimestampsTest{ - { - name: "mock SetAttr passes (function succeeds)", - ts: fs.TimeSpec{ - ATime: now, - MTime: now, - }, - }, - { - name: "mock SetAttr passes, times are 0 (function succeeds)", - ts: fs.TimeSpec{}, - }, - { - name: "mock SetAttr passes, times are 0 and not system time (function succeeds)", - ts: fs.TimeSpec{ - ATimeSetSystemTime: false, - MTimeSetSystemTime: false, - }, - }, - { - name: "mock SetAttr passes, times are set to system time (function succeeds)", - ts: fs.TimeSpec{ - ATimeSetSystemTime: true, - MTimeSetSystemTime: true, - }, - }, - { - name: "mock SetAttr passes, times are omitted (function succeeds)", - ts: fs.TimeSpec{ - ATimeOmit: true, - MTimeOmit: true, - }, - }, - } - - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - // Set up mock. - rootFile, rootInode, err := root(ctx, cacheNone, p9.PermissionsMask, 0) - if err != nil { - t.Fatalf("error creating root: %v", err) - } - - // Call function. - err = rootInode.SetTimestamps(ctx, nil /* Dirent */, test.ts) - - // Check return values. - if err != nil { - t.Errorf("SetTimestamps failed: got error %v, want nil", err) - } - - // Check mock parameters. - if !(test.ts.ATimeOmit && test.ts.MTimeOmit) && !rootFile.SetAttrMock.Called { - t.Errorf("TestSetTimestamps failed: SetAttr not called") - return - } - - // Check what was passed to the mock function. - attr := rootFile.SetAttrMock.Attr - atimeGiven := ktime.FromUnix(int64(attr.ATimeSeconds), int64(attr.ATimeNanoSeconds)) - if test.ts.ATimeOmit { - if rootFile.SetAttrMock.Valid.ATime { - t.Errorf("ATime got set true in mask, wanted false") - } - } else { - if got, want := rootFile.SetAttrMock.Valid.ATimeNotSystemTime, !test.ts.ATimeSetSystemTime; got != want { - t.Errorf("got ATimeNotSystemTime %v, want %v", got, want) - } - if !test.ts.ATimeSetSystemTime && !test.ts.ATime.Equal(atimeGiven) { - t.Errorf("ATime got %v, want %v", atimeGiven, test.ts.ATime) - } - } - - mtimeGiven := ktime.FromUnix(int64(attr.MTimeSeconds), int64(attr.MTimeNanoSeconds)) - if test.ts.MTimeOmit { - if rootFile.SetAttrMock.Valid.MTime { - t.Errorf("MTime got set true in mask, wanted false") - } - } else { - if got, want := rootFile.SetAttrMock.Valid.MTimeNotSystemTime, !test.ts.MTimeSetSystemTime; got != want { - t.Errorf("got MTimeNotSystemTime %v, want %v", got, want) - } - if !test.ts.MTimeSetSystemTime && !test.ts.MTime.Equal(mtimeGiven) { - t.Errorf("MTime got %v, want %v", mtimeGiven, test.ts.MTime) - } - } - }) - } -} - -func TestSetPermissions(t *testing.T) { - // Test parameters. - type setPermissionsTest struct { - // Name of the test. - name string - - // SetPermissions input parameters. - perms fs.FilePermissions - - // Error that SetAttr mock should return. - setAttrErr error - - // Expected return value. - want bool - } - - tests := []setPermissionsTest{ - { - name: "SetAttr mock succeeds (function succeeds)", - perms: fs.FilePermissions{User: fs.PermMask{Read: true, Write: true, Execute: true}}, - want: true, - setAttrErr: nil, - }, - { - name: "SetAttr mock fails (function fails)", - perms: fs.FilePermissions{User: fs.PermMask{Read: true, Write: true}}, - want: false, - setAttrErr: syscall.ENOENT, - }, - } - - ctx := contexttest.Context(t) - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - // Set up mock. - rootFile, rootInode, err := root(ctx, cacheNone, 0, 0) - if err != nil { - t.Fatalf("error creating root: %v", err) - } - rootFile.SetAttrMock.Err = test.setAttrErr - - ok := rootInode.SetPermissions(ctx, nil /* Dirent */, test.perms) - - // Check return value. - if ok != test.want { - t.Errorf("SetPermissions got %v, want %v", ok, test.want) - } - - // Check mock parameters. - pattr := rootFile.SetAttrMock.Attr - if !rootFile.SetAttrMock.Called { - t.Errorf("SetAttr not called") - return - } - if !rootFile.SetAttrMock.Valid.Permissions { - t.Errorf("SetAttr did not get right request (got false, expected SetAttrMask.Permissions true)") - } - if got := fs.FilePermsFromP9(pattr.Permissions); got != test.perms { - t.Errorf("SetAttr did not get right permissions -- got %v, want %v", got, test.perms) - } - }) - } -} - -func TestClose(t *testing.T) { - ctx := contexttest.Context(t) - // Set up mock. - rootFile, rootInode, err := root(ctx, cacheNone, p9.PermissionsMask, 0) - if err != nil { - t.Fatalf("error creating root: %v", err) - } - - // Call function. - rootInode.InodeOperations.Release(ctx) - - // Check mock parameters. - if !rootFile.CloseMock.Called { - t.Errorf("TestClose failed: Close not called") - } -} - -func TestRename(t *testing.T) { - // Test parameters. - type renameTest struct { - // Name of the test. - name string - - // Input parameters. - newParent *fs.Inode - newName string - - // Rename mock parameters. - renameErr error - renameCalled bool - - // Error want to return given the parameters. (Same as what - // we expect and tell rename to return.) - want error - } - ctx := contexttest.Context(t) - rootFile, rootInode, err := root(ctx, cacheNone, p9.PermissionsMask, 0) - if err != nil { - t.Fatalf("error creating root: %v", err) - } - - tests := []renameTest{ - { - name: "mock Rename succeeds (function succeeds)", - newParent: rootInode, - newName: "foo2", - want: nil, - renameErr: nil, - renameCalled: true, - }, - { - name: "mock Rename fails (function fails)", - newParent: rootInode, - newName: "foo2", - want: syscall.ENOENT, - renameErr: syscall.ENOENT, - renameCalled: true, - }, - { - name: "newParent is not inodeOperations but should be (function fails)", - newParent: fs.NewMockInode(ctx, fs.NewMockMountSource(nil), fs.StableAttr{Type: fs.Directory}), - newName: "foo2", - want: syscall.EXDEV, - renameErr: nil, - renameCalled: false, - }, - } - - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - mockFile := goodMockFile(p9.PermissionsMask, 0) - rootFile.WalkGetAttrMock.QIDs = []p9.QID{{}} - rootFile.WalkGetAttrMock.File = mockFile - - dirent, err := rootInode.Lookup(ctx, "foo") - if err != nil { - t.Fatalf("root.Walk failed: %v", err) - } - mockFile.RenameMock.Err = test.renameErr - mockFile.RenameMock.Called = false - - // Use a dummy oldParent to acquire write access to that directory. - oldParent := &inodeOperations{ - readdirCache: fs.NewSortedDentryMap(nil), - } - oldInode := fs.NewInode(oldParent, fs.NewMockMountSource(nil), fs.StableAttr{Type: fs.Directory}) - - // Call function. - err = dirent.Inode.InodeOperations.Rename(ctx, oldInode, "", test.newParent, test.newName) - - // Check return value. - if err != test.want { - t.Errorf("Rename got %v, want %v", err, test.want) - } - - // Check mock parameters. - if got, want := mockFile.RenameMock.Called, test.renameCalled; got != want { - t.Errorf("renameCalled got %v want %v", got, want) - } - }) - } -} - -// This file is read from in TestPreadv. -type readAtFileFake struct { - p9test.FileMock - - // Parameters for faking ReadAt. - FileLength int - Err error - ChunkSize int - Called bool - LengthRead int -} - -func (r *readAtFileFake) ReadAt(p []byte, offset uint64) (int, error) { - r.Called = true - log.Warningf("ReadAt fake: length read so far = %d, len(p) = %d, offset = %d", r.LengthRead, len(p), offset) - if int(offset) != r.LengthRead { - return 0, fmt.Errorf("offset got %d; expected %d", offset, r.LengthRead) - } - - if r.Err != nil { - return 0, r.Err - } - - if r.LengthRead >= r.FileLength { - return 0, io.EOF - } - - // Read at most ChunkSize and read at most what's left in the file. - toBeRead := len(p) - if r.LengthRead+toBeRead >= r.FileLength { - toBeRead = r.FileLength - int(offset) - } - if toBeRead > r.ChunkSize { - toBeRead = r.ChunkSize - } - - r.LengthRead += toBeRead - if r.LengthRead == r.FileLength { - return toBeRead, io.EOF - } - return toBeRead, nil -} - -func TestPreadv(t *testing.T) { - // Test parameters. - type preadvTest struct { - // Name of the test. - name string - - // Mock parameters - mode p9.FileMode - - // Buffer to read into. - buffer [512]byte - sliceSize int - - // How much readAt returns at a time. - chunkSize int - - // Whether or not we expect ReadAt to be called. - readAtCalled bool - readAtErr error - - // Expected return values. - want error - } - - tests := []preadvTest{ - { - name: "fake ReadAt succeeds, 512 bytes requested, 512 byte chunks (function succeeds)", - want: nil, - readAtErr: nil, - mode: p9.PermissionsMask, - readAtCalled: true, - sliceSize: 512, - chunkSize: 512, - }, - { - name: "fake ReadAt succeeds, 512 bytes requested, 200 byte chunks (function succeeds)", - want: nil, - readAtErr: nil, - mode: p9.PermissionsMask, - readAtCalled: true, - sliceSize: 512, - chunkSize: 200, - }, - { - name: "fake ReadAt succeeds, 0 bytes requested (function succeeds)", - want: nil, - readAtErr: nil, - mode: p9.PermissionsMask, - readAtCalled: false, - sliceSize: 0, - chunkSize: 100, - }, - { - name: "fake ReadAt returns 0 bytes and EOF (function fails)", - want: io.EOF, - readAtErr: io.EOF, - mode: p9.PermissionsMask, - readAtCalled: true, - sliceSize: 512, - chunkSize: 512, - }, - } - - ctx := contexttest.Context(t) - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - // Set up mock. - rootFile, rootInode, err := root(ctx, cacheNone, test.mode, 1024) - if err != nil { - t.Fatalf("error creating root: %v", err) - } - - // Set up the read buffer. - dst := usermem.BytesIOSequence(test.buffer[:test.sliceSize]) - - // This file will be read from. - openFile := &readAtFileFake{ - Err: test.readAtErr, - FileLength: test.sliceSize, - ChunkSize: test.chunkSize, - } - rootFile.WalkGetAttrMock.File = openFile - rootFile.WalkGetAttrMock.Attr.Mode = test.mode - rootFile.WalkGetAttrMock.Valid.Mode = true - - f := NewFile( - ctx, - fs.NewDirent(rootInode, ""), - "", - fs.FileFlags{Read: true}, - rootInode.InodeOperations.(*inodeOperations), - &handles{File: contextFile{file: openFile}}, - ) - - // Call function. - _, err = f.Preadv(ctx, dst, 0) - - // Check return value. - if err != test.want { - t.Errorf("Preadv got %v, want %v", err, test.want) - } - - // Check mock parameters. - if test.readAtCalled != openFile.Called { - t.Errorf("ReadAt called: %v, but expected opposite", openFile.Called) - } - }) - } -} - -func TestReadlink(t *testing.T) { - // Test parameters. - type readlinkTest struct { - // Name of the test. - name string - - // Mock parameters - mode p9.FileMode - - // Whether or not we expect ReadAt to be called and what error - // it shall return. - readlinkCalled bool - readlinkErr error - - // Expected return values. - want error - } - - tests := []readlinkTest{ - { - name: "file is not symlink (function fails)", - want: syscall.ENOLINK, - mode: p9.PermissionsMask, - readlinkCalled: false, - readlinkErr: nil, - }, - { - name: "mock Readlink succeeds (function succeeds)", - want: nil, - mode: p9.PermissionsMask | p9.ModeSymlink, - readlinkCalled: true, - readlinkErr: nil, - }, - { - name: "mock Readlink fails (function fails)", - want: syscall.ENOENT, - mode: p9.PermissionsMask | p9.ModeSymlink, - readlinkCalled: true, - readlinkErr: syscall.ENOENT, - }, - } - - ctx := contexttest.Context(t) - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - // Set up mock. - rootFile, rootInode, err := root(ctx, cacheNone, test.mode, 0) - if err != nil { - t.Fatalf("error creating root: %v", err) - } - - openFile := goodMockFile(test.mode, 0) - rootFile.WalkMock.File = openFile - rootFile.ReadlinkMock.Err = test.readlinkErr - - // Call function. - _, err = rootInode.Readlink(ctx) - - // Check return value. - if err != test.want { - t.Errorf("Readlink got %v, want %v", err, test.want) - } - - // Check mock parameters. - if test.readlinkCalled && !rootFile.ReadlinkMock.Called { - t.Errorf("Readlink not called") - } - }) - } -} - -// This file is write from in TestPwritev. -type writeAtFileFake struct { - p9test.FileMock - - // Parameters for faking WriteAt. - Err error - ChunkSize int - Called bool - LengthWritten int -} - -func (r *writeAtFileFake) WriteAt(p []byte, offset uint64) (int, error) { - r.Called = true - log.Warningf("WriteAt fake: length written so far = %d, len(p) = %d, offset = %d", r.LengthWritten, len(p), offset) - if int(offset) != r.LengthWritten { - return 0, fmt.Errorf("offset got %d; want %d", offset, r.LengthWritten) - } - - if r.Err != nil { - return 0, r.Err - } - - // Write at most ChunkSize. - toBeWritten := len(p) - if toBeWritten > r.ChunkSize { - toBeWritten = r.ChunkSize - } - r.LengthWritten += toBeWritten - return toBeWritten, nil -} - -func TestPwritev(t *testing.T) { - // Test parameters. - type pwritevTest struct { - // Name of the test. - name string - - // Mock parameters - mode p9.FileMode - - allowWrite bool - - // Buffer to write into. - buffer [512]byte - sliceSize int - chunkSize int - - // Whether or not we expect writeAt to be called. - writeAtCalled bool - writeAtErr error - - // Expected return values. - want error - } - - tests := []pwritevTest{ - { - name: "fake writeAt succeeds, one chunk (function succeeds)", - want: nil, - writeAtErr: nil, - mode: p9.PermissionsMask, - allowWrite: true, - writeAtCalled: true, - sliceSize: 512, - chunkSize: 512, - }, - { - name: "fake writeAt fails, short write (function fails)", - want: io.ErrShortWrite, - writeAtErr: nil, - mode: p9.PermissionsMask, - allowWrite: true, - writeAtCalled: true, - sliceSize: 512, - chunkSize: 200, - }, - { - name: "fake writeAt succeeds, len 0 (function succeeds)", - want: nil, - writeAtErr: nil, - mode: p9.PermissionsMask, - allowWrite: true, - writeAtCalled: false, - sliceSize: 0, - chunkSize: 0, - }, - { - name: "writeAt can still write despite file permissions read only (function succeeds)", - want: nil, - writeAtErr: nil, - mode: p9.PermissionsMask, - allowWrite: false, - writeAtCalled: true, - sliceSize: 512, - chunkSize: 512, - }, - } - - ctx := contexttest.Context(t) - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - // Set up mock. - _, rootInode, err := root(ctx, cacheNone, test.mode, 0) - if err != nil { - t.Fatalf("error creating root: %v", err) - } - - src := usermem.BytesIOSequence(test.buffer[:test.sliceSize]) - - // This is the file that will be used for writing. - openFile := &writeAtFileFake{ - Err: test.writeAtErr, - ChunkSize: test.chunkSize, - } - - f := NewFile( - ctx, - fs.NewDirent(rootInode, ""), - "", - fs.FileFlags{Write: true}, - rootInode.InodeOperations.(*inodeOperations), - &handles{File: contextFile{file: openFile}}, - ) - - // Call function. - _, err = f.Pwritev(ctx, src, 0) - - // Check return value. - if err != test.want { - t.Errorf("Pwritev got %v, want %v", err, test.want) - } - - // Check mock parameters. - if test.writeAtCalled != openFile.Called { - t.Errorf("WriteAt called: %v, but expected opposite", openFile.Called) - return + t.Errorf("Lookup with cachePolicy=%s got new dirent and error %v, wanted old dirent and nil error", test.cachePolicy, err) } - if openFile.Called && test.writeAtErr != nil && openFile.LengthWritten != test.sliceSize { - t.Errorf("wrote %d bytes, expected %d bytes written", openFile.LengthWritten, test.sliceSize) + if err == nil { + newDirent.DecRef() // See above. } }) } diff --git a/pkg/sentry/fs/gofer/session.go b/pkg/sentry/fs/gofer/session.go index 7552216f3..f76a83cd9 100644 --- a/pkg/sentry/fs/gofer/session.go +++ b/pkg/sentry/fs/gofer/session.go @@ -91,10 +91,6 @@ func (e *endpointMaps) get(key device.MultiDeviceKey) transport.BoundEndpoint { type session struct { refs.AtomicRefCount - // conn is a unet.Socket that wraps the readFD/writeFD mount option, - // see fs/gofer/fs.go. - conn *unet.Socket `state:"nosave"` - // msize is the value of the msize mount option, see fs/gofer/fs.go. msize uint32 `state:"wait"` @@ -142,7 +138,7 @@ type session struct { // Destroy tears down the session. func (s *session) Destroy() { - s.conn.Close() + s.client.Close() } // Revalidate implements MountSource.Revalidate. @@ -235,7 +231,6 @@ func Root(ctx context.Context, dev string, filesystem fs.Filesystem, superBlockF // Construct the session. s := &session{ connID: dev, - conn: conn, msize: o.msize, version: o.version, cachePolicy: o.policy, @@ -252,7 +247,7 @@ func Root(ctx context.Context, dev string, filesystem fs.Filesystem, superBlockF m := fs.NewMountSource(s, filesystem, superBlockFlags) // Send the Tversion request. - s.client, err = p9.NewClient(s.conn, s.msize, s.version) + s.client, err = p9.NewClient(conn, s.msize, s.version) if err != nil { // Drop our reference on the session, it needs to be torn down. s.DecRef() diff --git a/pkg/sentry/fs/gofer/session_state.go b/pkg/sentry/fs/gofer/session_state.go index f657135fc..d9fd7a221 100644 --- a/pkg/sentry/fs/gofer/session_state.go +++ b/pkg/sentry/fs/gofer/session_state.go @@ -84,13 +84,13 @@ func (s *session) afterLoad() { } // Manually restore the connection. - s.conn, err = unet.NewSocket(opts.fd) + conn, err := unet.NewSocket(opts.fd) if err != nil { panic(fmt.Sprintf("failed to create Socket for FD %d: %v", opts.fd, err)) } // Manually restore the client. - s.client, err = p9.NewClient(s.conn, s.msize, s.version) + s.client, err = p9.NewClient(conn, s.msize, s.version) if err != nil { panic(fmt.Sprintf("failed to connect client to server: %v", err)) } diff --git a/pkg/sentry/fs/proc/device/BUILD b/pkg/sentry/fs/proc/device/BUILD index 34582f275..ff7dacf07 100644 --- a/pkg/sentry/fs/proc/device/BUILD +++ b/pkg/sentry/fs/proc/device/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "device", srcs = ["device.go"], diff --git a/pkg/sentry/hostcpu/BUILD b/pkg/sentry/hostcpu/BUILD index f362d15c8..33197cf14 100644 --- a/pkg/sentry/hostcpu/BUILD +++ b/pkg/sentry/hostcpu/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "hostcpu", srcs = [ diff --git a/pkg/sentry/kernel/kdefs/BUILD b/pkg/sentry/kernel/kdefs/BUILD index fe6fa2260..3f8fa206c 100644 --- a/pkg/sentry/kernel/kdefs/BUILD +++ b/pkg/sentry/kernel/kdefs/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "kdefs", srcs = ["kdefs.go"], diff --git a/pkg/sentry/kernel/memevent/BUILD b/pkg/sentry/kernel/memevent/BUILD index 66899910c..e903badd3 100644 --- a/pkg/sentry/kernel/memevent/BUILD +++ b/pkg/sentry/kernel/memevent/BUILD @@ -1,8 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "memevent", srcs = ["memory_events.go"], diff --git a/pkg/sentry/kernel/sched/BUILD b/pkg/sentry/kernel/sched/BUILD index 125792f39..52e226a39 100644 --- a/pkg/sentry/kernel/sched/BUILD +++ b/pkg/sentry/kernel/sched/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "sched", srcs = [ diff --git a/pkg/sentry/loader/BUILD b/pkg/sentry/loader/BUILD index 0beb4561b..83cad186a 100644 --- a/pkg/sentry/loader/BUILD +++ b/pkg/sentry/loader/BUILD @@ -1,6 +1,7 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_embed_data") + package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_embed_data") load("//tools/go_stateify:defs.bzl", "go_library") go_embed_data( diff --git a/pkg/sentry/memutil/BUILD b/pkg/sentry/memutil/BUILD index 341b30b98..88738d65d 100644 --- a/pkg/sentry/memutil/BUILD +++ b/pkg/sentry/memutil/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "memutil", srcs = [ diff --git a/pkg/sentry/platform/interrupt/BUILD b/pkg/sentry/platform/interrupt/BUILD index 35121321a..dbafa3204 100644 --- a/pkg/sentry/platform/interrupt/BUILD +++ b/pkg/sentry/platform/interrupt/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "interrupt", srcs = [ diff --git a/pkg/sentry/platform/kvm/BUILD b/pkg/sentry/platform/kvm/BUILD index 4ef9e20d7..1b71e629f 100644 --- a/pkg/sentry/platform/kvm/BUILD +++ b/pkg/sentry/platform/kvm/BUILD @@ -1,6 +1,7 @@ +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") + package(licenses = ["notice"]) # Apache 2.0 -load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/sentry/platform/kvm/testutil/BUILD b/pkg/sentry/platform/kvm/testutil/BUILD index e779e3893..1dffe94a4 100644 --- a/pkg/sentry/platform/kvm/testutil/BUILD +++ b/pkg/sentry/platform/kvm/testutil/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "testutil", testonly = 1, diff --git a/pkg/sentry/platform/procid/BUILD b/pkg/sentry/platform/procid/BUILD index ba68d48f4..d3398d1e8 100644 --- a/pkg/sentry/platform/procid/BUILD +++ b/pkg/sentry/platform/procid/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "procid", srcs = [ diff --git a/pkg/sentry/platform/ptrace/BUILD b/pkg/sentry/platform/ptrace/BUILD index debae058b..2eb354ad4 100644 --- a/pkg/sentry/platform/ptrace/BUILD +++ b/pkg/sentry/platform/ptrace/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "ptrace", srcs = [ diff --git a/pkg/sentry/platform/ring0/BUILD b/pkg/sentry/platform/ring0/BUILD index 2485eb2eb..c35d49f2d 100644 --- a/pkg/sentry/platform/ring0/BUILD +++ b/pkg/sentry/platform/ring0/BUILD @@ -1,6 +1,7 @@ +load("//tools/go_stateify:defs.bzl", "go_library") + package(licenses = ["notice"]) # Apache 2.0 -load("//tools/go_stateify:defs.bzl", "go_library") load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") go_template( diff --git a/pkg/sentry/platform/ring0/gen_offsets/BUILD b/pkg/sentry/platform/ring0/gen_offsets/BUILD index 3bce56985..b76d7974e 100644 --- a/pkg/sentry/platform/ring0/gen_offsets/BUILD +++ b/pkg/sentry/platform/ring0/gen_offsets/BUILD @@ -1,6 +1,7 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_binary") + package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_binary") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/sentry/platform/ring0/pagetables/BUILD b/pkg/sentry/platform/ring0/pagetables/BUILD index 7a86e2234..de1b920af 100644 --- a/pkg/sentry/platform/ring0/pagetables/BUILD +++ b/pkg/sentry/platform/ring0/pagetables/BUILD @@ -1,6 +1,7 @@ +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") + package(licenses = ["notice"]) # Apache 2.0 -load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") go_template( diff --git a/pkg/sentry/platform/safecopy/BUILD b/pkg/sentry/platform/safecopy/BUILD index 7dcf6e561..614d9e21e 100644 --- a/pkg/sentry/platform/safecopy/BUILD +++ b/pkg/sentry/platform/safecopy/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "safecopy", srcs = [ diff --git a/pkg/sentry/safemem/BUILD b/pkg/sentry/safemem/BUILD index e96509ce1..87a9bff12 100644 --- a/pkg/sentry/safemem/BUILD +++ b/pkg/sentry/safemem/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "safemem", srcs = [ diff --git a/pkg/sentry/sighandling/BUILD b/pkg/sentry/sighandling/BUILD index 751176747..41313d334 100644 --- a/pkg/sentry/sighandling/BUILD +++ b/pkg/sentry/sighandling/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "sighandling", srcs = [ diff --git a/pkg/sentry/socket/rpcinet/BUILD b/pkg/sentry/socket/rpcinet/BUILD index 38fa54283..06e121946 100644 --- a/pkg/sentry/socket/rpcinet/BUILD +++ b/pkg/sentry/socket/rpcinet/BUILD @@ -1,8 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "rpcinet", srcs = [ diff --git a/pkg/sentry/socket/rpcinet/conn/BUILD b/pkg/sentry/socket/rpcinet/conn/BUILD index c51ca14b1..a16977f29 100644 --- a/pkg/sentry/socket/rpcinet/conn/BUILD +++ b/pkg/sentry/socket/rpcinet/conn/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # BSD - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # BSD + go_library( name = "conn", srcs = ["conn.go"], diff --git a/pkg/sentry/socket/rpcinet/notifier/BUILD b/pkg/sentry/socket/rpcinet/notifier/BUILD index 2ae902b3f..2bab01774 100644 --- a/pkg/sentry/socket/rpcinet/notifier/BUILD +++ b/pkg/sentry/socket/rpcinet/notifier/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # BSD - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # BSD + go_library( name = "notifier", srcs = ["notifier.go"], diff --git a/pkg/sentry/state/BUILD b/pkg/sentry/state/BUILD index a57a8298e..f1f6fdb7d 100644 --- a/pkg/sentry/state/BUILD +++ b/pkg/sentry/state/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "state", srcs = [ diff --git a/pkg/sentry/strace/BUILD b/pkg/sentry/strace/BUILD index 674554081..52c7f325c 100644 --- a/pkg/sentry/strace/BUILD +++ b/pkg/sentry/strace/BUILD @@ -1,8 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "strace", srcs = [ diff --git a/pkg/sentry/syscalls/BUILD b/pkg/sentry/syscalls/BUILD index 2a9f0915e..35192ff49 100644 --- a/pkg/sentry/syscalls/BUILD +++ b/pkg/sentry/syscalls/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "syscalls", srcs = [ diff --git a/pkg/sentry/time/BUILD b/pkg/sentry/time/BUILD index 9452787fb..5dadb8a2d 100644 --- a/pkg/sentry/time/BUILD +++ b/pkg/sentry/time/BUILD @@ -1,6 +1,7 @@ +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") + package(licenses = ["notice"]) # Apache 2.0 -load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/sentry/unimpl/BUILD b/pkg/sentry/unimpl/BUILD index 63da5e81f..42e24ace5 100644 --- a/pkg/sentry/unimpl/BUILD +++ b/pkg/sentry/unimpl/BUILD @@ -1,8 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +package(licenses = ["notice"]) # Apache 2.0 + proto_library( name = "unimplemented_syscall_proto", srcs = ["unimplemented_syscall.proto"], diff --git a/pkg/sentry/uniqueid/BUILD b/pkg/sentry/uniqueid/BUILD index 68b82af47..0929497c3 100644 --- a/pkg/sentry/uniqueid/BUILD +++ b/pkg/sentry/uniqueid/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "uniqueid", srcs = ["context.go"], diff --git a/pkg/sentry/watchdog/BUILD b/pkg/sentry/watchdog/BUILD index 13bc33eb1..b2c687b20 100644 --- a/pkg/sentry/watchdog/BUILD +++ b/pkg/sentry/watchdog/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "watchdog", srcs = ["watchdog.go"], diff --git a/pkg/sleep/BUILD b/pkg/sleep/BUILD index 05e4ca540..338fd9336 100644 --- a/pkg/sleep/BUILD +++ b/pkg/sleep/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "sleep", srcs = [ diff --git a/pkg/state/BUILD b/pkg/state/BUILD index 6a5b2d4ff..dd0f250fa 100644 --- a/pkg/state/BUILD +++ b/pkg/state/BUILD @@ -1,7 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") + +package(licenses = ["notice"]) # Apache 2.0 + load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/state/statefile/BUILD b/pkg/state/statefile/BUILD index 6be78dc9b..66c8f3807 100644 --- a/pkg/state/statefile/BUILD +++ b/pkg/state/statefile/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "statefile", srcs = ["statefile.go"], diff --git a/pkg/sync/atomicptrtest/BUILD b/pkg/sync/atomicptrtest/BUILD index 4fa959df0..9cb7f66fe 100644 --- a/pkg/sync/atomicptrtest/BUILD +++ b/pkg/sync/atomicptrtest/BUILD @@ -1,6 +1,7 @@ +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") + package(licenses = ["notice"]) # Apache 2.0 -load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/sync/seqatomictest/BUILD b/pkg/sync/seqatomictest/BUILD index 07b4f85ab..54f8e59b1 100644 --- a/pkg/sync/seqatomictest/BUILD +++ b/pkg/sync/seqatomictest/BUILD @@ -1,6 +1,7 @@ +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") + package(licenses = ["notice"]) # Apache 2.0 -load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/syserr/BUILD b/pkg/syserr/BUILD index 5dd2e90bb..30ae20772 100644 --- a/pkg/syserr/BUILD +++ b/pkg/syserr/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "syserr", srcs = [ diff --git a/pkg/syserror/BUILD b/pkg/syserror/BUILD index e050c2043..d4c6da97a 100644 --- a/pkg/syserror/BUILD +++ b/pkg/syserror/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "syserror", srcs = ["syserror.go"], diff --git a/pkg/tcpip/adapters/gonet/BUILD b/pkg/tcpip/adapters/gonet/BUILD index bf618831a..723ad668f 100644 --- a/pkg/tcpip/adapters/gonet/BUILD +++ b/pkg/tcpip/adapters/gonet/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "gonet", srcs = ["gonet.go"], diff --git a/pkg/tcpip/checker/BUILD b/pkg/tcpip/checker/BUILD index e8a524918..a1de808b9 100644 --- a/pkg/tcpip/checker/BUILD +++ b/pkg/tcpip/checker/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "checker", testonly = 1, diff --git a/pkg/tcpip/link/channel/BUILD b/pkg/tcpip/link/channel/BUILD index 9a6f49c45..25f6c1457 100644 --- a/pkg/tcpip/link/channel/BUILD +++ b/pkg/tcpip/link/channel/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "channel", srcs = ["channel.go"], diff --git a/pkg/tcpip/link/fdbased/BUILD b/pkg/tcpip/link/fdbased/BUILD index 6e75e9f47..94391433c 100644 --- a/pkg/tcpip/link/fdbased/BUILD +++ b/pkg/tcpip/link/fdbased/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "fdbased", srcs = ["endpoint.go"], diff --git a/pkg/tcpip/link/loopback/BUILD b/pkg/tcpip/link/loopback/BUILD index cc4247ffd..a46ba7f11 100644 --- a/pkg/tcpip/link/loopback/BUILD +++ b/pkg/tcpip/link/loopback/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "loopback", srcs = ["loopback.go"], diff --git a/pkg/tcpip/link/rawfile/BUILD b/pkg/tcpip/link/rawfile/BUILD index 10b35a37e..829ea7c42 100644 --- a/pkg/tcpip/link/rawfile/BUILD +++ b/pkg/tcpip/link/rawfile/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "rawfile", srcs = [ diff --git a/pkg/tcpip/link/sharedmem/BUILD b/pkg/tcpip/link/sharedmem/BUILD index 5390257c5..d7f1e66ef 100644 --- a/pkg/tcpip/link/sharedmem/BUILD +++ b/pkg/tcpip/link/sharedmem/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "sharedmem", srcs = [ diff --git a/pkg/tcpip/link/sharedmem/pipe/BUILD b/pkg/tcpip/link/sharedmem/pipe/BUILD index ff798ae6f..12e813509 100644 --- a/pkg/tcpip/link/sharedmem/pipe/BUILD +++ b/pkg/tcpip/link/sharedmem/pipe/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "pipe", srcs = [ diff --git a/pkg/tcpip/link/sharedmem/queue/BUILD b/pkg/tcpip/link/sharedmem/queue/BUILD index c4a7879c4..661037bb2 100644 --- a/pkg/tcpip/link/sharedmem/queue/BUILD +++ b/pkg/tcpip/link/sharedmem/queue/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "queue", srcs = [ diff --git a/pkg/tcpip/link/sniffer/BUILD b/pkg/tcpip/link/sniffer/BUILD index 7155aea66..52e237c25 100644 --- a/pkg/tcpip/link/sniffer/BUILD +++ b/pkg/tcpip/link/sniffer/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "sniffer", srcs = [ diff --git a/pkg/tcpip/link/tun/BUILD b/pkg/tcpip/link/tun/BUILD index a8bb03661..5ec01cec9 100644 --- a/pkg/tcpip/link/tun/BUILD +++ b/pkg/tcpip/link/tun/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "tun", srcs = ["tun_unsafe.go"], diff --git a/pkg/tcpip/link/waitable/BUILD b/pkg/tcpip/link/waitable/BUILD index 7582df32e..ba495c437 100644 --- a/pkg/tcpip/link/waitable/BUILD +++ b/pkg/tcpip/link/waitable/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "waitable", srcs = [ diff --git a/pkg/tcpip/network/BUILD b/pkg/tcpip/network/BUILD index 25a3c98b6..a2a07f533 100644 --- a/pkg/tcpip/network/BUILD +++ b/pkg/tcpip/network/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_test( name = "ip_test", size = "small", diff --git a/pkg/tcpip/network/arp/BUILD b/pkg/tcpip/network/arp/BUILD index 44f2b66e5..f6fb7daf7 100644 --- a/pkg/tcpip/network/arp/BUILD +++ b/pkg/tcpip/network/arp/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "arp", srcs = ["arp.go"], diff --git a/pkg/tcpip/network/hash/BUILD b/pkg/tcpip/network/hash/BUILD index 1c22c52fc..401dce646 100644 --- a/pkg/tcpip/network/hash/BUILD +++ b/pkg/tcpip/network/hash/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "hash", srcs = ["hash.go"], diff --git a/pkg/tcpip/network/ipv4/BUILD b/pkg/tcpip/network/ipv4/BUILD index 90d65d531..e72317e9f 100644 --- a/pkg/tcpip/network/ipv4/BUILD +++ b/pkg/tcpip/network/ipv4/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "ipv4", srcs = [ diff --git a/pkg/tcpip/network/ipv6/BUILD b/pkg/tcpip/network/ipv6/BUILD index 2f19a659e..808c37df3 100644 --- a/pkg/tcpip/network/ipv6/BUILD +++ b/pkg/tcpip/network/ipv6/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "ipv6", srcs = [ diff --git a/pkg/tcpip/ports/BUILD b/pkg/tcpip/ports/BUILD index 3c3374275..c69fc0744 100644 --- a/pkg/tcpip/ports/BUILD +++ b/pkg/tcpip/ports/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "ports", srcs = ["ports.go"], diff --git a/pkg/tcpip/sample/tun_tcp_connect/BUILD b/pkg/tcpip/sample/tun_tcp_connect/BUILD index 21d32245d..32baf2115 100644 --- a/pkg/tcpip/sample/tun_tcp_connect/BUILD +++ b/pkg/tcpip/sample/tun_tcp_connect/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_binary") +package(licenses = ["notice"]) # Apache 2.0 + go_binary( name = "tun_tcp_connect", srcs = ["main.go"], diff --git a/pkg/tcpip/sample/tun_tcp_echo/BUILD b/pkg/tcpip/sample/tun_tcp_echo/BUILD index d7402aaa2..760445843 100644 --- a/pkg/tcpip/sample/tun_tcp_echo/BUILD +++ b/pkg/tcpip/sample/tun_tcp_echo/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_binary") +package(licenses = ["notice"]) # Apache 2.0 + go_binary( name = "tun_tcp_echo", srcs = ["main.go"], diff --git a/pkg/tcpip/transport/tcp/testing/context/BUILD b/pkg/tcpip/transport/tcp/testing/context/BUILD index 7a95594ef..814e5c1ea 100644 --- a/pkg/tcpip/transport/tcp/testing/context/BUILD +++ b/pkg/tcpip/transport/tcp/testing/context/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "context", testonly = 1, diff --git a/pkg/tcpip/transport/tcpconntrack/BUILD b/pkg/tcpip/transport/tcpconntrack/BUILD index 46da3e6f1..ac1a94d4d 100644 --- a/pkg/tcpip/transport/tcpconntrack/BUILD +++ b/pkg/tcpip/transport/tcpconntrack/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "tcpconntrack", srcs = ["tcp_conntrack.go"], diff --git a/pkg/tmutex/BUILD b/pkg/tmutex/BUILD index d18338fff..c20df7005 100644 --- a/pkg/tmutex/BUILD +++ b/pkg/tmutex/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "tmutex", srcs = ["tmutex.go"], diff --git a/pkg/unet/BUILD b/pkg/unet/BUILD index acdfd7cb6..f90e43c89 100644 --- a/pkg/unet/BUILD +++ b/pkg/unet/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "unet", srcs = [ diff --git a/pkg/urpc/BUILD b/pkg/urpc/BUILD index d32c57d1a..21008cf6c 100644 --- a/pkg/urpc/BUILD +++ b/pkg/urpc/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "urpc", srcs = ["urpc.go"], diff --git a/pkg/waiter/fdnotifier/BUILD b/pkg/waiter/fdnotifier/BUILD index 4e582755d..af6baa303 100644 --- a/pkg/waiter/fdnotifier/BUILD +++ b/pkg/waiter/fdnotifier/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "fdnotifier", srcs = [ diff --git a/runsc/boot/BUILD b/runsc/boot/BUILD index 04cc0e854..07afce807 100644 --- a/runsc/boot/BUILD +++ b/runsc/boot/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "boot", srcs = [ diff --git a/runsc/boot/filter/BUILD b/runsc/boot/filter/BUILD index 48f2c8024..004222242 100644 --- a/runsc/boot/filter/BUILD +++ b/runsc/boot/filter/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "filter", srcs = [ diff --git a/runsc/cgroup/BUILD b/runsc/cgroup/BUILD index 10a8e5feb..bf2f373a9 100644 --- a/runsc/cgroup/BUILD +++ b/runsc/cgroup/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "cgroup", srcs = ["cgroup.go"], diff --git a/runsc/cmd/BUILD b/runsc/cmd/BUILD index 7040eb4ec..394bb0e1f 100644 --- a/runsc/cmd/BUILD +++ b/runsc/cmd/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "cmd", srcs = [ diff --git a/runsc/console/BUILD b/runsc/console/BUILD index fa1a7d430..ff4ccff69 100644 --- a/runsc/console/BUILD +++ b/runsc/console/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "console", srcs = ["console.go"], diff --git a/runsc/container/BUILD b/runsc/container/BUILD index f4c6f1525..bdd93aaba 100644 --- a/runsc/container/BUILD +++ b/runsc/container/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "container", srcs = [ diff --git a/runsc/fsgofer/BUILD b/runsc/fsgofer/BUILD index 24e172f48..f28e4fa77 100644 --- a/runsc/fsgofer/BUILD +++ b/runsc/fsgofer/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "fsgofer", srcs = [ diff --git a/runsc/fsgofer/filter/BUILD b/runsc/fsgofer/filter/BUILD index 40f4f2205..c7848d10c 100644 --- a/runsc/fsgofer/filter/BUILD +++ b/runsc/fsgofer/filter/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "filter", srcs = [ diff --git a/runsc/fsgofer/fsgofer.go b/runsc/fsgofer/fsgofer.go index e03bb7752..fd913831a 100644 --- a/runsc/fsgofer/fsgofer.go +++ b/runsc/fsgofer/fsgofer.go @@ -26,7 +26,6 @@ import ( "math" "os" "path" - "strings" "sync" "syscall" @@ -181,18 +180,6 @@ func (a *attachPoint) makeQID(stat syscall.Stat_t) p9.QID { } } -func isNameValid(name string) bool { - if name == "" || name == "." || name == ".." { - log.Warningf("Invalid name: %s", name) - return false - } - if strings.IndexByte(name, '/') >= 0 { - log.Warningf("Invalid name: %s", name) - return false - } - return true -} - // localFile implements p9.File wrapping a local file. The underlying file // is opened during Walk() and stored in 'controlFile' to be used with other // operations. The mode in which the file is opened varies depending on the @@ -228,11 +215,7 @@ type localFile struct { // attachPoint is the attachPoint that serves this localFile. attachPoint *attachPoint - // mu protects 'hostPath' when file is renamed. - mu sync.Mutex - - // TODO: hostPath is not safe to use as path needs to be walked - // everytime (and can change underneath us). Remove all usages. + // hostPath will be safely updated by the Renamed hook. hostPath string // controlFile is opened when localFile is created and it's never nil. @@ -246,6 +229,7 @@ type localFile struct { // if localFile isn't opened. mode p9.OpenFlags + // ft is the fileType for this file. ft fileType // readDirMu protects against concurrent Readdir calls. @@ -296,10 +280,7 @@ func openAnyFile(parent *localFile, name string) (*os.File, string, error) { return nil, "", extractErrno(err) } - parent.mu.Lock() - defer parent.mu.Unlock() newPath := path.Join(parent.hostPath, name) - return os.NewFile(uintptr(fd), newPath), newPath, nil } @@ -382,13 +363,10 @@ func (l *localFile) Open(mode p9.OpenFlags) (*fd.FD, p9.QID, uint32, error) { log.Debugf("Open reopening file, mode: %v, %q", mode, l.controlFile.Name()) var err error - l.mu.Lock() newFile, err = os.OpenFile(l.hostPath, openFlags|mode.OSFlags(), 0) if err != nil { - l.mu.Unlock() return nil, p9.QID{}, 0, extractErrno(err) } - l.mu.Unlock() } stat, err := stat(int(newFile.Fd())) @@ -418,9 +396,6 @@ func (l *localFile) Create(name string, mode p9.OpenFlags, perm p9.FileMode, uid } return nil, nil, p9.QID{}, 0, syscall.EBADF } - if !isNameValid(name) { - return nil, nil, p9.QID{}, 0, syscall.EINVAL - } // Use a single file for both 'controlFile' and 'openedFile'. Mode must include read for control // and whichever else was requested by caller. Note that resulting file might have a wider mode @@ -452,9 +427,6 @@ func (l *localFile) Create(name string, mode p9.OpenFlags, perm p9.FileMode, uid return nil, nil, p9.QID{}, 0, extractErrno(err) } - l.mu.Lock() - defer l.mu.Unlock() - cPath := path.Join(l.hostPath, name) f := os.NewFile(uintptr(fd), cPath) c := &localFile{ @@ -477,10 +449,6 @@ func (l *localFile) Mkdir(name string, perm p9.FileMode, uid p9.UID, gid p9.GID) return p9.QID{}, syscall.EBADF } - if !isNameValid(name) { - return p9.QID{}, syscall.EINVAL - } - if err := syscall.Mkdirat(l.controlFD(), name, uint32(perm.Permissions())); err != nil { return p9.QID{}, extractErrno(err) } @@ -517,9 +485,6 @@ func (l *localFile) Walk(names []string) ([]p9.QID, p9.File, error) { return nil, nil, extractErrno(err) } - l.mu.Lock() - defer l.mu.Unlock() - c := &localFile{ attachPoint: l.attachPoint, hostPath: l.hostPath, @@ -532,10 +497,6 @@ func (l *localFile) Walk(names []string) ([]p9.QID, p9.File, error) { var qids []p9.QID last := l for _, name := range names { - if !isNameValid(name) { - return nil, nil, syscall.EINVAL - } - f, path, err := openAnyFile(last, name) if err != nil { return nil, nil, extractErrno(err) @@ -761,15 +722,15 @@ func (l *localFile) SetAttr(valid p9.SetAttrMask, attr p9.SetAttr) error { return err } -// Remove implements p9.File. -// -// This is deprecated in favor of UnlinkAt. -func (*localFile) Remove() error { - return syscall.ENOSYS +// Rename implements p9.File; this should never be called. +func (l *localFile) Rename(p9.File, string) error { + panic("rename called directly") } -// Rename implements p9.File. -func (l *localFile) Rename(directory p9.File, name string) error { +// RenameAt implements p9.File.RenameAt. +// +// TODO: change to renameat(2). +func (l *localFile) RenameAt(oldName string, directory p9.File, newName string) error { conf := l.attachPoint.conf if conf.ROMount { if conf.PanicOnWrite { @@ -777,34 +738,16 @@ func (l *localFile) Rename(directory p9.File, name string) error { } return syscall.EBADF } - if !isNameValid(name) { - return syscall.EINVAL - } - - l.mu.Lock() - defer l.mu.Unlock() - // TODO: change to renameat(2) - parent := directory.(*localFile) - newPath := path.Join(parent.hostPath, name) - if err := syscall.Rename(l.hostPath, newPath); err != nil { + newParent := directory.(*localFile) + oldPath := path.Join(l.hostPath, oldName) + newPath := path.Join(newParent.hostPath, newName) + if err := syscall.Rename(oldPath, newPath); err != nil { return extractErrno(err) } - - // Update path on success. - // TODO: this doesn't cover cases where any of the - // parents have been renamed. - l.hostPath = newPath return nil } -// RenameAt implements p9.File.RenameAt. -// -// Code still uses [deprecated] Rename(). -func (*localFile) RenameAt(_ string, _ p9.File, _ string) error { - return syscall.ENOSYS -} - // ReadAt implements p9.File. func (l *localFile) ReadAt(p []byte, offset uint64) (int, error) { if l.mode != p9.ReadOnly && l.mode != p9.ReadWrite { @@ -848,9 +791,6 @@ func (l *localFile) Symlink(target, newName string, uid p9.UID, gid p9.GID) (p9. } return p9.QID{}, syscall.EBADF } - if !isNameValid(newName) { - return p9.QID{}, syscall.EINVAL - } if err := unix.Symlinkat(target, l.controlFD(), newName); err != nil { return p9.QID{}, extractErrno(err) @@ -882,9 +822,6 @@ func (l *localFile) Link(target p9.File, newName string) error { } return syscall.EBADF } - if !isNameValid(newName) { - return syscall.EINVAL - } targetFile := target.(*localFile) if err := unix.Linkat(targetFile.controlFD(), "", l.controlFD(), newName, linux.AT_EMPTY_PATH); err != nil { @@ -909,9 +846,7 @@ func (l *localFile) UnlinkAt(name string, flags uint32) error { } return syscall.EBADF } - if !isNameValid(name) { - return syscall.EINVAL - } + if err := unix.Unlinkat(l.controlFD(), name, int(flags)); err != nil { return extractErrno(err) } @@ -1000,6 +935,11 @@ func (l *localFile) Close() error { return err } +// Renamed implements p9.Renamed. +func (l *localFile) Renamed(newDir p9.File, newName string) { + l.hostPath = path.Join(newDir.(*localFile).hostPath, newName) +} + // extractErrno tries to determine the errno. func extractErrno(err error) syscall.Errno { if err == nil { diff --git a/runsc/fsgofer/fsgofer_test.go b/runsc/fsgofer/fsgofer_test.go index 48860f952..34033245b 100644 --- a/runsc/fsgofer/fsgofer_test.go +++ b/runsc/fsgofer/fsgofer_test.go @@ -415,22 +415,22 @@ func TestLink(t *testing.T) { func TestROMountChecks(t *testing.T) { runCustom(t, allTypes, roConfs, func(t *testing.T, s state) { - if _, _, _, _, err := s.file.Create("..", p9.ReadWrite, 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EBADF { + if _, _, _, _, err := s.file.Create("some_file", p9.ReadWrite, 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EBADF { t.Errorf("%v: Create() should have failed, got: %v, expected: syscall.EBADF", s, err) } - if _, err := s.file.Mkdir("..", 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EBADF { + if _, err := s.file.Mkdir("some_dir", 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EBADF { t.Errorf("%v: MkDir() should have failed, got: %v, expected: syscall.EBADF", s, err) } - if err := s.file.Rename(s.file, ".."); err != syscall.EBADF { + if err := s.file.RenameAt("some_file", s.file, "other_file"); err != syscall.EBADF { t.Errorf("%v: Rename() should have failed, got: %v, expected: syscall.EBADF", s, err) } - if _, err := s.file.Symlink("some_place", "..", p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EBADF { + if _, err := s.file.Symlink("some_place", "some_symlink", p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EBADF { t.Errorf("%v: Symlink() should have failed, got: %v, expected: syscall.EBADF", s, err) } - if err := s.file.UnlinkAt("..", 0); err != syscall.EBADF { + if err := s.file.UnlinkAt("some_file", 0); err != syscall.EBADF { t.Errorf("%v: UnlinkAt() should have failed, got: %v, expected: syscall.EBADF", s, err) } - if err := s.file.Link(s.file, ".."); err != syscall.EBADF { + if err := s.file.Link(s.file, "some_link"); err != syscall.EBADF { t.Errorf("%v: Link() should have failed, got: %v, expected: syscall.EBADF", s, err) } @@ -445,12 +445,12 @@ func TestROMountChecks(t *testing.T) { func TestROMountPanics(t *testing.T) { conf := Config{ROMount: true, PanicOnWrite: true} runCustom(t, allTypes, []Config{conf}, func(t *testing.T, s state) { - assertPanic(t, func() { s.file.Create("..", p9.ReadWrite, 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())) }) - assertPanic(t, func() { s.file.Mkdir("..", 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())) }) - assertPanic(t, func() { s.file.Rename(s.file, "..") }) - assertPanic(t, func() { s.file.Symlink("some_place", "..", p9.UID(os.Getuid()), p9.GID(os.Getgid())) }) - assertPanic(t, func() { s.file.UnlinkAt("..", 0) }) - assertPanic(t, func() { s.file.Link(s.file, "..") }) + assertPanic(t, func() { s.file.Create("some_file", p9.ReadWrite, 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())) }) + assertPanic(t, func() { s.file.Mkdir("some_dir", 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())) }) + assertPanic(t, func() { s.file.RenameAt("some_file", s.file, "other_file") }) + assertPanic(t, func() { s.file.Symlink("some_place", "some_symlink", p9.UID(os.Getuid()), p9.GID(os.Getgid())) }) + assertPanic(t, func() { s.file.UnlinkAt("some_file", 0) }) + assertPanic(t, func() { s.file.Link(s.file, "some_link") }) valid := p9.SetAttrMask{Size: true} attr := p9.SetAttr{Size: 0} @@ -458,60 +458,6 @@ func TestROMountPanics(t *testing.T) { }) } -func TestInvalidName(t *testing.T) { - runCustom(t, []fileType{regular}, rwConfs, func(t *testing.T, s state) { - if _, _, _, _, err := s.file.Create("..", p9.ReadWrite, 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EINVAL { - t.Errorf("%v: Create() should have failed, got: %v, expected: syscall.EINVAL", s, err) - } - if _, _, err := s.file.Walk([]string{".."}); err != syscall.EINVAL { - t.Errorf("%v: Walk() should have failed, got: %v, expected: syscall.EINVAL", s, err) - } - if _, err := s.file.Mkdir("..", 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EINVAL { - t.Errorf("%v: MkDir() should have failed, got: %v, expected: syscall.EINVAL", s, err) - } - if err := s.file.Rename(s.file, ".."); err != syscall.EINVAL { - t.Errorf("%v: Rename() should have failed, got: %v, expected: syscall.EINVAL", s, err) - } - if _, err := s.file.Symlink("some_place", "..", p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EINVAL { - t.Errorf("%v: Symlink() should have failed, got: %v, expected: syscall.EINVAL", s, err) - } - if err := s.file.UnlinkAt("..", 0); err != syscall.EINVAL { - t.Errorf("%v: UnlinkAt() should have failed, got: %v, expected: syscall.EINVAL", s, err) - } - if err := s.file.Link(s.file, ".."); err != syscall.EINVAL { - t.Errorf("%v: Link() should have failed, got: %v, expected: syscall.EINVAL", s, err) - } - }) -} - -func TestIsNameValid(t *testing.T) { - valid := []string{ - "name", - "123", - "!@#$%^&*()", - ".name", - "..name", - "...", - } - for _, s := range valid { - if got := isNameValid(s); !got { - t.Errorf("isNameValid(%s) failed, got: %v, expected: true", s, got) - } - } - invalid := []string{ - ".", - "..", - "name/name", - "/name", - "name/", - } - for _, s := range invalid { - if got := isNameValid(s); got { - t.Errorf("isNameValid(%s) failed, got: %v, expected: false", s, got) - } - } -} - func TestWalkNotFound(t *testing.T) { runCustom(t, []fileType{directory}, allConfs, func(t *testing.T, s state) { if _, _, err := s.file.Walk([]string{"nobody-here"}); err != syscall.ENOENT { diff --git a/runsc/sandbox/BUILD b/runsc/sandbox/BUILD index eb9c4cd76..d6043bcf7 100644 --- a/runsc/sandbox/BUILD +++ b/runsc/sandbox/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "sandbox", srcs = [ diff --git a/runsc/specutils/BUILD b/runsc/specutils/BUILD index e73b2293f..a1e5da3f5 100644 --- a/runsc/specutils/BUILD +++ b/runsc/specutils/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "specutils", srcs = [ diff --git a/runsc/test/image/BUILD b/runsc/test/image/BUILD index c41161d50..22b3ebd2a 100644 --- a/runsc/test/image/BUILD +++ b/runsc/test/image/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_test( name = "image_test", size = "large", diff --git a/runsc/test/integration/BUILD b/runsc/test/integration/BUILD index 726ebf49e..e7204dc66 100644 --- a/runsc/test/integration/BUILD +++ b/runsc/test/integration/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_test( name = "integration_test", size = "large", diff --git a/runsc/test/root/BUILD b/runsc/test/root/BUILD index c69249b52..c2567ef23 100644 --- a/runsc/test/root/BUILD +++ b/runsc/test/root/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "root", srcs = ["root.go"], diff --git a/runsc/test/testutil/BUILD b/runsc/test/testutil/BUILD index da2535bfa..128bd80fb 100644 --- a/runsc/test/testutil/BUILD +++ b/runsc/test/testutil/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "testutil", srcs = [ diff --git a/runsc/tools/dockercfg/BUILD b/runsc/tools/dockercfg/BUILD index 5abb0c90a..a80b3abab 100644 --- a/runsc/tools/dockercfg/BUILD +++ b/runsc/tools/dockercfg/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_binary") +package(licenses = ["notice"]) # Apache 2.0 + go_binary( name = "dockercfg", srcs = ["dockercfg.go"], diff --git a/tools/go_generics/BUILD b/tools/go_generics/BUILD index 1afc58625..22c2e62c3 100644 --- a/tools/go_generics/BUILD +++ b/tools/go_generics/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_binary") +package(licenses = ["notice"]) # Apache 2.0 + go_binary( name = "go_generics", srcs = [ diff --git a/tools/go_generics/globals/BUILD b/tools/go_generics/globals/BUILD index a238becab..c26ac56d2 100644 --- a/tools/go_generics/globals/BUILD +++ b/tools/go_generics/globals/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "globals", srcs = [ diff --git a/tools/go_generics/rules_tests/BUILD b/tools/go_generics/rules_tests/BUILD index 2d9a6fa9d..23b2d656d 100644 --- a/tools/go_generics/rules_tests/BUILD +++ b/tools/go_generics/rules_tests/BUILD @@ -1,6 +1,7 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_test") load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") go_template_instance( diff --git a/tools/go_stateify/BUILD b/tools/go_stateify/BUILD index edbeb4e2d..68d37f5d7 100644 --- a/tools/go_stateify/BUILD +++ b/tools/go_stateify/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_binary") +package(licenses = ["notice"]) # Apache 2.0 + go_binary( name = "stateify", srcs = ["main.go"], -- cgit v1.2.3 From ad8f293e1af99f3c04d1020bb51b46c0dba60e45 Mon Sep 17 00:00:00 2001 From: Zhaozhong Ni Date: Tue, 4 Dec 2018 13:13:13 -0800 Subject: sentry: save copy of tcp segment's delivered views to avoid in-struct pointers. PiperOrigin-RevId: 224033238 Change-Id: Ie5b1854b29340843b02c123766d290a8738d7631 --- pkg/state/encode.go | 5 +++-- pkg/tcpip/transport/tcp/segment.go | 2 +- pkg/tcpip/transport/tcp/segment_state.go | 11 ++++++++++- 3 files changed, 14 insertions(+), 4 deletions(-) (limited to 'pkg/state') diff --git a/pkg/state/encode.go b/pkg/state/encode.go index 577aaf051..fe8512bbf 100644 --- a/pkg/state/encode.go +++ b/pkg/state/encode.go @@ -119,9 +119,10 @@ func (es *encodeState) register(obj reflect.Value) uint64 { if size := typ.Size(); size > 0 { r := addrRange{addr, addr + size} if !es.values.IsEmptyRange(r) { - panic(fmt.Errorf("overlapping objects: [new object] %#v [existing object] %#v", obj.Interface(), es.values.FindSegment(addr).Value().Elem().Interface())) + old := es.values.LowerBoundSegment(addr).Value().Interface().(recoverable) + panic(fmt.Errorf("overlapping objects: [new object] %#v [existing object path] %s", obj.Interface(), old.path())) } - es.values.Add(r, obj) + es.values.Add(r, reflect.ValueOf(es.recoverable.copy())) } } else { // Push back the map itself; when maps are encoded from the diff --git a/pkg/tcpip/transport/tcp/segment.go b/pkg/tcpip/transport/tcp/segment.go index fc87a05fd..87c6d7d20 100644 --- a/pkg/tcpip/transport/tcp/segment.go +++ b/pkg/tcpip/transport/tcp/segment.go @@ -46,7 +46,7 @@ type segment struct { data buffer.VectorisedView `state:".(buffer.VectorisedView)"` // views is used as buffer for data when its length is large // enough to store a VectorisedView. - views [8]buffer.View + views [8]buffer.View `state:"nosave"` // viewToDeliver keeps track of the next View that should be // delivered by the Read endpoint. viewToDeliver int diff --git a/pkg/tcpip/transport/tcp/segment_state.go b/pkg/tcpip/transport/tcp/segment_state.go index 46b6d85a6..d4bd6cf95 100644 --- a/pkg/tcpip/transport/tcp/segment_state.go +++ b/pkg/tcpip/transport/tcp/segment_state.go @@ -22,7 +22,16 @@ import ( func (s *segment) saveData() buffer.VectorisedView { // We cannot save s.data directly as s.data.views may alias to s.views, // which is not allowed by state framework (in-struct pointer). - return s.data.Clone(nil) + v := make([]buffer.View, len(s.data.Views())) + // For views already delivered, we cannot save them directly as they may + // have already been sliced and saved elsewhere (e.g., readViews). + for i := 0; i < s.viewToDeliver; i++ { + v[i] = append([]byte(nil), s.data.Views()[i]...) + } + for i := s.viewToDeliver; i < len(v); i++ { + v[i] = s.data.Views()[i] + } + return buffer.NewVectorisedView(s.data.Size(), v) } // loadData is invoked by stateify. -- cgit v1.2.3 From 71f0d5108bfcfd6be2a61dc32579973f141f97f3 Mon Sep 17 00:00:00 2001 From: Michael Pratt Date: Fri, 21 Dec 2018 14:28:20 -0800 Subject: Internal Change PiperOrigin-RevId: 226542979 Change-Id: Ife11ebd0a85b8a63078e6daa71b4a99a82080ac9 --- pkg/compressio/compressio.go | 4 ++-- pkg/state/statefile/statefile.go | 4 ++-- pkg/tcpip/transport/tcp/accept.go | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) (limited to 'pkg/state') diff --git a/pkg/compressio/compressio.go b/pkg/compressio/compressio.go index 205536812..4daaa82b6 100644 --- a/pkg/compressio/compressio.go +++ b/pkg/compressio/compressio.go @@ -46,14 +46,14 @@ package compressio import ( "bytes" "compress/flate" + "crypto/hmac" + "crypto/sha256" "errors" "hash" "io" "runtime" "sync" - "crypto/hmac" - "crypto/sha256" "gvisor.googlesource.com/gvisor/pkg/binary" ) diff --git a/pkg/state/statefile/statefile.go b/pkg/state/statefile/statefile.go index 99158fd02..c21e3bb0e 100644 --- a/pkg/state/statefile/statefile.go +++ b/pkg/state/statefile/statefile.go @@ -46,6 +46,8 @@ package statefile import ( "bytes" "compress/flate" + "crypto/hmac" + "crypto/sha256" "encoding/json" "fmt" "hash" @@ -53,8 +55,6 @@ import ( "strings" "time" - "crypto/hmac" - "crypto/sha256" "gvisor.googlesource.com/gvisor/pkg/binary" "gvisor.googlesource.com/gvisor/pkg/compressio" ) diff --git a/pkg/tcpip/transport/tcp/accept.go b/pkg/tcpip/transport/tcp/accept.go index 32f65367c..d0e1d6782 100644 --- a/pkg/tcpip/transport/tcp/accept.go +++ b/pkg/tcpip/transport/tcp/accept.go @@ -15,13 +15,13 @@ package tcp import ( + "crypto/sha1" "encoding/binary" "hash" "io" "sync" "time" - "crypto/sha1" "gvisor.googlesource.com/gvisor/pkg/rand" "gvisor.googlesource.com/gvisor/pkg/sleep" "gvisor.googlesource.com/gvisor/pkg/tcpip" -- cgit v1.2.3 From 2a0c69b19f4b55c3f9777f0098a72af123ccff3c Mon Sep 17 00:00:00 2001 From: Michael Pratt Date: Thu, 31 Jan 2019 11:11:44 -0800 Subject: Remove license comments Nothing reads them and they can simply get stale. Generated with: $ sed -i "s/licenses(\(.*\)).*/licenses(\1)/" **/BUILD PiperOrigin-RevId: 231818945 Change-Id: Ibc3f9838546b7e94f13f217060d31f4ada9d4bf0 --- pkg/abi/BUILD | 2 +- pkg/abi/linux/BUILD | 2 +- pkg/amutex/BUILD | 2 +- pkg/atomicbitops/BUILD | 2 +- pkg/binary/BUILD | 2 +- pkg/bits/BUILD | 2 +- pkg/bpf/BUILD | 2 +- pkg/compressio/BUILD | 2 +- pkg/control/client/BUILD | 2 +- pkg/control/server/BUILD | 2 +- pkg/cpuid/BUILD | 2 +- pkg/dhcp/BUILD | 2 +- pkg/eventchannel/BUILD | 2 +- pkg/fd/BUILD | 2 +- pkg/fdnotifier/BUILD | 2 +- pkg/gate/BUILD | 2 +- pkg/ilist/BUILD | 2 +- pkg/linewriter/BUILD | 2 +- pkg/log/BUILD | 2 +- pkg/metric/BUILD | 2 +- pkg/p9/BUILD | 2 +- pkg/p9/local_server/BUILD | 2 +- pkg/p9/p9test/BUILD | 2 +- pkg/rand/BUILD | 2 +- pkg/refs/BUILD | 2 +- pkg/seccomp/BUILD | 2 +- pkg/secio/BUILD | 2 +- pkg/segment/BUILD | 2 +- pkg/segment/test/BUILD | 2 +- pkg/sentry/BUILD | 2 +- pkg/sentry/arch/BUILD | 2 +- pkg/sentry/context/BUILD | 2 +- pkg/sentry/context/contexttest/BUILD | 2 +- pkg/sentry/control/BUILD | 2 +- pkg/sentry/device/BUILD | 2 +- pkg/sentry/fs/BUILD | 2 +- pkg/sentry/fs/anon/BUILD | 2 +- pkg/sentry/fs/ashmem/BUILD | 2 +- pkg/sentry/fs/binder/BUILD | 2 +- pkg/sentry/fs/dev/BUILD | 2 +- pkg/sentry/fs/fdpipe/BUILD | 2 +- pkg/sentry/fs/filetest/BUILD | 2 +- pkg/sentry/fs/fsutil/BUILD | 2 +- pkg/sentry/fs/gofer/BUILD | 2 +- pkg/sentry/fs/host/BUILD | 2 +- pkg/sentry/fs/lock/BUILD | 2 +- pkg/sentry/fs/proc/BUILD | 2 +- pkg/sentry/fs/proc/device/BUILD | 2 +- pkg/sentry/fs/proc/seqfile/BUILD | 2 +- pkg/sentry/fs/ramfs/BUILD | 2 +- pkg/sentry/fs/sys/BUILD | 2 +- pkg/sentry/fs/timerfd/BUILD | 2 +- pkg/sentry/fs/tmpfs/BUILD | 2 +- pkg/sentry/fs/tty/BUILD | 2 +- pkg/sentry/hostcpu/BUILD | 2 +- pkg/sentry/inet/BUILD | 2 +- pkg/sentry/kernel/BUILD | 2 +- pkg/sentry/kernel/auth/BUILD | 2 +- pkg/sentry/kernel/contexttest/BUILD | 2 +- pkg/sentry/kernel/epoll/BUILD | 2 +- pkg/sentry/kernel/eventfd/BUILD | 2 +- pkg/sentry/kernel/fasync/BUILD | 2 +- pkg/sentry/kernel/futex/BUILD | 2 +- pkg/sentry/kernel/kdefs/BUILD | 2 +- pkg/sentry/kernel/memevent/BUILD | 2 +- pkg/sentry/kernel/pipe/BUILD | 2 +- pkg/sentry/kernel/sched/BUILD | 2 +- pkg/sentry/kernel/semaphore/BUILD | 2 +- pkg/sentry/kernel/shm/BUILD | 2 +- pkg/sentry/kernel/time/BUILD | 2 +- pkg/sentry/limits/BUILD | 2 +- pkg/sentry/loader/BUILD | 2 +- pkg/sentry/memmap/BUILD | 2 +- pkg/sentry/memutil/BUILD | 2 +- pkg/sentry/mm/BUILD | 2 +- pkg/sentry/platform/BUILD | 2 +- pkg/sentry/platform/filemem/BUILD | 2 +- pkg/sentry/platform/interrupt/BUILD | 2 +- pkg/sentry/platform/kvm/BUILD | 2 +- pkg/sentry/platform/kvm/testutil/BUILD | 2 +- pkg/sentry/platform/procid/BUILD | 2 +- pkg/sentry/platform/ptrace/BUILD | 2 +- pkg/sentry/platform/ring0/BUILD | 2 +- pkg/sentry/platform/ring0/gen_offsets/BUILD | 2 +- pkg/sentry/platform/ring0/pagetables/BUILD | 2 +- pkg/sentry/platform/safecopy/BUILD | 2 +- pkg/sentry/safemem/BUILD | 2 +- pkg/sentry/sighandling/BUILD | 2 +- pkg/sentry/socket/BUILD | 2 +- pkg/sentry/socket/control/BUILD | 2 +- pkg/sentry/socket/epsocket/BUILD | 2 +- pkg/sentry/socket/hostinet/BUILD | 2 +- pkg/sentry/socket/netlink/BUILD | 2 +- pkg/sentry/socket/netlink/port/BUILD | 2 +- pkg/sentry/socket/netlink/route/BUILD | 2 +- pkg/sentry/socket/rpcinet/BUILD | 2 +- pkg/sentry/socket/rpcinet/conn/BUILD | 2 +- pkg/sentry/socket/rpcinet/notifier/BUILD | 2 +- pkg/sentry/socket/unix/BUILD | 2 +- pkg/sentry/socket/unix/transport/BUILD | 2 +- pkg/sentry/state/BUILD | 2 +- pkg/sentry/strace/BUILD | 2 +- pkg/sentry/syscalls/BUILD | 2 +- pkg/sentry/syscalls/linux/BUILD | 2 +- pkg/sentry/time/BUILD | 2 +- pkg/sentry/unimpl/BUILD | 2 +- pkg/sentry/uniqueid/BUILD | 2 +- pkg/sentry/usage/BUILD | 2 +- pkg/sentry/usermem/BUILD | 2 +- pkg/sentry/watchdog/BUILD | 2 +- pkg/sleep/BUILD | 2 +- pkg/state/BUILD | 2 +- pkg/state/statefile/BUILD | 2 +- pkg/sync/BUILD | 2 +- pkg/sync/atomicptrtest/BUILD | 2 +- pkg/sync/seqatomictest/BUILD | 2 +- pkg/syserr/BUILD | 2 +- pkg/syserror/BUILD | 2 +- pkg/tcpip/BUILD | 2 +- pkg/tcpip/adapters/gonet/BUILD | 2 +- pkg/tcpip/buffer/BUILD | 2 +- pkg/tcpip/checker/BUILD | 2 +- pkg/tcpip/hash/jenkins/BUILD | 2 +- pkg/tcpip/header/BUILD | 2 +- pkg/tcpip/link/channel/BUILD | 2 +- pkg/tcpip/link/fdbased/BUILD | 2 +- pkg/tcpip/link/loopback/BUILD | 2 +- pkg/tcpip/link/rawfile/BUILD | 2 +- pkg/tcpip/link/sharedmem/BUILD | 2 +- pkg/tcpip/link/sharedmem/pipe/BUILD | 2 +- pkg/tcpip/link/sharedmem/queue/BUILD | 2 +- pkg/tcpip/link/sniffer/BUILD | 2 +- pkg/tcpip/link/tun/BUILD | 2 +- pkg/tcpip/link/waitable/BUILD | 2 +- pkg/tcpip/network/BUILD | 2 +- pkg/tcpip/network/arp/BUILD | 2 +- pkg/tcpip/network/fragmentation/BUILD | 2 +- pkg/tcpip/network/hash/BUILD | 2 +- pkg/tcpip/network/ipv4/BUILD | 2 +- pkg/tcpip/network/ipv6/BUILD | 2 +- pkg/tcpip/ports/BUILD | 2 +- pkg/tcpip/sample/tun_tcp_connect/BUILD | 2 +- pkg/tcpip/sample/tun_tcp_echo/BUILD | 2 +- pkg/tcpip/seqnum/BUILD | 2 +- pkg/tcpip/stack/BUILD | 2 +- pkg/tcpip/transport/ping/BUILD | 2 +- pkg/tcpip/transport/tcp/BUILD | 2 +- pkg/tcpip/transport/tcp/testing/context/BUILD | 2 +- pkg/tcpip/transport/tcpconntrack/BUILD | 2 +- pkg/tcpip/transport/udp/BUILD | 2 +- pkg/tmutex/BUILD | 2 +- pkg/unet/BUILD | 2 +- pkg/urpc/BUILD | 2 +- pkg/waiter/BUILD | 2 +- runsc/boot/BUILD | 2 +- runsc/boot/filter/BUILD | 2 +- runsc/cgroup/BUILD | 2 +- runsc/cmd/BUILD | 2 +- runsc/console/BUILD | 2 +- runsc/container/BUILD | 2 +- runsc/fsgofer/BUILD | 2 +- runsc/fsgofer/filter/BUILD | 2 +- runsc/sandbox/BUILD | 2 +- runsc/specutils/BUILD | 2 +- runsc/test/image/BUILD | 2 +- runsc/test/integration/BUILD | 2 +- runsc/test/root/BUILD | 2 +- runsc/test/root/testdata/BUILD | 2 +- runsc/test/testutil/BUILD | 2 +- runsc/tools/dockercfg/BUILD | 2 +- test/syscalls/BUILD | 2 +- test/syscalls/gtest/BUILD | 2 +- test/syscalls/linux/BUILD | 2 +- test/util/BUILD | 2 +- tools/go_generics/BUILD | 2 +- tools/go_generics/globals/BUILD | 2 +- tools/go_generics/go_merge/BUILD | 2 +- tools/go_generics/rules_tests/BUILD | 2 +- tools/go_stateify/BUILD | 2 +- vdso/BUILD | 2 +- 180 files changed, 180 insertions(+), 180 deletions(-) (limited to 'pkg/state') diff --git a/pkg/abi/BUILD b/pkg/abi/BUILD index 1ba4f3a46..323263ebf 100644 --- a/pkg/abi/BUILD +++ b/pkg/abi/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/abi/linux/BUILD b/pkg/abi/linux/BUILD index e6043abf4..7648c9469 100644 --- a/pkg/abi/linux/BUILD +++ b/pkg/abi/linux/BUILD @@ -2,7 +2,7 @@ # Linux kernel. It should be used instead of syscall or golang.org/x/sys/unix # when the host OS may not be Linux. -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/amutex/BUILD b/pkg/amutex/BUILD index 7cda07418..bdb6e8f2c 100644 --- a/pkg/amutex/BUILD +++ b/pkg/amutex/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "amutex", diff --git a/pkg/atomicbitops/BUILD b/pkg/atomicbitops/BUILD index 235188531..9555bf645 100644 --- a/pkg/atomicbitops/BUILD +++ b/pkg/atomicbitops/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "atomicbitops", diff --git a/pkg/binary/BUILD b/pkg/binary/BUILD index 571151f72..bd37376b0 100644 --- a/pkg/binary/BUILD +++ b/pkg/binary/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "binary", diff --git a/pkg/bits/BUILD b/pkg/bits/BUILD index 46794bdb8..5214b2c24 100644 --- a/pkg/bits/BUILD +++ b/pkg/bits/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") diff --git a/pkg/bpf/BUILD b/pkg/bpf/BUILD index 564df3af5..3c7ae3103 100644 --- a/pkg/bpf/BUILD +++ b/pkg/bpf/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/compressio/BUILD b/pkg/compressio/BUILD index 72952d735..3a0ac64e6 100644 --- a/pkg/compressio/BUILD +++ b/pkg/compressio/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "compressio", diff --git a/pkg/control/client/BUILD b/pkg/control/client/BUILD index 32853875d..22a4a4a5a 100644 --- a/pkg/control/client/BUILD +++ b/pkg/control/client/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "client", diff --git a/pkg/control/server/BUILD b/pkg/control/server/BUILD index ba2b1be9f..76b2e9787 100644 --- a/pkg/control/server/BUILD +++ b/pkg/control/server/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "server", diff --git a/pkg/cpuid/BUILD b/pkg/cpuid/BUILD index 46fc4703b..29cc38778 100644 --- a/pkg/cpuid/BUILD +++ b/pkg/cpuid/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/dhcp/BUILD b/pkg/dhcp/BUILD index c97dfc14b..003620b48 100644 --- a/pkg/dhcp/BUILD +++ b/pkg/dhcp/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "dhcp", diff --git a/pkg/eventchannel/BUILD b/pkg/eventchannel/BUILD index 18348ef54..5c2a44aa1 100644 --- a/pkg/eventchannel/BUILD +++ b/pkg/eventchannel/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "eventchannel", diff --git a/pkg/fd/BUILD b/pkg/fd/BUILD index 06cfd445e..ab1109157 100644 --- a/pkg/fd/BUILD +++ b/pkg/fd/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "fd", diff --git a/pkg/fdnotifier/BUILD b/pkg/fdnotifier/BUILD index 27d378d5b..8c8d193cc 100644 --- a/pkg/fdnotifier/BUILD +++ b/pkg/fdnotifier/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "fdnotifier", diff --git a/pkg/gate/BUILD b/pkg/gate/BUILD index 9a87a3a31..83679f2da 100644 --- a/pkg/gate/BUILD +++ b/pkg/gate/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "gate", diff --git a/pkg/ilist/BUILD b/pkg/ilist/BUILD index a67aa2cff..dbd65ab12 100644 --- a/pkg/ilist/BUILD +++ b/pkg/ilist/BUILD @@ -1,7 +1,7 @@ load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "ilist", diff --git a/pkg/linewriter/BUILD b/pkg/linewriter/BUILD index 3f28ba867..d1aa2e7d6 100644 --- a/pkg/linewriter/BUILD +++ b/pkg/linewriter/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "linewriter", diff --git a/pkg/log/BUILD b/pkg/log/BUILD index 94ac66db3..b2d18eddb 100644 --- a/pkg/log/BUILD +++ b/pkg/log/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "log", diff --git a/pkg/metric/BUILD b/pkg/metric/BUILD index d96e5563b..4b2c7a00e 100644 --- a/pkg/metric/BUILD +++ b/pkg/metric/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "metric", diff --git a/pkg/p9/BUILD b/pkg/p9/BUILD index 2c224e65b..5d972309d 100644 --- a/pkg/p9/BUILD +++ b/pkg/p9/BUILD @@ -2,7 +2,7 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") package( default_visibility = ["//visibility:public"], - licenses = ["notice"], # Apache 2.0 + licenses = ["notice"], ) go_library( diff --git a/pkg/p9/local_server/BUILD b/pkg/p9/local_server/BUILD index b17ebb79d..aa6db186c 100644 --- a/pkg/p9/local_server/BUILD +++ b/pkg/p9/local_server/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_binary( name = "local_server", diff --git a/pkg/p9/p9test/BUILD b/pkg/p9/p9test/BUILD index 7c4b875ce..cf22edde8 100644 --- a/pkg/p9/p9test/BUILD +++ b/pkg/p9/p9test/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) alias( name = "mockgen", diff --git a/pkg/rand/BUILD b/pkg/rand/BUILD index 0c9efc709..4eec3a4dd 100644 --- a/pkg/rand/BUILD +++ b/pkg/rand/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "rand", diff --git a/pkg/refs/BUILD b/pkg/refs/BUILD index 98150ba8f..fc562f821 100644 --- a/pkg/refs/BUILD +++ b/pkg/refs/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/seccomp/BUILD b/pkg/seccomp/BUILD index 657f923ed..0e9c4692d 100644 --- a/pkg/seccomp/BUILD +++ b/pkg/seccomp/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_embed_data") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_binary( name = "victim", diff --git a/pkg/secio/BUILD b/pkg/secio/BUILD index 29f751725..2b4b87c61 100644 --- a/pkg/secio/BUILD +++ b/pkg/secio/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "secio", diff --git a/pkg/segment/BUILD b/pkg/segment/BUILD index 964d73af8..700385907 100644 --- a/pkg/segment/BUILD +++ b/pkg/segment/BUILD @@ -1,6 +1,6 @@ package( default_visibility = ["//:sandbox"], - licenses = ["notice"], # Apache 2.0 + licenses = ["notice"], ) load("//tools/go_generics:defs.bzl", "go_template") diff --git a/pkg/segment/test/BUILD b/pkg/segment/test/BUILD index bdf53e24e..81e929b8c 100644 --- a/pkg/segment/test/BUILD +++ b/pkg/segment/test/BUILD @@ -2,7 +2,7 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") package( default_visibility = ["//visibility:private"], - licenses = ["notice"], # Apache 2.0 + licenses = ["notice"], ) load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/sentry/BUILD b/pkg/sentry/BUILD index d18cf3555..53989301f 100644 --- a/pkg/sentry/BUILD +++ b/pkg/sentry/BUILD @@ -1,7 +1,7 @@ # This BUILD file defines a package_group that allows for interdependencies for # sentry-internal packages. -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) package_group( name = "internal", diff --git a/pkg/sentry/arch/BUILD b/pkg/sentry/arch/BUILD index 9bf04360a..0c044bc33 100644 --- a/pkg/sentry/arch/BUILD +++ b/pkg/sentry/arch/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/context/BUILD b/pkg/sentry/context/BUILD index 02d24defd..a3c8d0177 100644 --- a/pkg/sentry/context/BUILD +++ b/pkg/sentry/context/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "context", diff --git a/pkg/sentry/context/contexttest/BUILD b/pkg/sentry/context/contexttest/BUILD index 01bb40b04..bed156b70 100644 --- a/pkg/sentry/context/contexttest/BUILD +++ b/pkg/sentry/context/contexttest/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/control/BUILD b/pkg/sentry/control/BUILD index c3b682d6f..f54e01ee8 100644 --- a/pkg/sentry/control/BUILD +++ b/pkg/sentry/control/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "control", diff --git a/pkg/sentry/device/BUILD b/pkg/sentry/device/BUILD index bebdb2939..01de708d3 100644 --- a/pkg/sentry/device/BUILD +++ b/pkg/sentry/device/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "device", diff --git a/pkg/sentry/fs/BUILD b/pkg/sentry/fs/BUILD index 6f368b0da..e58333da3 100644 --- a/pkg/sentry/fs/BUILD +++ b/pkg/sentry/fs/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/anon/BUILD b/pkg/sentry/fs/anon/BUILD index 4bd912e95..2111df2e8 100644 --- a/pkg/sentry/fs/anon/BUILD +++ b/pkg/sentry/fs/anon/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "anon", diff --git a/pkg/sentry/fs/ashmem/BUILD b/pkg/sentry/fs/ashmem/BUILD index e5bb661b5..dcf620dca 100644 --- a/pkg/sentry/fs/ashmem/BUILD +++ b/pkg/sentry/fs/ashmem/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/sentry/fs/binder/BUILD b/pkg/sentry/fs/binder/BUILD index 27155819e..8a448175f 100644 --- a/pkg/sentry/fs/binder/BUILD +++ b/pkg/sentry/fs/binder/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/fs/dev/BUILD b/pkg/sentry/fs/dev/BUILD index 85371032a..e5b962c8c 100644 --- a/pkg/sentry/fs/dev/BUILD +++ b/pkg/sentry/fs/dev/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/fs/fdpipe/BUILD b/pkg/sentry/fs/fdpipe/BUILD index 8a0937cda..098463e97 100644 --- a/pkg/sentry/fs/fdpipe/BUILD +++ b/pkg/sentry/fs/fdpipe/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/filetest/BUILD b/pkg/sentry/fs/filetest/BUILD index d137fee4c..05ca72aa0 100644 --- a/pkg/sentry/fs/filetest/BUILD +++ b/pkg/sentry/fs/filetest/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/fs/fsutil/BUILD b/pkg/sentry/fs/fsutil/BUILD index d4767642b..7dff970ea 100644 --- a/pkg/sentry/fs/fsutil/BUILD +++ b/pkg/sentry/fs/fsutil/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/gofer/BUILD b/pkg/sentry/fs/gofer/BUILD index 35ffadd13..f2c79b475 100644 --- a/pkg/sentry/fs/gofer/BUILD +++ b/pkg/sentry/fs/gofer/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/host/BUILD b/pkg/sentry/fs/host/BUILD index 6877eb161..ea2ca11bf 100644 --- a/pkg/sentry/fs/host/BUILD +++ b/pkg/sentry/fs/host/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/lock/BUILD b/pkg/sentry/fs/lock/BUILD index 3159ff1da..7164744b8 100644 --- a/pkg/sentry/fs/lock/BUILD +++ b/pkg/sentry/fs/lock/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/proc/BUILD b/pkg/sentry/fs/proc/BUILD index 74954f213..f6bc90634 100644 --- a/pkg/sentry/fs/proc/BUILD +++ b/pkg/sentry/fs/proc/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/proc/device/BUILD b/pkg/sentry/fs/proc/device/BUILD index ff7dacf07..64b0c5a3a 100644 --- a/pkg/sentry/fs/proc/device/BUILD +++ b/pkg/sentry/fs/proc/device/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "device", diff --git a/pkg/sentry/fs/proc/seqfile/BUILD b/pkg/sentry/fs/proc/seqfile/BUILD index b4ba64e10..6b44c0075 100644 --- a/pkg/sentry/fs/proc/seqfile/BUILD +++ b/pkg/sentry/fs/proc/seqfile/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/ramfs/BUILD b/pkg/sentry/fs/ramfs/BUILD index 4a629e38e..f36e4a5e8 100644 --- a/pkg/sentry/fs/ramfs/BUILD +++ b/pkg/sentry/fs/ramfs/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/sys/BUILD b/pkg/sentry/fs/sys/BUILD index 7de928e16..42e98230e 100644 --- a/pkg/sentry/fs/sys/BUILD +++ b/pkg/sentry/fs/sys/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/fs/timerfd/BUILD b/pkg/sentry/fs/timerfd/BUILD index ffdd7e0dc..0e06a5028 100644 --- a/pkg/sentry/fs/timerfd/BUILD +++ b/pkg/sentry/fs/timerfd/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/fs/tmpfs/BUILD b/pkg/sentry/fs/tmpfs/BUILD index c5ec85460..bf5b68869 100644 --- a/pkg/sentry/fs/tmpfs/BUILD +++ b/pkg/sentry/fs/tmpfs/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/tty/BUILD b/pkg/sentry/fs/tty/BUILD index 011cb6955..bee2db3f3 100644 --- a/pkg/sentry/fs/tty/BUILD +++ b/pkg/sentry/fs/tty/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/hostcpu/BUILD b/pkg/sentry/hostcpu/BUILD index 33197cf14..b5067ae6d 100644 --- a/pkg/sentry/hostcpu/BUILD +++ b/pkg/sentry/hostcpu/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "hostcpu", diff --git a/pkg/sentry/inet/BUILD b/pkg/sentry/inet/BUILD index 159c50efb..e288d34e9 100644 --- a/pkg/sentry/inet/BUILD +++ b/pkg/sentry/inet/BUILD @@ -1,6 +1,6 @@ package( default_visibility = ["//:sandbox"], - licenses = ["notice"], # Apache 2.0 + licenses = ["notice"], ) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD index 7d41626dc..b230aff98 100644 --- a/pkg/sentry/kernel/BUILD +++ b/pkg/sentry/kernel/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/kernel/auth/BUILD b/pkg/sentry/kernel/auth/BUILD index a81085372..abd4f2dae 100644 --- a/pkg/sentry/kernel/auth/BUILD +++ b/pkg/sentry/kernel/auth/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/kernel/contexttest/BUILD b/pkg/sentry/kernel/contexttest/BUILD index 391986291..5769a3b28 100644 --- a/pkg/sentry/kernel/contexttest/BUILD +++ b/pkg/sentry/kernel/contexttest/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/kernel/epoll/BUILD b/pkg/sentry/kernel/epoll/BUILD index 5e8b36ed6..1567d5050 100644 --- a/pkg/sentry/kernel/epoll/BUILD +++ b/pkg/sentry/kernel/epoll/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/kernel/eventfd/BUILD b/pkg/sentry/kernel/eventfd/BUILD index d96803fc9..f2f1a1223 100644 --- a/pkg/sentry/kernel/eventfd/BUILD +++ b/pkg/sentry/kernel/eventfd/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/kernel/fasync/BUILD b/pkg/sentry/kernel/fasync/BUILD index 17749c0de..5faf95909 100644 --- a/pkg/sentry/kernel/fasync/BUILD +++ b/pkg/sentry/kernel/fasync/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/kernel/futex/BUILD b/pkg/sentry/kernel/futex/BUILD index afd35985f..da24c36c1 100644 --- a/pkg/sentry/kernel/futex/BUILD +++ b/pkg/sentry/kernel/futex/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/kernel/kdefs/BUILD b/pkg/sentry/kernel/kdefs/BUILD index 3f8fa206c..38aaca134 100644 --- a/pkg/sentry/kernel/kdefs/BUILD +++ b/pkg/sentry/kernel/kdefs/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "kdefs", diff --git a/pkg/sentry/kernel/memevent/BUILD b/pkg/sentry/kernel/memevent/BUILD index dfd8dd062..347a69062 100644 --- a/pkg/sentry/kernel/memevent/BUILD +++ b/pkg/sentry/kernel/memevent/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "memevent", diff --git a/pkg/sentry/kernel/pipe/BUILD b/pkg/sentry/kernel/pipe/BUILD index 19b23c6d2..011a3f349 100644 --- a/pkg/sentry/kernel/pipe/BUILD +++ b/pkg/sentry/kernel/pipe/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/kernel/sched/BUILD b/pkg/sentry/kernel/sched/BUILD index 52e226a39..184e8a35b 100644 --- a/pkg/sentry/kernel/sched/BUILD +++ b/pkg/sentry/kernel/sched/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "sched", diff --git a/pkg/sentry/kernel/semaphore/BUILD b/pkg/sentry/kernel/semaphore/BUILD index bdcf4ce5c..840943ca8 100644 --- a/pkg/sentry/kernel/semaphore/BUILD +++ b/pkg/sentry/kernel/semaphore/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/kernel/shm/BUILD b/pkg/sentry/kernel/shm/BUILD index 40e641355..f45770eef 100644 --- a/pkg/sentry/kernel/shm/BUILD +++ b/pkg/sentry/kernel/shm/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/kernel/time/BUILD b/pkg/sentry/kernel/time/BUILD index 5d8db2273..584f7c7cc 100644 --- a/pkg/sentry/kernel/time/BUILD +++ b/pkg/sentry/kernel/time/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/limits/BUILD b/pkg/sentry/limits/BUILD index 90f4395d4..800166675 100644 --- a/pkg/sentry/limits/BUILD +++ b/pkg/sentry/limits/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/loader/BUILD b/pkg/sentry/loader/BUILD index 24e734b49..1ea260a4e 100644 --- a/pkg/sentry/loader/BUILD +++ b/pkg/sentry/loader/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_embed_data") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/memmap/BUILD b/pkg/sentry/memmap/BUILD index c9e0b95a0..9c2cbd18b 100644 --- a/pkg/sentry/memmap/BUILD +++ b/pkg/sentry/memmap/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/memutil/BUILD b/pkg/sentry/memutil/BUILD index 88738d65d..68b03d4cc 100644 --- a/pkg/sentry/memutil/BUILD +++ b/pkg/sentry/memutil/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "memutil", diff --git a/pkg/sentry/mm/BUILD b/pkg/sentry/mm/BUILD index 0997ec0a7..f679262d0 100644 --- a/pkg/sentry/mm/BUILD +++ b/pkg/sentry/mm/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/platform/BUILD b/pkg/sentry/platform/BUILD index af9ba5394..ac8a6cb7f 100644 --- a/pkg/sentry/platform/BUILD +++ b/pkg/sentry/platform/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/platform/filemem/BUILD b/pkg/sentry/platform/filemem/BUILD index 2a5982763..1a61cfaa5 100644 --- a/pkg/sentry/platform/filemem/BUILD +++ b/pkg/sentry/platform/filemem/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/platform/interrupt/BUILD b/pkg/sentry/platform/interrupt/BUILD index dbafa3204..eeccd4d0e 100644 --- a/pkg/sentry/platform/interrupt/BUILD +++ b/pkg/sentry/platform/interrupt/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "interrupt", diff --git a/pkg/sentry/platform/kvm/BUILD b/pkg/sentry/platform/kvm/BUILD index 1b71e629f..6e40b3177 100644 --- a/pkg/sentry/platform/kvm/BUILD +++ b/pkg/sentry/platform/kvm/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/sentry/platform/kvm/testutil/BUILD b/pkg/sentry/platform/kvm/testutil/BUILD index 1dffe94a4..e10087e8e 100644 --- a/pkg/sentry/platform/kvm/testutil/BUILD +++ b/pkg/sentry/platform/kvm/testutil/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "testutil", diff --git a/pkg/sentry/platform/procid/BUILD b/pkg/sentry/platform/procid/BUILD index 20c8bc02c..277509624 100644 --- a/pkg/sentry/platform/procid/BUILD +++ b/pkg/sentry/platform/procid/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "procid", diff --git a/pkg/sentry/platform/ptrace/BUILD b/pkg/sentry/platform/ptrace/BUILD index 2eb354ad4..f86790942 100644 --- a/pkg/sentry/platform/ptrace/BUILD +++ b/pkg/sentry/platform/ptrace/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "ptrace", diff --git a/pkg/sentry/platform/ring0/BUILD b/pkg/sentry/platform/ring0/BUILD index c35d49f2d..ecb3e9a9c 100644 --- a/pkg/sentry/platform/ring0/BUILD +++ b/pkg/sentry/platform/ring0/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") diff --git a/pkg/sentry/platform/ring0/gen_offsets/BUILD b/pkg/sentry/platform/ring0/gen_offsets/BUILD index b76d7974e..d7029d5a9 100644 --- a/pkg/sentry/platform/ring0/gen_offsets/BUILD +++ b/pkg/sentry/platform/ring0/gen_offsets/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/sentry/platform/ring0/pagetables/BUILD b/pkg/sentry/platform/ring0/pagetables/BUILD index de1b920af..fe93d3030 100644 --- a/pkg/sentry/platform/ring0/pagetables/BUILD +++ b/pkg/sentry/platform/ring0/pagetables/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") diff --git a/pkg/sentry/platform/safecopy/BUILD b/pkg/sentry/platform/safecopy/BUILD index cb8347dd8..05a6a61ae 100644 --- a/pkg/sentry/platform/safecopy/BUILD +++ b/pkg/sentry/platform/safecopy/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0, portions BSD, MIT +package(licenses = ["notice"]) go_library( name = "safecopy", diff --git a/pkg/sentry/safemem/BUILD b/pkg/sentry/safemem/BUILD index 87a9bff12..3ab453718 100644 --- a/pkg/sentry/safemem/BUILD +++ b/pkg/sentry/safemem/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "safemem", diff --git a/pkg/sentry/sighandling/BUILD b/pkg/sentry/sighandling/BUILD index 41313d334..cec3af92e 100644 --- a/pkg/sentry/sighandling/BUILD +++ b/pkg/sentry/sighandling/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "sighandling", diff --git a/pkg/sentry/socket/BUILD b/pkg/sentry/socket/BUILD index 3a8044b5f..076f953e7 100644 --- a/pkg/sentry/socket/BUILD +++ b/pkg/sentry/socket/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/socket/control/BUILD b/pkg/sentry/socket/control/BUILD index d3a63f15f..9f4763906 100644 --- a/pkg/sentry/socket/control/BUILD +++ b/pkg/sentry/socket/control/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/socket/epsocket/BUILD b/pkg/sentry/socket/epsocket/BUILD index da4aaf510..45e418db3 100644 --- a/pkg/sentry/socket/epsocket/BUILD +++ b/pkg/sentry/socket/epsocket/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/socket/hostinet/BUILD b/pkg/sentry/socket/hostinet/BUILD index b8dceb102..a469af7ac 100644 --- a/pkg/sentry/socket/hostinet/BUILD +++ b/pkg/sentry/socket/hostinet/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/socket/netlink/BUILD b/pkg/sentry/socket/netlink/BUILD index cff922cb8..148306329 100644 --- a/pkg/sentry/socket/netlink/BUILD +++ b/pkg/sentry/socket/netlink/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/socket/netlink/port/BUILD b/pkg/sentry/socket/netlink/port/BUILD index 3a7dbc5ed..a7370a4ec 100644 --- a/pkg/sentry/socket/netlink/port/BUILD +++ b/pkg/sentry/socket/netlink/port/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/socket/netlink/route/BUILD b/pkg/sentry/socket/netlink/route/BUILD index e1bcfe252..be0419679 100644 --- a/pkg/sentry/socket/netlink/route/BUILD +++ b/pkg/sentry/socket/netlink/route/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/socket/rpcinet/BUILD b/pkg/sentry/socket/rpcinet/BUILD index 06e121946..4da14a1e0 100644 --- a/pkg/sentry/socket/rpcinet/BUILD +++ b/pkg/sentry/socket/rpcinet/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "rpcinet", diff --git a/pkg/sentry/socket/rpcinet/conn/BUILD b/pkg/sentry/socket/rpcinet/conn/BUILD index a16977f29..4336ae9b4 100644 --- a/pkg/sentry/socket/rpcinet/conn/BUILD +++ b/pkg/sentry/socket/rpcinet/conn/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # BSD +package(licenses = ["notice"]) go_library( name = "conn", diff --git a/pkg/sentry/socket/rpcinet/notifier/BUILD b/pkg/sentry/socket/rpcinet/notifier/BUILD index 2bab01774..b0b107ddb 100644 --- a/pkg/sentry/socket/rpcinet/notifier/BUILD +++ b/pkg/sentry/socket/rpcinet/notifier/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # BSD +package(licenses = ["notice"]) go_library( name = "notifier", diff --git a/pkg/sentry/socket/unix/BUILD b/pkg/sentry/socket/unix/BUILD index a12fa93db..fe6871cc6 100644 --- a/pkg/sentry/socket/unix/BUILD +++ b/pkg/sentry/socket/unix/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/socket/unix/transport/BUILD b/pkg/sentry/socket/unix/transport/BUILD index 5a90837bc..5a2de0c4c 100644 --- a/pkg/sentry/socket/unix/transport/BUILD +++ b/pkg/sentry/socket/unix/transport/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/sentry/state/BUILD b/pkg/sentry/state/BUILD index f1f6fdb7d..42c459acc 100644 --- a/pkg/sentry/state/BUILD +++ b/pkg/sentry/state/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "state", diff --git a/pkg/sentry/strace/BUILD b/pkg/sentry/strace/BUILD index 8517db1ac..552e79686 100644 --- a/pkg/sentry/strace/BUILD +++ b/pkg/sentry/strace/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "strace", diff --git a/pkg/sentry/syscalls/BUILD b/pkg/sentry/syscalls/BUILD index 35192ff49..6b5469e45 100644 --- a/pkg/sentry/syscalls/BUILD +++ b/pkg/sentry/syscalls/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "syscalls", diff --git a/pkg/sentry/syscalls/linux/BUILD b/pkg/sentry/syscalls/linux/BUILD index 7621bfdbd..846601881 100644 --- a/pkg/sentry/syscalls/linux/BUILD +++ b/pkg/sentry/syscalls/linux/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/time/BUILD b/pkg/sentry/time/BUILD index 1191010e6..c4b6dcc63 100644 --- a/pkg/sentry/time/BUILD +++ b/pkg/sentry/time/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0, portions BSD +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/sentry/unimpl/BUILD b/pkg/sentry/unimpl/BUILD index 42e24ace5..b608867a9 100644 --- a/pkg/sentry/unimpl/BUILD +++ b/pkg/sentry/unimpl/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) proto_library( name = "unimplemented_syscall_proto", diff --git a/pkg/sentry/uniqueid/BUILD b/pkg/sentry/uniqueid/BUILD index 0929497c3..ccc5a28d3 100644 --- a/pkg/sentry/uniqueid/BUILD +++ b/pkg/sentry/uniqueid/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "uniqueid", diff --git a/pkg/sentry/usage/BUILD b/pkg/sentry/usage/BUILD index 868dfd400..09198496b 100644 --- a/pkg/sentry/usage/BUILD +++ b/pkg/sentry/usage/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/usermem/BUILD b/pkg/sentry/usermem/BUILD index dae41ed0e..1a560b6f3 100644 --- a/pkg/sentry/usermem/BUILD +++ b/pkg/sentry/usermem/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/watchdog/BUILD b/pkg/sentry/watchdog/BUILD index b2c687b20..0bbf3705c 100644 --- a/pkg/sentry/watchdog/BUILD +++ b/pkg/sentry/watchdog/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "watchdog", diff --git a/pkg/sleep/BUILD b/pkg/sleep/BUILD index 338fd9336..2b005bf66 100644 --- a/pkg/sleep/BUILD +++ b/pkg/sleep/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "sleep", diff --git a/pkg/state/BUILD b/pkg/state/BUILD index dd0f250fa..0a975e162 100644 --- a/pkg/state/BUILD +++ b/pkg/state/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/state/statefile/BUILD b/pkg/state/statefile/BUILD index 66c8f3807..5967781e8 100644 --- a/pkg/state/statefile/BUILD +++ b/pkg/state/statefile/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "statefile", diff --git a/pkg/sync/BUILD b/pkg/sync/BUILD index 6ddc6e812..1624e681c 100644 --- a/pkg/sync/BUILD +++ b/pkg/sync/BUILD @@ -2,7 +2,7 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") package( default_visibility = ["//:sandbox"], - licenses = ["notice"], # Apache 2.0, portions BSD + licenses = ["notice"], ) load("//tools/go_generics:defs.bzl", "go_template") diff --git a/pkg/sync/atomicptrtest/BUILD b/pkg/sync/atomicptrtest/BUILD index 9cb7f66fe..198fbb895 100644 --- a/pkg/sync/atomicptrtest/BUILD +++ b/pkg/sync/atomicptrtest/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/sync/seqatomictest/BUILD b/pkg/sync/seqatomictest/BUILD index 54f8e59b1..23132650a 100644 --- a/pkg/sync/seqatomictest/BUILD +++ b/pkg/sync/seqatomictest/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/syserr/BUILD b/pkg/syserr/BUILD index 30ae20772..0d65115ef 100644 --- a/pkg/syserr/BUILD +++ b/pkg/syserr/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "syserr", diff --git a/pkg/syserror/BUILD b/pkg/syserror/BUILD index d4c6da97a..ac478d0ff 100644 --- a/pkg/syserror/BUILD +++ b/pkg/syserror/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "syserror", diff --git a/pkg/tcpip/BUILD b/pkg/tcpip/BUILD index daff9a0a0..83524cc8a 100644 --- a/pkg/tcpip/BUILD +++ b/pkg/tcpip/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/tcpip/adapters/gonet/BUILD b/pkg/tcpip/adapters/gonet/BUILD index 723ad668f..ee2417238 100644 --- a/pkg/tcpip/adapters/gonet/BUILD +++ b/pkg/tcpip/adapters/gonet/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "gonet", diff --git a/pkg/tcpip/buffer/BUILD b/pkg/tcpip/buffer/BUILD index 11a725423..648d12cdf 100644 --- a/pkg/tcpip/buffer/BUILD +++ b/pkg/tcpip/buffer/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/tcpip/checker/BUILD b/pkg/tcpip/checker/BUILD index a1de808b9..f597d0b24 100644 --- a/pkg/tcpip/checker/BUILD +++ b/pkg/tcpip/checker/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "checker", diff --git a/pkg/tcpip/hash/jenkins/BUILD b/pkg/tcpip/hash/jenkins/BUILD index bbb764db8..ce2194a4d 100644 --- a/pkg/tcpip/hash/jenkins/BUILD +++ b/pkg/tcpip/hash/jenkins/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "jenkins", diff --git a/pkg/tcpip/header/BUILD b/pkg/tcpip/header/BUILD index 8e455fe1e..a5c7290ee 100644 --- a/pkg/tcpip/header/BUILD +++ b/pkg/tcpip/header/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/tcpip/link/channel/BUILD b/pkg/tcpip/link/channel/BUILD index 25f6c1457..ae285e495 100644 --- a/pkg/tcpip/link/channel/BUILD +++ b/pkg/tcpip/link/channel/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "channel", diff --git a/pkg/tcpip/link/fdbased/BUILD b/pkg/tcpip/link/fdbased/BUILD index a4aa3feec..0d78c9b15 100644 --- a/pkg/tcpip/link/fdbased/BUILD +++ b/pkg/tcpip/link/fdbased/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "fdbased", diff --git a/pkg/tcpip/link/loopback/BUILD b/pkg/tcpip/link/loopback/BUILD index a46ba7f11..710a05ede 100644 --- a/pkg/tcpip/link/loopback/BUILD +++ b/pkg/tcpip/link/loopback/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "loopback", diff --git a/pkg/tcpip/link/rawfile/BUILD b/pkg/tcpip/link/rawfile/BUILD index 2746d4ced..f01bb2c07 100644 --- a/pkg/tcpip/link/rawfile/BUILD +++ b/pkg/tcpip/link/rawfile/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "rawfile", diff --git a/pkg/tcpip/link/sharedmem/BUILD b/pkg/tcpip/link/sharedmem/BUILD index d7f1e66ef..dc8f1543e 100644 --- a/pkg/tcpip/link/sharedmem/BUILD +++ b/pkg/tcpip/link/sharedmem/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "sharedmem", diff --git a/pkg/tcpip/link/sharedmem/pipe/BUILD b/pkg/tcpip/link/sharedmem/pipe/BUILD index 12e813509..85deafa38 100644 --- a/pkg/tcpip/link/sharedmem/pipe/BUILD +++ b/pkg/tcpip/link/sharedmem/pipe/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "pipe", diff --git a/pkg/tcpip/link/sharedmem/queue/BUILD b/pkg/tcpip/link/sharedmem/queue/BUILD index 661037bb2..d7dc631eb 100644 --- a/pkg/tcpip/link/sharedmem/queue/BUILD +++ b/pkg/tcpip/link/sharedmem/queue/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "queue", diff --git a/pkg/tcpip/link/sniffer/BUILD b/pkg/tcpip/link/sniffer/BUILD index 52e237c25..7d0d1781e 100644 --- a/pkg/tcpip/link/sniffer/BUILD +++ b/pkg/tcpip/link/sniffer/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "sniffer", diff --git a/pkg/tcpip/link/tun/BUILD b/pkg/tcpip/link/tun/BUILD index 5ec01cec9..e54852d3f 100644 --- a/pkg/tcpip/link/tun/BUILD +++ b/pkg/tcpip/link/tun/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "tun", diff --git a/pkg/tcpip/link/waitable/BUILD b/pkg/tcpip/link/waitable/BUILD index ba495c437..89a9eee23 100644 --- a/pkg/tcpip/link/waitable/BUILD +++ b/pkg/tcpip/link/waitable/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "waitable", diff --git a/pkg/tcpip/network/BUILD b/pkg/tcpip/network/BUILD index a2a07f533..f36f49453 100644 --- a/pkg/tcpip/network/BUILD +++ b/pkg/tcpip/network/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_test( name = "ip_test", diff --git a/pkg/tcpip/network/arp/BUILD b/pkg/tcpip/network/arp/BUILD index f6fb7daf7..ef18bb93d 100644 --- a/pkg/tcpip/network/arp/BUILD +++ b/pkg/tcpip/network/arp/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "arp", diff --git a/pkg/tcpip/network/fragmentation/BUILD b/pkg/tcpip/network/fragmentation/BUILD index aaabfcb9a..bf0a7b99c 100644 --- a/pkg/tcpip/network/fragmentation/BUILD +++ b/pkg/tcpip/network/fragmentation/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/tcpip/network/hash/BUILD b/pkg/tcpip/network/hash/BUILD index 401dce646..ea520c6ed 100644 --- a/pkg/tcpip/network/hash/BUILD +++ b/pkg/tcpip/network/hash/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "hash", diff --git a/pkg/tcpip/network/ipv4/BUILD b/pkg/tcpip/network/ipv4/BUILD index e72317e9f..7a5341def 100644 --- a/pkg/tcpip/network/ipv4/BUILD +++ b/pkg/tcpip/network/ipv4/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "ipv4", diff --git a/pkg/tcpip/network/ipv6/BUILD b/pkg/tcpip/network/ipv6/BUILD index 808c37df3..000e00dba 100644 --- a/pkg/tcpip/network/ipv6/BUILD +++ b/pkg/tcpip/network/ipv6/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "ipv6", diff --git a/pkg/tcpip/ports/BUILD b/pkg/tcpip/ports/BUILD index a2fa9b84a..3ee80c62b 100644 --- a/pkg/tcpip/ports/BUILD +++ b/pkg/tcpip/ports/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "ports", diff --git a/pkg/tcpip/sample/tun_tcp_connect/BUILD b/pkg/tcpip/sample/tun_tcp_connect/BUILD index 32baf2115..996939581 100644 --- a/pkg/tcpip/sample/tun_tcp_connect/BUILD +++ b/pkg/tcpip/sample/tun_tcp_connect/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_binary( name = "tun_tcp_connect", diff --git a/pkg/tcpip/sample/tun_tcp_echo/BUILD b/pkg/tcpip/sample/tun_tcp_echo/BUILD index 760445843..dad8ef399 100644 --- a/pkg/tcpip/sample/tun_tcp_echo/BUILD +++ b/pkg/tcpip/sample/tun_tcp_echo/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_binary( name = "tun_tcp_echo", diff --git a/pkg/tcpip/seqnum/BUILD b/pkg/tcpip/seqnum/BUILD index c5c889239..a63665efc 100644 --- a/pkg/tcpip/seqnum/BUILD +++ b/pkg/tcpip/seqnum/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/tcpip/stack/BUILD b/pkg/tcpip/stack/BUILD index 8a598c57d..551c3c73e 100644 --- a/pkg/tcpip/stack/BUILD +++ b/pkg/tcpip/stack/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/tcpip/transport/ping/BUILD b/pkg/tcpip/transport/ping/BUILD index 982b6795c..4d4241d4b 100644 --- a/pkg/tcpip/transport/ping/BUILD +++ b/pkg/tcpip/transport/ping/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/tcpip/transport/tcp/BUILD b/pkg/tcpip/transport/tcp/BUILD index 726107739..e5c05f8c0 100644 --- a/pkg/tcpip/transport/tcp/BUILD +++ b/pkg/tcpip/transport/tcp/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/tcpip/transport/tcp/testing/context/BUILD b/pkg/tcpip/transport/tcp/testing/context/BUILD index 814e5c1ea..1584e4095 100644 --- a/pkg/tcpip/transport/tcp/testing/context/BUILD +++ b/pkg/tcpip/transport/tcp/testing/context/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "context", diff --git a/pkg/tcpip/transport/tcpconntrack/BUILD b/pkg/tcpip/transport/tcpconntrack/BUILD index ac1a94d4d..31a845dee 100644 --- a/pkg/tcpip/transport/tcpconntrack/BUILD +++ b/pkg/tcpip/transport/tcpconntrack/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "tcpconntrack", diff --git a/pkg/tcpip/transport/udp/BUILD b/pkg/tcpip/transport/udp/BUILD index 4225e28dc..8ccb79c48 100644 --- a/pkg/tcpip/transport/udp/BUILD +++ b/pkg/tcpip/transport/udp/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/tmutex/BUILD b/pkg/tmutex/BUILD index c20df7005..69035044d 100644 --- a/pkg/tmutex/BUILD +++ b/pkg/tmutex/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "tmutex", diff --git a/pkg/unet/BUILD b/pkg/unet/BUILD index f90e43c89..5e177e78e 100644 --- a/pkg/unet/BUILD +++ b/pkg/unet/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "unet", diff --git a/pkg/urpc/BUILD b/pkg/urpc/BUILD index 21008cf6c..36cae67e1 100644 --- a/pkg/urpc/BUILD +++ b/pkg/urpc/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "urpc", diff --git a/pkg/waiter/BUILD b/pkg/waiter/BUILD index 5e611c54f..b748246da 100644 --- a/pkg/waiter/BUILD +++ b/pkg/waiter/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/runsc/boot/BUILD b/runsc/boot/BUILD index 15a7cdae1..540e99151 100644 --- a/runsc/boot/BUILD +++ b/runsc/boot/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "boot", diff --git a/runsc/boot/filter/BUILD b/runsc/boot/filter/BUILD index 004222242..3b6020cf3 100644 --- a/runsc/boot/filter/BUILD +++ b/runsc/boot/filter/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "filter", diff --git a/runsc/cgroup/BUILD b/runsc/cgroup/BUILD index 4f9a25a25..620d33a19 100644 --- a/runsc/cgroup/BUILD +++ b/runsc/cgroup/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "cgroup", diff --git a/runsc/cmd/BUILD b/runsc/cmd/BUILD index a908172af..9e2be0d37 100644 --- a/runsc/cmd/BUILD +++ b/runsc/cmd/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "cmd", diff --git a/runsc/console/BUILD b/runsc/console/BUILD index ff4ccff69..3ff9eba27 100644 --- a/runsc/console/BUILD +++ b/runsc/console/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "console", diff --git a/runsc/container/BUILD b/runsc/container/BUILD index 354ce2661..3b25ff79a 100644 --- a/runsc/container/BUILD +++ b/runsc/container/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "container", diff --git a/runsc/fsgofer/BUILD b/runsc/fsgofer/BUILD index 756c20ad7..4adc9c1bc 100644 --- a/runsc/fsgofer/BUILD +++ b/runsc/fsgofer/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "fsgofer", diff --git a/runsc/fsgofer/filter/BUILD b/runsc/fsgofer/filter/BUILD index c7848d10c..78c5b526c 100644 --- a/runsc/fsgofer/filter/BUILD +++ b/runsc/fsgofer/filter/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "filter", diff --git a/runsc/sandbox/BUILD b/runsc/sandbox/BUILD index 899fd99de..2ed793333 100644 --- a/runsc/sandbox/BUILD +++ b/runsc/sandbox/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "sandbox", diff --git a/runsc/specutils/BUILD b/runsc/specutils/BUILD index 77a10e2b6..372799850 100644 --- a/runsc/specutils/BUILD +++ b/runsc/specutils/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "specutils", diff --git a/runsc/test/image/BUILD b/runsc/test/image/BUILD index 22b3ebd2a..e8b629c6a 100644 --- a/runsc/test/image/BUILD +++ b/runsc/test/image/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_test( name = "image_test", diff --git a/runsc/test/integration/BUILD b/runsc/test/integration/BUILD index e7204dc66..779d30ec9 100644 --- a/runsc/test/integration/BUILD +++ b/runsc/test/integration/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_test( name = "integration_test", diff --git a/runsc/test/root/BUILD b/runsc/test/root/BUILD index 77dcbd79e..75826a521 100644 --- a/runsc/test/root/BUILD +++ b/runsc/test/root/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "root", diff --git a/runsc/test/root/testdata/BUILD b/runsc/test/root/testdata/BUILD index 6c9fe0aea..7f272dcd3 100644 --- a/runsc/test/root/testdata/BUILD +++ b/runsc/test/root/testdata/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "testdata", diff --git a/runsc/test/testutil/BUILD b/runsc/test/testutil/BUILD index 8c3919320..ddec81444 100644 --- a/runsc/test/testutil/BUILD +++ b/runsc/test/testutil/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "testutil", diff --git a/runsc/tools/dockercfg/BUILD b/runsc/tools/dockercfg/BUILD index a80b3abab..fd406ab93 100644 --- a/runsc/tools/dockercfg/BUILD +++ b/runsc/tools/dockercfg/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_binary( name = "dockercfg", diff --git a/test/syscalls/BUILD b/test/syscalls/BUILD index 8c391c8a6..148d9c366 100644 --- a/test/syscalls/BUILD +++ b/test/syscalls/BUILD @@ -1,7 +1,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") load("//test/syscalls:build_defs.bzl", "syscall_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) syscall_test(test = "//test/syscalls/linux:32bit_test") diff --git a/test/syscalls/gtest/BUILD b/test/syscalls/gtest/BUILD index d078fd3d5..22e061652 100644 --- a/test/syscalls/gtest/BUILD +++ b/test/syscalls/gtest/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "gtest", diff --git a/test/syscalls/linux/BUILD b/test/syscalls/linux/BUILD index e70742875..a311ca12c 100644 --- a/test/syscalls/linux/BUILD +++ b/test/syscalls/linux/BUILD @@ -1,6 +1,6 @@ package( default_visibility = ["//:sandbox"], - licenses = ["notice"], # Apache 2.0 + licenses = ["notice"], ) cc_binary( diff --git a/test/util/BUILD b/test/util/BUILD index f2e563507..fac0730b4 100644 --- a/test/util/BUILD +++ b/test/util/BUILD @@ -1,6 +1,6 @@ package( default_visibility = ["//:sandbox"], - licenses = ["notice"], # Apache 2.0 + licenses = ["notice"], ) cc_library( diff --git a/tools/go_generics/BUILD b/tools/go_generics/BUILD index 2d97d99dc..39318b877 100644 --- a/tools/go_generics/BUILD +++ b/tools/go_generics/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_binary( name = "go_generics", diff --git a/tools/go_generics/globals/BUILD b/tools/go_generics/globals/BUILD index c26ac56d2..6628132f5 100644 --- a/tools/go_generics/globals/BUILD +++ b/tools/go_generics/globals/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "globals", diff --git a/tools/go_generics/go_merge/BUILD b/tools/go_generics/go_merge/BUILD index a60437962..02b09120e 100644 --- a/tools/go_generics/go_merge/BUILD +++ b/tools/go_generics/go_merge/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_binary( name = "go_merge", diff --git a/tools/go_generics/rules_tests/BUILD b/tools/go_generics/rules_tests/BUILD index 23b2d656d..a6f8cdd3c 100644 --- a/tools/go_generics/rules_tests/BUILD +++ b/tools/go_generics/rules_tests/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") diff --git a/tools/go_stateify/BUILD b/tools/go_stateify/BUILD index 68d37f5d7..bb53f8ae9 100644 --- a/tools/go_stateify/BUILD +++ b/tools/go_stateify/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_binary( name = "stateify", diff --git a/vdso/BUILD b/vdso/BUILD index fd395511c..c43d24070 100644 --- a/vdso/BUILD +++ b/vdso/BUILD @@ -3,7 +3,7 @@ # normal system VDSO (time, gettimeofday, clock_gettimeofday) but which uses # timekeeping parameters managed by the sandbox kernel. -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) genrule( name = "vdso", -- cgit v1.2.3 From f86c35a51ff92718e36ff6075339300be11e09b3 Mon Sep 17 00:00:00 2001 From: Michael Pratt Date: Mon, 22 Apr 2019 18:17:25 -0700 Subject: Clean up state error handling PiperOrigin-RevId: 244773836 Change-Id: I32223f79d2314fe1ac4ddfc63004fc22ff634adf --- pkg/sentry/control/state.go | 2 +- pkg/sentry/kernel/kernel.go | 23 ++++++++++++----------- pkg/state/state.go | 20 +++++++++++++++----- 3 files changed, 28 insertions(+), 17 deletions(-) (limited to 'pkg/state') diff --git a/pkg/sentry/control/state.go b/pkg/sentry/control/state.go index 0a480c84a..b6bbf69fa 100644 --- a/pkg/sentry/control/state.go +++ b/pkg/sentry/control/state.go @@ -64,7 +64,7 @@ func (s *State) Save(o *SaveOpts, _ *struct{}) error { log.Infof("Save succeeded: exiting...") } else { log.Warningf("Save failed: exiting...") - s.Kernel.SetExitError(err) + s.Kernel.SetSaveError(err) } s.Kernel.Kill(kernel.ExitStatus{}) }, diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go index 290c4a53c..ee6334509 100644 --- a/pkg/sentry/kernel/kernel.go +++ b/pkg/sentry/kernel/kernel.go @@ -175,9 +175,9 @@ type Kernel struct { // netlinkPorts manages allocation of netlink socket port IDs. netlinkPorts *port.Manager - // exitErr is the error causing the sandbox to exit, if any. It is - // protected by extMu. - exitErr error `state:"nosave"` + // saveErr is the error causing the sandbox to exit during save, if + // any. It is protected by extMu. + saveErr error `state:"nosave"` // danglingEndpoints is used to save / restore tcpip.DanglingEndpoints. danglingEndpoints struct{} `state:".([]tcpip.Endpoint)"` @@ -1029,20 +1029,21 @@ func (k *Kernel) NetlinkPorts() *port.Manager { return k.netlinkPorts } -// ExitError returns the sandbox error that caused the kernel to exit. -func (k *Kernel) ExitError() error { +// SaveError returns the sandbox error that caused the kernel to exit during +// save. +func (k *Kernel) SaveError() error { k.extMu.Lock() defer k.extMu.Unlock() - return k.exitErr + return k.saveErr } -// SetExitError sets the sandbox error that caused the kernel to exit, if one is -// not already set. -func (k *Kernel) SetExitError(err error) { +// SetSaveError sets the sandbox error that caused the kernel to exit during +// save, if one is not already set. +func (k *Kernel) SetSaveError(err error) { k.extMu.Lock() defer k.extMu.Unlock() - if k.exitErr == nil { - k.exitErr = err + if k.saveErr == nil { + k.saveErr = err } } diff --git a/pkg/state/state.go b/pkg/state/state.go index 4b141777e..4486f83a7 100644 --- a/pkg/state/state.go +++ b/pkg/state/state.go @@ -60,8 +60,8 @@ import ( // ErrState is returned when an error is encountered during encode/decode. type ErrState struct { - // Err is the underlying error. - Err error + // err is the underlying error. + err error // path is the visit path from root to the current object. path string @@ -72,7 +72,17 @@ type ErrState struct { // Error returns a sensible description of the state error. func (e *ErrState) Error() string { - return fmt.Sprintf("%v:\nstate path: %s\n%s", e.Err, e.path, e.trace) + return fmt.Sprintf("%v:\nstate path: %s\n%s", e.err, e.path, e.trace) +} + +// UnwrapErrState returns the underlying error in ErrState. +// +// If err is not *ErrState, err is returned directly. +func UnwrapErrState(err error) error { + if e, ok := err.(*ErrState); ok { + return e.err + } + return err } // Save saves the given object state. @@ -318,9 +328,9 @@ func (sr *recoverable) safely(fn func()) (err error) { if r := recover(); r != nil { es := new(ErrState) if e, ok := r.(error); ok { - es.Err = e + es.err = e } else { - es.Err = fmt.Errorf("%v", r) + es.err = fmt.Errorf("%v", r) } es.path = sr.path() -- cgit v1.2.3 From 4d52a5520101a88424fb63dd99412a1db33fbd06 Mon Sep 17 00:00:00 2001 From: Michael Pratt Date: Mon, 29 Apr 2019 14:25:05 -0700 Subject: Change copyright notice to "The gVisor Authors" Based on the guidelines at https://opensource.google.com/docs/releasing/authors/. 1. $ rg -l "Google LLC" | xargs sed -i 's/Google LLC.*/The gVisor Authors./' 2. Manual fixup of "Google Inc" references. 3. Add AUTHORS file. Authors may request to be added to this file. 4. Point netstack AUTHORS to gVisor AUTHORS. Drop CONTRIBUTORS. Fixes #209 PiperOrigin-RevId: 245823212 Change-Id: I64530b24ad021a7d683137459cafc510f5ee1de9 --- AUTHORS | 8 ++++++++ kokoro/run_build.sh | 2 +- kokoro/run_tests.sh | 2 +- pkg/abi/abi.go | 2 +- pkg/abi/abi_linux.go | 2 +- pkg/abi/flag.go | 2 +- pkg/abi/linux/aio.go | 2 +- pkg/abi/linux/ashmem.go | 2 +- pkg/abi/linux/audit.go | 2 +- pkg/abi/linux/binder.go | 2 +- pkg/abi/linux/bpf.go | 2 +- pkg/abi/linux/capability.go | 2 +- pkg/abi/linux/dev.go | 2 +- pkg/abi/linux/elf.go | 2 +- pkg/abi/linux/errors.go | 2 +- pkg/abi/linux/eventfd.go | 2 +- pkg/abi/linux/exec.go | 2 +- pkg/abi/linux/fcntl.go | 2 +- pkg/abi/linux/file.go | 2 +- pkg/abi/linux/fs.go | 2 +- pkg/abi/linux/futex.go | 2 +- pkg/abi/linux/inotify.go | 2 +- pkg/abi/linux/ioctl.go | 2 +- pkg/abi/linux/ip.go | 2 +- pkg/abi/linux/ipc.go | 2 +- pkg/abi/linux/limits.go | 2 +- pkg/abi/linux/linux.go | 2 +- pkg/abi/linux/mm.go | 2 +- pkg/abi/linux/netdevice.go | 2 +- pkg/abi/linux/netlink.go | 2 +- pkg/abi/linux/netlink_route.go | 2 +- pkg/abi/linux/poll.go | 2 +- pkg/abi/linux/prctl.go | 2 +- pkg/abi/linux/ptrace.go | 2 +- pkg/abi/linux/rusage.go | 2 +- pkg/abi/linux/sched.go | 2 +- pkg/abi/linux/seccomp.go | 2 +- pkg/abi/linux/sem.go | 2 +- pkg/abi/linux/shm.go | 2 +- pkg/abi/linux/signal.go | 2 +- pkg/abi/linux/socket.go | 2 +- pkg/abi/linux/tcp.go | 2 +- pkg/abi/linux/time.go | 2 +- pkg/abi/linux/timer.go | 2 +- pkg/abi/linux/tty.go | 2 +- pkg/abi/linux/uio.go | 2 +- pkg/abi/linux/utsname.go | 2 +- pkg/amutex/amutex.go | 2 +- pkg/amutex/amutex_test.go | 2 +- pkg/atomicbitops/atomic_bitops.go | 2 +- pkg/atomicbitops/atomic_bitops_amd64.s | 2 +- pkg/atomicbitops/atomic_bitops_common.go | 2 +- pkg/atomicbitops/atomic_bitops_test.go | 2 +- pkg/binary/binary.go | 2 +- pkg/binary/binary_test.go | 2 +- pkg/bits/bits.go | 2 +- pkg/bits/bits_template.go | 2 +- pkg/bits/uint64_arch_amd64.go | 2 +- pkg/bits/uint64_arch_amd64_asm.s | 2 +- pkg/bits/uint64_arch_generic.go | 2 +- pkg/bits/uint64_test.go | 2 +- pkg/bpf/bpf.go | 2 +- pkg/bpf/decoder.go | 2 +- pkg/bpf/decoder_test.go | 2 +- pkg/bpf/input_bytes.go | 2 +- pkg/bpf/interpreter.go | 2 +- pkg/bpf/interpreter_test.go | 2 +- pkg/bpf/program_builder.go | 2 +- pkg/bpf/program_builder_test.go | 2 +- pkg/compressio/compressio.go | 2 +- pkg/compressio/compressio_test.go | 2 +- pkg/control/client/client.go | 2 +- pkg/control/server/server.go | 2 +- pkg/cpuid/cpu_amd64.s | 2 +- pkg/cpuid/cpuid.go | 2 +- pkg/cpuid/cpuid_parse_test.go | 2 +- pkg/cpuid/cpuid_test.go | 2 +- pkg/dhcp/client.go | 2 +- pkg/dhcp/dhcp.go | 2 +- pkg/dhcp/dhcp_string.go | 2 +- pkg/dhcp/dhcp_test.go | 2 +- pkg/dhcp/server.go | 2 +- pkg/eventchannel/event.go | 2 +- pkg/eventchannel/event.proto | 2 +- pkg/fd/fd.go | 2 +- pkg/fd/fd_test.go | 2 +- pkg/fdnotifier/fdnotifier.go | 2 +- pkg/fdnotifier/poll_unsafe.go | 2 +- pkg/gate/gate.go | 2 +- pkg/gate/gate_test.go | 2 +- pkg/ilist/list.go | 2 +- pkg/ilist/list_test.go | 2 +- pkg/linewriter/linewriter.go | 2 +- pkg/linewriter/linewriter_test.go | 2 +- pkg/log/glog.go | 2 +- pkg/log/glog_unsafe.go | 2 +- pkg/log/json.go | 2 +- pkg/log/json_k8s.go | 2 +- pkg/log/json_test.go | 2 +- pkg/log/log.go | 2 +- pkg/log/log_test.go | 2 +- pkg/metric/metric.go | 2 +- pkg/metric/metric.proto | 2 +- pkg/metric/metric_test.go | 2 +- pkg/p9/buffer.go | 2 +- pkg/p9/buffer_test.go | 2 +- pkg/p9/client.go | 2 +- pkg/p9/client_file.go | 2 +- pkg/p9/client_test.go | 2 +- pkg/p9/file.go | 2 +- pkg/p9/handlers.go | 2 +- pkg/p9/local_server/local_server.go | 2 +- pkg/p9/messages.go | 2 +- pkg/p9/messages_test.go | 2 +- pkg/p9/p9.go | 2 +- pkg/p9/p9_test.go | 2 +- pkg/p9/p9test/client_test.go | 2 +- pkg/p9/p9test/p9test.go | 2 +- pkg/p9/path_tree.go | 2 +- pkg/p9/pool.go | 2 +- pkg/p9/pool_test.go | 2 +- pkg/p9/server.go | 2 +- pkg/p9/transport.go | 2 +- pkg/p9/transport_test.go | 2 +- pkg/p9/version.go | 2 +- pkg/p9/version_test.go | 2 +- pkg/rand/rand.go | 2 +- pkg/rand/rand_linux.go | 2 +- pkg/refs/refcounter.go | 2 +- pkg/refs/refcounter_state.go | 2 +- pkg/refs/refcounter_test.go | 2 +- pkg/seccomp/seccomp.go | 2 +- pkg/seccomp/seccomp_rules.go | 2 +- pkg/seccomp/seccomp_test.go | 2 +- pkg/seccomp/seccomp_test_victim.go | 2 +- pkg/seccomp/seccomp_unsafe.go | 2 +- pkg/secio/full_reader.go | 2 +- pkg/secio/secio.go | 2 +- pkg/secio/secio_test.go | 2 +- pkg/segment/range.go | 2 +- pkg/segment/set.go | 2 +- pkg/segment/set_state.go | 2 +- pkg/segment/test/segment_test.go | 2 +- pkg/segment/test/set_functions.go | 2 +- pkg/sentry/arch/aligned.go | 2 +- pkg/sentry/arch/arch.go | 2 +- pkg/sentry/arch/arch_amd64.go | 2 +- pkg/sentry/arch/arch_amd64.s | 2 +- pkg/sentry/arch/arch_state_x86.go | 2 +- pkg/sentry/arch/arch_x86.go | 2 +- pkg/sentry/arch/auxv.go | 2 +- pkg/sentry/arch/registers.proto | 2 +- pkg/sentry/arch/signal_act.go | 2 +- pkg/sentry/arch/signal_amd64.go | 2 +- pkg/sentry/arch/signal_info.go | 2 +- pkg/sentry/arch/signal_stack.go | 2 +- pkg/sentry/arch/stack.go | 2 +- pkg/sentry/arch/syscalls_amd64.go | 2 +- pkg/sentry/context/context.go | 2 +- pkg/sentry/context/contexttest/contexttest.go | 2 +- pkg/sentry/control/control.go | 2 +- pkg/sentry/control/pprof.go | 2 +- pkg/sentry/control/proc.go | 2 +- pkg/sentry/control/proc_test.go | 2 +- pkg/sentry/control/state.go | 2 +- pkg/sentry/device/device.go | 2 +- pkg/sentry/device/device_test.go | 2 +- pkg/sentry/fs/anon/anon.go | 2 +- pkg/sentry/fs/anon/device.go | 2 +- pkg/sentry/fs/ashmem/area.go | 2 +- pkg/sentry/fs/ashmem/device.go | 2 +- pkg/sentry/fs/ashmem/pin_board.go | 2 +- pkg/sentry/fs/ashmem/pin_board_test.go | 2 +- pkg/sentry/fs/attr.go | 2 +- pkg/sentry/fs/binder/binder.go | 2 +- pkg/sentry/fs/context.go | 2 +- pkg/sentry/fs/copy_up.go | 2 +- pkg/sentry/fs/copy_up_test.go | 2 +- pkg/sentry/fs/dentry.go | 2 +- pkg/sentry/fs/dev/dev.go | 2 +- pkg/sentry/fs/dev/device.go | 2 +- pkg/sentry/fs/dev/fs.go | 2 +- pkg/sentry/fs/dev/full.go | 2 +- pkg/sentry/fs/dev/null.go | 2 +- pkg/sentry/fs/dev/random.go | 2 +- pkg/sentry/fs/dirent.go | 2 +- pkg/sentry/fs/dirent_cache.go | 2 +- pkg/sentry/fs/dirent_cache_limiter.go | 2 +- pkg/sentry/fs/dirent_cache_test.go | 2 +- pkg/sentry/fs/dirent_refs_test.go | 2 +- pkg/sentry/fs/dirent_state.go | 2 +- pkg/sentry/fs/fdpipe/pipe.go | 2 +- pkg/sentry/fs/fdpipe/pipe_opener.go | 2 +- pkg/sentry/fs/fdpipe/pipe_opener_test.go | 2 +- pkg/sentry/fs/fdpipe/pipe_state.go | 2 +- pkg/sentry/fs/fdpipe/pipe_test.go | 2 +- pkg/sentry/fs/file.go | 2 +- pkg/sentry/fs/file_operations.go | 2 +- pkg/sentry/fs/file_overlay.go | 2 +- pkg/sentry/fs/file_overlay_test.go | 2 +- pkg/sentry/fs/file_state.go | 2 +- pkg/sentry/fs/file_test.go | 2 +- pkg/sentry/fs/filesystems.go | 2 +- pkg/sentry/fs/filetest/filetest.go | 2 +- pkg/sentry/fs/flags.go | 2 +- pkg/sentry/fs/fs.go | 2 +- pkg/sentry/fs/fsutil/dirty_set.go | 2 +- pkg/sentry/fs/fsutil/dirty_set_test.go | 2 +- pkg/sentry/fs/fsutil/file.go | 2 +- pkg/sentry/fs/fsutil/file_range_set.go | 2 +- pkg/sentry/fs/fsutil/frame_ref_set.go | 2 +- pkg/sentry/fs/fsutil/fsutil.go | 2 +- pkg/sentry/fs/fsutil/host_file_mapper.go | 2 +- pkg/sentry/fs/fsutil/host_file_mapper_state.go | 2 +- pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go | 2 +- pkg/sentry/fs/fsutil/host_mappable.go | 2 +- pkg/sentry/fs/fsutil/inode.go | 2 +- pkg/sentry/fs/fsutil/inode_cached.go | 2 +- pkg/sentry/fs/fsutil/inode_cached_test.go | 2 +- pkg/sentry/fs/gofer/attr.go | 2 +- pkg/sentry/fs/gofer/cache_policy.go | 2 +- pkg/sentry/fs/gofer/context_file.go | 2 +- pkg/sentry/fs/gofer/device.go | 2 +- pkg/sentry/fs/gofer/file.go | 2 +- pkg/sentry/fs/gofer/file_state.go | 2 +- pkg/sentry/fs/gofer/fs.go | 2 +- pkg/sentry/fs/gofer/gofer_test.go | 2 +- pkg/sentry/fs/gofer/handles.go | 2 +- pkg/sentry/fs/gofer/inode.go | 2 +- pkg/sentry/fs/gofer/inode_state.go | 2 +- pkg/sentry/fs/gofer/path.go | 2 +- pkg/sentry/fs/gofer/session.go | 2 +- pkg/sentry/fs/gofer/session_state.go | 2 +- pkg/sentry/fs/gofer/socket.go | 2 +- pkg/sentry/fs/gofer/util.go | 2 +- pkg/sentry/fs/host/control.go | 2 +- pkg/sentry/fs/host/descriptor.go | 2 +- pkg/sentry/fs/host/descriptor_state.go | 2 +- pkg/sentry/fs/host/descriptor_test.go | 2 +- pkg/sentry/fs/host/device.go | 2 +- pkg/sentry/fs/host/file.go | 2 +- pkg/sentry/fs/host/fs.go | 2 +- pkg/sentry/fs/host/fs_test.go | 2 +- pkg/sentry/fs/host/inode.go | 2 +- pkg/sentry/fs/host/inode_state.go | 2 +- pkg/sentry/fs/host/inode_test.go | 2 +- pkg/sentry/fs/host/ioctl_unsafe.go | 2 +- pkg/sentry/fs/host/socket.go | 2 +- pkg/sentry/fs/host/socket_iovec.go | 2 +- pkg/sentry/fs/host/socket_state.go | 2 +- pkg/sentry/fs/host/socket_test.go | 2 +- pkg/sentry/fs/host/socket_unsafe.go | 2 +- pkg/sentry/fs/host/tty.go | 2 +- pkg/sentry/fs/host/util.go | 2 +- pkg/sentry/fs/host/util_unsafe.go | 2 +- pkg/sentry/fs/host/wait_test.go | 2 +- pkg/sentry/fs/inode.go | 2 +- pkg/sentry/fs/inode_inotify.go | 2 +- pkg/sentry/fs/inode_operations.go | 2 +- pkg/sentry/fs/inode_overlay.go | 2 +- pkg/sentry/fs/inode_overlay_test.go | 2 +- pkg/sentry/fs/inotify.go | 2 +- pkg/sentry/fs/inotify_event.go | 2 +- pkg/sentry/fs/inotify_watch.go | 2 +- pkg/sentry/fs/lock/lock.go | 2 +- pkg/sentry/fs/lock/lock_range_test.go | 2 +- pkg/sentry/fs/lock/lock_set_functions.go | 2 +- pkg/sentry/fs/lock/lock_test.go | 2 +- pkg/sentry/fs/mock.go | 2 +- pkg/sentry/fs/mount.go | 2 +- pkg/sentry/fs/mount_overlay.go | 2 +- pkg/sentry/fs/mount_test.go | 2 +- pkg/sentry/fs/mounts.go | 2 +- pkg/sentry/fs/mounts_test.go | 2 +- pkg/sentry/fs/offset.go | 2 +- pkg/sentry/fs/overlay.go | 2 +- pkg/sentry/fs/path.go | 2 +- pkg/sentry/fs/path_test.go | 2 +- pkg/sentry/fs/proc/cpuinfo.go | 2 +- pkg/sentry/fs/proc/device/device.go | 2 +- pkg/sentry/fs/proc/exec_args.go | 2 +- pkg/sentry/fs/proc/fds.go | 2 +- pkg/sentry/fs/proc/filesystems.go | 2 +- pkg/sentry/fs/proc/fs.go | 2 +- pkg/sentry/fs/proc/inode.go | 2 +- pkg/sentry/fs/proc/loadavg.go | 2 +- pkg/sentry/fs/proc/meminfo.go | 2 +- pkg/sentry/fs/proc/mounts.go | 2 +- pkg/sentry/fs/proc/net.go | 2 +- pkg/sentry/fs/proc/net_test.go | 2 +- pkg/sentry/fs/proc/proc.go | 2 +- pkg/sentry/fs/proc/rpcinet_proc.go | 2 +- pkg/sentry/fs/proc/seqfile/seqfile.go | 2 +- pkg/sentry/fs/proc/seqfile/seqfile_test.go | 2 +- pkg/sentry/fs/proc/stat.go | 2 +- pkg/sentry/fs/proc/sys.go | 2 +- pkg/sentry/fs/proc/sys_net.go | 2 +- pkg/sentry/fs/proc/sys_net_state.go | 2 +- pkg/sentry/fs/proc/sys_net_test.go | 2 +- pkg/sentry/fs/proc/task.go | 2 +- pkg/sentry/fs/proc/uid_gid_map.go | 2 +- pkg/sentry/fs/proc/uptime.go | 2 +- pkg/sentry/fs/proc/version.go | 2 +- pkg/sentry/fs/ramfs/dir.go | 2 +- pkg/sentry/fs/ramfs/socket.go | 2 +- pkg/sentry/fs/ramfs/symlink.go | 2 +- pkg/sentry/fs/ramfs/tree.go | 2 +- pkg/sentry/fs/ramfs/tree_test.go | 2 +- pkg/sentry/fs/restore.go | 2 +- pkg/sentry/fs/save.go | 2 +- pkg/sentry/fs/seek.go | 2 +- pkg/sentry/fs/sync.go | 2 +- pkg/sentry/fs/sys/device.go | 2 +- pkg/sentry/fs/sys/devices.go | 2 +- pkg/sentry/fs/sys/fs.go | 2 +- pkg/sentry/fs/sys/sys.go | 2 +- pkg/sentry/fs/timerfd/timerfd.go | 2 +- pkg/sentry/fs/tmpfs/device.go | 2 +- pkg/sentry/fs/tmpfs/file_regular.go | 2 +- pkg/sentry/fs/tmpfs/file_test.go | 2 +- pkg/sentry/fs/tmpfs/fs.go | 2 +- pkg/sentry/fs/tmpfs/inode_file.go | 2 +- pkg/sentry/fs/tmpfs/tmpfs.go | 2 +- pkg/sentry/fs/tty/dir.go | 2 +- pkg/sentry/fs/tty/fs.go | 2 +- pkg/sentry/fs/tty/line_discipline.go | 2 +- pkg/sentry/fs/tty/master.go | 2 +- pkg/sentry/fs/tty/queue.go | 2 +- pkg/sentry/fs/tty/slave.go | 2 +- pkg/sentry/fs/tty/terminal.go | 2 +- pkg/sentry/fs/tty/tty_test.go | 2 +- pkg/sentry/hostcpu/getcpu_amd64.s | 2 +- pkg/sentry/hostcpu/hostcpu.go | 2 +- pkg/sentry/hostcpu/hostcpu_test.go | 2 +- pkg/sentry/inet/context.go | 2 +- pkg/sentry/inet/inet.go | 2 +- pkg/sentry/inet/test_stack.go | 2 +- pkg/sentry/kernel/abstract_socket_namespace.go | 2 +- pkg/sentry/kernel/auth/auth.go | 2 +- pkg/sentry/kernel/auth/capability_set.go | 2 +- pkg/sentry/kernel/auth/context.go | 2 +- pkg/sentry/kernel/auth/credentials.go | 2 +- pkg/sentry/kernel/auth/id.go | 2 +- pkg/sentry/kernel/auth/id_map.go | 2 +- pkg/sentry/kernel/auth/id_map_functions.go | 2 +- pkg/sentry/kernel/auth/user_namespace.go | 2 +- pkg/sentry/kernel/context.go | 2 +- pkg/sentry/kernel/contexttest/contexttest.go | 2 +- pkg/sentry/kernel/epoll/epoll.go | 2 +- pkg/sentry/kernel/epoll/epoll_state.go | 2 +- pkg/sentry/kernel/epoll/epoll_test.go | 2 +- pkg/sentry/kernel/eventfd/eventfd.go | 2 +- pkg/sentry/kernel/eventfd/eventfd_test.go | 2 +- pkg/sentry/kernel/fasync/fasync.go | 2 +- pkg/sentry/kernel/fd_map.go | 2 +- pkg/sentry/kernel/fd_map_test.go | 2 +- pkg/sentry/kernel/fs_context.go | 2 +- pkg/sentry/kernel/futex/futex.go | 2 +- pkg/sentry/kernel/futex/futex_test.go | 2 +- pkg/sentry/kernel/ipc_namespace.go | 2 +- pkg/sentry/kernel/kdefs/kdefs.go | 2 +- pkg/sentry/kernel/kernel.go | 2 +- pkg/sentry/kernel/kernel_state.go | 2 +- pkg/sentry/kernel/memevent/memory_events.go | 2 +- pkg/sentry/kernel/memevent/memory_events.proto | 2 +- pkg/sentry/kernel/pending_signals.go | 2 +- pkg/sentry/kernel/pending_signals_state.go | 2 +- pkg/sentry/kernel/pipe/buffers.go | 2 +- pkg/sentry/kernel/pipe/device.go | 2 +- pkg/sentry/kernel/pipe/node.go | 2 +- pkg/sentry/kernel/pipe/node_test.go | 2 +- pkg/sentry/kernel/pipe/pipe.go | 2 +- pkg/sentry/kernel/pipe/pipe_test.go | 2 +- pkg/sentry/kernel/pipe/reader.go | 2 +- pkg/sentry/kernel/pipe/reader_writer.go | 2 +- pkg/sentry/kernel/pipe/writer.go | 2 +- pkg/sentry/kernel/posixtimer.go | 2 +- pkg/sentry/kernel/ptrace.go | 2 +- pkg/sentry/kernel/ptrace_amd64.go | 2 +- pkg/sentry/kernel/ptrace_arm64.go | 2 +- pkg/sentry/kernel/rseq.go | 2 +- pkg/sentry/kernel/sched/cpuset.go | 2 +- pkg/sentry/kernel/sched/cpuset_test.go | 2 +- pkg/sentry/kernel/sched/sched.go | 2 +- pkg/sentry/kernel/seccomp.go | 2 +- pkg/sentry/kernel/semaphore/semaphore.go | 2 +- pkg/sentry/kernel/semaphore/semaphore_test.go | 2 +- pkg/sentry/kernel/sessions.go | 2 +- pkg/sentry/kernel/shm/device.go | 2 +- pkg/sentry/kernel/shm/shm.go | 2 +- pkg/sentry/kernel/signal.go | 2 +- pkg/sentry/kernel/signal_handlers.go | 2 +- pkg/sentry/kernel/syscalls.go | 2 +- pkg/sentry/kernel/syscalls_state.go | 2 +- pkg/sentry/kernel/syslog.go | 2 +- pkg/sentry/kernel/table_test.go | 2 +- pkg/sentry/kernel/task.go | 2 +- pkg/sentry/kernel/task_acct.go | 2 +- pkg/sentry/kernel/task_block.go | 2 +- pkg/sentry/kernel/task_clone.go | 2 +- pkg/sentry/kernel/task_context.go | 2 +- pkg/sentry/kernel/task_exec.go | 2 +- pkg/sentry/kernel/task_exit.go | 2 +- pkg/sentry/kernel/task_futex.go | 2 +- pkg/sentry/kernel/task_identity.go | 2 +- pkg/sentry/kernel/task_log.go | 2 +- pkg/sentry/kernel/task_net.go | 2 +- pkg/sentry/kernel/task_run.go | 2 +- pkg/sentry/kernel/task_sched.go | 2 +- pkg/sentry/kernel/task_signals.go | 2 +- pkg/sentry/kernel/task_start.go | 2 +- pkg/sentry/kernel/task_stop.go | 2 +- pkg/sentry/kernel/task_syscall.go | 2 +- pkg/sentry/kernel/task_test.go | 2 +- pkg/sentry/kernel/task_usermem.go | 2 +- pkg/sentry/kernel/thread_group.go | 2 +- pkg/sentry/kernel/threads.go | 2 +- pkg/sentry/kernel/time/context.go | 2 +- pkg/sentry/kernel/time/time.go | 2 +- pkg/sentry/kernel/timekeeper.go | 2 +- pkg/sentry/kernel/timekeeper_state.go | 2 +- pkg/sentry/kernel/timekeeper_test.go | 2 +- pkg/sentry/kernel/uncaught_signal.proto | 2 +- pkg/sentry/kernel/uts_namespace.go | 2 +- pkg/sentry/kernel/vdso.go | 2 +- pkg/sentry/kernel/version.go | 2 +- pkg/sentry/limits/context.go | 2 +- pkg/sentry/limits/limits.go | 2 +- pkg/sentry/limits/limits_test.go | 2 +- pkg/sentry/limits/linux.go | 2 +- pkg/sentry/loader/elf.go | 2 +- pkg/sentry/loader/interpreter.go | 2 +- pkg/sentry/loader/loader.go | 2 +- pkg/sentry/loader/vdso.go | 2 +- pkg/sentry/loader/vdso_state.go | 2 +- pkg/sentry/memmap/mapping_set.go | 2 +- pkg/sentry/memmap/mapping_set_test.go | 2 +- pkg/sentry/memmap/memmap.go | 2 +- pkg/sentry/memutil/memutil.go | 2 +- pkg/sentry/memutil/memutil_unsafe.go | 2 +- pkg/sentry/mm/address_space.go | 2 +- pkg/sentry/mm/aio_context.go | 2 +- pkg/sentry/mm/aio_context_state.go | 2 +- pkg/sentry/mm/debug.go | 2 +- pkg/sentry/mm/io.go | 2 +- pkg/sentry/mm/lifecycle.go | 2 +- pkg/sentry/mm/metadata.go | 2 +- pkg/sentry/mm/mm.go | 2 +- pkg/sentry/mm/mm_test.go | 2 +- pkg/sentry/mm/pma.go | 2 +- pkg/sentry/mm/procfs.go | 2 +- pkg/sentry/mm/save_restore.go | 2 +- pkg/sentry/mm/shm.go | 2 +- pkg/sentry/mm/special_mappable.go | 2 +- pkg/sentry/mm/syscalls.go | 2 +- pkg/sentry/mm/vma.go | 2 +- pkg/sentry/pgalloc/context.go | 2 +- pkg/sentry/pgalloc/pgalloc.go | 2 +- pkg/sentry/pgalloc/pgalloc_test.go | 2 +- pkg/sentry/pgalloc/pgalloc_unsafe.go | 2 +- pkg/sentry/pgalloc/save_restore.go | 2 +- pkg/sentry/platform/context.go | 2 +- pkg/sentry/platform/interrupt/interrupt.go | 2 +- pkg/sentry/platform/interrupt/interrupt_test.go | 2 +- pkg/sentry/platform/kvm/address_space.go | 2 +- pkg/sentry/platform/kvm/allocator.go | 2 +- pkg/sentry/platform/kvm/bluepill.go | 2 +- pkg/sentry/platform/kvm/bluepill_amd64.go | 2 +- pkg/sentry/platform/kvm/bluepill_amd64.s | 2 +- pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go | 2 +- pkg/sentry/platform/kvm/bluepill_fault.go | 2 +- pkg/sentry/platform/kvm/bluepill_unsafe.go | 2 +- pkg/sentry/platform/kvm/context.go | 2 +- pkg/sentry/platform/kvm/kvm.go | 2 +- pkg/sentry/platform/kvm/kvm_amd64.go | 2 +- pkg/sentry/platform/kvm/kvm_amd64_unsafe.go | 2 +- pkg/sentry/platform/kvm/kvm_const.go | 2 +- pkg/sentry/platform/kvm/kvm_test.go | 2 +- pkg/sentry/platform/kvm/machine.go | 2 +- pkg/sentry/platform/kvm/machine_amd64.go | 2 +- pkg/sentry/platform/kvm/machine_amd64_unsafe.go | 2 +- pkg/sentry/platform/kvm/machine_unsafe.go | 2 +- pkg/sentry/platform/kvm/physical_map.go | 2 +- pkg/sentry/platform/kvm/testutil/testutil.go | 2 +- pkg/sentry/platform/kvm/testutil/testutil_amd64.go | 2 +- pkg/sentry/platform/kvm/testutil/testutil_amd64.s | 2 +- pkg/sentry/platform/kvm/virtual_map.go | 2 +- pkg/sentry/platform/kvm/virtual_map_test.go | 2 +- pkg/sentry/platform/mmap_min_addr.go | 2 +- pkg/sentry/platform/platform.go | 2 +- pkg/sentry/platform/procid/procid.go | 2 +- pkg/sentry/platform/procid/procid_amd64.s | 2 +- pkg/sentry/platform/procid/procid_arm64.s | 2 +- pkg/sentry/platform/procid/procid_net_test.go | 2 +- pkg/sentry/platform/procid/procid_test.go | 2 +- pkg/sentry/platform/ptrace/ptrace.go | 2 +- pkg/sentry/platform/ptrace/ptrace_unsafe.go | 2 +- pkg/sentry/platform/ptrace/stub_amd64.s | 2 +- pkg/sentry/platform/ptrace/stub_unsafe.go | 2 +- pkg/sentry/platform/ptrace/subprocess.go | 2 +- pkg/sentry/platform/ptrace/subprocess_amd64.go | 2 +- pkg/sentry/platform/ptrace/subprocess_linux.go | 2 +- pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go | 2 +- pkg/sentry/platform/ptrace/subprocess_unsafe.go | 2 +- pkg/sentry/platform/ring0/defs.go | 2 +- pkg/sentry/platform/ring0/defs_amd64.go | 2 +- pkg/sentry/platform/ring0/entry_amd64.go | 2 +- pkg/sentry/platform/ring0/entry_amd64.s | 2 +- pkg/sentry/platform/ring0/gen_offsets/main.go | 2 +- pkg/sentry/platform/ring0/kernel.go | 2 +- pkg/sentry/platform/ring0/kernel_amd64.go | 2 +- pkg/sentry/platform/ring0/kernel_unsafe.go | 2 +- pkg/sentry/platform/ring0/lib_amd64.go | 2 +- pkg/sentry/platform/ring0/lib_amd64.s | 2 +- pkg/sentry/platform/ring0/offsets_amd64.go | 2 +- pkg/sentry/platform/ring0/pagetables/allocator.go | 2 +- pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables_test.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables_x86.go | 2 +- pkg/sentry/platform/ring0/pagetables/pcids_x86.go | 2 +- pkg/sentry/platform/ring0/pagetables/walker_amd64.go | 2 +- pkg/sentry/platform/ring0/ring0.go | 2 +- pkg/sentry/platform/ring0/x86.go | 2 +- pkg/sentry/platform/safecopy/atomic_amd64.s | 2 +- pkg/sentry/platform/safecopy/safecopy.go | 2 +- pkg/sentry/platform/safecopy/safecopy_test.go | 2 +- pkg/sentry/platform/safecopy/safecopy_unsafe.go | 2 +- pkg/sentry/platform/safecopy/sighandler_amd64.s | 2 +- pkg/sentry/platform/safecopy/sighandler_arm64.s | 2 +- pkg/sentry/safemem/block_unsafe.go | 2 +- pkg/sentry/safemem/io.go | 2 +- pkg/sentry/safemem/io_test.go | 2 +- pkg/sentry/safemem/safemem.go | 2 +- pkg/sentry/safemem/seq_test.go | 2 +- pkg/sentry/safemem/seq_unsafe.go | 2 +- pkg/sentry/sighandling/sighandling.go | 2 +- pkg/sentry/sighandling/sighandling_unsafe.go | 2 +- pkg/sentry/socket/control/control.go | 2 +- pkg/sentry/socket/epsocket/device.go | 2 +- pkg/sentry/socket/epsocket/epsocket.go | 2 +- pkg/sentry/socket/epsocket/provider.go | 2 +- pkg/sentry/socket/epsocket/save_restore.go | 2 +- pkg/sentry/socket/epsocket/stack.go | 2 +- pkg/sentry/socket/hostinet/device.go | 2 +- pkg/sentry/socket/hostinet/hostinet.go | 2 +- pkg/sentry/socket/hostinet/save_restore.go | 2 +- pkg/sentry/socket/hostinet/socket.go | 2 +- pkg/sentry/socket/hostinet/socket_unsafe.go | 2 +- pkg/sentry/socket/hostinet/stack.go | 2 +- pkg/sentry/socket/netlink/message.go | 2 +- pkg/sentry/socket/netlink/port/port.go | 2 +- pkg/sentry/socket/netlink/port/port_test.go | 2 +- pkg/sentry/socket/netlink/provider.go | 2 +- pkg/sentry/socket/netlink/route/protocol.go | 2 +- pkg/sentry/socket/netlink/socket.go | 2 +- pkg/sentry/socket/rpcinet/conn/conn.go | 2 +- pkg/sentry/socket/rpcinet/device.go | 2 +- pkg/sentry/socket/rpcinet/notifier/notifier.go | 2 +- pkg/sentry/socket/rpcinet/rpcinet.go | 2 +- pkg/sentry/socket/rpcinet/socket.go | 2 +- pkg/sentry/socket/rpcinet/stack.go | 2 +- pkg/sentry/socket/rpcinet/stack_unsafe.go | 2 +- pkg/sentry/socket/socket.go | 2 +- pkg/sentry/socket/unix/device.go | 2 +- pkg/sentry/socket/unix/io.go | 2 +- pkg/sentry/socket/unix/transport/connectioned.go | 2 +- pkg/sentry/socket/unix/transport/connectioned_state.go | 2 +- pkg/sentry/socket/unix/transport/connectionless.go | 2 +- pkg/sentry/socket/unix/transport/queue.go | 2 +- pkg/sentry/socket/unix/transport/unix.go | 2 +- pkg/sentry/socket/unix/unix.go | 2 +- pkg/sentry/state/state.go | 2 +- pkg/sentry/state/state_metadata.go | 2 +- pkg/sentry/state/state_unsafe.go | 2 +- pkg/sentry/strace/capability.go | 2 +- pkg/sentry/strace/clone.go | 2 +- pkg/sentry/strace/futex.go | 2 +- pkg/sentry/strace/linux64.go | 2 +- pkg/sentry/strace/open.go | 2 +- pkg/sentry/strace/poll.go | 2 +- pkg/sentry/strace/ptrace.go | 2 +- pkg/sentry/strace/signal.go | 2 +- pkg/sentry/strace/socket.go | 2 +- pkg/sentry/strace/strace.go | 2 +- pkg/sentry/strace/strace.proto | 2 +- pkg/sentry/strace/syscalls.go | 2 +- pkg/sentry/syscalls/epoll.go | 2 +- pkg/sentry/syscalls/linux/error.go | 2 +- pkg/sentry/syscalls/linux/flags.go | 2 +- pkg/sentry/syscalls/linux/linux64.go | 2 +- pkg/sentry/syscalls/linux/sigset.go | 2 +- pkg/sentry/syscalls/linux/sys_aio.go | 2 +- pkg/sentry/syscalls/linux/sys_capability.go | 2 +- pkg/sentry/syscalls/linux/sys_epoll.go | 2 +- pkg/sentry/syscalls/linux/sys_eventfd.go | 2 +- pkg/sentry/syscalls/linux/sys_file.go | 2 +- pkg/sentry/syscalls/linux/sys_futex.go | 2 +- pkg/sentry/syscalls/linux/sys_getdents.go | 2 +- pkg/sentry/syscalls/linux/sys_identity.go | 2 +- pkg/sentry/syscalls/linux/sys_inotify.go | 2 +- pkg/sentry/syscalls/linux/sys_lseek.go | 2 +- pkg/sentry/syscalls/linux/sys_mmap.go | 2 +- pkg/sentry/syscalls/linux/sys_mount.go | 2 +- pkg/sentry/syscalls/linux/sys_pipe.go | 2 +- pkg/sentry/syscalls/linux/sys_poll.go | 2 +- pkg/sentry/syscalls/linux/sys_prctl.go | 2 +- pkg/sentry/syscalls/linux/sys_random.go | 2 +- pkg/sentry/syscalls/linux/sys_read.go | 2 +- pkg/sentry/syscalls/linux/sys_rlimit.go | 2 +- pkg/sentry/syscalls/linux/sys_rusage.go | 2 +- pkg/sentry/syscalls/linux/sys_sched.go | 2 +- pkg/sentry/syscalls/linux/sys_seccomp.go | 2 +- pkg/sentry/syscalls/linux/sys_sem.go | 2 +- pkg/sentry/syscalls/linux/sys_shm.go | 2 +- pkg/sentry/syscalls/linux/sys_signal.go | 2 +- pkg/sentry/syscalls/linux/sys_socket.go | 2 +- pkg/sentry/syscalls/linux/sys_stat.go | 2 +- pkg/sentry/syscalls/linux/sys_sync.go | 2 +- pkg/sentry/syscalls/linux/sys_sysinfo.go | 2 +- pkg/sentry/syscalls/linux/sys_syslog.go | 2 +- pkg/sentry/syscalls/linux/sys_thread.go | 2 +- pkg/sentry/syscalls/linux/sys_time.go | 2 +- pkg/sentry/syscalls/linux/sys_timer.go | 2 +- pkg/sentry/syscalls/linux/sys_timerfd.go | 2 +- pkg/sentry/syscalls/linux/sys_tls.go | 2 +- pkg/sentry/syscalls/linux/sys_utsname.go | 2 +- pkg/sentry/syscalls/linux/sys_write.go | 2 +- pkg/sentry/syscalls/linux/timespec.go | 2 +- pkg/sentry/syscalls/syscalls.go | 2 +- pkg/sentry/time/calibrated_clock.go | 2 +- pkg/sentry/time/calibrated_clock_test.go | 2 +- pkg/sentry/time/clock_id.go | 2 +- pkg/sentry/time/clocks.go | 2 +- pkg/sentry/time/muldiv_amd64.s | 2 +- pkg/sentry/time/muldiv_arm64.s | 2 +- pkg/sentry/time/parameters.go | 2 +- pkg/sentry/time/parameters_test.go | 2 +- pkg/sentry/time/sampler.go | 2 +- pkg/sentry/time/sampler_test.go | 2 +- pkg/sentry/time/sampler_unsafe.go | 2 +- pkg/sentry/time/tsc_amd64.s | 2 +- pkg/sentry/time/tsc_arm64.s | 2 +- pkg/sentry/unimpl/events.go | 2 +- pkg/sentry/unimpl/unimplemented_syscall.proto | 2 +- pkg/sentry/uniqueid/context.go | 2 +- pkg/sentry/usage/cpu.go | 2 +- pkg/sentry/usage/io.go | 2 +- pkg/sentry/usage/memory.go | 2 +- pkg/sentry/usage/memory_unsafe.go | 2 +- pkg/sentry/usage/usage.go | 2 +- pkg/sentry/usermem/access_type.go | 2 +- pkg/sentry/usermem/addr.go | 2 +- pkg/sentry/usermem/addr_range_seq_test.go | 2 +- pkg/sentry/usermem/addr_range_seq_unsafe.go | 2 +- pkg/sentry/usermem/bytes_io.go | 2 +- pkg/sentry/usermem/bytes_io_unsafe.go | 2 +- pkg/sentry/usermem/usermem.go | 2 +- pkg/sentry/usermem/usermem_arm64.go | 2 +- pkg/sentry/usermem/usermem_test.go | 2 +- pkg/sentry/usermem/usermem_unsafe.go | 2 +- pkg/sentry/usermem/usermem_x86.go | 2 +- pkg/sentry/watchdog/watchdog.go | 2 +- pkg/sleep/commit_amd64.s | 2 +- pkg/sleep/commit_asm.go | 2 +- pkg/sleep/commit_noasm.go | 2 +- pkg/sleep/empty.s | 2 +- pkg/sleep/sleep_test.go | 2 +- pkg/sleep/sleep_unsafe.go | 2 +- pkg/state/decode.go | 2 +- pkg/state/encode.go | 2 +- pkg/state/encode_unsafe.go | 2 +- pkg/state/map.go | 2 +- pkg/state/object.proto | 2 +- pkg/state/printer.go | 2 +- pkg/state/state.go | 2 +- pkg/state/state_test.go | 2 +- pkg/state/statefile/statefile.go | 2 +- pkg/state/statefile/statefile_test.go | 2 +- pkg/state/stats.go | 2 +- pkg/syserr/host_linux.go | 2 +- pkg/syserr/netstack.go | 2 +- pkg/syserr/syserr.go | 2 +- pkg/syserror/syserror.go | 2 +- pkg/syserror/syserror_test.go | 2 +- pkg/tcpip/adapters/gonet/gonet.go | 2 +- pkg/tcpip/adapters/gonet/gonet_test.go | 2 +- pkg/tcpip/buffer/prependable.go | 2 +- pkg/tcpip/buffer/view.go | 2 +- pkg/tcpip/buffer/view_test.go | 2 +- pkg/tcpip/checker/checker.go | 2 +- pkg/tcpip/hash/jenkins/jenkins.go | 2 +- pkg/tcpip/hash/jenkins/jenkins_test.go | 2 +- pkg/tcpip/header/arp.go | 2 +- pkg/tcpip/header/checksum.go | 2 +- pkg/tcpip/header/eth.go | 2 +- pkg/tcpip/header/gue.go | 2 +- pkg/tcpip/header/icmpv4.go | 2 +- pkg/tcpip/header/icmpv6.go | 2 +- pkg/tcpip/header/interfaces.go | 2 +- pkg/tcpip/header/ipv4.go | 2 +- pkg/tcpip/header/ipv6.go | 2 +- pkg/tcpip/header/ipv6_fragment.go | 2 +- pkg/tcpip/header/ipversion_test.go | 2 +- pkg/tcpip/header/tcp.go | 2 +- pkg/tcpip/header/tcp_test.go | 2 +- pkg/tcpip/header/udp.go | 2 +- pkg/tcpip/link/channel/channel.go | 2 +- pkg/tcpip/link/fdbased/endpoint.go | 2 +- pkg/tcpip/link/fdbased/endpoint_test.go | 2 +- pkg/tcpip/link/fdbased/endpoint_unsafe.go | 2 +- pkg/tcpip/link/fdbased/mmap.go | 2 +- pkg/tcpip/link/fdbased/mmap_amd64_unsafe.go | 2 +- pkg/tcpip/link/loopback/loopback.go | 2 +- pkg/tcpip/link/muxed/injectable.go | 2 +- pkg/tcpip/link/muxed/injectable_test.go | 2 +- pkg/tcpip/link/rawfile/blockingpoll_amd64.s | 2 +- pkg/tcpip/link/rawfile/blockingpoll_amd64_unsafe.go | 2 +- pkg/tcpip/link/rawfile/blockingpoll_unsafe.go | 2 +- pkg/tcpip/link/rawfile/errors.go | 2 +- pkg/tcpip/link/rawfile/rawfile_unsafe.go | 2 +- pkg/tcpip/link/sharedmem/pipe/pipe.go | 2 +- pkg/tcpip/link/sharedmem/pipe/pipe_test.go | 2 +- pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go | 2 +- pkg/tcpip/link/sharedmem/pipe/rx.go | 2 +- pkg/tcpip/link/sharedmem/pipe/tx.go | 2 +- pkg/tcpip/link/sharedmem/queue/queue_test.go | 2 +- pkg/tcpip/link/sharedmem/queue/rx.go | 2 +- pkg/tcpip/link/sharedmem/queue/tx.go | 2 +- pkg/tcpip/link/sharedmem/rx.go | 2 +- pkg/tcpip/link/sharedmem/sharedmem.go | 2 +- pkg/tcpip/link/sharedmem/sharedmem_test.go | 2 +- pkg/tcpip/link/sharedmem/sharedmem_unsafe.go | 2 +- pkg/tcpip/link/sharedmem/tx.go | 2 +- pkg/tcpip/link/sniffer/pcap.go | 2 +- pkg/tcpip/link/sniffer/sniffer.go | 2 +- pkg/tcpip/link/tun/tun_unsafe.go | 2 +- pkg/tcpip/link/waitable/waitable.go | 2 +- pkg/tcpip/link/waitable/waitable_test.go | 2 +- pkg/tcpip/network/arp/arp.go | 2 +- pkg/tcpip/network/arp/arp_test.go | 2 +- pkg/tcpip/network/fragmentation/frag_heap.go | 2 +- pkg/tcpip/network/fragmentation/frag_heap_test.go | 2 +- pkg/tcpip/network/fragmentation/fragmentation.go | 2 +- pkg/tcpip/network/fragmentation/fragmentation_test.go | 2 +- pkg/tcpip/network/fragmentation/reassembler.go | 2 +- pkg/tcpip/network/fragmentation/reassembler_test.go | 2 +- pkg/tcpip/network/hash/hash.go | 2 +- pkg/tcpip/network/ip_test.go | 2 +- pkg/tcpip/network/ipv4/icmp.go | 2 +- pkg/tcpip/network/ipv4/ipv4.go | 2 +- pkg/tcpip/network/ipv4/ipv4_test.go | 2 +- pkg/tcpip/network/ipv6/icmp.go | 2 +- pkg/tcpip/network/ipv6/icmp_test.go | 2 +- pkg/tcpip/network/ipv6/ipv6.go | 2 +- pkg/tcpip/ports/ports.go | 2 +- pkg/tcpip/ports/ports_test.go | 2 +- pkg/tcpip/sample/tun_tcp_connect/main.go | 2 +- pkg/tcpip/sample/tun_tcp_echo/main.go | 2 +- pkg/tcpip/seqnum/seqnum.go | 2 +- pkg/tcpip/stack/linkaddrcache.go | 2 +- pkg/tcpip/stack/linkaddrcache_test.go | 2 +- pkg/tcpip/stack/nic.go | 2 +- pkg/tcpip/stack/registration.go | 2 +- pkg/tcpip/stack/route.go | 2 +- pkg/tcpip/stack/stack.go | 2 +- pkg/tcpip/stack/stack_global_state.go | 2 +- pkg/tcpip/stack/stack_test.go | 2 +- pkg/tcpip/stack/transport_demuxer.go | 2 +- pkg/tcpip/stack/transport_test.go | 2 +- pkg/tcpip/tcpip.go | 2 +- pkg/tcpip/tcpip_test.go | 2 +- pkg/tcpip/time.s | 2 +- pkg/tcpip/time_unsafe.go | 2 +- pkg/tcpip/transport/icmp/endpoint.go | 2 +- pkg/tcpip/transport/icmp/endpoint_state.go | 2 +- pkg/tcpip/transport/icmp/protocol.go | 2 +- pkg/tcpip/transport/raw/raw.go | 2 +- pkg/tcpip/transport/raw/state.go | 2 +- pkg/tcpip/transport/tcp/accept.go | 2 +- pkg/tcpip/transport/tcp/connect.go | 2 +- pkg/tcpip/transport/tcp/cubic.go | 2 +- pkg/tcpip/transport/tcp/dual_stack_test.go | 2 +- pkg/tcpip/transport/tcp/endpoint.go | 2 +- pkg/tcpip/transport/tcp/endpoint_state.go | 2 +- pkg/tcpip/transport/tcp/forwarder.go | 2 +- pkg/tcpip/transport/tcp/protocol.go | 2 +- pkg/tcpip/transport/tcp/rcv.go | 2 +- pkg/tcpip/transport/tcp/reno.go | 2 +- pkg/tcpip/transport/tcp/sack.go | 2 +- pkg/tcpip/transport/tcp/sack_scoreboard.go | 2 +- pkg/tcpip/transport/tcp/sack_scoreboard_test.go | 2 +- pkg/tcpip/transport/tcp/segment.go | 2 +- pkg/tcpip/transport/tcp/segment_heap.go | 2 +- pkg/tcpip/transport/tcp/segment_queue.go | 2 +- pkg/tcpip/transport/tcp/segment_state.go | 2 +- pkg/tcpip/transport/tcp/snd.go | 2 +- pkg/tcpip/transport/tcp/snd_state.go | 2 +- pkg/tcpip/transport/tcp/tcp_sack_test.go | 2 +- pkg/tcpip/transport/tcp/tcp_test.go | 2 +- pkg/tcpip/transport/tcp/tcp_timestamp_test.go | 2 +- pkg/tcpip/transport/tcp/testing/context/context.go | 2 +- pkg/tcpip/transport/tcp/timer.go | 2 +- pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go | 2 +- pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go | 2 +- pkg/tcpip/transport/udp/endpoint.go | 2 +- pkg/tcpip/transport/udp/endpoint_state.go | 2 +- pkg/tcpip/transport/udp/forwarder.go | 2 +- pkg/tcpip/transport/udp/protocol.go | 2 +- pkg/tcpip/transport/udp/udp_test.go | 2 +- pkg/tmutex/tmutex.go | 2 +- pkg/tmutex/tmutex_test.go | 2 +- pkg/unet/unet.go | 2 +- pkg/unet/unet_test.go | 2 +- pkg/unet/unet_unsafe.go | 2 +- pkg/urpc/urpc.go | 2 +- pkg/urpc/urpc_test.go | 2 +- pkg/waiter/waiter.go | 2 +- pkg/waiter/waiter_test.go | 2 +- runsc/boot/compat.go | 2 +- runsc/boot/compat_amd64.go | 2 +- runsc/boot/compat_test.go | 2 +- runsc/boot/config.go | 2 +- runsc/boot/controller.go | 2 +- runsc/boot/debug.go | 2 +- runsc/boot/events.go | 2 +- runsc/boot/fds.go | 2 +- runsc/boot/filter/config.go | 2 +- runsc/boot/filter/extra_filters.go | 2 +- runsc/boot/filter/extra_filters_msan.go | 2 +- runsc/boot/filter/extra_filters_race.go | 2 +- runsc/boot/filter/filter.go | 2 +- runsc/boot/fs.go | 2 +- runsc/boot/limits.go | 2 +- runsc/boot/loader.go | 2 +- runsc/boot/loader_test.go | 2 +- runsc/boot/network.go | 2 +- runsc/boot/strace.go | 2 +- runsc/cgroup/cgroup.go | 2 +- runsc/cgroup/cgroup_test.go | 2 +- runsc/cmd/boot.go | 2 +- runsc/cmd/capability.go | 2 +- runsc/cmd/capability_test.go | 2 +- runsc/cmd/checkpoint.go | 2 +- runsc/cmd/chroot.go | 2 +- runsc/cmd/cmd.go | 2 +- runsc/cmd/create.go | 2 +- runsc/cmd/debug.go | 2 +- runsc/cmd/delete.go | 2 +- runsc/cmd/delete_test.go | 2 +- runsc/cmd/do.go | 2 +- runsc/cmd/events.go | 2 +- runsc/cmd/exec.go | 2 +- runsc/cmd/exec_test.go | 2 +- runsc/cmd/gofer.go | 2 +- runsc/cmd/gofer_test.go | 2 +- runsc/cmd/kill.go | 2 +- runsc/cmd/list.go | 2 +- runsc/cmd/path.go | 2 +- runsc/cmd/pause.go | 2 +- runsc/cmd/ps.go | 2 +- runsc/cmd/restore.go | 2 +- runsc/cmd/resume.go | 2 +- runsc/cmd/run.go | 2 +- runsc/cmd/spec.go | 2 +- runsc/cmd/start.go | 2 +- runsc/cmd/state.go | 2 +- runsc/cmd/wait.go | 2 +- runsc/console/console.go | 2 +- runsc/container/console_test.go | 2 +- runsc/container/container.go | 2 +- runsc/container/container_test.go | 2 +- runsc/container/hook.go | 2 +- runsc/container/multi_container_test.go | 2 +- runsc/container/shared_volume_test.go | 2 +- runsc/container/status.go | 2 +- runsc/container/test_app.go | 2 +- runsc/fsgofer/filter/config.go | 2 +- runsc/fsgofer/filter/extra_filters.go | 2 +- runsc/fsgofer/filter/extra_filters_msan.go | 2 +- runsc/fsgofer/filter/extra_filters_race.go | 2 +- runsc/fsgofer/filter/filter.go | 2 +- runsc/fsgofer/fsgofer.go | 2 +- runsc/fsgofer/fsgofer_test.go | 2 +- runsc/fsgofer/fsgofer_unsafe.go | 2 +- runsc/main.go | 2 +- runsc/sandbox/network.go | 2 +- runsc/sandbox/network_unsafe.go | 2 +- runsc/sandbox/sandbox.go | 2 +- runsc/specutils/fs.go | 2 +- runsc/specutils/namespace.go | 2 +- runsc/specutils/specutils.go | 2 +- runsc/specutils/specutils_test.go | 2 +- runsc/test/image/image.go | 2 +- runsc/test/image/image_test.go | 2 +- runsc/test/image/mysql.sql | 2 +- runsc/test/image/ruby.rb | 2 +- runsc/test/image/ruby.sh | 2 +- runsc/test/install.sh | 2 +- runsc/test/integration/exec_test.go | 2 +- runsc/test/integration/integration.go | 2 +- runsc/test/integration/integration_test.go | 2 +- runsc/test/root/cgroup_test.go | 2 +- runsc/test/root/chroot_test.go | 2 +- runsc/test/root/crictl_test.go | 2 +- runsc/test/root/root.go | 2 +- runsc/test/root/testdata/busybox.go | 2 +- runsc/test/root/testdata/containerd_config.go | 2 +- runsc/test/root/testdata/httpd.go | 2 +- runsc/test/root/testdata/httpd_mount_paths.go | 2 +- runsc/test/root/testdata/sandbox.go | 2 +- runsc/test/testutil/crictl.go | 2 +- runsc/test/testutil/docker.go | 2 +- runsc/test/testutil/testutil.go | 2 +- runsc/test/testutil/testutil_race.go | 2 +- runsc/tools/dockercfg/dockercfg.go | 2 +- runsc/version.go | 2 +- test/syscalls/gtest/gtest.go | 2 +- test/syscalls/linux/32bit.cc | 2 +- test/syscalls/linux/accept_bind.cc | 2 +- test/syscalls/linux/accept_bind_stream.cc | 2 +- test/syscalls/linux/access.cc | 2 +- test/syscalls/linux/affinity.cc | 2 +- test/syscalls/linux/aio.cc | 2 +- test/syscalls/linux/alarm.cc | 2 +- test/syscalls/linux/arch_prctl.cc | 2 +- test/syscalls/linux/bad.cc | 2 +- test/syscalls/linux/base_poll_test.cc | 2 +- test/syscalls/linux/base_poll_test.h | 2 +- test/syscalls/linux/bind.cc | 2 +- test/syscalls/linux/brk.cc | 2 +- test/syscalls/linux/chdir.cc | 2 +- test/syscalls/linux/chmod.cc | 2 +- test/syscalls/linux/chown.cc | 2 +- test/syscalls/linux/chroot.cc | 2 +- test/syscalls/linux/clock_getres.cc | 2 +- test/syscalls/linux/clock_gettime.cc | 2 +- test/syscalls/linux/clock_nanosleep.cc | 2 +- test/syscalls/linux/concurrency.cc | 2 +- test/syscalls/linux/creat.cc | 2 +- test/syscalls/linux/dev.cc | 2 +- test/syscalls/linux/dup.cc | 2 +- test/syscalls/linux/epoll.cc | 2 +- test/syscalls/linux/eventfd.cc | 2 +- test/syscalls/linux/exceptions.cc | 2 +- test/syscalls/linux/exec.cc | 2 +- test/syscalls/linux/exec.h | 2 +- test/syscalls/linux/exec_assert_closed_workload.cc | 2 +- test/syscalls/linux/exec_basic_workload.cc | 2 +- test/syscalls/linux/exec_binary.cc | 2 +- test/syscalls/linux/exec_proc_exe_workload.cc | 2 +- test/syscalls/linux/exec_state_workload.cc | 2 +- test/syscalls/linux/exit.cc | 2 +- test/syscalls/linux/exit_script.sh | 2 +- test/syscalls/linux/fadvise64.cc | 2 +- test/syscalls/linux/fallocate.cc | 2 +- test/syscalls/linux/fault.cc | 2 +- test/syscalls/linux/fchdir.cc | 2 +- test/syscalls/linux/fcntl.cc | 2 +- test/syscalls/linux/file_base.h | 2 +- test/syscalls/linux/flock.cc | 2 +- test/syscalls/linux/fork.cc | 2 +- test/syscalls/linux/fpsig_fork.cc | 2 +- test/syscalls/linux/fpsig_nested.cc | 2 +- test/syscalls/linux/fsync.cc | 2 +- test/syscalls/linux/futex.cc | 2 +- test/syscalls/linux/getcpu.cc | 2 +- test/syscalls/linux/getdents.cc | 2 +- test/syscalls/linux/getrandom.cc | 2 +- test/syscalls/linux/getrusage.cc | 2 +- test/syscalls/linux/inotify.cc | 2 +- test/syscalls/linux/ioctl.cc | 2 +- test/syscalls/linux/ip_socket_test_util.cc | 2 +- test/syscalls/linux/ip_socket_test_util.h | 2 +- test/syscalls/linux/itimer.cc | 2 +- test/syscalls/linux/kill.cc | 2 +- test/syscalls/linux/link.cc | 2 +- test/syscalls/linux/lseek.cc | 2 +- test/syscalls/linux/madvise.cc | 2 +- test/syscalls/linux/memfd.cc | 2 +- test/syscalls/linux/memory_accounting.cc | 2 +- test/syscalls/linux/mempolicy.cc | 2 +- test/syscalls/linux/mincore.cc | 2 +- test/syscalls/linux/mkdir.cc | 2 +- test/syscalls/linux/mknod.cc | 2 +- test/syscalls/linux/mlock.cc | 2 +- test/syscalls/linux/mmap.cc | 2 +- test/syscalls/linux/mount.cc | 2 +- test/syscalls/linux/mremap.cc | 2 +- test/syscalls/linux/msync.cc | 2 +- test/syscalls/linux/munmap.cc | 2 +- test/syscalls/linux/open.cc | 2 +- test/syscalls/linux/open_create.cc | 2 +- test/syscalls/linux/partial_bad_buffer.cc | 2 +- test/syscalls/linux/pause.cc | 2 +- test/syscalls/linux/pipe.cc | 2 +- test/syscalls/linux/poll.cc | 2 +- test/syscalls/linux/ppoll.cc | 2 +- test/syscalls/linux/prctl.cc | 2 +- test/syscalls/linux/prctl_setuid.cc | 2 +- test/syscalls/linux/pread64.cc | 2 +- test/syscalls/linux/preadv.cc | 2 +- test/syscalls/linux/preadv2.cc | 2 +- test/syscalls/linux/priority.cc | 2 +- test/syscalls/linux/priority_execve.cc | 2 +- test/syscalls/linux/proc.cc | 2 +- test/syscalls/linux/proc_net.cc | 2 +- test/syscalls/linux/proc_net_unix.cc | 2 +- test/syscalls/linux/proc_pid_smaps.cc | 2 +- test/syscalls/linux/proc_pid_uid_gid_map.cc | 2 +- test/syscalls/linux/pselect.cc | 2 +- test/syscalls/linux/ptrace.cc | 2 +- test/syscalls/linux/pty.cc | 2 +- test/syscalls/linux/pwrite64.cc | 2 +- test/syscalls/linux/pwritev2.cc | 2 +- test/syscalls/linux/raw_socket_ipv4.cc | 2 +- test/syscalls/linux/read.cc | 2 +- test/syscalls/linux/readv.cc | 2 +- test/syscalls/linux/readv_common.cc | 2 +- test/syscalls/linux/readv_common.h | 2 +- test/syscalls/linux/readv_socket.cc | 2 +- test/syscalls/linux/rename.cc | 2 +- test/syscalls/linux/rlimits.cc | 2 +- test/syscalls/linux/rtsignal.cc | 2 +- test/syscalls/linux/sched.cc | 2 +- test/syscalls/linux/sched_yield.cc | 2 +- test/syscalls/linux/seccomp.cc | 2 +- test/syscalls/linux/select.cc | 2 +- test/syscalls/linux/semaphore.cc | 2 +- test/syscalls/linux/sendfile.cc | 2 +- test/syscalls/linux/sendfile_socket.cc | 2 +- test/syscalls/linux/shm.cc | 2 +- test/syscalls/linux/sigaction.cc | 2 +- test/syscalls/linux/sigaltstack.cc | 2 +- test/syscalls/linux/sigaltstack_check.cc | 2 +- test/syscalls/linux/sigiret.cc | 2 +- test/syscalls/linux/sigprocmask.cc | 2 +- test/syscalls/linux/sigstop.cc | 2 +- test/syscalls/linux/sigtimedwait.cc | 2 +- test/syscalls/linux/socket_abstract.cc | 2 +- test/syscalls/linux/socket_blocking.cc | 2 +- test/syscalls/linux/socket_blocking.h | 2 +- test/syscalls/linux/socket_filesystem.cc | 2 +- test/syscalls/linux/socket_generic.cc | 2 +- test/syscalls/linux/socket_generic.h | 2 +- test/syscalls/linux/socket_inet_loopback.cc | 2 +- test/syscalls/linux/socket_ip_loopback_blocking.cc | 2 +- test/syscalls/linux/socket_ip_tcp_generic.cc | 2 +- test/syscalls/linux/socket_ip_tcp_generic.h | 2 +- test/syscalls/linux/socket_ip_tcp_generic_loopback.cc | 2 +- test/syscalls/linux/socket_ip_tcp_loopback.cc | 2 +- test/syscalls/linux/socket_ip_tcp_loopback_blocking.cc | 2 +- test/syscalls/linux/socket_ip_tcp_loopback_nonblock.cc | 2 +- test/syscalls/linux/socket_ip_tcp_udp_generic.cc | 2 +- test/syscalls/linux/socket_ip_udp_generic.cc | 2 +- test/syscalls/linux/socket_ip_udp_generic.h | 2 +- test/syscalls/linux/socket_ip_udp_loopback.cc | 2 +- test/syscalls/linux/socket_ip_udp_loopback_blocking.cc | 2 +- test/syscalls/linux/socket_ip_udp_loopback_nonblock.cc | 2 +- .../syscalls/linux/socket_ipv4_tcp_unbound_external_networking.cc | 2 +- test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.h | 2 +- .../linux/socket_ipv4_tcp_unbound_external_networking_test.cc | 2 +- test/syscalls/linux/socket_ipv4_udp_unbound.cc | 2 +- test/syscalls/linux/socket_ipv4_udp_unbound.h | 2 +- .../syscalls/linux/socket_ipv4_udp_unbound_external_networking.cc | 2 +- test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.h | 2 +- .../linux/socket_ipv4_udp_unbound_external_networking_test.cc | 2 +- test/syscalls/linux/socket_ipv4_udp_unbound_loopback.cc | 2 +- test/syscalls/linux/socket_netdevice.cc | 2 +- test/syscalls/linux/socket_netlink_route.cc | 2 +- test/syscalls/linux/socket_netlink_util.cc | 2 +- test/syscalls/linux/socket_netlink_util.h | 2 +- test/syscalls/linux/socket_non_blocking.cc | 2 +- test/syscalls/linux/socket_non_blocking.h | 2 +- test/syscalls/linux/socket_non_stream.cc | 2 +- test/syscalls/linux/socket_non_stream.h | 2 +- test/syscalls/linux/socket_non_stream_blocking.cc | 2 +- test/syscalls/linux/socket_non_stream_blocking.h | 2 +- test/syscalls/linux/socket_stream.cc | 2 +- test/syscalls/linux/socket_stream.h | 2 +- test/syscalls/linux/socket_stream_blocking.cc | 2 +- test/syscalls/linux/socket_stream_blocking.h | 2 +- test/syscalls/linux/socket_stream_nonblock.cc | 2 +- test/syscalls/linux/socket_stream_nonblock.h | 2 +- test/syscalls/linux/socket_test_util.cc | 2 +- test/syscalls/linux/socket_test_util.h | 2 +- test/syscalls/linux/socket_unix.cc | 2 +- test/syscalls/linux/socket_unix.h | 2 +- test/syscalls/linux/socket_unix_abstract.cc | 2 +- test/syscalls/linux/socket_unix_abstract_nonblock.cc | 2 +- test/syscalls/linux/socket_unix_blocking_local.cc | 2 +- test/syscalls/linux/socket_unix_dgram.cc | 2 +- test/syscalls/linux/socket_unix_dgram.h | 2 +- test/syscalls/linux/socket_unix_dgram_local.cc | 2 +- test/syscalls/linux/socket_unix_dgram_non_blocking.cc | 2 +- test/syscalls/linux/socket_unix_domain.cc | 2 +- test/syscalls/linux/socket_unix_filesystem.cc | 2 +- test/syscalls/linux/socket_unix_filesystem_nonblock.cc | 2 +- test/syscalls/linux/socket_unix_non_stream.cc | 2 +- test/syscalls/linux/socket_unix_non_stream.h | 2 +- test/syscalls/linux/socket_unix_non_stream_blocking_local.cc | 2 +- test/syscalls/linux/socket_unix_pair.cc | 2 +- test/syscalls/linux/socket_unix_pair_nonblock.cc | 2 +- test/syscalls/linux/socket_unix_seqpacket.cc | 2 +- test/syscalls/linux/socket_unix_seqpacket.h | 2 +- test/syscalls/linux/socket_unix_seqpacket_local.cc | 2 +- test/syscalls/linux/socket_unix_stream.cc | 2 +- test/syscalls/linux/socket_unix_stream_blocking_local.cc | 2 +- test/syscalls/linux/socket_unix_stream_local.cc | 2 +- test/syscalls/linux/socket_unix_stream_nonblock_local.cc | 2 +- test/syscalls/linux/socket_unix_unbound_abstract.cc | 2 +- test/syscalls/linux/socket_unix_unbound_dgram.cc | 2 +- test/syscalls/linux/socket_unix_unbound_filesystem.cc | 2 +- test/syscalls/linux/socket_unix_unbound_seqpacket.cc | 2 +- test/syscalls/linux/socket_unix_unbound_stream.cc | 2 +- test/syscalls/linux/stat.cc | 2 +- test/syscalls/linux/stat_times.cc | 2 +- test/syscalls/linux/statfs.cc | 2 +- test/syscalls/linux/sticky.cc | 2 +- test/syscalls/linux/symlink.cc | 2 +- test/syscalls/linux/sync.cc | 2 +- test/syscalls/linux/sync_file_range.cc | 2 +- test/syscalls/linux/sysinfo.cc | 2 +- test/syscalls/linux/syslog.cc | 2 +- test/syscalls/linux/sysret.cc | 2 +- test/syscalls/linux/tcp_socket.cc | 2 +- test/syscalls/linux/temp_umask.h | 2 +- test/syscalls/linux/tgkill.cc | 2 +- test/syscalls/linux/time.cc | 2 +- test/syscalls/linux/timerfd.cc | 2 +- test/syscalls/linux/timers.cc | 2 +- test/syscalls/linux/tkill.cc | 2 +- test/syscalls/linux/truncate.cc | 2 +- test/syscalls/linux/udp_bind.cc | 2 +- test/syscalls/linux/udp_socket.cc | 2 +- test/syscalls/linux/uidgid.cc | 2 +- test/syscalls/linux/uname.cc | 2 +- test/syscalls/linux/unix_domain_socket_test_util.cc | 2 +- test/syscalls/linux/unix_domain_socket_test_util.h | 2 +- test/syscalls/linux/unlink.cc | 2 +- test/syscalls/linux/unshare.cc | 2 +- test/syscalls/linux/utimes.cc | 2 +- test/syscalls/linux/vdso.cc | 2 +- test/syscalls/linux/vdso_clock_gettime.cc | 2 +- test/syscalls/linux/vfork.cc | 2 +- test/syscalls/linux/vsyscall.cc | 2 +- test/syscalls/linux/wait.cc | 2 +- test/syscalls/linux/write.cc | 2 +- test/syscalls/syscall_test_runner.go | 2 +- test/syscalls/syscall_test_runner.sh | 2 +- test/util/capability_util.cc | 2 +- test/util/capability_util.h | 2 +- test/util/cleanup.h | 2 +- test/util/epoll_util.cc | 2 +- test/util/epoll_util.h | 2 +- test/util/eventfd_util.h | 2 +- test/util/file_descriptor.h | 2 +- test/util/fs_util.cc | 2 +- test/util/fs_util.h | 2 +- test/util/fs_util_test.cc | 2 +- test/util/logging.cc | 2 +- test/util/logging.h | 2 +- test/util/memory_util.h | 2 +- test/util/mount_util.h | 2 +- test/util/multiprocess_util.cc | 2 +- test/util/multiprocess_util.h | 2 +- test/util/posix_error.cc | 2 +- test/util/posix_error.h | 2 +- test/util/posix_error_test.cc | 2 +- test/util/proc_util.cc | 2 +- test/util/proc_util.h | 2 +- test/util/proc_util_test.cc | 2 +- test/util/rlimit_util.cc | 2 +- test/util/rlimit_util.h | 2 +- test/util/save_util.cc | 2 +- test/util/save_util.h | 2 +- test/util/signal_util.cc | 2 +- test/util/signal_util.h | 2 +- test/util/temp_path.cc | 2 +- test/util/temp_path.h | 2 +- test/util/test_main.cc | 2 +- test/util/test_util.cc | 2 +- test/util/test_util.h | 2 +- test/util/test_util_test.cc | 2 +- test/util/thread_util.h | 2 +- test/util/timer_util.cc | 2 +- test/util/timer_util.h | 2 +- third_party/gvsync/atomicptr_unsafe.go | 2 +- third_party/gvsync/atomicptrtest/atomicptr_test.go | 2 +- third_party/gvsync/downgradable_rwmutex_test.go | 2 +- third_party/gvsync/downgradable_rwmutex_unsafe.go | 2 +- third_party/gvsync/gvsync.go | 2 +- third_party/gvsync/memmove_unsafe.go | 2 +- third_party/gvsync/norace_unsafe.go | 2 +- third_party/gvsync/race_unsafe.go | 2 +- third_party/gvsync/seqatomic_unsafe.go | 2 +- third_party/gvsync/seqatomictest/seqatomic_test.go | 2 +- third_party/gvsync/seqcount.go | 2 +- third_party/gvsync/seqcount_test.go | 2 +- tools/go_generics/generics.go | 2 +- tools/go_generics/generics_tests/all_stmts/input.go | 2 +- tools/go_generics/generics_tests/all_stmts/output/output.go | 2 +- tools/go_generics/generics_tests/all_types/input.go | 2 +- tools/go_generics/generics_tests/all_types/lib/lib.go | 2 +- tools/go_generics/generics_tests/all_types/output/output.go | 2 +- tools/go_generics/generics_tests/consts/input.go | 2 +- tools/go_generics/generics_tests/consts/output/output.go | 2 +- tools/go_generics/generics_tests/imports/input.go | 2 +- tools/go_generics/generics_tests/imports/output/output.go | 2 +- tools/go_generics/generics_tests/remove_typedef/input.go | 2 +- tools/go_generics/generics_tests/remove_typedef/output/output.go | 2 +- tools/go_generics/generics_tests/simple/input.go | 2 +- tools/go_generics/generics_tests/simple/output/output.go | 2 +- tools/go_generics/globals/globals_visitor.go | 2 +- tools/go_generics/globals/scope.go | 2 +- tools/go_generics/go_generics_unittest.sh | 2 +- tools/go_generics/go_merge/main.go | 2 +- tools/go_generics/imports.go | 2 +- tools/go_generics/remove.go | 2 +- tools/go_generics/rules_tests/template.go | 2 +- tools/go_generics/rules_tests/template_test.go | 2 +- tools/go_stateify/main.go | 2 +- tools/tag_release.sh | 2 +- tools/workspace_status.sh | 2 +- vdso/barrier.h | 2 +- vdso/check_vdso.py | 2 +- vdso/compiler.h | 2 +- vdso/cycle_clock.h | 2 +- vdso/seqlock.h | 2 +- vdso/syscalls.h | 2 +- vdso/vdso.cc | 2 +- vdso/vdso_time.cc | 2 +- vdso/vdso_time.h | 2 +- 1235 files changed, 1242 insertions(+), 1234 deletions(-) create mode 100644 AUTHORS (limited to 'pkg/state') diff --git a/AUTHORS b/AUTHORS new file mode 100644 index 000000000..01ba46567 --- /dev/null +++ b/AUTHORS @@ -0,0 +1,8 @@ +# This is the list of gVisor authors for copyright purposes. +# +# This does not necessarily list everyone who has contributed code, since in +# some cases, their employer may be the copyright holder. To see the full list +# of contributors, see the revision history in source control. +# +# Please send a patch if you would like to be included in this list. +Google LLC diff --git a/kokoro/run_build.sh b/kokoro/run_build.sh index 89e24b037..63fffda48 100755 --- a/kokoro/run_build.sh +++ b/kokoro/run_build.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google LLC +# Copyright 2018 The gVisor Authors. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/kokoro/run_tests.sh b/kokoro/run_tests.sh index 8a3ce7402..08f678e39 100755 --- a/kokoro/run_tests.sh +++ b/kokoro/run_tests.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google LLC +# Copyright 2018 The gVisor Authors. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/pkg/abi/abi.go b/pkg/abi/abi.go index 7770f0405..d56c481c9 100644 --- a/pkg/abi/abi.go +++ b/pkg/abi/abi.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/abi_linux.go b/pkg/abi/abi_linux.go index 9d9f361a4..3059479bd 100644 --- a/pkg/abi/abi_linux.go +++ b/pkg/abi/abi_linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/flag.go b/pkg/abi/flag.go index b48757da8..dcdd66d4e 100644 --- a/pkg/abi/flag.go +++ b/pkg/abi/flag.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/aio.go b/pkg/abi/linux/aio.go index 1b7ca714a..3c6e0079d 100644 --- a/pkg/abi/linux/aio.go +++ b/pkg/abi/linux/aio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/ashmem.go b/pkg/abi/linux/ashmem.go index ced1e44d4..2a722abe0 100644 --- a/pkg/abi/linux/ashmem.go +++ b/pkg/abi/linux/ashmem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/audit.go b/pkg/abi/linux/audit.go index b39ba4515..6cca69af9 100644 --- a/pkg/abi/linux/audit.go +++ b/pkg/abi/linux/audit.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/binder.go b/pkg/abi/linux/binder.go index 522dc6f53..63b08324a 100644 --- a/pkg/abi/linux/binder.go +++ b/pkg/abi/linux/binder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/bpf.go b/pkg/abi/linux/bpf.go index d9cd09948..aa3d3ce70 100644 --- a/pkg/abi/linux/bpf.go +++ b/pkg/abi/linux/bpf.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/capability.go b/pkg/abi/linux/capability.go index 7d96f013e..c120cac64 100644 --- a/pkg/abi/linux/capability.go +++ b/pkg/abi/linux/capability.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/dev.go b/pkg/abi/linux/dev.go index 5b1199aac..421e11256 100644 --- a/pkg/abi/linux/dev.go +++ b/pkg/abi/linux/dev.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/elf.go b/pkg/abi/linux/elf.go index 928067c04..fb1c679d2 100644 --- a/pkg/abi/linux/elf.go +++ b/pkg/abi/linux/elf.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/errors.go b/pkg/abi/linux/errors.go index e5f6f3f07..93f85a864 100644 --- a/pkg/abi/linux/errors.go +++ b/pkg/abi/linux/errors.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/eventfd.go b/pkg/abi/linux/eventfd.go index 5614f5cf1..9c479fc8f 100644 --- a/pkg/abi/linux/eventfd.go +++ b/pkg/abi/linux/eventfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/exec.go b/pkg/abi/linux/exec.go index a07c29243..579d46c41 100644 --- a/pkg/abi/linux/exec.go +++ b/pkg/abi/linux/exec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/fcntl.go b/pkg/abi/linux/fcntl.go index c8558933a..cc8f2702d 100644 --- a/pkg/abi/linux/fcntl.go +++ b/pkg/abi/linux/fcntl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/file.go b/pkg/abi/linux/file.go index 46b10ca97..753fec3ed 100644 --- a/pkg/abi/linux/file.go +++ b/pkg/abi/linux/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/fs.go b/pkg/abi/linux/fs.go index a9f2ba132..c82ab9b5b 100644 --- a/pkg/abi/linux/fs.go +++ b/pkg/abi/linux/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/futex.go b/pkg/abi/linux/futex.go index afdf4123b..08bfde3b5 100644 --- a/pkg/abi/linux/futex.go +++ b/pkg/abi/linux/futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/inotify.go b/pkg/abi/linux/inotify.go index 79c5d3593..2d08194ba 100644 --- a/pkg/abi/linux/inotify.go +++ b/pkg/abi/linux/inotify.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/ioctl.go b/pkg/abi/linux/ioctl.go index 191b26e4d..04bb767dc 100644 --- a/pkg/abi/linux/ioctl.go +++ b/pkg/abi/linux/ioctl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/ip.go b/pkg/abi/linux/ip.go index 77ac1062c..31e56ffa6 100644 --- a/pkg/abi/linux/ip.go +++ b/pkg/abi/linux/ip.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/ipc.go b/pkg/abi/linux/ipc.go index 10681768b..2ef8d6cbb 100644 --- a/pkg/abi/linux/ipc.go +++ b/pkg/abi/linux/ipc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/limits.go b/pkg/abi/linux/limits.go index e0aa5b31d..c74dfcd53 100644 --- a/pkg/abi/linux/limits.go +++ b/pkg/abi/linux/limits.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/linux.go b/pkg/abi/linux/linux.go index d365f693d..8a8f831cd 100644 --- a/pkg/abi/linux/linux.go +++ b/pkg/abi/linux/linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/mm.go b/pkg/abi/linux/mm.go index eda8d9788..0b02f938a 100644 --- a/pkg/abi/linux/mm.go +++ b/pkg/abi/linux/mm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/netdevice.go b/pkg/abi/linux/netdevice.go index e3b6b1e40..aef1acf75 100644 --- a/pkg/abi/linux/netdevice.go +++ b/pkg/abi/linux/netdevice.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/netlink.go b/pkg/abi/linux/netlink.go index 25c5e17fd..5e718c363 100644 --- a/pkg/abi/linux/netlink.go +++ b/pkg/abi/linux/netlink.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/netlink_route.go b/pkg/abi/linux/netlink_route.go index 4200b6506..630dc339a 100644 --- a/pkg/abi/linux/netlink_route.go +++ b/pkg/abi/linux/netlink_route.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/poll.go b/pkg/abi/linux/poll.go index 9f0b15d1c..c04d26e4c 100644 --- a/pkg/abi/linux/poll.go +++ b/pkg/abi/linux/poll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/prctl.go b/pkg/abi/linux/prctl.go index db3206f36..dae2de290 100644 --- a/pkg/abi/linux/prctl.go +++ b/pkg/abi/linux/prctl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/ptrace.go b/pkg/abi/linux/ptrace.go index 7db4f5464..23e605ab2 100644 --- a/pkg/abi/linux/ptrace.go +++ b/pkg/abi/linux/ptrace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/rusage.go b/pkg/abi/linux/rusage.go index 7fea4b589..d8302dc85 100644 --- a/pkg/abi/linux/rusage.go +++ b/pkg/abi/linux/rusage.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/sched.go b/pkg/abi/linux/sched.go index ef96a3801..193d9a242 100644 --- a/pkg/abi/linux/sched.go +++ b/pkg/abi/linux/sched.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/seccomp.go b/pkg/abi/linux/seccomp.go index 8673a27bf..4eeb5cd7a 100644 --- a/pkg/abi/linux/seccomp.go +++ b/pkg/abi/linux/seccomp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/sem.go b/pkg/abi/linux/sem.go index b80c93daf..de422c519 100644 --- a/pkg/abi/linux/sem.go +++ b/pkg/abi/linux/sem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/shm.go b/pkg/abi/linux/shm.go index 82a80e609..e45aadb10 100644 --- a/pkg/abi/linux/shm.go +++ b/pkg/abi/linux/shm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/signal.go b/pkg/abi/linux/signal.go index 395f9f31e..9cbd77dda 100644 --- a/pkg/abi/linux/signal.go +++ b/pkg/abi/linux/signal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/socket.go b/pkg/abi/linux/socket.go index 6fa4e7c3e..417840731 100644 --- a/pkg/abi/linux/socket.go +++ b/pkg/abi/linux/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/tcp.go b/pkg/abi/linux/tcp.go index 67908deb9..174d470e2 100644 --- a/pkg/abi/linux/tcp.go +++ b/pkg/abi/linux/tcp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/time.go b/pkg/abi/linux/time.go index bbd21e726..fa9ee27e1 100644 --- a/pkg/abi/linux/time.go +++ b/pkg/abi/linux/time.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/timer.go b/pkg/abi/linux/timer.go index a6f420bdb..e32d09e10 100644 --- a/pkg/abi/linux/timer.go +++ b/pkg/abi/linux/timer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/tty.go b/pkg/abi/linux/tty.go index bff882d89..8ac02aee8 100644 --- a/pkg/abi/linux/tty.go +++ b/pkg/abi/linux/tty.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/uio.go b/pkg/abi/linux/uio.go index 7e00d9959..1fd1e9802 100644 --- a/pkg/abi/linux/uio.go +++ b/pkg/abi/linux/uio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/utsname.go b/pkg/abi/linux/utsname.go index f80ed7d4a..60f220a67 100644 --- a/pkg/abi/linux/utsname.go +++ b/pkg/abi/linux/utsname.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/amutex/amutex.go b/pkg/amutex/amutex.go index 26b674435..85e819304 100644 --- a/pkg/amutex/amutex.go +++ b/pkg/amutex/amutex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/amutex/amutex_test.go b/pkg/amutex/amutex_test.go index 104e0dab1..6a0af006e 100644 --- a/pkg/amutex/amutex_test.go +++ b/pkg/amutex/amutex_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/atomicbitops/atomic_bitops.go b/pkg/atomicbitops/atomic_bitops.go index 9a57f9599..63aa2b7f1 100644 --- a/pkg/atomicbitops/atomic_bitops.go +++ b/pkg/atomicbitops/atomic_bitops.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/atomicbitops/atomic_bitops_amd64.s b/pkg/atomicbitops/atomic_bitops_amd64.s index b37e3aad3..db0972001 100644 --- a/pkg/atomicbitops/atomic_bitops_amd64.s +++ b/pkg/atomicbitops/atomic_bitops_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/atomicbitops/atomic_bitops_common.go b/pkg/atomicbitops/atomic_bitops_common.go index b03242baa..b2a943dcb 100644 --- a/pkg/atomicbitops/atomic_bitops_common.go +++ b/pkg/atomicbitops/atomic_bitops_common.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/atomicbitops/atomic_bitops_test.go b/pkg/atomicbitops/atomic_bitops_test.go index ee6207cb3..965e9be79 100644 --- a/pkg/atomicbitops/atomic_bitops_test.go +++ b/pkg/atomicbitops/atomic_bitops_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/binary/binary.go b/pkg/binary/binary.go index 02f7e9fb8..631785f7b 100644 --- a/pkg/binary/binary.go +++ b/pkg/binary/binary.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/binary/binary_test.go b/pkg/binary/binary_test.go index 200961c70..4d609a438 100644 --- a/pkg/binary/binary_test.go +++ b/pkg/binary/binary_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/bits.go b/pkg/bits/bits.go index eb3c80f49..a26433ad6 100644 --- a/pkg/bits/bits.go +++ b/pkg/bits/bits.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/bits_template.go b/pkg/bits/bits_template.go index 8c578cca2..93a435b80 100644 --- a/pkg/bits/bits_template.go +++ b/pkg/bits/bits_template.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/uint64_arch_amd64.go b/pkg/bits/uint64_arch_amd64.go index 1fef89394..faccaa61a 100644 --- a/pkg/bits/uint64_arch_amd64.go +++ b/pkg/bits/uint64_arch_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/uint64_arch_amd64_asm.s b/pkg/bits/uint64_arch_amd64_asm.s index 8c7322f0f..8ff364181 100644 --- a/pkg/bits/uint64_arch_amd64_asm.s +++ b/pkg/bits/uint64_arch_amd64_asm.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/uint64_arch_generic.go b/pkg/bits/uint64_arch_generic.go index cfb47400b..7dd2d1480 100644 --- a/pkg/bits/uint64_arch_generic.go +++ b/pkg/bits/uint64_arch_generic.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/uint64_test.go b/pkg/bits/uint64_test.go index d6dbaf602..1b018d808 100644 --- a/pkg/bits/uint64_test.go +++ b/pkg/bits/uint64_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/bpf.go b/pkg/bpf/bpf.go index 98d44d911..eb546f48f 100644 --- a/pkg/bpf/bpf.go +++ b/pkg/bpf/bpf.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/decoder.go b/pkg/bpf/decoder.go index ae6b8839a..45c192215 100644 --- a/pkg/bpf/decoder.go +++ b/pkg/bpf/decoder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/decoder_test.go b/pkg/bpf/decoder_test.go index f093e1e41..8c4bdad21 100644 --- a/pkg/bpf/decoder_test.go +++ b/pkg/bpf/decoder_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/input_bytes.go b/pkg/bpf/input_bytes.go index 745c0749b..86b216cfc 100644 --- a/pkg/bpf/input_bytes.go +++ b/pkg/bpf/input_bytes.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/interpreter.go b/pkg/bpf/interpreter.go index 86c7add4d..86de523a2 100644 --- a/pkg/bpf/interpreter.go +++ b/pkg/bpf/interpreter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/interpreter_test.go b/pkg/bpf/interpreter_test.go index c46a43991..67b00ffe3 100644 --- a/pkg/bpf/interpreter_test.go +++ b/pkg/bpf/interpreter_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/program_builder.go b/pkg/bpf/program_builder.go index b4ce228e1..fc9d27203 100644 --- a/pkg/bpf/program_builder.go +++ b/pkg/bpf/program_builder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/program_builder_test.go b/pkg/bpf/program_builder_test.go index 0e0b79d88..5b2ad67de 100644 --- a/pkg/bpf/program_builder_test.go +++ b/pkg/bpf/program_builder_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/compressio/compressio.go b/pkg/compressio/compressio.go index 4daaa82b6..8c14ccbfa 100644 --- a/pkg/compressio/compressio.go +++ b/pkg/compressio/compressio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/compressio/compressio_test.go b/pkg/compressio/compressio_test.go index 1bbabee79..86dc47e44 100644 --- a/pkg/compressio/compressio_test.go +++ b/pkg/compressio/compressio_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/control/client/client.go b/pkg/control/client/client.go index 0d0c9f148..3fec27846 100644 --- a/pkg/control/client/client.go +++ b/pkg/control/client/client.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/control/server/server.go b/pkg/control/server/server.go index c46b5d70b..1a15da1a8 100644 --- a/pkg/control/server/server.go +++ b/pkg/control/server/server.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/cpuid/cpu_amd64.s b/pkg/cpuid/cpu_amd64.s index 905c1d12e..ac80d3c8a 100644 --- a/pkg/cpuid/cpu_amd64.s +++ b/pkg/cpuid/cpu_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/cpuid/cpuid.go b/pkg/cpuid/cpuid.go index 61441150e..3eb2bcd2b 100644 --- a/pkg/cpuid/cpuid.go +++ b/pkg/cpuid/cpuid.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/cpuid/cpuid_parse_test.go b/pkg/cpuid/cpuid_parse_test.go index e8f87a10e..dd9969db4 100644 --- a/pkg/cpuid/cpuid_parse_test.go +++ b/pkg/cpuid/cpuid_parse_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/cpuid/cpuid_test.go b/pkg/cpuid/cpuid_test.go index 64ade1cbe..6ae14d2da 100644 --- a/pkg/cpuid/cpuid_test.go +++ b/pkg/cpuid/cpuid_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/dhcp/client.go b/pkg/dhcp/client.go index 2ba79be32..b7cde3819 100644 --- a/pkg/dhcp/client.go +++ b/pkg/dhcp/client.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/dhcp/dhcp.go b/pkg/dhcp/dhcp.go index 6945bcd35..f96ffd891 100644 --- a/pkg/dhcp/dhcp.go +++ b/pkg/dhcp/dhcp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/dhcp/dhcp_string.go b/pkg/dhcp/dhcp_string.go index 8533895bd..29ce98593 100644 --- a/pkg/dhcp/dhcp_string.go +++ b/pkg/dhcp/dhcp_string.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/dhcp/dhcp_test.go b/pkg/dhcp/dhcp_test.go index e1d8ef603..751626bb0 100644 --- a/pkg/dhcp/dhcp_test.go +++ b/pkg/dhcp/dhcp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/dhcp/server.go b/pkg/dhcp/server.go index 9549ff705..6a1972860 100644 --- a/pkg/dhcp/server.go +++ b/pkg/dhcp/server.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/eventchannel/event.go b/pkg/eventchannel/event.go index 41a7b5ed3..4c8ae573b 100644 --- a/pkg/eventchannel/event.go +++ b/pkg/eventchannel/event.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/eventchannel/event.proto b/pkg/eventchannel/event.proto index c1679c7e7..34468f072 100644 --- a/pkg/eventchannel/event.proto +++ b/pkg/eventchannel/event.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/fd/fd.go b/pkg/fd/fd.go index d40758c22..2785243a2 100644 --- a/pkg/fd/fd.go +++ b/pkg/fd/fd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/fd/fd_test.go b/pkg/fd/fd_test.go index 42bb3ef6c..5fb0ad47d 100644 --- a/pkg/fd/fd_test.go +++ b/pkg/fd/fd_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/fdnotifier/fdnotifier.go b/pkg/fdnotifier/fdnotifier.go index aa4906ca0..f0b028b0b 100644 --- a/pkg/fdnotifier/fdnotifier.go +++ b/pkg/fdnotifier/fdnotifier.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/fdnotifier/poll_unsafe.go b/pkg/fdnotifier/poll_unsafe.go index 05be9aeb5..bc5e0ac44 100644 --- a/pkg/fdnotifier/poll_unsafe.go +++ b/pkg/fdnotifier/poll_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/gate/gate.go b/pkg/gate/gate.go index 48122bf5a..bda6aae09 100644 --- a/pkg/gate/gate.go +++ b/pkg/gate/gate.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/gate/gate_test.go b/pkg/gate/gate_test.go index 95620fa8e..7467e7d07 100644 --- a/pkg/gate/gate_test.go +++ b/pkg/gate/gate_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/ilist/list.go b/pkg/ilist/list.go index 51c9b6df3..019caadca 100644 --- a/pkg/ilist/list.go +++ b/pkg/ilist/list.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/ilist/list_test.go b/pkg/ilist/list_test.go index f37946dc2..3f9abfb56 100644 --- a/pkg/ilist/list_test.go +++ b/pkg/ilist/list_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/linewriter/linewriter.go b/pkg/linewriter/linewriter.go index 5fbd4e779..cd6e4e2ce 100644 --- a/pkg/linewriter/linewriter.go +++ b/pkg/linewriter/linewriter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/linewriter/linewriter_test.go b/pkg/linewriter/linewriter_test.go index 9140ee6af..96dc7e6e0 100644 --- a/pkg/linewriter/linewriter_test.go +++ b/pkg/linewriter/linewriter_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/glog.go b/pkg/log/glog.go index 24d5390d7..5732785b4 100644 --- a/pkg/log/glog.go +++ b/pkg/log/glog.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/glog_unsafe.go b/pkg/log/glog_unsafe.go index bb06aa7d3..ea17ae349 100644 --- a/pkg/log/glog_unsafe.go +++ b/pkg/log/glog_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/json.go b/pkg/log/json.go index 96bd13d87..a278c8fc8 100644 --- a/pkg/log/json.go +++ b/pkg/log/json.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/json_k8s.go b/pkg/log/json_k8s.go index 9c2f8d2b7..c2c019915 100644 --- a/pkg/log/json_k8s.go +++ b/pkg/log/json_k8s.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/json_test.go b/pkg/log/json_test.go index b8c7a795e..f25224fe1 100644 --- a/pkg/log/json_test.go +++ b/pkg/log/json_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/log.go b/pkg/log/log.go index b8d456aae..7d563241e 100644 --- a/pkg/log/log.go +++ b/pkg/log/log.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/log_test.go b/pkg/log/log_test.go index a59d457dd..0634e7c1f 100644 --- a/pkg/log/log_test.go +++ b/pkg/log/log_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/metric/metric.go b/pkg/metric/metric.go index e5eb95f89..803709cc4 100644 --- a/pkg/metric/metric.go +++ b/pkg/metric/metric.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/metric/metric.proto b/pkg/metric/metric.proto index 917fda1ac..a2c2bd1ba 100644 --- a/pkg/metric/metric.proto +++ b/pkg/metric/metric.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/metric/metric_test.go b/pkg/metric/metric_test.go index 40034a589..b8b124c83 100644 --- a/pkg/metric/metric_test.go +++ b/pkg/metric/metric_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/buffer.go b/pkg/p9/buffer.go index b7bb14ef9..4c8c6555d 100644 --- a/pkg/p9/buffer.go +++ b/pkg/p9/buffer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/buffer_test.go b/pkg/p9/buffer_test.go index 18d55e5c0..a9c75f86b 100644 --- a/pkg/p9/buffer_test.go +++ b/pkg/p9/buffer_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/client.go b/pkg/p9/client.go index 67887874a..2f9c716d0 100644 --- a/pkg/p9/client.go +++ b/pkg/p9/client.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/client_file.go b/pkg/p9/client_file.go index 992d1daf7..63c65129a 100644 --- a/pkg/p9/client_file.go +++ b/pkg/p9/client_file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/client_test.go b/pkg/p9/client_test.go index f7145452d..fc49729d8 100644 --- a/pkg/p9/client_test.go +++ b/pkg/p9/client_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/file.go b/pkg/p9/file.go index 55ceb52e1..a52a0f3e7 100644 --- a/pkg/p9/file.go +++ b/pkg/p9/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/handlers.go b/pkg/p9/handlers.go index c1d1ac1e8..6da2ce4e3 100644 --- a/pkg/p9/handlers.go +++ b/pkg/p9/handlers.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/local_server/local_server.go b/pkg/p9/local_server/local_server.go index 69b90c6cd..f4077a9d4 100644 --- a/pkg/p9/local_server/local_server.go +++ b/pkg/p9/local_server/local_server.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/messages.go b/pkg/p9/messages.go index 97decd3cc..833defbd6 100644 --- a/pkg/p9/messages.go +++ b/pkg/p9/messages.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/messages_test.go b/pkg/p9/messages_test.go index 68395a396..10a0587cf 100644 --- a/pkg/p9/messages_test.go +++ b/pkg/p9/messages_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/p9.go b/pkg/p9/p9.go index 4ea9f2f9a..78c7d3f86 100644 --- a/pkg/p9/p9.go +++ b/pkg/p9/p9.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/p9_test.go b/pkg/p9/p9_test.go index 02498346c..8dda6cc64 100644 --- a/pkg/p9/p9_test.go +++ b/pkg/p9/p9_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/p9test/client_test.go b/pkg/p9/p9test/client_test.go index 242d81b95..e00dd03ab 100644 --- a/pkg/p9/p9test/client_test.go +++ b/pkg/p9/p9test/client_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/p9test/p9test.go b/pkg/p9/p9test/p9test.go index f9bacbf84..1c8eff200 100644 --- a/pkg/p9/p9test/p9test.go +++ b/pkg/p9/p9test/p9test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/path_tree.go b/pkg/p9/path_tree.go index 60b20578e..f37ad4ab2 100644 --- a/pkg/p9/path_tree.go +++ b/pkg/p9/path_tree.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/pool.go b/pkg/p9/pool.go index 34ed898e8..52de889e1 100644 --- a/pkg/p9/pool.go +++ b/pkg/p9/pool.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/pool_test.go b/pkg/p9/pool_test.go index 71052d8c4..e4746b8da 100644 --- a/pkg/p9/pool_test.go +++ b/pkg/p9/pool_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/server.go b/pkg/p9/server.go index 3ef151595..b2a86d8fa 100644 --- a/pkg/p9/server.go +++ b/pkg/p9/server.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/transport.go b/pkg/p9/transport.go index bafb377de..ef59077ff 100644 --- a/pkg/p9/transport.go +++ b/pkg/p9/transport.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/transport_test.go b/pkg/p9/transport_test.go index b7b7825bd..c833d1c9c 100644 --- a/pkg/p9/transport_test.go +++ b/pkg/p9/transport_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/version.go b/pkg/p9/version.go index ceb6fabbf..a36a499a1 100644 --- a/pkg/p9/version.go +++ b/pkg/p9/version.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/version_test.go b/pkg/p9/version_test.go index c053614c9..291e8580e 100644 --- a/pkg/p9/version_test.go +++ b/pkg/p9/version_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/rand/rand.go b/pkg/rand/rand.go index 593a14380..a2714784d 100644 --- a/pkg/rand/rand.go +++ b/pkg/rand/rand.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/rand/rand_linux.go b/pkg/rand/rand_linux.go index 7ebe8f3b0..2b92db3e6 100644 --- a/pkg/rand/rand_linux.go +++ b/pkg/rand/rand_linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/refs/refcounter.go b/pkg/refs/refcounter.go index 8f08c74c7..20f515391 100644 --- a/pkg/refs/refcounter.go +++ b/pkg/refs/refcounter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/refs/refcounter_state.go b/pkg/refs/refcounter_state.go index 136f06fbf..7c99fd2b5 100644 --- a/pkg/refs/refcounter_state.go +++ b/pkg/refs/refcounter_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/refs/refcounter_test.go b/pkg/refs/refcounter_test.go index abaa87453..ffd3d3f07 100644 --- a/pkg/refs/refcounter_test.go +++ b/pkg/refs/refcounter_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/seccomp/seccomp.go b/pkg/seccomp/seccomp.go index e113f3574..50c9409e4 100644 --- a/pkg/seccomp/seccomp.go +++ b/pkg/seccomp/seccomp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/seccomp/seccomp_rules.go b/pkg/seccomp/seccomp_rules.go index a9278c64b..29eec8db1 100644 --- a/pkg/seccomp/seccomp_rules.go +++ b/pkg/seccomp/seccomp_rules.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/seccomp/seccomp_test.go b/pkg/seccomp/seccomp_test.go index 11ed90eb4..47ecac6f7 100644 --- a/pkg/seccomp/seccomp_test.go +++ b/pkg/seccomp/seccomp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/seccomp/seccomp_test_victim.go b/pkg/seccomp/seccomp_test_victim.go index dd5ed0041..afc2f755f 100644 --- a/pkg/seccomp/seccomp_test_victim.go +++ b/pkg/seccomp/seccomp_test_victim.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/seccomp/seccomp_unsafe.go b/pkg/seccomp/seccomp_unsafe.go index a31c6471d..ccd40d9db 100644 --- a/pkg/seccomp/seccomp_unsafe.go +++ b/pkg/seccomp/seccomp_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/secio/full_reader.go b/pkg/secio/full_reader.go index 90b1772a7..aed2564bd 100644 --- a/pkg/secio/full_reader.go +++ b/pkg/secio/full_reader.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/secio/secio.go b/pkg/secio/secio.go index e5f74a497..b43226035 100644 --- a/pkg/secio/secio.go +++ b/pkg/secio/secio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/secio/secio_test.go b/pkg/secio/secio_test.go index 8304c4f74..d1d905187 100644 --- a/pkg/secio/secio_test.go +++ b/pkg/secio/secio_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/segment/range.go b/pkg/segment/range.go index 057bcd7ff..4d4aeffef 100644 --- a/pkg/segment/range.go +++ b/pkg/segment/range.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/segment/set.go b/pkg/segment/set.go index 74a916ea3..982eb3fdd 100644 --- a/pkg/segment/set.go +++ b/pkg/segment/set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/segment/set_state.go b/pkg/segment/set_state.go index b86e1b75f..76de92591 100644 --- a/pkg/segment/set_state.go +++ b/pkg/segment/set_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/segment/test/segment_test.go b/pkg/segment/test/segment_test.go index 0825105db..f19a005f3 100644 --- a/pkg/segment/test/segment_test.go +++ b/pkg/segment/test/segment_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/segment/test/set_functions.go b/pkg/segment/test/set_functions.go index 41f649011..bcddb39bb 100644 --- a/pkg/segment/test/set_functions.go +++ b/pkg/segment/test/set_functions.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/aligned.go b/pkg/sentry/arch/aligned.go index c88c034f6..df01a903d 100644 --- a/pkg/sentry/arch/aligned.go +++ b/pkg/sentry/arch/aligned.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/arch.go b/pkg/sentry/arch/arch.go index 16d8eb2b2..53f0c9018 100644 --- a/pkg/sentry/arch/arch.go +++ b/pkg/sentry/arch/arch.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/arch_amd64.go b/pkg/sentry/arch/arch_amd64.go index 7ec2f2c84..135c2ee1f 100644 --- a/pkg/sentry/arch/arch_amd64.go +++ b/pkg/sentry/arch/arch_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/arch_amd64.s b/pkg/sentry/arch/arch_amd64.s index fa9857df7..bd61402cf 100644 --- a/pkg/sentry/arch/arch_amd64.s +++ b/pkg/sentry/arch/arch_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/arch_state_x86.go b/pkg/sentry/arch/arch_state_x86.go index 01949049d..bb52d8db0 100644 --- a/pkg/sentry/arch/arch_state_x86.go +++ b/pkg/sentry/arch/arch_state_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/arch_x86.go b/pkg/sentry/arch/arch_x86.go index 4305fe2cb..4d167ce98 100644 --- a/pkg/sentry/arch/arch_x86.go +++ b/pkg/sentry/arch/arch_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/auxv.go b/pkg/sentry/arch/auxv.go index 5df65a691..80c923103 100644 --- a/pkg/sentry/arch/auxv.go +++ b/pkg/sentry/arch/auxv.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/registers.proto b/pkg/sentry/arch/registers.proto index f4c2f7043..9dc83e241 100644 --- a/pkg/sentry/arch/registers.proto +++ b/pkg/sentry/arch/registers.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/signal_act.go b/pkg/sentry/arch/signal_act.go index ad098c746..f9ca2e74e 100644 --- a/pkg/sentry/arch/signal_act.go +++ b/pkg/sentry/arch/signal_act.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/signal_amd64.go b/pkg/sentry/arch/signal_amd64.go index 7f76eba27..aa030fd70 100644 --- a/pkg/sentry/arch/signal_amd64.go +++ b/pkg/sentry/arch/signal_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/signal_info.go b/pkg/sentry/arch/signal_info.go index fa0ecbec5..f93ee8b46 100644 --- a/pkg/sentry/arch/signal_info.go +++ b/pkg/sentry/arch/signal_info.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/signal_stack.go b/pkg/sentry/arch/signal_stack.go index c02ae3b7c..a442f9fdc 100644 --- a/pkg/sentry/arch/signal_stack.go +++ b/pkg/sentry/arch/signal_stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/stack.go b/pkg/sentry/arch/stack.go index 2e33ccdf5..7e6324e82 100644 --- a/pkg/sentry/arch/stack.go +++ b/pkg/sentry/arch/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/syscalls_amd64.go b/pkg/sentry/arch/syscalls_amd64.go index 47c31d4b9..8b4f23007 100644 --- a/pkg/sentry/arch/syscalls_amd64.go +++ b/pkg/sentry/arch/syscalls_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/context/context.go b/pkg/sentry/context/context.go index eefc3e1b4..d70f3a5c3 100644 --- a/pkg/sentry/context/context.go +++ b/pkg/sentry/context/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/context/contexttest/contexttest.go b/pkg/sentry/context/contexttest/contexttest.go index a29087775..a42038711 100644 --- a/pkg/sentry/context/contexttest/contexttest.go +++ b/pkg/sentry/context/contexttest/contexttest.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/control/control.go b/pkg/sentry/control/control.go index 32d30b6ea..6060b9b4f 100644 --- a/pkg/sentry/control/control.go +++ b/pkg/sentry/control/control.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/control/pprof.go b/pkg/sentry/control/pprof.go index 1af092af3..94ed149f2 100644 --- a/pkg/sentry/control/pprof.go +++ b/pkg/sentry/control/pprof.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/control/proc.go b/pkg/sentry/control/proc.go index aca2267a7..f7f02a3e1 100644 --- a/pkg/sentry/control/proc.go +++ b/pkg/sentry/control/proc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/control/proc_test.go b/pkg/sentry/control/proc_test.go index 5d52cd829..b7895d03c 100644 --- a/pkg/sentry/control/proc_test.go +++ b/pkg/sentry/control/proc_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/control/state.go b/pkg/sentry/control/state.go index b6bbf69fa..11efcaba1 100644 --- a/pkg/sentry/control/state.go +++ b/pkg/sentry/control/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/device/device.go b/pkg/sentry/device/device.go index ae4fa1d93..458d03b30 100644 --- a/pkg/sentry/device/device.go +++ b/pkg/sentry/device/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/device/device_test.go b/pkg/sentry/device/device_test.go index 5d8805c2f..e3f51ce4f 100644 --- a/pkg/sentry/device/device_test.go +++ b/pkg/sentry/device/device_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/anon/anon.go b/pkg/sentry/fs/anon/anon.go index a5e8c4f0d..a6ea8b9e7 100644 --- a/pkg/sentry/fs/anon/anon.go +++ b/pkg/sentry/fs/anon/anon.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/anon/device.go b/pkg/sentry/fs/anon/device.go index 2d1249299..5927bd11e 100644 --- a/pkg/sentry/fs/anon/device.go +++ b/pkg/sentry/fs/anon/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ashmem/area.go b/pkg/sentry/fs/ashmem/area.go index 1f61c5711..b53746519 100644 --- a/pkg/sentry/fs/ashmem/area.go +++ b/pkg/sentry/fs/ashmem/area.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ashmem/device.go b/pkg/sentry/fs/ashmem/device.go index 5369d1b0d..5e005bc2e 100644 --- a/pkg/sentry/fs/ashmem/device.go +++ b/pkg/sentry/fs/ashmem/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ashmem/pin_board.go b/pkg/sentry/fs/ashmem/pin_board.go index 7c997f533..bdf23b371 100644 --- a/pkg/sentry/fs/ashmem/pin_board.go +++ b/pkg/sentry/fs/ashmem/pin_board.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ashmem/pin_board_test.go b/pkg/sentry/fs/ashmem/pin_board_test.go index 736e628dc..24f5d86d6 100644 --- a/pkg/sentry/fs/ashmem/pin_board_test.go +++ b/pkg/sentry/fs/ashmem/pin_board_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/attr.go b/pkg/sentry/fs/attr.go index 3523b068a..591e35e6a 100644 --- a/pkg/sentry/fs/attr.go +++ b/pkg/sentry/fs/attr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/binder/binder.go b/pkg/sentry/fs/binder/binder.go index d9f1559de..acbbd5466 100644 --- a/pkg/sentry/fs/binder/binder.go +++ b/pkg/sentry/fs/binder/binder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/context.go b/pkg/sentry/fs/context.go index 4869428a8..c80ea0175 100644 --- a/pkg/sentry/fs/context.go +++ b/pkg/sentry/fs/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/copy_up.go b/pkg/sentry/fs/copy_up.go index ba69e718d..ee2d3d115 100644 --- a/pkg/sentry/fs/copy_up.go +++ b/pkg/sentry/fs/copy_up.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/copy_up_test.go b/pkg/sentry/fs/copy_up_test.go index 98a0b7638..54810afca 100644 --- a/pkg/sentry/fs/copy_up_test.go +++ b/pkg/sentry/fs/copy_up_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dentry.go b/pkg/sentry/fs/dentry.go index 29fb155a4..fe656cc24 100644 --- a/pkg/sentry/fs/dentry.go +++ b/pkg/sentry/fs/dentry.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/dev.go b/pkg/sentry/fs/dev/dev.go index fbc750a71..34ac01173 100644 --- a/pkg/sentry/fs/dev/dev.go +++ b/pkg/sentry/fs/dev/dev.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/device.go b/pkg/sentry/fs/dev/device.go index 3cecdf6e2..9f4e41fc9 100644 --- a/pkg/sentry/fs/dev/device.go +++ b/pkg/sentry/fs/dev/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/fs.go b/pkg/sentry/fs/dev/fs.go index cf4e7d00f..6096a40f8 100644 --- a/pkg/sentry/fs/dev/fs.go +++ b/pkg/sentry/fs/dev/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/full.go b/pkg/sentry/fs/dev/full.go index 82da9aae9..6b11afa44 100644 --- a/pkg/sentry/fs/dev/full.go +++ b/pkg/sentry/fs/dev/full.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/null.go b/pkg/sentry/fs/dev/null.go index 5d306d352..069212b6d 100644 --- a/pkg/sentry/fs/dev/null.go +++ b/pkg/sentry/fs/dev/null.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/random.go b/pkg/sentry/fs/dev/random.go index ffd5cf6c3..de0f3e5e5 100644 --- a/pkg/sentry/fs/dev/random.go +++ b/pkg/sentry/fs/dev/random.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dirent.go b/pkg/sentry/fs/dirent.go index 54fc11fe1..c0bc261a2 100644 --- a/pkg/sentry/fs/dirent.go +++ b/pkg/sentry/fs/dirent.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dirent_cache.go b/pkg/sentry/fs/dirent_cache.go index d26a06971..71f2d11de 100644 --- a/pkg/sentry/fs/dirent_cache.go +++ b/pkg/sentry/fs/dirent_cache.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dirent_cache_limiter.go b/pkg/sentry/fs/dirent_cache_limiter.go index 024c7b2d5..ebb80bd50 100644 --- a/pkg/sentry/fs/dirent_cache_limiter.go +++ b/pkg/sentry/fs/dirent_cache_limiter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dirent_cache_test.go b/pkg/sentry/fs/dirent_cache_test.go index 93e8d415f..395c879f5 100644 --- a/pkg/sentry/fs/dirent_cache_test.go +++ b/pkg/sentry/fs/dirent_cache_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dirent_refs_test.go b/pkg/sentry/fs/dirent_refs_test.go index 325404e27..db88d850e 100644 --- a/pkg/sentry/fs/dirent_refs_test.go +++ b/pkg/sentry/fs/dirent_refs_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dirent_state.go b/pkg/sentry/fs/dirent_state.go index 5cf151dab..18652b809 100644 --- a/pkg/sentry/fs/dirent_state.go +++ b/pkg/sentry/fs/dirent_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fdpipe/pipe.go b/pkg/sentry/fs/fdpipe/pipe.go index 98483ab68..95e66ea8d 100644 --- a/pkg/sentry/fs/fdpipe/pipe.go +++ b/pkg/sentry/fs/fdpipe/pipe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fdpipe/pipe_opener.go b/pkg/sentry/fs/fdpipe/pipe_opener.go index 92ab6ff0e..0cabe2e18 100644 --- a/pkg/sentry/fs/fdpipe/pipe_opener.go +++ b/pkg/sentry/fs/fdpipe/pipe_opener.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fdpipe/pipe_opener_test.go b/pkg/sentry/fs/fdpipe/pipe_opener_test.go index 69516e048..8c8b1b40c 100644 --- a/pkg/sentry/fs/fdpipe/pipe_opener_test.go +++ b/pkg/sentry/fs/fdpipe/pipe_opener_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fdpipe/pipe_state.go b/pkg/sentry/fs/fdpipe/pipe_state.go index 4395666ad..8b347aa11 100644 --- a/pkg/sentry/fs/fdpipe/pipe_state.go +++ b/pkg/sentry/fs/fdpipe/pipe_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fdpipe/pipe_test.go b/pkg/sentry/fs/fdpipe/pipe_test.go index 7e3ee5257..b59a6aa0e 100644 --- a/pkg/sentry/fs/fdpipe/pipe_test.go +++ b/pkg/sentry/fs/fdpipe/pipe_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file.go b/pkg/sentry/fs/file.go index 5d5026661..62b35dabc 100644 --- a/pkg/sentry/fs/file.go +++ b/pkg/sentry/fs/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file_operations.go b/pkg/sentry/fs/file_operations.go index e0fa5135f..ab0acb6eb 100644 --- a/pkg/sentry/fs/file_operations.go +++ b/pkg/sentry/fs/file_operations.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file_overlay.go b/pkg/sentry/fs/file_overlay.go index 6e680f0a4..948ce9c6f 100644 --- a/pkg/sentry/fs/file_overlay.go +++ b/pkg/sentry/fs/file_overlay.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file_overlay_test.go b/pkg/sentry/fs/file_overlay_test.go index a4ac58763..6a2b8007c 100644 --- a/pkg/sentry/fs/file_overlay_test.go +++ b/pkg/sentry/fs/file_overlay_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file_state.go b/pkg/sentry/fs/file_state.go index 1c3bae3e8..523182d59 100644 --- a/pkg/sentry/fs/file_state.go +++ b/pkg/sentry/fs/file_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file_test.go b/pkg/sentry/fs/file_test.go index f3ed9a70b..d867a0257 100644 --- a/pkg/sentry/fs/file_test.go +++ b/pkg/sentry/fs/file_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/filesystems.go b/pkg/sentry/fs/filesystems.go index a6b27c402..acd84dfcc 100644 --- a/pkg/sentry/fs/filesystems.go +++ b/pkg/sentry/fs/filesystems.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/filetest/filetest.go b/pkg/sentry/fs/filetest/filetest.go index 388a1ce36..f6b827800 100644 --- a/pkg/sentry/fs/filetest/filetest.go +++ b/pkg/sentry/fs/filetest/filetest.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/flags.go b/pkg/sentry/fs/flags.go index bf2a20b33..5c8cb773f 100644 --- a/pkg/sentry/fs/flags.go +++ b/pkg/sentry/fs/flags.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fs.go b/pkg/sentry/fs/fs.go index 119689776..632055cce 100644 --- a/pkg/sentry/fs/fs.go +++ b/pkg/sentry/fs/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/dirty_set.go b/pkg/sentry/fs/fsutil/dirty_set.go index 5add16ac4..9cd196d7d 100644 --- a/pkg/sentry/fs/fsutil/dirty_set.go +++ b/pkg/sentry/fs/fsutil/dirty_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/dirty_set_test.go b/pkg/sentry/fs/fsutil/dirty_set_test.go index f5c9d9215..d9c68baa3 100644 --- a/pkg/sentry/fs/fsutil/dirty_set_test.go +++ b/pkg/sentry/fs/fsutil/dirty_set_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/file.go b/pkg/sentry/fs/fsutil/file.go index 42afdd11c..e355d8594 100644 --- a/pkg/sentry/fs/fsutil/file.go +++ b/pkg/sentry/fs/fsutil/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/file_range_set.go b/pkg/sentry/fs/fsutil/file_range_set.go index 32ebf64ff..b5ac6c71c 100644 --- a/pkg/sentry/fs/fsutil/file_range_set.go +++ b/pkg/sentry/fs/fsutil/file_range_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/frame_ref_set.go b/pkg/sentry/fs/fsutil/frame_ref_set.go index b6e783614..6565c28c8 100644 --- a/pkg/sentry/fs/fsutil/frame_ref_set.go +++ b/pkg/sentry/fs/fsutil/frame_ref_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/fsutil.go b/pkg/sentry/fs/fsutil/fsutil.go index 319c4841b..c9587b1d9 100644 --- a/pkg/sentry/fs/fsutil/fsutil.go +++ b/pkg/sentry/fs/fsutil/fsutil.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/host_file_mapper.go b/pkg/sentry/fs/fsutil/host_file_mapper.go index 9599665f0..2bdfc0db6 100644 --- a/pkg/sentry/fs/fsutil/host_file_mapper.go +++ b/pkg/sentry/fs/fsutil/host_file_mapper.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/host_file_mapper_state.go b/pkg/sentry/fs/fsutil/host_file_mapper_state.go index bbd15b30b..576d2a3df 100644 --- a/pkg/sentry/fs/fsutil/host_file_mapper_state.go +++ b/pkg/sentry/fs/fsutil/host_file_mapper_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go b/pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go index 86df76822..7167be263 100644 --- a/pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go +++ b/pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/host_mappable.go b/pkg/sentry/fs/fsutil/host_mappable.go index 4a182baa1..28686f3b3 100644 --- a/pkg/sentry/fs/fsutil/host_mappable.go +++ b/pkg/sentry/fs/fsutil/host_mappable.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/inode.go b/pkg/sentry/fs/fsutil/inode.go index 468171a9b..b6366d906 100644 --- a/pkg/sentry/fs/fsutil/inode.go +++ b/pkg/sentry/fs/fsutil/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/inode_cached.go b/pkg/sentry/fs/fsutil/inode_cached.go index ba33b9912..919d2534c 100644 --- a/pkg/sentry/fs/fsutil/inode_cached.go +++ b/pkg/sentry/fs/fsutil/inode_cached.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/inode_cached_test.go b/pkg/sentry/fs/fsutil/inode_cached_test.go index 2a8a1639c..661ec41f6 100644 --- a/pkg/sentry/fs/fsutil/inode_cached_test.go +++ b/pkg/sentry/fs/fsutil/inode_cached_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/attr.go b/pkg/sentry/fs/gofer/attr.go index 98700d014..c572f3396 100644 --- a/pkg/sentry/fs/gofer/attr.go +++ b/pkg/sentry/fs/gofer/attr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/cache_policy.go b/pkg/sentry/fs/gofer/cache_policy.go index 51c573aef..35cd0c1d6 100644 --- a/pkg/sentry/fs/gofer/cache_policy.go +++ b/pkg/sentry/fs/gofer/cache_policy.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/context_file.go b/pkg/sentry/fs/gofer/context_file.go index 455953237..d512afefc 100644 --- a/pkg/sentry/fs/gofer/context_file.go +++ b/pkg/sentry/fs/gofer/context_file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/device.go b/pkg/sentry/fs/gofer/device.go index 52c5acf48..1de6c247c 100644 --- a/pkg/sentry/fs/gofer/device.go +++ b/pkg/sentry/fs/gofer/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/file.go b/pkg/sentry/fs/gofer/file.go index 35caa42cd..bc2be546e 100644 --- a/pkg/sentry/fs/gofer/file.go +++ b/pkg/sentry/fs/gofer/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/file_state.go b/pkg/sentry/fs/gofer/file_state.go index d0c64003c..31264e065 100644 --- a/pkg/sentry/fs/gofer/file_state.go +++ b/pkg/sentry/fs/gofer/file_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/fs.go b/pkg/sentry/fs/gofer/fs.go index adff0abac..6ab89fcc2 100644 --- a/pkg/sentry/fs/gofer/fs.go +++ b/pkg/sentry/fs/gofer/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/gofer_test.go b/pkg/sentry/fs/gofer/gofer_test.go index 36201f017..29d34da7e 100644 --- a/pkg/sentry/fs/gofer/gofer_test.go +++ b/pkg/sentry/fs/gofer/gofer_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/handles.go b/pkg/sentry/fs/gofer/handles.go index 0b33e80c3..c7098cd36 100644 --- a/pkg/sentry/fs/gofer/handles.go +++ b/pkg/sentry/fs/gofer/handles.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/inode.go b/pkg/sentry/fs/gofer/inode.go index 1181a24cc..f6f20844d 100644 --- a/pkg/sentry/fs/gofer/inode.go +++ b/pkg/sentry/fs/gofer/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/inode_state.go b/pkg/sentry/fs/gofer/inode_state.go index 44d76ba9f..ac22ee4b1 100644 --- a/pkg/sentry/fs/gofer/inode_state.go +++ b/pkg/sentry/fs/gofer/inode_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/path.go b/pkg/sentry/fs/gofer/path.go index 8ae33d286..4cbf9e9d9 100644 --- a/pkg/sentry/fs/gofer/path.go +++ b/pkg/sentry/fs/gofer/path.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/session.go b/pkg/sentry/fs/gofer/session.go index 4ed688ce5..4cb65e7c6 100644 --- a/pkg/sentry/fs/gofer/session.go +++ b/pkg/sentry/fs/gofer/session.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/session_state.go b/pkg/sentry/fs/gofer/session_state.go index b1f299be5..68fbf3417 100644 --- a/pkg/sentry/fs/gofer/session_state.go +++ b/pkg/sentry/fs/gofer/session_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/socket.go b/pkg/sentry/fs/gofer/socket.go index ce6d3d5c3..cbd5b9a84 100644 --- a/pkg/sentry/fs/gofer/socket.go +++ b/pkg/sentry/fs/gofer/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/util.go b/pkg/sentry/fs/gofer/util.go index 1a759370d..d0e1096ce 100644 --- a/pkg/sentry/fs/gofer/util.go +++ b/pkg/sentry/fs/gofer/util.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/control.go b/pkg/sentry/fs/host/control.go index 0753640a2..480f0c8f4 100644 --- a/pkg/sentry/fs/host/control.go +++ b/pkg/sentry/fs/host/control.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/descriptor.go b/pkg/sentry/fs/host/descriptor.go index 554e1693a..ffcd57a94 100644 --- a/pkg/sentry/fs/host/descriptor.go +++ b/pkg/sentry/fs/host/descriptor.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/descriptor_state.go b/pkg/sentry/fs/host/descriptor_state.go index 530c0109f..8167390a9 100644 --- a/pkg/sentry/fs/host/descriptor_state.go +++ b/pkg/sentry/fs/host/descriptor_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/descriptor_test.go b/pkg/sentry/fs/host/descriptor_test.go index 5dec84ab2..ff08e43af 100644 --- a/pkg/sentry/fs/host/descriptor_test.go +++ b/pkg/sentry/fs/host/descriptor_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/device.go b/pkg/sentry/fs/host/device.go index b5adedf44..055024c44 100644 --- a/pkg/sentry/fs/host/device.go +++ b/pkg/sentry/fs/host/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/file.go b/pkg/sentry/fs/host/file.go index 2a8f285ff..82e2ae3b9 100644 --- a/pkg/sentry/fs/host/file.go +++ b/pkg/sentry/fs/host/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/fs.go b/pkg/sentry/fs/host/fs.go index de349a41a..b1b8dc0b6 100644 --- a/pkg/sentry/fs/host/fs.go +++ b/pkg/sentry/fs/host/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/fs_test.go b/pkg/sentry/fs/host/fs_test.go index c83b29a16..16c89ddf1 100644 --- a/pkg/sentry/fs/host/fs_test.go +++ b/pkg/sentry/fs/host/fs_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/inode.go b/pkg/sentry/fs/host/inode.go index 69c648f67..20e077f77 100644 --- a/pkg/sentry/fs/host/inode.go +++ b/pkg/sentry/fs/host/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/inode_state.go b/pkg/sentry/fs/host/inode_state.go index b7c1a9581..26cc755bc 100644 --- a/pkg/sentry/fs/host/inode_state.go +++ b/pkg/sentry/fs/host/inode_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/inode_test.go b/pkg/sentry/fs/host/inode_test.go index 9f1561bd5..ad1878b5a 100644 --- a/pkg/sentry/fs/host/inode_test.go +++ b/pkg/sentry/fs/host/inode_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/ioctl_unsafe.go b/pkg/sentry/fs/host/ioctl_unsafe.go index 175dca613..b5a85c4d9 100644 --- a/pkg/sentry/fs/host/ioctl_unsafe.go +++ b/pkg/sentry/fs/host/ioctl_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/socket.go b/pkg/sentry/fs/host/socket.go index be2c3581f..3034e9441 100644 --- a/pkg/sentry/fs/host/socket.go +++ b/pkg/sentry/fs/host/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/socket_iovec.go b/pkg/sentry/fs/host/socket_iovec.go index d4ce4a8c1..5efbb3ae8 100644 --- a/pkg/sentry/fs/host/socket_iovec.go +++ b/pkg/sentry/fs/host/socket_iovec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/socket_state.go b/pkg/sentry/fs/host/socket_state.go index 2932c1f16..5676c451a 100644 --- a/pkg/sentry/fs/host/socket_state.go +++ b/pkg/sentry/fs/host/socket_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/socket_test.go b/pkg/sentry/fs/host/socket_test.go index 83e8e1b3c..cc760a7e1 100644 --- a/pkg/sentry/fs/host/socket_test.go +++ b/pkg/sentry/fs/host/socket_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/socket_unsafe.go b/pkg/sentry/fs/host/socket_unsafe.go index f35e2492d..8873705c0 100644 --- a/pkg/sentry/fs/host/socket_unsafe.go +++ b/pkg/sentry/fs/host/socket_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/tty.go b/pkg/sentry/fs/host/tty.go index c5cb75df7..e45b339f5 100644 --- a/pkg/sentry/fs/host/tty.go +++ b/pkg/sentry/fs/host/tty.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/util.go b/pkg/sentry/fs/host/util.go index 40c450660..94ff7708e 100644 --- a/pkg/sentry/fs/host/util.go +++ b/pkg/sentry/fs/host/util.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/util_unsafe.go b/pkg/sentry/fs/host/util_unsafe.go index a8721d197..b95a57c3f 100644 --- a/pkg/sentry/fs/host/util_unsafe.go +++ b/pkg/sentry/fs/host/util_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/wait_test.go b/pkg/sentry/fs/host/wait_test.go index 9ca8c399f..afcb74724 100644 --- a/pkg/sentry/fs/host/wait_test.go +++ b/pkg/sentry/fs/host/wait_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inode.go b/pkg/sentry/fs/inode.go index fe411a766..d764ef93d 100644 --- a/pkg/sentry/fs/inode.go +++ b/pkg/sentry/fs/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inode_inotify.go b/pkg/sentry/fs/inode_inotify.go index d2b653bc7..0f2a66a79 100644 --- a/pkg/sentry/fs/inode_inotify.go +++ b/pkg/sentry/fs/inode_inotify.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inode_operations.go b/pkg/sentry/fs/inode_operations.go index ff8b75f31..ac287e1e4 100644 --- a/pkg/sentry/fs/inode_operations.go +++ b/pkg/sentry/fs/inode_operations.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inode_overlay.go b/pkg/sentry/fs/inode_overlay.go index bda3e1861..3d015328e 100644 --- a/pkg/sentry/fs/inode_overlay.go +++ b/pkg/sentry/fs/inode_overlay.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inode_overlay_test.go b/pkg/sentry/fs/inode_overlay_test.go index fa8accf6c..66b3da2d0 100644 --- a/pkg/sentry/fs/inode_overlay_test.go +++ b/pkg/sentry/fs/inode_overlay_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inotify.go b/pkg/sentry/fs/inotify.go index 59fa662f3..2652582c3 100644 --- a/pkg/sentry/fs/inotify.go +++ b/pkg/sentry/fs/inotify.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inotify_event.go b/pkg/sentry/fs/inotify_event.go index f09928b68..d52f956e4 100644 --- a/pkg/sentry/fs/inotify_event.go +++ b/pkg/sentry/fs/inotify_event.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inotify_watch.go b/pkg/sentry/fs/inotify_watch.go index d33e7e498..a0b488467 100644 --- a/pkg/sentry/fs/inotify_watch.go +++ b/pkg/sentry/fs/inotify_watch.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/lock/lock.go b/pkg/sentry/fs/lock/lock.go index 5ff800d2d..f2aee4512 100644 --- a/pkg/sentry/fs/lock/lock.go +++ b/pkg/sentry/fs/lock/lock.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/lock/lock_range_test.go b/pkg/sentry/fs/lock/lock_range_test.go index b0ab882b9..6221199d1 100644 --- a/pkg/sentry/fs/lock/lock_range_test.go +++ b/pkg/sentry/fs/lock/lock_range_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/lock/lock_set_functions.go b/pkg/sentry/fs/lock/lock_set_functions.go index 395592a4b..8a3ace0c1 100644 --- a/pkg/sentry/fs/lock/lock_set_functions.go +++ b/pkg/sentry/fs/lock/lock_set_functions.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/lock/lock_test.go b/pkg/sentry/fs/lock/lock_test.go index 67fa4b1dd..ba002aeb7 100644 --- a/pkg/sentry/fs/lock/lock_test.go +++ b/pkg/sentry/fs/lock/lock_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mock.go b/pkg/sentry/fs/mock.go index 118e30f63..cf359a1f1 100644 --- a/pkg/sentry/fs/mock.go +++ b/pkg/sentry/fs/mock.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mount.go b/pkg/sentry/fs/mount.go index 4d1693204..a169ea4c9 100644 --- a/pkg/sentry/fs/mount.go +++ b/pkg/sentry/fs/mount.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mount_overlay.go b/pkg/sentry/fs/mount_overlay.go index fb60a1aec..535f812c8 100644 --- a/pkg/sentry/fs/mount_overlay.go +++ b/pkg/sentry/fs/mount_overlay.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mount_test.go b/pkg/sentry/fs/mount_test.go index d7605b2c9..9f7fbeff2 100644 --- a/pkg/sentry/fs/mount_test.go +++ b/pkg/sentry/fs/mount_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mounts.go b/pkg/sentry/fs/mounts.go index f6f7be0aa..01eb4607e 100644 --- a/pkg/sentry/fs/mounts.go +++ b/pkg/sentry/fs/mounts.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mounts_test.go b/pkg/sentry/fs/mounts_test.go index 54000614f..56d726dd1 100644 --- a/pkg/sentry/fs/mounts_test.go +++ b/pkg/sentry/fs/mounts_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/offset.go b/pkg/sentry/fs/offset.go index 38aee765a..3f68da149 100644 --- a/pkg/sentry/fs/offset.go +++ b/pkg/sentry/fs/offset.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/overlay.go b/pkg/sentry/fs/overlay.go index f3e2d5cbe..db89a5f70 100644 --- a/pkg/sentry/fs/overlay.go +++ b/pkg/sentry/fs/overlay.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/path.go b/pkg/sentry/fs/path.go index 52139b648..e4dc02dbb 100644 --- a/pkg/sentry/fs/path.go +++ b/pkg/sentry/fs/path.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/path_test.go b/pkg/sentry/fs/path_test.go index 4ba1498f6..e6f57ebba 100644 --- a/pkg/sentry/fs/path_test.go +++ b/pkg/sentry/fs/path_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/cpuinfo.go b/pkg/sentry/fs/proc/cpuinfo.go index f756c45bf..15031234e 100644 --- a/pkg/sentry/fs/proc/cpuinfo.go +++ b/pkg/sentry/fs/proc/cpuinfo.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/device/device.go b/pkg/sentry/fs/proc/device/device.go index 04b687bcf..0de466c73 100644 --- a/pkg/sentry/fs/proc/device/device.go +++ b/pkg/sentry/fs/proc/device/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/exec_args.go b/pkg/sentry/fs/proc/exec_args.go index fc21dfbbd..d49dad685 100644 --- a/pkg/sentry/fs/proc/exec_args.go +++ b/pkg/sentry/fs/proc/exec_args.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/fds.go b/pkg/sentry/fs/proc/fds.go index f2329e623..744b31c74 100644 --- a/pkg/sentry/fs/proc/fds.go +++ b/pkg/sentry/fs/proc/fds.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/filesystems.go b/pkg/sentry/fs/proc/filesystems.go index c050a00be..7bb081d0e 100644 --- a/pkg/sentry/fs/proc/filesystems.go +++ b/pkg/sentry/fs/proc/filesystems.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/fs.go b/pkg/sentry/fs/proc/fs.go index 666a2d054..7c5f8484a 100644 --- a/pkg/sentry/fs/proc/fs.go +++ b/pkg/sentry/fs/proc/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/inode.go b/pkg/sentry/fs/proc/inode.go index 8dde2ea46..b03807043 100644 --- a/pkg/sentry/fs/proc/inode.go +++ b/pkg/sentry/fs/proc/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/loadavg.go b/pkg/sentry/fs/proc/loadavg.go index 3ee0e570a..2dfe7089a 100644 --- a/pkg/sentry/fs/proc/loadavg.go +++ b/pkg/sentry/fs/proc/loadavg.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/meminfo.go b/pkg/sentry/fs/proc/meminfo.go index 75cbf3e77..d2b9b92c7 100644 --- a/pkg/sentry/fs/proc/meminfo.go +++ b/pkg/sentry/fs/proc/meminfo.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/mounts.go b/pkg/sentry/fs/proc/mounts.go index fe62b167b..37ed30724 100644 --- a/pkg/sentry/fs/proc/mounts.go +++ b/pkg/sentry/fs/proc/mounts.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/net.go b/pkg/sentry/fs/proc/net.go index d24b2d370..4a107c739 100644 --- a/pkg/sentry/fs/proc/net.go +++ b/pkg/sentry/fs/proc/net.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/net_test.go b/pkg/sentry/fs/proc/net_test.go index 94677cc1d..9aed5fdca 100644 --- a/pkg/sentry/fs/proc/net_test.go +++ b/pkg/sentry/fs/proc/net_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/proc.go b/pkg/sentry/fs/proc/proc.go index 64e1e1998..196fa5128 100644 --- a/pkg/sentry/fs/proc/proc.go +++ b/pkg/sentry/fs/proc/proc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/rpcinet_proc.go b/pkg/sentry/fs/proc/rpcinet_proc.go index 81f64a28b..db53686f6 100644 --- a/pkg/sentry/fs/proc/rpcinet_proc.go +++ b/pkg/sentry/fs/proc/rpcinet_proc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/seqfile/seqfile.go b/pkg/sentry/fs/proc/seqfile/seqfile.go index 0a0eb45e2..10ea1f55d 100644 --- a/pkg/sentry/fs/proc/seqfile/seqfile.go +++ b/pkg/sentry/fs/proc/seqfile/seqfile.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/seqfile/seqfile_test.go b/pkg/sentry/fs/proc/seqfile/seqfile_test.go index 35403ab7f..c4de565eb 100644 --- a/pkg/sentry/fs/proc/seqfile/seqfile_test.go +++ b/pkg/sentry/fs/proc/seqfile/seqfile_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/stat.go b/pkg/sentry/fs/proc/stat.go index 18bd8e9b6..397f9ec6b 100644 --- a/pkg/sentry/fs/proc/stat.go +++ b/pkg/sentry/fs/proc/stat.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/sys.go b/pkg/sentry/fs/proc/sys.go index a7bc9198e..b889ed625 100644 --- a/pkg/sentry/fs/proc/sys.go +++ b/pkg/sentry/fs/proc/sys.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/sys_net.go b/pkg/sentry/fs/proc/sys_net.go index 0ce77f04f..e49794a48 100644 --- a/pkg/sentry/fs/proc/sys_net.go +++ b/pkg/sentry/fs/proc/sys_net.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/sys_net_state.go b/pkg/sentry/fs/proc/sys_net_state.go index 5f481a1cf..6eba709c6 100644 --- a/pkg/sentry/fs/proc/sys_net_state.go +++ b/pkg/sentry/fs/proc/sys_net_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/sys_net_test.go b/pkg/sentry/fs/proc/sys_net_test.go index ea0d94fce..78135ba13 100644 --- a/pkg/sentry/fs/proc/sys_net_test.go +++ b/pkg/sentry/fs/proc/sys_net_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/task.go b/pkg/sentry/fs/proc/task.go index 9f65a8337..0f400e80f 100644 --- a/pkg/sentry/fs/proc/task.go +++ b/pkg/sentry/fs/proc/task.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/uid_gid_map.go b/pkg/sentry/fs/proc/uid_gid_map.go index d433632cf..d649da0f1 100644 --- a/pkg/sentry/fs/proc/uid_gid_map.go +++ b/pkg/sentry/fs/proc/uid_gid_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/uptime.go b/pkg/sentry/fs/proc/uptime.go index d7ae26fcf..1ddf9fafa 100644 --- a/pkg/sentry/fs/proc/uptime.go +++ b/pkg/sentry/fs/proc/uptime.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/version.go b/pkg/sentry/fs/proc/version.go index 58e0c793c..a5479990c 100644 --- a/pkg/sentry/fs/proc/version.go +++ b/pkg/sentry/fs/proc/version.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/dir.go b/pkg/sentry/fs/ramfs/dir.go index c0400b67d..a6b6a5c33 100644 --- a/pkg/sentry/fs/ramfs/dir.go +++ b/pkg/sentry/fs/ramfs/dir.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/socket.go b/pkg/sentry/fs/ramfs/socket.go index 5bcb6c364..9406a07ca 100644 --- a/pkg/sentry/fs/ramfs/socket.go +++ b/pkg/sentry/fs/ramfs/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/symlink.go b/pkg/sentry/fs/ramfs/symlink.go index 35dabdad2..f7835fe05 100644 --- a/pkg/sentry/fs/ramfs/symlink.go +++ b/pkg/sentry/fs/ramfs/symlink.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/tree.go b/pkg/sentry/fs/ramfs/tree.go index c1ac8a78b..8c6b31f70 100644 --- a/pkg/sentry/fs/ramfs/tree.go +++ b/pkg/sentry/fs/ramfs/tree.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/tree_test.go b/pkg/sentry/fs/ramfs/tree_test.go index 8bee9cfc1..27abeb6ba 100644 --- a/pkg/sentry/fs/ramfs/tree_test.go +++ b/pkg/sentry/fs/ramfs/tree_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/restore.go b/pkg/sentry/fs/restore.go index a6645b41e..f10168125 100644 --- a/pkg/sentry/fs/restore.go +++ b/pkg/sentry/fs/restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/save.go b/pkg/sentry/fs/save.go index 90988d385..2eaf6ab69 100644 --- a/pkg/sentry/fs/save.go +++ b/pkg/sentry/fs/save.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/seek.go b/pkg/sentry/fs/seek.go index 72f3fb632..0f43918ad 100644 --- a/pkg/sentry/fs/seek.go +++ b/pkg/sentry/fs/seek.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/sync.go b/pkg/sentry/fs/sync.go index 6dcc2fe8d..1fff8059c 100644 --- a/pkg/sentry/fs/sync.go +++ b/pkg/sentry/fs/sync.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/sys/device.go b/pkg/sentry/fs/sys/device.go index 38ecd0c18..128d3a9d9 100644 --- a/pkg/sentry/fs/sys/device.go +++ b/pkg/sentry/fs/sys/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/sys/devices.go b/pkg/sentry/fs/sys/devices.go index 8b728a4e4..db91de435 100644 --- a/pkg/sentry/fs/sys/devices.go +++ b/pkg/sentry/fs/sys/devices.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/sys/fs.go b/pkg/sentry/fs/sys/fs.go index 44ae43754..f0c2322e0 100644 --- a/pkg/sentry/fs/sys/fs.go +++ b/pkg/sentry/fs/sys/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/sys/sys.go b/pkg/sentry/fs/sys/sys.go index c5b56fe69..d20ef91fa 100644 --- a/pkg/sentry/fs/sys/sys.go +++ b/pkg/sentry/fs/sys/sys.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/timerfd/timerfd.go b/pkg/sentry/fs/timerfd/timerfd.go index ef9a08854..749961f51 100644 --- a/pkg/sentry/fs/timerfd/timerfd.go +++ b/pkg/sentry/fs/timerfd/timerfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/device.go b/pkg/sentry/fs/tmpfs/device.go index aade93c26..179c3a46f 100644 --- a/pkg/sentry/fs/tmpfs/device.go +++ b/pkg/sentry/fs/tmpfs/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/file_regular.go b/pkg/sentry/fs/tmpfs/file_regular.go index d0c9b8bea..1ef256511 100644 --- a/pkg/sentry/fs/tmpfs/file_regular.go +++ b/pkg/sentry/fs/tmpfs/file_regular.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/file_test.go b/pkg/sentry/fs/tmpfs/file_test.go index 743061190..b44c06556 100644 --- a/pkg/sentry/fs/tmpfs/file_test.go +++ b/pkg/sentry/fs/tmpfs/file_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/fs.go b/pkg/sentry/fs/tmpfs/fs.go index 8e44421b6..b7c29a4d1 100644 --- a/pkg/sentry/fs/tmpfs/fs.go +++ b/pkg/sentry/fs/tmpfs/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/inode_file.go b/pkg/sentry/fs/tmpfs/inode_file.go index 4450e1363..f89d86c83 100644 --- a/pkg/sentry/fs/tmpfs/inode_file.go +++ b/pkg/sentry/fs/tmpfs/inode_file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/tmpfs.go b/pkg/sentry/fs/tmpfs/tmpfs.go index 5bb4922cb..832914453 100644 --- a/pkg/sentry/fs/tmpfs/tmpfs.go +++ b/pkg/sentry/fs/tmpfs/tmpfs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/dir.go b/pkg/sentry/fs/tty/dir.go index f8713471a..0fc777e67 100644 --- a/pkg/sentry/fs/tty/dir.go +++ b/pkg/sentry/fs/tty/dir.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/fs.go b/pkg/sentry/fs/tty/fs.go index a53448c47..701b2f7d9 100644 --- a/pkg/sentry/fs/tty/fs.go +++ b/pkg/sentry/fs/tty/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/line_discipline.go b/pkg/sentry/fs/tty/line_discipline.go index c4a364edb..20d29d130 100644 --- a/pkg/sentry/fs/tty/line_discipline.go +++ b/pkg/sentry/fs/tty/line_discipline.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/master.go b/pkg/sentry/fs/tty/master.go index e2686a074..45e167e5f 100644 --- a/pkg/sentry/fs/tty/master.go +++ b/pkg/sentry/fs/tty/master.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/queue.go b/pkg/sentry/fs/tty/queue.go index 5e88d84d9..11fb92be3 100644 --- a/pkg/sentry/fs/tty/queue.go +++ b/pkg/sentry/fs/tty/queue.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/slave.go b/pkg/sentry/fs/tty/slave.go index ed080ca0f..0ae57a02c 100644 --- a/pkg/sentry/fs/tty/slave.go +++ b/pkg/sentry/fs/tty/slave.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/terminal.go b/pkg/sentry/fs/tty/terminal.go index 79f9d76d7..2b4160ba5 100644 --- a/pkg/sentry/fs/tty/terminal.go +++ b/pkg/sentry/fs/tty/terminal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/tty_test.go b/pkg/sentry/fs/tty/tty_test.go index ad535838f..d2e75a511 100644 --- a/pkg/sentry/fs/tty/tty_test.go +++ b/pkg/sentry/fs/tty/tty_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/hostcpu/getcpu_amd64.s b/pkg/sentry/hostcpu/getcpu_amd64.s index 409db1450..aa00316da 100644 --- a/pkg/sentry/hostcpu/getcpu_amd64.s +++ b/pkg/sentry/hostcpu/getcpu_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/hostcpu/hostcpu.go b/pkg/sentry/hostcpu/hostcpu.go index 3adc847bb..d78f78402 100644 --- a/pkg/sentry/hostcpu/hostcpu.go +++ b/pkg/sentry/hostcpu/hostcpu.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/hostcpu/hostcpu_test.go b/pkg/sentry/hostcpu/hostcpu_test.go index 38de0e1f6..7d6885c9e 100644 --- a/pkg/sentry/hostcpu/hostcpu_test.go +++ b/pkg/sentry/hostcpu/hostcpu_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/inet/context.go b/pkg/sentry/inet/context.go index d05e96f15..8550c4793 100644 --- a/pkg/sentry/inet/context.go +++ b/pkg/sentry/inet/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/inet/inet.go b/pkg/sentry/inet/inet.go index 8206377cc..7c104fd47 100644 --- a/pkg/sentry/inet/inet.go +++ b/pkg/sentry/inet/inet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/inet/test_stack.go b/pkg/sentry/inet/test_stack.go index 05c1a1792..624371eb6 100644 --- a/pkg/sentry/inet/test_stack.go +++ b/pkg/sentry/inet/test_stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/abstract_socket_namespace.go b/pkg/sentry/kernel/abstract_socket_namespace.go index 1ea2cee36..5ce52e66c 100644 --- a/pkg/sentry/kernel/abstract_socket_namespace.go +++ b/pkg/sentry/kernel/abstract_socket_namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/auth.go b/pkg/sentry/kernel/auth/auth.go index 19f15fd36..847d121aa 100644 --- a/pkg/sentry/kernel/auth/auth.go +++ b/pkg/sentry/kernel/auth/auth.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/capability_set.go b/pkg/sentry/kernel/auth/capability_set.go index 88d6243aa..7a0c967cd 100644 --- a/pkg/sentry/kernel/auth/capability_set.go +++ b/pkg/sentry/kernel/auth/capability_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/context.go b/pkg/sentry/kernel/auth/context.go index f7e945599..16d110610 100644 --- a/pkg/sentry/kernel/auth/context.go +++ b/pkg/sentry/kernel/auth/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/credentials.go b/pkg/sentry/kernel/auth/credentials.go index 2055da196..1511a0324 100644 --- a/pkg/sentry/kernel/auth/credentials.go +++ b/pkg/sentry/kernel/auth/credentials.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/id.go b/pkg/sentry/kernel/auth/id.go index e5bed44d7..0a58ba17c 100644 --- a/pkg/sentry/kernel/auth/id.go +++ b/pkg/sentry/kernel/auth/id.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/id_map.go b/pkg/sentry/kernel/auth/id_map.go index 43f439825..e5d6028d6 100644 --- a/pkg/sentry/kernel/auth/id_map.go +++ b/pkg/sentry/kernel/auth/id_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/id_map_functions.go b/pkg/sentry/kernel/auth/id_map_functions.go index 8f1a189ec..432dbfb6d 100644 --- a/pkg/sentry/kernel/auth/id_map_functions.go +++ b/pkg/sentry/kernel/auth/id_map_functions.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/user_namespace.go b/pkg/sentry/kernel/auth/user_namespace.go index 159940a69..a40dd668f 100644 --- a/pkg/sentry/kernel/auth/user_namespace.go +++ b/pkg/sentry/kernel/auth/user_namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/context.go b/pkg/sentry/kernel/context.go index b629521eb..a1a084eab 100644 --- a/pkg/sentry/kernel/context.go +++ b/pkg/sentry/kernel/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/contexttest/contexttest.go b/pkg/sentry/kernel/contexttest/contexttest.go index eb56a6a07..ae67e2a25 100644 --- a/pkg/sentry/kernel/contexttest/contexttest.go +++ b/pkg/sentry/kernel/contexttest/contexttest.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/epoll/epoll.go b/pkg/sentry/kernel/epoll/epoll.go index befefb11c..2399ae6f2 100644 --- a/pkg/sentry/kernel/epoll/epoll.go +++ b/pkg/sentry/kernel/epoll/epoll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/epoll/epoll_state.go b/pkg/sentry/kernel/epoll/epoll_state.go index f6e3e4825..4c3c38f9e 100644 --- a/pkg/sentry/kernel/epoll/epoll_state.go +++ b/pkg/sentry/kernel/epoll/epoll_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/epoll/epoll_test.go b/pkg/sentry/kernel/epoll/epoll_test.go index d89c1b745..49b781b69 100644 --- a/pkg/sentry/kernel/epoll/epoll_test.go +++ b/pkg/sentry/kernel/epoll/epoll_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/eventfd/eventfd.go b/pkg/sentry/kernel/eventfd/eventfd.go index b448ad813..5d3139eef 100644 --- a/pkg/sentry/kernel/eventfd/eventfd.go +++ b/pkg/sentry/kernel/eventfd/eventfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/eventfd/eventfd_test.go b/pkg/sentry/kernel/eventfd/eventfd_test.go index 14e8996d9..1159638e5 100644 --- a/pkg/sentry/kernel/eventfd/eventfd_test.go +++ b/pkg/sentry/kernel/eventfd/eventfd_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/fasync/fasync.go b/pkg/sentry/kernel/fasync/fasync.go index 298d988ea..84cd08501 100644 --- a/pkg/sentry/kernel/fasync/fasync.go +++ b/pkg/sentry/kernel/fasync/fasync.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/fd_map.go b/pkg/sentry/kernel/fd_map.go index 715f4714d..c5636d233 100644 --- a/pkg/sentry/kernel/fd_map.go +++ b/pkg/sentry/kernel/fd_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/fd_map_test.go b/pkg/sentry/kernel/fd_map_test.go index 9e76f0a2d..22db4c7cf 100644 --- a/pkg/sentry/kernel/fd_map_test.go +++ b/pkg/sentry/kernel/fd_map_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/fs_context.go b/pkg/sentry/kernel/fs_context.go index 3cf0db280..d8115f59a 100644 --- a/pkg/sentry/kernel/fs_context.go +++ b/pkg/sentry/kernel/fs_context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/futex/futex.go b/pkg/sentry/kernel/futex/futex.go index cd7d51621..bb38eb81e 100644 --- a/pkg/sentry/kernel/futex/futex.go +++ b/pkg/sentry/kernel/futex/futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/futex/futex_test.go b/pkg/sentry/kernel/futex/futex_test.go index 9d44ee8e5..2de5239bf 100644 --- a/pkg/sentry/kernel/futex/futex_test.go +++ b/pkg/sentry/kernel/futex/futex_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/ipc_namespace.go b/pkg/sentry/kernel/ipc_namespace.go index 9ceb9bd92..ebe12812c 100644 --- a/pkg/sentry/kernel/ipc_namespace.go +++ b/pkg/sentry/kernel/ipc_namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/kdefs/kdefs.go b/pkg/sentry/kernel/kdefs/kdefs.go index 8eafe810b..304da2032 100644 --- a/pkg/sentry/kernel/kdefs/kdefs.go +++ b/pkg/sentry/kernel/kdefs/kdefs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go index a1b2d7161..0468dd678 100644 --- a/pkg/sentry/kernel/kernel.go +++ b/pkg/sentry/kernel/kernel.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/kernel_state.go b/pkg/sentry/kernel/kernel_state.go index aae6f9ad2..48c3ff5a9 100644 --- a/pkg/sentry/kernel/kernel_state.go +++ b/pkg/sentry/kernel/kernel_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/memevent/memory_events.go b/pkg/sentry/kernel/memevent/memory_events.go index d09d6debf..0e2cee807 100644 --- a/pkg/sentry/kernel/memevent/memory_events.go +++ b/pkg/sentry/kernel/memevent/memory_events.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/memevent/memory_events.proto b/pkg/sentry/kernel/memevent/memory_events.proto index 43b8deb76..bf8029ff5 100644 --- a/pkg/sentry/kernel/memevent/memory_events.proto +++ b/pkg/sentry/kernel/memevent/memory_events.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pending_signals.go b/pkg/sentry/kernel/pending_signals.go index deff6def9..c93f6598a 100644 --- a/pkg/sentry/kernel/pending_signals.go +++ b/pkg/sentry/kernel/pending_signals.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pending_signals_state.go b/pkg/sentry/kernel/pending_signals_state.go index 72be6702f..2c902c7e3 100644 --- a/pkg/sentry/kernel/pending_signals_state.go +++ b/pkg/sentry/kernel/pending_signals_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/buffers.go b/pkg/sentry/kernel/pipe/buffers.go index 54e059f8b..ba53fd482 100644 --- a/pkg/sentry/kernel/pipe/buffers.go +++ b/pkg/sentry/kernel/pipe/buffers.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/device.go b/pkg/sentry/kernel/pipe/device.go index eec5c5de8..eb59e15a1 100644 --- a/pkg/sentry/kernel/pipe/device.go +++ b/pkg/sentry/kernel/pipe/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/node.go b/pkg/sentry/kernel/pipe/node.go index 1336b6293..99188dddf 100644 --- a/pkg/sentry/kernel/pipe/node.go +++ b/pkg/sentry/kernel/pipe/node.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/node_test.go b/pkg/sentry/kernel/pipe/node_test.go index ad103b195..7ddecdad8 100644 --- a/pkg/sentry/kernel/pipe/node_test.go +++ b/pkg/sentry/kernel/pipe/node_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/pipe.go b/pkg/sentry/kernel/pipe/pipe.go index 357d1162e..bd7649d2f 100644 --- a/pkg/sentry/kernel/pipe/pipe.go +++ b/pkg/sentry/kernel/pipe/pipe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/pipe_test.go b/pkg/sentry/kernel/pipe/pipe_test.go index 3b9895927..de340c40c 100644 --- a/pkg/sentry/kernel/pipe/pipe_test.go +++ b/pkg/sentry/kernel/pipe/pipe_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/reader.go b/pkg/sentry/kernel/pipe/reader.go index f27379969..48fab45d1 100644 --- a/pkg/sentry/kernel/pipe/reader.go +++ b/pkg/sentry/kernel/pipe/reader.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/reader_writer.go b/pkg/sentry/kernel/pipe/reader_writer.go index 1090432d7..ddcc5e09a 100644 --- a/pkg/sentry/kernel/pipe/reader_writer.go +++ b/pkg/sentry/kernel/pipe/reader_writer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/writer.go b/pkg/sentry/kernel/pipe/writer.go index 6fea9769c..0f29fbc43 100644 --- a/pkg/sentry/kernel/pipe/writer.go +++ b/pkg/sentry/kernel/pipe/writer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/posixtimer.go b/pkg/sentry/kernel/posixtimer.go index 40b5acca3..a016b4087 100644 --- a/pkg/sentry/kernel/posixtimer.go +++ b/pkg/sentry/kernel/posixtimer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/ptrace.go b/pkg/sentry/kernel/ptrace.go index 15f2e2964..4423e7efd 100644 --- a/pkg/sentry/kernel/ptrace.go +++ b/pkg/sentry/kernel/ptrace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/ptrace_amd64.go b/pkg/sentry/kernel/ptrace_amd64.go index 1f88efca3..048eeaa3f 100644 --- a/pkg/sentry/kernel/ptrace_amd64.go +++ b/pkg/sentry/kernel/ptrace_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google Inc. +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/ptrace_arm64.go b/pkg/sentry/kernel/ptrace_arm64.go index 4636405e6..4899c813f 100644 --- a/pkg/sentry/kernel/ptrace_arm64.go +++ b/pkg/sentry/kernel/ptrace_arm64.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google Inc. +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/rseq.go b/pkg/sentry/kernel/rseq.go index 6d3314e81..c4fb2c56c 100644 --- a/pkg/sentry/kernel/rseq.go +++ b/pkg/sentry/kernel/rseq.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/sched/cpuset.go b/pkg/sentry/kernel/sched/cpuset.go index 41ac1067d..c6c436690 100644 --- a/pkg/sentry/kernel/sched/cpuset.go +++ b/pkg/sentry/kernel/sched/cpuset.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/sched/cpuset_test.go b/pkg/sentry/kernel/sched/cpuset_test.go index a036ed513..3af9f1197 100644 --- a/pkg/sentry/kernel/sched/cpuset_test.go +++ b/pkg/sentry/kernel/sched/cpuset_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/sched/sched.go b/pkg/sentry/kernel/sched/sched.go index e59909baf..de18c9d02 100644 --- a/pkg/sentry/kernel/sched/sched.go +++ b/pkg/sentry/kernel/sched/sched.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/seccomp.go b/pkg/sentry/kernel/seccomp.go index 4bed4d373..cc75eb08a 100644 --- a/pkg/sentry/kernel/seccomp.go +++ b/pkg/sentry/kernel/seccomp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/semaphore/semaphore.go b/pkg/sentry/kernel/semaphore/semaphore.go index 2b7c1a9bc..9d0620e02 100644 --- a/pkg/sentry/kernel/semaphore/semaphore.go +++ b/pkg/sentry/kernel/semaphore/semaphore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/semaphore/semaphore_test.go b/pkg/sentry/kernel/semaphore/semaphore_test.go index 2e51e6ee5..abfcd0fb4 100644 --- a/pkg/sentry/kernel/semaphore/semaphore_test.go +++ b/pkg/sentry/kernel/semaphore/semaphore_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/sessions.go b/pkg/sentry/kernel/sessions.go index 070c2f930..610e199da 100644 --- a/pkg/sentry/kernel/sessions.go +++ b/pkg/sentry/kernel/sessions.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/shm/device.go b/pkg/sentry/kernel/shm/device.go index bbc653ed8..3cb759072 100644 --- a/pkg/sentry/kernel/shm/device.go +++ b/pkg/sentry/kernel/shm/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/shm/shm.go b/pkg/sentry/kernel/shm/shm.go index d4812a065..00393b5f0 100644 --- a/pkg/sentry/kernel/shm/shm.go +++ b/pkg/sentry/kernel/shm/shm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/signal.go b/pkg/sentry/kernel/signal.go index 22a56c6fc..b528ec0dc 100644 --- a/pkg/sentry/kernel/signal.go +++ b/pkg/sentry/kernel/signal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/signal_handlers.go b/pkg/sentry/kernel/signal_handlers.go index 60cbe85b8..ce8bcb5e5 100644 --- a/pkg/sentry/kernel/signal_handlers.go +++ b/pkg/sentry/kernel/signal_handlers.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/syscalls.go b/pkg/sentry/kernel/syscalls.go index 293b21249..0572053db 100644 --- a/pkg/sentry/kernel/syscalls.go +++ b/pkg/sentry/kernel/syscalls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/syscalls_state.go b/pkg/sentry/kernel/syscalls_state.go index 981455d46..00358326b 100644 --- a/pkg/sentry/kernel/syscalls_state.go +++ b/pkg/sentry/kernel/syscalls_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/syslog.go b/pkg/sentry/kernel/syslog.go index 2aecf3eea..175d1b247 100644 --- a/pkg/sentry/kernel/syslog.go +++ b/pkg/sentry/kernel/syslog.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/table_test.go b/pkg/sentry/kernel/table_test.go index 3b29d3c6a..8f7cdb9f3 100644 --- a/pkg/sentry/kernel/table_test.go +++ b/pkg/sentry/kernel/table_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task.go b/pkg/sentry/kernel/task.go index ed2175c37..f9378c2de 100644 --- a/pkg/sentry/kernel/task.go +++ b/pkg/sentry/kernel/task.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_acct.go b/pkg/sentry/kernel/task_acct.go index 24230af89..1ca2a82eb 100644 --- a/pkg/sentry/kernel/task_acct.go +++ b/pkg/sentry/kernel/task_acct.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_block.go b/pkg/sentry/kernel/task_block.go index e5027e551..30a7f6b1e 100644 --- a/pkg/sentry/kernel/task_block.go +++ b/pkg/sentry/kernel/task_block.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_clone.go b/pkg/sentry/kernel/task_clone.go index daf974920..bba8ddd39 100644 --- a/pkg/sentry/kernel/task_clone.go +++ b/pkg/sentry/kernel/task_clone.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_context.go b/pkg/sentry/kernel/task_context.go index ac38dd157..bbd294141 100644 --- a/pkg/sentry/kernel/task_context.go +++ b/pkg/sentry/kernel/task_context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_exec.go b/pkg/sentry/kernel/task_exec.go index b49f902a5..5d1425d5c 100644 --- a/pkg/sentry/kernel/task_exec.go +++ b/pkg/sentry/kernel/task_exec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_exit.go b/pkg/sentry/kernel/task_exit.go index a07956208..6e9701b01 100644 --- a/pkg/sentry/kernel/task_exit.go +++ b/pkg/sentry/kernel/task_exit.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_futex.go b/pkg/sentry/kernel/task_futex.go index 351cf47d7..f98097c2c 100644 --- a/pkg/sentry/kernel/task_futex.go +++ b/pkg/sentry/kernel/task_futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_identity.go b/pkg/sentry/kernel/task_identity.go index 6c9608f8d..17f08729a 100644 --- a/pkg/sentry/kernel/task_identity.go +++ b/pkg/sentry/kernel/task_identity.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_log.go b/pkg/sentry/kernel/task_log.go index f4c881c2d..e0e57e8bd 100644 --- a/pkg/sentry/kernel/task_log.go +++ b/pkg/sentry/kernel/task_log.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_net.go b/pkg/sentry/kernel/task_net.go index fc7cefc1f..04c684c1a 100644 --- a/pkg/sentry/kernel/task_net.go +++ b/pkg/sentry/kernel/task_net.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_run.go b/pkg/sentry/kernel/task_run.go index 7115aa967..4549b437e 100644 --- a/pkg/sentry/kernel/task_run.go +++ b/pkg/sentry/kernel/task_run.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_sched.go b/pkg/sentry/kernel/task_sched.go index 3d654bf93..5455f6ea9 100644 --- a/pkg/sentry/kernel/task_sched.go +++ b/pkg/sentry/kernel/task_sched.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_signals.go b/pkg/sentry/kernel/task_signals.go index 7f2e0df72..654cf7525 100644 --- a/pkg/sentry/kernel/task_signals.go +++ b/pkg/sentry/kernel/task_signals.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_start.go b/pkg/sentry/kernel/task_start.go index b7534c0a2..b42531e57 100644 --- a/pkg/sentry/kernel/task_start.go +++ b/pkg/sentry/kernel/task_start.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_stop.go b/pkg/sentry/kernel/task_stop.go index 1302cadc1..e735a5dd0 100644 --- a/pkg/sentry/kernel/task_stop.go +++ b/pkg/sentry/kernel/task_stop.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_syscall.go b/pkg/sentry/kernel/task_syscall.go index 52f5fde8d..a9283d0df 100644 --- a/pkg/sentry/kernel/task_syscall.go +++ b/pkg/sentry/kernel/task_syscall.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_test.go b/pkg/sentry/kernel/task_test.go index 3f37f505d..b895361d0 100644 --- a/pkg/sentry/kernel/task_test.go +++ b/pkg/sentry/kernel/task_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_usermem.go b/pkg/sentry/kernel/task_usermem.go index cb68799d3..461bd7316 100644 --- a/pkg/sentry/kernel/task_usermem.go +++ b/pkg/sentry/kernel/task_usermem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/thread_group.go b/pkg/sentry/kernel/thread_group.go index 58f3a7ec9..8bd53928e 100644 --- a/pkg/sentry/kernel/thread_group.go +++ b/pkg/sentry/kernel/thread_group.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/threads.go b/pkg/sentry/kernel/threads.go index 4fd6cf4e2..656bbd46c 100644 --- a/pkg/sentry/kernel/threads.go +++ b/pkg/sentry/kernel/threads.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/time/context.go b/pkg/sentry/kernel/time/context.go index 3675ea20d..c0660d362 100644 --- a/pkg/sentry/kernel/time/context.go +++ b/pkg/sentry/kernel/time/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/time/time.go b/pkg/sentry/kernel/time/time.go index ca0f4ba2e..3846cf1ea 100644 --- a/pkg/sentry/kernel/time/time.go +++ b/pkg/sentry/kernel/time/time.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/timekeeper.go b/pkg/sentry/kernel/timekeeper.go index d7bd85e78..505a4fa4f 100644 --- a/pkg/sentry/kernel/timekeeper.go +++ b/pkg/sentry/kernel/timekeeper.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/timekeeper_state.go b/pkg/sentry/kernel/timekeeper_state.go index f3a3ed543..6ce358a05 100644 --- a/pkg/sentry/kernel/timekeeper_state.go +++ b/pkg/sentry/kernel/timekeeper_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/timekeeper_test.go b/pkg/sentry/kernel/timekeeper_test.go index 6084bcb18..a92ad689e 100644 --- a/pkg/sentry/kernel/timekeeper_test.go +++ b/pkg/sentry/kernel/timekeeper_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/uncaught_signal.proto b/pkg/sentry/kernel/uncaught_signal.proto index c7f6a1978..0bdb062cb 100644 --- a/pkg/sentry/kernel/uncaught_signal.proto +++ b/pkg/sentry/kernel/uncaught_signal.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/uts_namespace.go b/pkg/sentry/kernel/uts_namespace.go index ed5f0c031..96fe3cbb9 100644 --- a/pkg/sentry/kernel/uts_namespace.go +++ b/pkg/sentry/kernel/uts_namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/vdso.go b/pkg/sentry/kernel/vdso.go index 3a35f1d00..d40ad74f4 100644 --- a/pkg/sentry/kernel/vdso.go +++ b/pkg/sentry/kernel/vdso.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/version.go b/pkg/sentry/kernel/version.go index 8d2f14209..5640dd71d 100644 --- a/pkg/sentry/kernel/version.go +++ b/pkg/sentry/kernel/version.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/limits/context.go b/pkg/sentry/limits/context.go index bf413eb7d..9200edb52 100644 --- a/pkg/sentry/limits/context.go +++ b/pkg/sentry/limits/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/limits/limits.go b/pkg/sentry/limits/limits.go index b0571739f..b6c22656b 100644 --- a/pkg/sentry/limits/limits.go +++ b/pkg/sentry/limits/limits.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/limits/limits_test.go b/pkg/sentry/limits/limits_test.go index 945428163..658a20f56 100644 --- a/pkg/sentry/limits/limits_test.go +++ b/pkg/sentry/limits/limits_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/limits/linux.go b/pkg/sentry/limits/linux.go index e09d0d2fb..a2b401e3d 100644 --- a/pkg/sentry/limits/linux.go +++ b/pkg/sentry/limits/linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/loader/elf.go b/pkg/sentry/loader/elf.go index 385ad0102..97e32c8ba 100644 --- a/pkg/sentry/loader/elf.go +++ b/pkg/sentry/loader/elf.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/loader/interpreter.go b/pkg/sentry/loader/interpreter.go index 35b83654d..b88062ae5 100644 --- a/pkg/sentry/loader/interpreter.go +++ b/pkg/sentry/loader/interpreter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/loader/loader.go b/pkg/sentry/loader/loader.go index 79051befa..dc1a52398 100644 --- a/pkg/sentry/loader/loader.go +++ b/pkg/sentry/loader/loader.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/loader/vdso.go b/pkg/sentry/loader/vdso.go index 8c196df84..207d8ed3d 100644 --- a/pkg/sentry/loader/vdso.go +++ b/pkg/sentry/loader/vdso.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/loader/vdso_state.go b/pkg/sentry/loader/vdso_state.go index b327f0e1e..db378e90a 100644 --- a/pkg/sentry/loader/vdso_state.go +++ b/pkg/sentry/loader/vdso_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/memmap/mapping_set.go b/pkg/sentry/memmap/mapping_set.go index bd07e9aac..3cf2b338f 100644 --- a/pkg/sentry/memmap/mapping_set.go +++ b/pkg/sentry/memmap/mapping_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/memmap/mapping_set_test.go b/pkg/sentry/memmap/mapping_set_test.go index 45d1d4688..c702555ce 100644 --- a/pkg/sentry/memmap/mapping_set_test.go +++ b/pkg/sentry/memmap/mapping_set_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/memmap/memmap.go b/pkg/sentry/memmap/memmap.go index 3f6f7ebd0..0106c857d 100644 --- a/pkg/sentry/memmap/memmap.go +++ b/pkg/sentry/memmap/memmap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/memutil/memutil.go b/pkg/sentry/memutil/memutil.go index 286d50ca4..a4154c42a 100644 --- a/pkg/sentry/memutil/memutil.go +++ b/pkg/sentry/memutil/memutil.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/memutil/memutil_unsafe.go b/pkg/sentry/memutil/memutil_unsafe.go index bc2c72f55..92eab8a26 100644 --- a/pkg/sentry/memutil/memutil_unsafe.go +++ b/pkg/sentry/memutil/memutil_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/address_space.go b/pkg/sentry/mm/address_space.go index 4dddcf7b5..06f587fde 100644 --- a/pkg/sentry/mm/address_space.go +++ b/pkg/sentry/mm/address_space.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/aio_context.go b/pkg/sentry/mm/aio_context.go index 7075792e0..5c61acf36 100644 --- a/pkg/sentry/mm/aio_context.go +++ b/pkg/sentry/mm/aio_context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/aio_context_state.go b/pkg/sentry/mm/aio_context_state.go index 192a6f744..c37fc9f7b 100644 --- a/pkg/sentry/mm/aio_context_state.go +++ b/pkg/sentry/mm/aio_context_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/debug.go b/pkg/sentry/mm/debug.go index d075ee1ca..fe58cfc4c 100644 --- a/pkg/sentry/mm/debug.go +++ b/pkg/sentry/mm/debug.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/io.go b/pkg/sentry/mm/io.go index 81787a6fd..e4c057d28 100644 --- a/pkg/sentry/mm/io.go +++ b/pkg/sentry/mm/io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/lifecycle.go b/pkg/sentry/mm/lifecycle.go index 2fe03172c..e6aa6f9ef 100644 --- a/pkg/sentry/mm/lifecycle.go +++ b/pkg/sentry/mm/lifecycle.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/metadata.go b/pkg/sentry/mm/metadata.go index 5ef1ba0b1..9768e51f1 100644 --- a/pkg/sentry/mm/metadata.go +++ b/pkg/sentry/mm/metadata.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/mm.go b/pkg/sentry/mm/mm.go index a3417a46e..d25aa5136 100644 --- a/pkg/sentry/mm/mm.go +++ b/pkg/sentry/mm/mm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/mm_test.go b/pkg/sentry/mm/mm_test.go index ae4fba478..f4917419f 100644 --- a/pkg/sentry/mm/mm_test.go +++ b/pkg/sentry/mm/mm_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/pma.go b/pkg/sentry/mm/pma.go index 0cca743ef..ece561ff0 100644 --- a/pkg/sentry/mm/pma.go +++ b/pkg/sentry/mm/pma.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/procfs.go b/pkg/sentry/mm/procfs.go index 7cdbf6e25..c8302a553 100644 --- a/pkg/sentry/mm/procfs.go +++ b/pkg/sentry/mm/procfs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/save_restore.go b/pkg/sentry/mm/save_restore.go index 46e0e0754..0385957bd 100644 --- a/pkg/sentry/mm/save_restore.go +++ b/pkg/sentry/mm/save_restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/shm.go b/pkg/sentry/mm/shm.go index 3bc48c7e7..12913007b 100644 --- a/pkg/sentry/mm/shm.go +++ b/pkg/sentry/mm/shm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/special_mappable.go b/pkg/sentry/mm/special_mappable.go index 3b5161998..687959005 100644 --- a/pkg/sentry/mm/special_mappable.go +++ b/pkg/sentry/mm/special_mappable.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/syscalls.go b/pkg/sentry/mm/syscalls.go index 7b675b9b5..a25318abb 100644 --- a/pkg/sentry/mm/syscalls.go +++ b/pkg/sentry/mm/syscalls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/vma.go b/pkg/sentry/mm/vma.go index 931995254..ad901344b 100644 --- a/pkg/sentry/mm/vma.go +++ b/pkg/sentry/mm/vma.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/pgalloc/context.go b/pkg/sentry/pgalloc/context.go index adc97e78f..cb9809b1f 100644 --- a/pkg/sentry/pgalloc/context.go +++ b/pkg/sentry/pgalloc/context.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google Inc. +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/pgalloc/pgalloc.go b/pkg/sentry/pgalloc/pgalloc.go index 0754e608f..411dafa07 100644 --- a/pkg/sentry/pgalloc/pgalloc.go +++ b/pkg/sentry/pgalloc/pgalloc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/pgalloc/pgalloc_test.go b/pkg/sentry/pgalloc/pgalloc_test.go index 726623c1a..14a39bb9e 100644 --- a/pkg/sentry/pgalloc/pgalloc_test.go +++ b/pkg/sentry/pgalloc/pgalloc_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/pgalloc/pgalloc_unsafe.go b/pkg/sentry/pgalloc/pgalloc_unsafe.go index 33b0a68a8..a4b5d581c 100644 --- a/pkg/sentry/pgalloc/pgalloc_unsafe.go +++ b/pkg/sentry/pgalloc/pgalloc_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/pgalloc/save_restore.go b/pkg/sentry/pgalloc/save_restore.go index 21024e656..cf169af55 100644 --- a/pkg/sentry/pgalloc/save_restore.go +++ b/pkg/sentry/pgalloc/save_restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/context.go b/pkg/sentry/platform/context.go index cca21a23e..793f57fd7 100644 --- a/pkg/sentry/platform/context.go +++ b/pkg/sentry/platform/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/interrupt/interrupt.go b/pkg/sentry/platform/interrupt/interrupt.go index 9c83f41eb..a4651f500 100644 --- a/pkg/sentry/platform/interrupt/interrupt.go +++ b/pkg/sentry/platform/interrupt/interrupt.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/interrupt/interrupt_test.go b/pkg/sentry/platform/interrupt/interrupt_test.go index fb3284395..0ecdf6e7a 100644 --- a/pkg/sentry/platform/interrupt/interrupt_test.go +++ b/pkg/sentry/platform/interrupt/interrupt_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/address_space.go b/pkg/sentry/platform/kvm/address_space.go index f2f7ab1e8..689122175 100644 --- a/pkg/sentry/platform/kvm/address_space.go +++ b/pkg/sentry/platform/kvm/address_space.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/allocator.go b/pkg/sentry/platform/kvm/allocator.go index b25cad155..42bcc9733 100644 --- a/pkg/sentry/platform/kvm/allocator.go +++ b/pkg/sentry/platform/kvm/allocator.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill.go b/pkg/sentry/platform/kvm/bluepill.go index f24f1c662..a926e6f8b 100644 --- a/pkg/sentry/platform/kvm/bluepill.go +++ b/pkg/sentry/platform/kvm/bluepill.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill_amd64.go b/pkg/sentry/platform/kvm/bluepill_amd64.go index 6520682d7..c258408f9 100644 --- a/pkg/sentry/platform/kvm/bluepill_amd64.go +++ b/pkg/sentry/platform/kvm/bluepill_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill_amd64.s b/pkg/sentry/platform/kvm/bluepill_amd64.s index 65b01f358..2bc34a435 100644 --- a/pkg/sentry/platform/kvm/bluepill_amd64.s +++ b/pkg/sentry/platform/kvm/bluepill_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go b/pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go index 21de2488e..92fde7ee0 100644 --- a/pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go +++ b/pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill_fault.go b/pkg/sentry/platform/kvm/bluepill_fault.go index e79a30ef2..3c452f5ba 100644 --- a/pkg/sentry/platform/kvm/bluepill_fault.go +++ b/pkg/sentry/platform/kvm/bluepill_fault.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill_unsafe.go b/pkg/sentry/platform/kvm/bluepill_unsafe.go index 2605f8c93..4184939e5 100644 --- a/pkg/sentry/platform/kvm/bluepill_unsafe.go +++ b/pkg/sentry/platform/kvm/bluepill_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/context.go b/pkg/sentry/platform/kvm/context.go index c75a4b415..0eb0020f7 100644 --- a/pkg/sentry/platform/kvm/context.go +++ b/pkg/sentry/platform/kvm/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/kvm.go b/pkg/sentry/platform/kvm/kvm.go index c5a4435b1..ed0521c3f 100644 --- a/pkg/sentry/platform/kvm/kvm.go +++ b/pkg/sentry/platform/kvm/kvm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/kvm_amd64.go b/pkg/sentry/platform/kvm/kvm_amd64.go index 70d0ac63b..61493ccaf 100644 --- a/pkg/sentry/platform/kvm/kvm_amd64.go +++ b/pkg/sentry/platform/kvm/kvm_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go b/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go index d0f6bb225..46c4b9113 100644 --- a/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go +++ b/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/kvm_const.go b/pkg/sentry/platform/kvm/kvm_const.go index cac8d9937..d05f05c29 100644 --- a/pkg/sentry/platform/kvm/kvm_const.go +++ b/pkg/sentry/platform/kvm/kvm_const.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/kvm_test.go b/pkg/sentry/platform/kvm/kvm_test.go index 361200622..e83db71e9 100644 --- a/pkg/sentry/platform/kvm/kvm_test.go +++ b/pkg/sentry/platform/kvm/kvm_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/machine.go b/pkg/sentry/platform/kvm/machine.go index b8b3c9a4a..f5953b96e 100644 --- a/pkg/sentry/platform/kvm/machine.go +++ b/pkg/sentry/platform/kvm/machine.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/machine_amd64.go b/pkg/sentry/platform/kvm/machine_amd64.go index ccfe837b5..b6821122a 100644 --- a/pkg/sentry/platform/kvm/machine_amd64.go +++ b/pkg/sentry/platform/kvm/machine_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/machine_amd64_unsafe.go b/pkg/sentry/platform/kvm/machine_amd64_unsafe.go index 69ba67ced..06a2e3b0c 100644 --- a/pkg/sentry/platform/kvm/machine_amd64_unsafe.go +++ b/pkg/sentry/platform/kvm/machine_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/machine_unsafe.go b/pkg/sentry/platform/kvm/machine_unsafe.go index 22ae60b63..452d88d7f 100644 --- a/pkg/sentry/platform/kvm/machine_unsafe.go +++ b/pkg/sentry/platform/kvm/machine_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/physical_map.go b/pkg/sentry/platform/kvm/physical_map.go index 9d7dca5b3..450eb8201 100644 --- a/pkg/sentry/platform/kvm/physical_map.go +++ b/pkg/sentry/platform/kvm/physical_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/testutil/testutil.go b/pkg/sentry/platform/kvm/testutil/testutil.go index 0d496561d..6cf2359a3 100644 --- a/pkg/sentry/platform/kvm/testutil/testutil.go +++ b/pkg/sentry/platform/kvm/testutil/testutil.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/testutil/testutil_amd64.go b/pkg/sentry/platform/kvm/testutil/testutil_amd64.go index fcba33813..203d71528 100644 --- a/pkg/sentry/platform/kvm/testutil/testutil_amd64.go +++ b/pkg/sentry/platform/kvm/testutil/testutil_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/testutil/testutil_amd64.s b/pkg/sentry/platform/kvm/testutil/testutil_amd64.s index f1da41a44..491ec0c2a 100644 --- a/pkg/sentry/platform/kvm/testutil/testutil_amd64.s +++ b/pkg/sentry/platform/kvm/testutil/testutil_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/virtual_map.go b/pkg/sentry/platform/kvm/virtual_map.go index 0343e9267..28a1b4414 100644 --- a/pkg/sentry/platform/kvm/virtual_map.go +++ b/pkg/sentry/platform/kvm/virtual_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/virtual_map_test.go b/pkg/sentry/platform/kvm/virtual_map_test.go index 935e0eb93..d03ec654a 100644 --- a/pkg/sentry/platform/kvm/virtual_map_test.go +++ b/pkg/sentry/platform/kvm/virtual_map_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/mmap_min_addr.go b/pkg/sentry/platform/mmap_min_addr.go index 1bcc1f8e9..90976735b 100644 --- a/pkg/sentry/platform/mmap_min_addr.go +++ b/pkg/sentry/platform/mmap_min_addr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/platform.go b/pkg/sentry/platform/platform.go index 0e48417b9..ae37276ad 100644 --- a/pkg/sentry/platform/platform.go +++ b/pkg/sentry/platform/platform.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/procid/procid.go b/pkg/sentry/platform/procid/procid.go index 3f49ab093..78b92422c 100644 --- a/pkg/sentry/platform/procid/procid.go +++ b/pkg/sentry/platform/procid/procid.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/procid/procid_amd64.s b/pkg/sentry/platform/procid/procid_amd64.s index ef3439c03..272c9fc14 100644 --- a/pkg/sentry/platform/procid/procid_amd64.s +++ b/pkg/sentry/platform/procid/procid_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/procid/procid_arm64.s b/pkg/sentry/platform/procid/procid_arm64.s index 02e907b6b..7a1684a18 100644 --- a/pkg/sentry/platform/procid/procid_arm64.s +++ b/pkg/sentry/platform/procid/procid_arm64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/procid/procid_net_test.go b/pkg/sentry/platform/procid/procid_net_test.go index e8dcc479d..b628e2285 100644 --- a/pkg/sentry/platform/procid/procid_net_test.go +++ b/pkg/sentry/platform/procid/procid_net_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/procid/procid_test.go b/pkg/sentry/platform/procid/procid_test.go index 7a57c7cdc..88dd0b3ae 100644 --- a/pkg/sentry/platform/procid/procid_test.go +++ b/pkg/sentry/platform/procid/procid_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/ptrace.go b/pkg/sentry/platform/ptrace/ptrace.go index 3c0713e95..6a890dd81 100644 --- a/pkg/sentry/platform/ptrace/ptrace.go +++ b/pkg/sentry/platform/ptrace/ptrace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/ptrace_unsafe.go b/pkg/sentry/platform/ptrace/ptrace_unsafe.go index 223b23199..585f6c1fb 100644 --- a/pkg/sentry/platform/ptrace/ptrace_unsafe.go +++ b/pkg/sentry/platform/ptrace/ptrace_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/stub_amd64.s b/pkg/sentry/platform/ptrace/stub_amd64.s index 63f98e40d..64c718d21 100644 --- a/pkg/sentry/platform/ptrace/stub_amd64.s +++ b/pkg/sentry/platform/ptrace/stub_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/stub_unsafe.go b/pkg/sentry/platform/ptrace/stub_unsafe.go index 48c16c4a1..54d5021a9 100644 --- a/pkg/sentry/platform/ptrace/stub_unsafe.go +++ b/pkg/sentry/platform/ptrace/stub_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/subprocess.go b/pkg/sentry/platform/ptrace/subprocess.go index 2a5d699ec..83b43057f 100644 --- a/pkg/sentry/platform/ptrace/subprocess.go +++ b/pkg/sentry/platform/ptrace/subprocess.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/subprocess_amd64.go b/pkg/sentry/platform/ptrace/subprocess_amd64.go index d23a1133e..77a0e908f 100644 --- a/pkg/sentry/platform/ptrace/subprocess_amd64.go +++ b/pkg/sentry/platform/ptrace/subprocess_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/subprocess_linux.go b/pkg/sentry/platform/ptrace/subprocess_linux.go index e2aab8135..2c07b4ac3 100644 --- a/pkg/sentry/platform/ptrace/subprocess_linux.go +++ b/pkg/sentry/platform/ptrace/subprocess_linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go b/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go index 0c9263060..1bf7eab28 100644 --- a/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go +++ b/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/subprocess_unsafe.go b/pkg/sentry/platform/ptrace/subprocess_unsafe.go index ca6c4ac97..17736b05b 100644 --- a/pkg/sentry/platform/ptrace/subprocess_unsafe.go +++ b/pkg/sentry/platform/ptrace/subprocess_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/defs.go b/pkg/sentry/platform/ring0/defs.go index 98d0a6de0..5bbd4612d 100644 --- a/pkg/sentry/platform/ring0/defs.go +++ b/pkg/sentry/platform/ring0/defs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/defs_amd64.go b/pkg/sentry/platform/ring0/defs_amd64.go index 67242b92b..413c3dbc4 100644 --- a/pkg/sentry/platform/ring0/defs_amd64.go +++ b/pkg/sentry/platform/ring0/defs_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/entry_amd64.go b/pkg/sentry/platform/ring0/entry_amd64.go index 4a9affe64..a5ce67885 100644 --- a/pkg/sentry/platform/ring0/entry_amd64.go +++ b/pkg/sentry/platform/ring0/entry_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/entry_amd64.s b/pkg/sentry/platform/ring0/entry_amd64.s index afb040a6f..8cb8c4996 100644 --- a/pkg/sentry/platform/ring0/entry_amd64.s +++ b/pkg/sentry/platform/ring0/entry_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/gen_offsets/main.go b/pkg/sentry/platform/ring0/gen_offsets/main.go index 11c49855f..a4927da2f 100644 --- a/pkg/sentry/platform/ring0/gen_offsets/main.go +++ b/pkg/sentry/platform/ring0/gen_offsets/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/kernel.go b/pkg/sentry/platform/ring0/kernel.go index 19ac6eb7c..900c0bba7 100644 --- a/pkg/sentry/platform/ring0/kernel.go +++ b/pkg/sentry/platform/ring0/kernel.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/kernel_amd64.go b/pkg/sentry/platform/ring0/kernel_amd64.go index 5ed4342dd..3577b5127 100644 --- a/pkg/sentry/platform/ring0/kernel_amd64.go +++ b/pkg/sentry/platform/ring0/kernel_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/kernel_unsafe.go b/pkg/sentry/platform/ring0/kernel_unsafe.go index faf4240e5..16955ad91 100644 --- a/pkg/sentry/platform/ring0/kernel_unsafe.go +++ b/pkg/sentry/platform/ring0/kernel_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/lib_amd64.go b/pkg/sentry/platform/ring0/lib_amd64.go index 2b95a0141..9c5f26962 100644 --- a/pkg/sentry/platform/ring0/lib_amd64.go +++ b/pkg/sentry/platform/ring0/lib_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/lib_amd64.s b/pkg/sentry/platform/ring0/lib_amd64.s index 98a130525..75d742750 100644 --- a/pkg/sentry/platform/ring0/lib_amd64.s +++ b/pkg/sentry/platform/ring0/lib_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/offsets_amd64.go b/pkg/sentry/platform/ring0/offsets_amd64.go index 806e07ec0..85cc3fdad 100644 --- a/pkg/sentry/platform/ring0/offsets_amd64.go +++ b/pkg/sentry/platform/ring0/offsets_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/allocator.go b/pkg/sentry/platform/ring0/pagetables/allocator.go index ee6e90a11..23fd5c352 100644 --- a/pkg/sentry/platform/ring0/pagetables/allocator.go +++ b/pkg/sentry/platform/ring0/pagetables/allocator.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go b/pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go index f48647b3a..1b996b4e2 100644 --- a/pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go +++ b/pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables.go b/pkg/sentry/platform/ring0/pagetables/pagetables.go index c7207ec18..e5dcaada7 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go b/pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go index 746f614e5..7aa6c524e 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go b/pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go index 2f82c4353..a1ec4b109 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables_test.go b/pkg/sentry/platform/ring0/pagetables/pagetables_test.go index 3e5dc7dc7..36e424495 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables_test.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables_x86.go b/pkg/sentry/platform/ring0/pagetables/pagetables_x86.go index 6bd8c3584..ff427fbe9 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables_x86.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pcids_x86.go b/pkg/sentry/platform/ring0/pagetables/pcids_x86.go index 0d9a51aa5..0f029f25d 100644 --- a/pkg/sentry/platform/ring0/pagetables/pcids_x86.go +++ b/pkg/sentry/platform/ring0/pagetables/pcids_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/walker_amd64.go b/pkg/sentry/platform/ring0/pagetables/walker_amd64.go index c4c71d23e..8f9dacd93 100644 --- a/pkg/sentry/platform/ring0/pagetables/walker_amd64.go +++ b/pkg/sentry/platform/ring0/pagetables/walker_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/ring0.go b/pkg/sentry/platform/ring0/ring0.go index 10c51e88d..cdeb1b43a 100644 --- a/pkg/sentry/platform/ring0/ring0.go +++ b/pkg/sentry/platform/ring0/ring0.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/x86.go b/pkg/sentry/platform/ring0/x86.go index 4c6daec22..7e5ceafdb 100644 --- a/pkg/sentry/platform/ring0/x86.go +++ b/pkg/sentry/platform/ring0/x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/atomic_amd64.s b/pkg/sentry/platform/safecopy/atomic_amd64.s index f90b4bfd1..a0cd78f33 100644 --- a/pkg/sentry/platform/safecopy/atomic_amd64.s +++ b/pkg/sentry/platform/safecopy/atomic_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/safecopy.go b/pkg/sentry/platform/safecopy/safecopy.go index 69c66a3b7..5126871eb 100644 --- a/pkg/sentry/platform/safecopy/safecopy.go +++ b/pkg/sentry/platform/safecopy/safecopy.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/safecopy_test.go b/pkg/sentry/platform/safecopy/safecopy_test.go index 1a682d28a..5818f7f9b 100644 --- a/pkg/sentry/platform/safecopy/safecopy_test.go +++ b/pkg/sentry/platform/safecopy/safecopy_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/safecopy_unsafe.go b/pkg/sentry/platform/safecopy/safecopy_unsafe.go index f84527484..eef028e68 100644 --- a/pkg/sentry/platform/safecopy/safecopy_unsafe.go +++ b/pkg/sentry/platform/safecopy/safecopy_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/sighandler_amd64.s b/pkg/sentry/platform/safecopy/sighandler_amd64.s index db7701a29..475ae48e9 100644 --- a/pkg/sentry/platform/safecopy/sighandler_amd64.s +++ b/pkg/sentry/platform/safecopy/sighandler_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/sighandler_arm64.s b/pkg/sentry/platform/safecopy/sighandler_arm64.s index cdfca8207..53e4ac2c1 100644 --- a/pkg/sentry/platform/safecopy/sighandler_arm64.s +++ b/pkg/sentry/platform/safecopy/sighandler_arm64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/block_unsafe.go b/pkg/sentry/safemem/block_unsafe.go index c3a9780d2..1f72deb61 100644 --- a/pkg/sentry/safemem/block_unsafe.go +++ b/pkg/sentry/safemem/block_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/io.go b/pkg/sentry/safemem/io.go index 6cb52439f..5c3d73eb7 100644 --- a/pkg/sentry/safemem/io.go +++ b/pkg/sentry/safemem/io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/io_test.go b/pkg/sentry/safemem/io_test.go index 2eda8c3bb..629741bee 100644 --- a/pkg/sentry/safemem/io_test.go +++ b/pkg/sentry/safemem/io_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/safemem.go b/pkg/sentry/safemem/safemem.go index 090932d3e..3e70d33a2 100644 --- a/pkg/sentry/safemem/safemem.go +++ b/pkg/sentry/safemem/safemem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/seq_test.go b/pkg/sentry/safemem/seq_test.go index fddcaf714..eba4bb535 100644 --- a/pkg/sentry/safemem/seq_test.go +++ b/pkg/sentry/safemem/seq_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/seq_unsafe.go b/pkg/sentry/safemem/seq_unsafe.go index 83a6b7183..354a95dde 100644 --- a/pkg/sentry/safemem/seq_unsafe.go +++ b/pkg/sentry/safemem/seq_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/sighandling/sighandling.go b/pkg/sentry/sighandling/sighandling.go index 571245ce5..659b43363 100644 --- a/pkg/sentry/sighandling/sighandling.go +++ b/pkg/sentry/sighandling/sighandling.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/sighandling/sighandling_unsafe.go b/pkg/sentry/sighandling/sighandling_unsafe.go index db6e71487..aca77888a 100644 --- a/pkg/sentry/sighandling/sighandling_unsafe.go +++ b/pkg/sentry/sighandling/sighandling_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/control/control.go b/pkg/sentry/socket/control/control.go index d44f5e88a..abda364c9 100644 --- a/pkg/sentry/socket/control/control.go +++ b/pkg/sentry/socket/control/control.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/epsocket/device.go b/pkg/sentry/socket/epsocket/device.go index 3cc138eb0..ab4083efe 100644 --- a/pkg/sentry/socket/epsocket/device.go +++ b/pkg/sentry/socket/epsocket/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/epsocket/epsocket.go b/pkg/sentry/socket/epsocket/epsocket.go index 768fa0dfa..520d82f68 100644 --- a/pkg/sentry/socket/epsocket/epsocket.go +++ b/pkg/sentry/socket/epsocket/epsocket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/epsocket/provider.go b/pkg/sentry/socket/epsocket/provider.go index 0d9c2df24..5a89a63fb 100644 --- a/pkg/sentry/socket/epsocket/provider.go +++ b/pkg/sentry/socket/epsocket/provider.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/epsocket/save_restore.go b/pkg/sentry/socket/epsocket/save_restore.go index f19afb6c0..feaafb7cc 100644 --- a/pkg/sentry/socket/epsocket/save_restore.go +++ b/pkg/sentry/socket/epsocket/save_restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/epsocket/stack.go b/pkg/sentry/socket/epsocket/stack.go index 37c48f4bc..edefa225b 100644 --- a/pkg/sentry/socket/epsocket/stack.go +++ b/pkg/sentry/socket/epsocket/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/device.go b/pkg/sentry/socket/hostinet/device.go index c5133f3bb..4267e3691 100644 --- a/pkg/sentry/socket/hostinet/device.go +++ b/pkg/sentry/socket/hostinet/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/hostinet.go b/pkg/sentry/socket/hostinet/hostinet.go index 7858892ab..0d6f51d2b 100644 --- a/pkg/sentry/socket/hostinet/hostinet.go +++ b/pkg/sentry/socket/hostinet/hostinet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/save_restore.go b/pkg/sentry/socket/hostinet/save_restore.go index 3827f082a..1dec33897 100644 --- a/pkg/sentry/socket/hostinet/save_restore.go +++ b/pkg/sentry/socket/hostinet/save_restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/socket.go b/pkg/sentry/socket/hostinet/socket.go index 49349074f..71884d3db 100644 --- a/pkg/sentry/socket/hostinet/socket.go +++ b/pkg/sentry/socket/hostinet/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/socket_unsafe.go b/pkg/sentry/socket/hostinet/socket_unsafe.go index 59c8910ca..eed0c7837 100644 --- a/pkg/sentry/socket/hostinet/socket_unsafe.go +++ b/pkg/sentry/socket/hostinet/socket_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/stack.go b/pkg/sentry/socket/hostinet/stack.go index 4ce73c1f1..9c45991ba 100644 --- a/pkg/sentry/socket/hostinet/stack.go +++ b/pkg/sentry/socket/hostinet/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/message.go b/pkg/sentry/socket/netlink/message.go index a95172cba..5bd3b49ce 100644 --- a/pkg/sentry/socket/netlink/message.go +++ b/pkg/sentry/socket/netlink/message.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/port/port.go b/pkg/sentry/socket/netlink/port/port.go index 20b9a6e37..e9d3275b1 100644 --- a/pkg/sentry/socket/netlink/port/port.go +++ b/pkg/sentry/socket/netlink/port/port.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/port/port_test.go b/pkg/sentry/socket/netlink/port/port_test.go index 49b3b48ab..516f6cd6c 100644 --- a/pkg/sentry/socket/netlink/port/port_test.go +++ b/pkg/sentry/socket/netlink/port/port_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/provider.go b/pkg/sentry/socket/netlink/provider.go index 06786bd50..76cf12fd4 100644 --- a/pkg/sentry/socket/netlink/provider.go +++ b/pkg/sentry/socket/netlink/provider.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/route/protocol.go b/pkg/sentry/socket/netlink/route/protocol.go index e414b829b..9f0a81403 100644 --- a/pkg/sentry/socket/netlink/route/protocol.go +++ b/pkg/sentry/socket/netlink/route/protocol.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/socket.go b/pkg/sentry/socket/netlink/socket.go index a34f9d3ca..dc688eb00 100644 --- a/pkg/sentry/socket/netlink/socket.go +++ b/pkg/sentry/socket/netlink/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/conn/conn.go b/pkg/sentry/socket/rpcinet/conn/conn.go index 64106c4b5..f537c7f63 100644 --- a/pkg/sentry/socket/rpcinet/conn/conn.go +++ b/pkg/sentry/socket/rpcinet/conn/conn.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/device.go b/pkg/sentry/socket/rpcinet/device.go index d2b9f9222..44c0a39b7 100644 --- a/pkg/sentry/socket/rpcinet/device.go +++ b/pkg/sentry/socket/rpcinet/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/notifier/notifier.go b/pkg/sentry/socket/rpcinet/notifier/notifier.go index f06d12231..601e05994 100644 --- a/pkg/sentry/socket/rpcinet/notifier/notifier.go +++ b/pkg/sentry/socket/rpcinet/notifier/notifier.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/rpcinet.go b/pkg/sentry/socket/rpcinet/rpcinet.go index 6c98e6acb..5d4fd4dac 100644 --- a/pkg/sentry/socket/rpcinet/rpcinet.go +++ b/pkg/sentry/socket/rpcinet/rpcinet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index cf8f69efb..c028ed4dd 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/stack.go b/pkg/sentry/socket/rpcinet/stack.go index cb8344ec6..a1be711df 100644 --- a/pkg/sentry/socket/rpcinet/stack.go +++ b/pkg/sentry/socket/rpcinet/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/stack_unsafe.go b/pkg/sentry/socket/rpcinet/stack_unsafe.go index d04fb2069..e53f578ba 100644 --- a/pkg/sentry/socket/rpcinet/stack_unsafe.go +++ b/pkg/sentry/socket/rpcinet/stack_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/socket.go b/pkg/sentry/socket/socket.go index 62ba13782..7e840b452 100644 --- a/pkg/sentry/socket/socket.go +++ b/pkg/sentry/socket/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/device.go b/pkg/sentry/socket/unix/device.go index 41820dbb3..734d39ee6 100644 --- a/pkg/sentry/socket/unix/device.go +++ b/pkg/sentry/socket/unix/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/io.go b/pkg/sentry/socket/unix/io.go index 7d80e4393..382911d51 100644 --- a/pkg/sentry/socket/unix/io.go +++ b/pkg/sentry/socket/unix/io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/transport/connectioned.go b/pkg/sentry/socket/unix/transport/connectioned.go index 62641bb34..18e492862 100644 --- a/pkg/sentry/socket/unix/transport/connectioned.go +++ b/pkg/sentry/socket/unix/transport/connectioned.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/transport/connectioned_state.go b/pkg/sentry/socket/unix/transport/connectioned_state.go index 608a6a97a..7e02a5db8 100644 --- a/pkg/sentry/socket/unix/transport/connectioned_state.go +++ b/pkg/sentry/socket/unix/transport/connectioned_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/transport/connectionless.go b/pkg/sentry/socket/unix/transport/connectionless.go index 728863f3f..43ff875e4 100644 --- a/pkg/sentry/socket/unix/transport/connectionless.go +++ b/pkg/sentry/socket/unix/transport/connectionless.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/transport/queue.go b/pkg/sentry/socket/unix/transport/queue.go index 45a58c600..b650caae7 100644 --- a/pkg/sentry/socket/unix/transport/queue.go +++ b/pkg/sentry/socket/unix/transport/queue.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/transport/unix.go b/pkg/sentry/socket/unix/transport/unix.go index 12b1576bd..d5f7f7aa8 100644 --- a/pkg/sentry/socket/unix/transport/unix.go +++ b/pkg/sentry/socket/unix/transport/unix.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/unix.go b/pkg/sentry/socket/unix/unix.go index 01efd24d3..e9607aa01 100644 --- a/pkg/sentry/socket/unix/unix.go +++ b/pkg/sentry/socket/unix/unix.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/state/state.go b/pkg/sentry/state/state.go index 224f8b709..27fde505b 100644 --- a/pkg/sentry/state/state.go +++ b/pkg/sentry/state/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/state/state_metadata.go b/pkg/sentry/state/state_metadata.go index 7f047b808..b8e128c40 100644 --- a/pkg/sentry/state/state_metadata.go +++ b/pkg/sentry/state/state_metadata.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/state/state_unsafe.go b/pkg/sentry/state/state_unsafe.go index f02e12b2a..7745b6ac6 100644 --- a/pkg/sentry/state/state_unsafe.go +++ b/pkg/sentry/state/state_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/capability.go b/pkg/sentry/strace/capability.go index 9001181e7..f85d6636e 100644 --- a/pkg/sentry/strace/capability.go +++ b/pkg/sentry/strace/capability.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/clone.go b/pkg/sentry/strace/clone.go index e18ce84dc..ff6a432c6 100644 --- a/pkg/sentry/strace/clone.go +++ b/pkg/sentry/strace/clone.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/futex.go b/pkg/sentry/strace/futex.go index f4aa7fcad..24301bda6 100644 --- a/pkg/sentry/strace/futex.go +++ b/pkg/sentry/strace/futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/linux64.go b/pkg/sentry/strace/linux64.go index 6043b8cb1..3650fd6e1 100644 --- a/pkg/sentry/strace/linux64.go +++ b/pkg/sentry/strace/linux64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/open.go b/pkg/sentry/strace/open.go index 3bf348d7a..140727b02 100644 --- a/pkg/sentry/strace/open.go +++ b/pkg/sentry/strace/open.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/poll.go b/pkg/sentry/strace/poll.go index b6b05423c..15605187d 100644 --- a/pkg/sentry/strace/poll.go +++ b/pkg/sentry/strace/poll.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/ptrace.go b/pkg/sentry/strace/ptrace.go index 8c4b79227..485aacb8a 100644 --- a/pkg/sentry/strace/ptrace.go +++ b/pkg/sentry/strace/ptrace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/signal.go b/pkg/sentry/strace/signal.go index 524be0e15..f82460e1c 100644 --- a/pkg/sentry/strace/signal.go +++ b/pkg/sentry/strace/signal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/socket.go b/pkg/sentry/strace/socket.go index 4c1a9d469..dbe53b9a2 100644 --- a/pkg/sentry/strace/socket.go +++ b/pkg/sentry/strace/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/strace.go b/pkg/sentry/strace/strace.go index 434a200d9..f4c1be4ce 100644 --- a/pkg/sentry/strace/strace.go +++ b/pkg/sentry/strace/strace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/strace.proto b/pkg/sentry/strace/strace.proto index f1fc539d6..4b2f73a5f 100644 --- a/pkg/sentry/strace/strace.proto +++ b/pkg/sentry/strace/strace.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/syscalls.go b/pkg/sentry/strace/syscalls.go index 8c897fcbe..eae2d6c12 100644 --- a/pkg/sentry/strace/syscalls.go +++ b/pkg/sentry/strace/syscalls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/epoll.go b/pkg/sentry/syscalls/epoll.go index b90d191b7..ec1eab331 100644 --- a/pkg/sentry/syscalls/epoll.go +++ b/pkg/sentry/syscalls/epoll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/error.go b/pkg/sentry/syscalls/linux/error.go index 304a12dde..1ba3695fb 100644 --- a/pkg/sentry/syscalls/linux/error.go +++ b/pkg/sentry/syscalls/linux/error.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/flags.go b/pkg/sentry/syscalls/linux/flags.go index d2aec963a..d83e12971 100644 --- a/pkg/sentry/syscalls/linux/flags.go +++ b/pkg/sentry/syscalls/linux/flags.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/linux64.go b/pkg/sentry/syscalls/linux/linux64.go index b9b4ccbd1..9a460ebdf 100644 --- a/pkg/sentry/syscalls/linux/linux64.go +++ b/pkg/sentry/syscalls/linux/linux64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sigset.go b/pkg/sentry/syscalls/linux/sigset.go index a033b7c70..5438b664b 100644 --- a/pkg/sentry/syscalls/linux/sigset.go +++ b/pkg/sentry/syscalls/linux/sigset.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_aio.go b/pkg/sentry/syscalls/linux/sys_aio.go index 61c2647bf..1b27b2415 100644 --- a/pkg/sentry/syscalls/linux/sys_aio.go +++ b/pkg/sentry/syscalls/linux/sys_aio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_capability.go b/pkg/sentry/syscalls/linux/sys_capability.go index cf972dc28..622cb8d0d 100644 --- a/pkg/sentry/syscalls/linux/sys_capability.go +++ b/pkg/sentry/syscalls/linux/sys_capability.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_epoll.go b/pkg/sentry/syscalls/linux/sys_epoll.go index 200c46355..1467feb4e 100644 --- a/pkg/sentry/syscalls/linux/sys_epoll.go +++ b/pkg/sentry/syscalls/linux/sys_epoll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_eventfd.go b/pkg/sentry/syscalls/linux/sys_eventfd.go index 903172890..ca4ead488 100644 --- a/pkg/sentry/syscalls/linux/sys_eventfd.go +++ b/pkg/sentry/syscalls/linux/sys_eventfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_file.go b/pkg/sentry/syscalls/linux/sys_file.go index 967464c85..893322647 100644 --- a/pkg/sentry/syscalls/linux/sys_file.go +++ b/pkg/sentry/syscalls/linux/sys_file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_futex.go b/pkg/sentry/syscalls/linux/sys_futex.go index f0c89cba4..7cef4b50c 100644 --- a/pkg/sentry/syscalls/linux/sys_futex.go +++ b/pkg/sentry/syscalls/linux/sys_futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_getdents.go b/pkg/sentry/syscalls/linux/sys_getdents.go index 4b441b31b..1b597d5bc 100644 --- a/pkg/sentry/syscalls/linux/sys_getdents.go +++ b/pkg/sentry/syscalls/linux/sys_getdents.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_identity.go b/pkg/sentry/syscalls/linux/sys_identity.go index 8d594aa83..27e765a2d 100644 --- a/pkg/sentry/syscalls/linux/sys_identity.go +++ b/pkg/sentry/syscalls/linux/sys_identity.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_inotify.go b/pkg/sentry/syscalls/linux/sys_inotify.go index 26a505782..20269a769 100644 --- a/pkg/sentry/syscalls/linux/sys_inotify.go +++ b/pkg/sentry/syscalls/linux/sys_inotify.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_lseek.go b/pkg/sentry/syscalls/linux/sys_lseek.go index ad3bfd761..8aadc6d8c 100644 --- a/pkg/sentry/syscalls/linux/sys_lseek.go +++ b/pkg/sentry/syscalls/linux/sys_lseek.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_mmap.go b/pkg/sentry/syscalls/linux/sys_mmap.go index 805b251b1..64a6e639c 100644 --- a/pkg/sentry/syscalls/linux/sys_mmap.go +++ b/pkg/sentry/syscalls/linux/sys_mmap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_mount.go b/pkg/sentry/syscalls/linux/sys_mount.go index e110a553f..cf613bad0 100644 --- a/pkg/sentry/syscalls/linux/sys_mount.go +++ b/pkg/sentry/syscalls/linux/sys_mount.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_pipe.go b/pkg/sentry/syscalls/linux/sys_pipe.go index 3652c429e..036845c13 100644 --- a/pkg/sentry/syscalls/linux/sys_pipe.go +++ b/pkg/sentry/syscalls/linux/sys_pipe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_poll.go b/pkg/sentry/syscalls/linux/sys_poll.go index 17b6768e5..e32099dd4 100644 --- a/pkg/sentry/syscalls/linux/sys_poll.go +++ b/pkg/sentry/syscalls/linux/sys_poll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_prctl.go b/pkg/sentry/syscalls/linux/sys_prctl.go index 7a29bd9b7..117ae1a0e 100644 --- a/pkg/sentry/syscalls/linux/sys_prctl.go +++ b/pkg/sentry/syscalls/linux/sys_prctl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_random.go b/pkg/sentry/syscalls/linux/sys_random.go index 452dff058..fc3959a7e 100644 --- a/pkg/sentry/syscalls/linux/sys_random.go +++ b/pkg/sentry/syscalls/linux/sys_random.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_read.go b/pkg/sentry/syscalls/linux/sys_read.go index 50c7d7a74..48b0fd49d 100644 --- a/pkg/sentry/syscalls/linux/sys_read.go +++ b/pkg/sentry/syscalls/linux/sys_read.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_rlimit.go b/pkg/sentry/syscalls/linux/sys_rlimit.go index 443334693..8b0379779 100644 --- a/pkg/sentry/syscalls/linux/sys_rlimit.go +++ b/pkg/sentry/syscalls/linux/sys_rlimit.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_rusage.go b/pkg/sentry/syscalls/linux/sys_rusage.go index ab07c77f9..003d718da 100644 --- a/pkg/sentry/syscalls/linux/sys_rusage.go +++ b/pkg/sentry/syscalls/linux/sys_rusage.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_sched.go b/pkg/sentry/syscalls/linux/sys_sched.go index e679a6694..8aea03abe 100644 --- a/pkg/sentry/syscalls/linux/sys_sched.go +++ b/pkg/sentry/syscalls/linux/sys_sched.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_seccomp.go b/pkg/sentry/syscalls/linux/sys_seccomp.go index f08fdf5cb..b4262162a 100644 --- a/pkg/sentry/syscalls/linux/sys_seccomp.go +++ b/pkg/sentry/syscalls/linux/sys_seccomp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_sem.go b/pkg/sentry/syscalls/linux/sys_sem.go index 86f850ef1..5bd61ab87 100644 --- a/pkg/sentry/syscalls/linux/sys_sem.go +++ b/pkg/sentry/syscalls/linux/sys_sem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_shm.go b/pkg/sentry/syscalls/linux/sys_shm.go index a0d3a73c5..d0eceac7c 100644 --- a/pkg/sentry/syscalls/linux/sys_shm.go +++ b/pkg/sentry/syscalls/linux/sys_shm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_signal.go b/pkg/sentry/syscalls/linux/sys_signal.go index a539354c5..7fbeb4fcd 100644 --- a/pkg/sentry/syscalls/linux/sys_signal.go +++ b/pkg/sentry/syscalls/linux/sys_signal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_socket.go b/pkg/sentry/syscalls/linux/sys_socket.go index c8748958a..69862f110 100644 --- a/pkg/sentry/syscalls/linux/sys_socket.go +++ b/pkg/sentry/syscalls/linux/sys_socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_stat.go b/pkg/sentry/syscalls/linux/sys_stat.go index 49c225011..10fc201ef 100644 --- a/pkg/sentry/syscalls/linux/sys_stat.go +++ b/pkg/sentry/syscalls/linux/sys_stat.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_sync.go b/pkg/sentry/syscalls/linux/sys_sync.go index 68488330f..4352482fb 100644 --- a/pkg/sentry/syscalls/linux/sys_sync.go +++ b/pkg/sentry/syscalls/linux/sys_sync.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_sysinfo.go b/pkg/sentry/syscalls/linux/sys_sysinfo.go index 6f7acf98f..ecf88edc1 100644 --- a/pkg/sentry/syscalls/linux/sys_sysinfo.go +++ b/pkg/sentry/syscalls/linux/sys_sysinfo.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_syslog.go b/pkg/sentry/syscalls/linux/sys_syslog.go index 7193b7aed..9efc58d34 100644 --- a/pkg/sentry/syscalls/linux/sys_syslog.go +++ b/pkg/sentry/syscalls/linux/sys_syslog.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_thread.go b/pkg/sentry/syscalls/linux/sys_thread.go index ddcb5b789..23c2f7035 100644 --- a/pkg/sentry/syscalls/linux/sys_thread.go +++ b/pkg/sentry/syscalls/linux/sys_thread.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_time.go b/pkg/sentry/syscalls/linux/sys_time.go index 063fbb106..b4f2609c0 100644 --- a/pkg/sentry/syscalls/linux/sys_time.go +++ b/pkg/sentry/syscalls/linux/sys_time.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_timer.go b/pkg/sentry/syscalls/linux/sys_timer.go index 6baf4599b..04ea7a4e9 100644 --- a/pkg/sentry/syscalls/linux/sys_timer.go +++ b/pkg/sentry/syscalls/linux/sys_timer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_timerfd.go b/pkg/sentry/syscalls/linux/sys_timerfd.go index f70d13682..ec0155cbb 100644 --- a/pkg/sentry/syscalls/linux/sys_timerfd.go +++ b/pkg/sentry/syscalls/linux/sys_timerfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_tls.go b/pkg/sentry/syscalls/linux/sys_tls.go index 8ea78093b..1e8312e00 100644 --- a/pkg/sentry/syscalls/linux/sys_tls.go +++ b/pkg/sentry/syscalls/linux/sys_tls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_utsname.go b/pkg/sentry/syscalls/linux/sys_utsname.go index f7545b965..fa81fe10e 100644 --- a/pkg/sentry/syscalls/linux/sys_utsname.go +++ b/pkg/sentry/syscalls/linux/sys_utsname.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_write.go b/pkg/sentry/syscalls/linux/sys_write.go index e405608c4..1da72d606 100644 --- a/pkg/sentry/syscalls/linux/sys_write.go +++ b/pkg/sentry/syscalls/linux/sys_write.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/timespec.go b/pkg/sentry/syscalls/linux/timespec.go index 752ec326d..fa6fcdc0b 100644 --- a/pkg/sentry/syscalls/linux/timespec.go +++ b/pkg/sentry/syscalls/linux/timespec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/syscalls.go b/pkg/sentry/syscalls/syscalls.go index 425ce900c..5d10b3824 100644 --- a/pkg/sentry/syscalls/syscalls.go +++ b/pkg/sentry/syscalls/syscalls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/calibrated_clock.go b/pkg/sentry/time/calibrated_clock.go index a98bcd7de..c27e391c9 100644 --- a/pkg/sentry/time/calibrated_clock.go +++ b/pkg/sentry/time/calibrated_clock.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/calibrated_clock_test.go b/pkg/sentry/time/calibrated_clock_test.go index a9237630e..d6622bfe2 100644 --- a/pkg/sentry/time/calibrated_clock_test.go +++ b/pkg/sentry/time/calibrated_clock_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/clock_id.go b/pkg/sentry/time/clock_id.go index 1317a5dad..724f59dd9 100644 --- a/pkg/sentry/time/clock_id.go +++ b/pkg/sentry/time/clock_id.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/clocks.go b/pkg/sentry/time/clocks.go index e26386520..837e86094 100644 --- a/pkg/sentry/time/clocks.go +++ b/pkg/sentry/time/clocks.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/muldiv_amd64.s b/pkg/sentry/time/muldiv_amd64.s index bfcb8c724..028c6684e 100644 --- a/pkg/sentry/time/muldiv_amd64.s +++ b/pkg/sentry/time/muldiv_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/muldiv_arm64.s b/pkg/sentry/time/muldiv_arm64.s index 5fa82a136..5ad57a8a3 100644 --- a/pkg/sentry/time/muldiv_arm64.s +++ b/pkg/sentry/time/muldiv_arm64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/parameters.go b/pkg/sentry/time/parameters.go index 8568b1193..63cf7c4a3 100644 --- a/pkg/sentry/time/parameters.go +++ b/pkg/sentry/time/parameters.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/parameters_test.go b/pkg/sentry/time/parameters_test.go index 4a0c4e880..e1b9084ac 100644 --- a/pkg/sentry/time/parameters_test.go +++ b/pkg/sentry/time/parameters_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/sampler.go b/pkg/sentry/time/sampler.go index 445690d49..2140a99b7 100644 --- a/pkg/sentry/time/sampler.go +++ b/pkg/sentry/time/sampler.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/sampler_test.go b/pkg/sentry/time/sampler_test.go index ec0e442b6..3e70a1134 100644 --- a/pkg/sentry/time/sampler_test.go +++ b/pkg/sentry/time/sampler_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/sampler_unsafe.go b/pkg/sentry/time/sampler_unsafe.go index 0f8eb4fc8..e76180217 100644 --- a/pkg/sentry/time/sampler_unsafe.go +++ b/pkg/sentry/time/sampler_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/tsc_amd64.s b/pkg/sentry/time/tsc_amd64.s index e53d477f7..6a8eed664 100644 --- a/pkg/sentry/time/tsc_amd64.s +++ b/pkg/sentry/time/tsc_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/tsc_arm64.s b/pkg/sentry/time/tsc_arm64.s index c1c9760ef..da9fa4112 100644 --- a/pkg/sentry/time/tsc_arm64.s +++ b/pkg/sentry/time/tsc_arm64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/unimpl/events.go b/pkg/sentry/unimpl/events.go index f78f8c981..d92766e2d 100644 --- a/pkg/sentry/unimpl/events.go +++ b/pkg/sentry/unimpl/events.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/unimpl/unimplemented_syscall.proto b/pkg/sentry/unimpl/unimplemented_syscall.proto index 41579b016..0d7a94be7 100644 --- a/pkg/sentry/unimpl/unimplemented_syscall.proto +++ b/pkg/sentry/unimpl/unimplemented_syscall.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/uniqueid/context.go b/pkg/sentry/uniqueid/context.go index 399d98c29..e55b89689 100644 --- a/pkg/sentry/uniqueid/context.go +++ b/pkg/sentry/uniqueid/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usage/cpu.go b/pkg/sentry/usage/cpu.go index cbd7cfe19..bfc282d69 100644 --- a/pkg/sentry/usage/cpu.go +++ b/pkg/sentry/usage/cpu.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usage/io.go b/pkg/sentry/usage/io.go index 8e27a0a88..dfcd3a49d 100644 --- a/pkg/sentry/usage/io.go +++ b/pkg/sentry/usage/io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usage/memory.go b/pkg/sentry/usage/memory.go index 5be9ed9c6..c316f1597 100644 --- a/pkg/sentry/usage/memory.go +++ b/pkg/sentry/usage/memory.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usage/memory_unsafe.go b/pkg/sentry/usage/memory_unsafe.go index a3ae668a5..9e0014ca0 100644 --- a/pkg/sentry/usage/memory_unsafe.go +++ b/pkg/sentry/usage/memory_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usage/usage.go b/pkg/sentry/usage/usage.go index ab327f8e2..e3d33a965 100644 --- a/pkg/sentry/usage/usage.go +++ b/pkg/sentry/usage/usage.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/access_type.go b/pkg/sentry/usermem/access_type.go index 9e6a27bcf..9c1742a59 100644 --- a/pkg/sentry/usermem/access_type.go +++ b/pkg/sentry/usermem/access_type.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/addr.go b/pkg/sentry/usermem/addr.go index 2a75aa60c..e79210804 100644 --- a/pkg/sentry/usermem/addr.go +++ b/pkg/sentry/usermem/addr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/addr_range_seq_test.go b/pkg/sentry/usermem/addr_range_seq_test.go index bd6a1ec8a..82f735026 100644 --- a/pkg/sentry/usermem/addr_range_seq_test.go +++ b/pkg/sentry/usermem/addr_range_seq_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/addr_range_seq_unsafe.go b/pkg/sentry/usermem/addr_range_seq_unsafe.go index f5fd446fa..c09337c15 100644 --- a/pkg/sentry/usermem/addr_range_seq_unsafe.go +++ b/pkg/sentry/usermem/addr_range_seq_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/bytes_io.go b/pkg/sentry/usermem/bytes_io.go index 274f568d0..f98d82168 100644 --- a/pkg/sentry/usermem/bytes_io.go +++ b/pkg/sentry/usermem/bytes_io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/bytes_io_unsafe.go b/pkg/sentry/usermem/bytes_io_unsafe.go index 7add8bc82..bb49d2ff3 100644 --- a/pkg/sentry/usermem/bytes_io_unsafe.go +++ b/pkg/sentry/usermem/bytes_io_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/usermem.go b/pkg/sentry/usermem/usermem.go index 4c7d5014a..31e4d6ada 100644 --- a/pkg/sentry/usermem/usermem.go +++ b/pkg/sentry/usermem/usermem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/usermem_arm64.go b/pkg/sentry/usermem/usermem_arm64.go index 7fd4ce963..fdfc30a66 100644 --- a/pkg/sentry/usermem/usermem_arm64.go +++ b/pkg/sentry/usermem/usermem_arm64.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/usermem_test.go b/pkg/sentry/usermem/usermem_test.go index 1991a9641..4a07118b7 100644 --- a/pkg/sentry/usermem/usermem_test.go +++ b/pkg/sentry/usermem/usermem_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/usermem_unsafe.go b/pkg/sentry/usermem/usermem_unsafe.go index 3895e7871..876783e78 100644 --- a/pkg/sentry/usermem/usermem_unsafe.go +++ b/pkg/sentry/usermem/usermem_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/usermem_x86.go b/pkg/sentry/usermem/usermem_x86.go index 9ec90f9ff..8059b72d2 100644 --- a/pkg/sentry/usermem/usermem_x86.go +++ b/pkg/sentry/usermem/usermem_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/watchdog/watchdog.go b/pkg/sentry/watchdog/watchdog.go index b4f1e3a4f..2fc4472dd 100644 --- a/pkg/sentry/watchdog/watchdog.go +++ b/pkg/sentry/watchdog/watchdog.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/commit_amd64.s b/pkg/sleep/commit_amd64.s index d08df7f37..bc4ac2c3c 100644 --- a/pkg/sleep/commit_amd64.s +++ b/pkg/sleep/commit_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/commit_asm.go b/pkg/sleep/commit_asm.go index 90eef4cbc..35e2cc337 100644 --- a/pkg/sleep/commit_asm.go +++ b/pkg/sleep/commit_asm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/commit_noasm.go b/pkg/sleep/commit_noasm.go index 967d22e24..686b1da3d 100644 --- a/pkg/sleep/commit_noasm.go +++ b/pkg/sleep/commit_noasm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/empty.s b/pkg/sleep/empty.s index 85d52cd9c..fb37360ac 100644 --- a/pkg/sleep/empty.s +++ b/pkg/sleep/empty.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/sleep_test.go b/pkg/sleep/sleep_test.go index 8feb9ffc2..130806c86 100644 --- a/pkg/sleep/sleep_test.go +++ b/pkg/sleep/sleep_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/sleep_unsafe.go b/pkg/sleep/sleep_unsafe.go index 45fb6f0ea..62e0abc34 100644 --- a/pkg/sleep/sleep_unsafe.go +++ b/pkg/sleep/sleep_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/decode.go b/pkg/state/decode.go index 54b5ad8b8..73a59f871 100644 --- a/pkg/state/decode.go +++ b/pkg/state/decode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/encode.go b/pkg/state/encode.go index fe8512bbf..b0714170b 100644 --- a/pkg/state/encode.go +++ b/pkg/state/encode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/encode_unsafe.go b/pkg/state/encode_unsafe.go index be94742a8..457e6dbb7 100644 --- a/pkg/state/encode_unsafe.go +++ b/pkg/state/encode_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/map.go b/pkg/state/map.go index 0035d7250..1fb9b47b8 100644 --- a/pkg/state/map.go +++ b/pkg/state/map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/object.proto b/pkg/state/object.proto index d3b46ea97..952289069 100644 --- a/pkg/state/object.proto +++ b/pkg/state/object.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/printer.go b/pkg/state/printer.go index aee4b69fb..5174c3ba3 100644 --- a/pkg/state/printer.go +++ b/pkg/state/printer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/state.go b/pkg/state/state.go index 4486f83a7..cf7df803a 100644 --- a/pkg/state/state.go +++ b/pkg/state/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/state_test.go b/pkg/state/state_test.go index 22bcad9e1..7c24bbcda 100644 --- a/pkg/state/state_test.go +++ b/pkg/state/state_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/statefile/statefile.go b/pkg/state/statefile/statefile.go index c21e3bb0e..ad4e3b43e 100644 --- a/pkg/state/statefile/statefile.go +++ b/pkg/state/statefile/statefile.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/statefile/statefile_test.go b/pkg/state/statefile/statefile_test.go index b4f400e01..60b769895 100644 --- a/pkg/state/statefile/statefile_test.go +++ b/pkg/state/statefile/statefile_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/stats.go b/pkg/state/stats.go index 17ca258fc..eb51cda47 100644 --- a/pkg/state/stats.go +++ b/pkg/state/stats.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/syserr/host_linux.go b/pkg/syserr/host_linux.go index 74bbe9f5b..fc6ef60a1 100644 --- a/pkg/syserr/host_linux.go +++ b/pkg/syserr/host_linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/syserr/netstack.go b/pkg/syserr/netstack.go index 1a23919ef..bd489b424 100644 --- a/pkg/syserr/netstack.go +++ b/pkg/syserr/netstack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/syserr/syserr.go b/pkg/syserr/syserr.go index 232634dd4..4ddbd3322 100644 --- a/pkg/syserr/syserr.go +++ b/pkg/syserr/syserr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/syserror/syserror.go b/pkg/syserror/syserror.go index 5558cccff..345653544 100644 --- a/pkg/syserror/syserror.go +++ b/pkg/syserror/syserror.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/syserror/syserror_test.go b/pkg/syserror/syserror_test.go index 0f0da5781..f2a10ee7b 100644 --- a/pkg/syserror/syserror_test.go +++ b/pkg/syserror/syserror_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/adapters/gonet/gonet.go b/pkg/tcpip/adapters/gonet/gonet.go index 628e28f57..df8bf435d 100644 --- a/pkg/tcpip/adapters/gonet/gonet.go +++ b/pkg/tcpip/adapters/gonet/gonet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/adapters/gonet/gonet_test.go b/pkg/tcpip/adapters/gonet/gonet_test.go index e84f73feb..2c81c5697 100644 --- a/pkg/tcpip/adapters/gonet/gonet_test.go +++ b/pkg/tcpip/adapters/gonet/gonet_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/buffer/prependable.go b/pkg/tcpip/buffer/prependable.go index d3a9a0f88..43cbbc74c 100644 --- a/pkg/tcpip/buffer/prependable.go +++ b/pkg/tcpip/buffer/prependable.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/buffer/view.go b/pkg/tcpip/buffer/view.go index 43cbb9461..1a9d40778 100644 --- a/pkg/tcpip/buffer/view.go +++ b/pkg/tcpip/buffer/view.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/buffer/view_test.go b/pkg/tcpip/buffer/view_test.go index 74a0a96fc..ebc3a17b7 100644 --- a/pkg/tcpip/buffer/view_test.go +++ b/pkg/tcpip/buffer/view_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/checker/checker.go b/pkg/tcpip/checker/checker.go index 5dfb3ca1d..6e7edf3ab 100644 --- a/pkg/tcpip/checker/checker.go +++ b/pkg/tcpip/checker/checker.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/hash/jenkins/jenkins.go b/pkg/tcpip/hash/jenkins/jenkins.go index e66d5f12b..52c22230e 100644 --- a/pkg/tcpip/hash/jenkins/jenkins.go +++ b/pkg/tcpip/hash/jenkins/jenkins.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/hash/jenkins/jenkins_test.go b/pkg/tcpip/hash/jenkins/jenkins_test.go index 9d86174aa..4c78b5808 100644 --- a/pkg/tcpip/hash/jenkins/jenkins_test.go +++ b/pkg/tcpip/hash/jenkins/jenkins_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/arp.go b/pkg/tcpip/header/arp.go index 22b259ccb..55fe7292c 100644 --- a/pkg/tcpip/header/arp.go +++ b/pkg/tcpip/header/arp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/checksum.go b/pkg/tcpip/header/checksum.go index 2e8c65fac..2eaa7938a 100644 --- a/pkg/tcpip/header/checksum.go +++ b/pkg/tcpip/header/checksum.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/eth.go b/pkg/tcpip/header/eth.go index 77365bc41..76143f454 100644 --- a/pkg/tcpip/header/eth.go +++ b/pkg/tcpip/header/eth.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/gue.go b/pkg/tcpip/header/gue.go index 2ad13955a..10d358c0e 100644 --- a/pkg/tcpip/header/gue.go +++ b/pkg/tcpip/header/gue.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/icmpv4.go b/pkg/tcpip/header/icmpv4.go index 3ac89cdae..782e1053c 100644 --- a/pkg/tcpip/header/icmpv4.go +++ b/pkg/tcpip/header/icmpv4.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/icmpv6.go b/pkg/tcpip/header/icmpv6.go index e317975e8..d0b10d849 100644 --- a/pkg/tcpip/header/icmpv6.go +++ b/pkg/tcpip/header/icmpv6.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/interfaces.go b/pkg/tcpip/header/interfaces.go index ac327d8a5..fb250ea30 100644 --- a/pkg/tcpip/header/interfaces.go +++ b/pkg/tcpip/header/interfaces.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/ipv4.go b/pkg/tcpip/header/ipv4.go index c3b8fb00e..96e461491 100644 --- a/pkg/tcpip/header/ipv4.go +++ b/pkg/tcpip/header/ipv4.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/ipv6.go b/pkg/tcpip/header/ipv6.go index 3d24736c7..66820a466 100644 --- a/pkg/tcpip/header/ipv6.go +++ b/pkg/tcpip/header/ipv6.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/ipv6_fragment.go b/pkg/tcpip/header/ipv6_fragment.go index e36d5177b..6d896355a 100644 --- a/pkg/tcpip/header/ipv6_fragment.go +++ b/pkg/tcpip/header/ipv6_fragment.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/ipversion_test.go b/pkg/tcpip/header/ipversion_test.go index 8301ba5cf..0c830180e 100644 --- a/pkg/tcpip/header/ipversion_test.go +++ b/pkg/tcpip/header/ipversion_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/tcp.go b/pkg/tcpip/header/tcp.go index e656ebb15..0cd89b992 100644 --- a/pkg/tcpip/header/tcp.go +++ b/pkg/tcpip/header/tcp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/tcp_test.go b/pkg/tcpip/header/tcp_test.go index 7cd98df3b..9a2b99489 100644 --- a/pkg/tcpip/header/tcp_test.go +++ b/pkg/tcpip/header/tcp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/udp.go b/pkg/tcpip/header/udp.go index e8c860436..2205fec18 100644 --- a/pkg/tcpip/header/udp.go +++ b/pkg/tcpip/header/udp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/channel/channel.go b/pkg/tcpip/link/channel/channel.go index f7501a1bc..ee9dd8700 100644 --- a/pkg/tcpip/link/channel/channel.go +++ b/pkg/tcpip/link/channel/channel.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/fdbased/endpoint.go b/pkg/tcpip/link/fdbased/endpoint.go index 8f4d67074..4da376774 100644 --- a/pkg/tcpip/link/fdbased/endpoint.go +++ b/pkg/tcpip/link/fdbased/endpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/fdbased/endpoint_test.go b/pkg/tcpip/link/fdbased/endpoint_test.go index c8b037d57..31138e4ac 100644 --- a/pkg/tcpip/link/fdbased/endpoint_test.go +++ b/pkg/tcpip/link/fdbased/endpoint_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/fdbased/endpoint_unsafe.go b/pkg/tcpip/link/fdbased/endpoint_unsafe.go index 36e7fe5a9..97a477b61 100644 --- a/pkg/tcpip/link/fdbased/endpoint_unsafe.go +++ b/pkg/tcpip/link/fdbased/endpoint_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/fdbased/mmap.go b/pkg/tcpip/link/fdbased/mmap.go index f1e71c233..430c85a42 100644 --- a/pkg/tcpip/link/fdbased/mmap.go +++ b/pkg/tcpip/link/fdbased/mmap.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/fdbased/mmap_amd64_unsafe.go b/pkg/tcpip/link/fdbased/mmap_amd64_unsafe.go index e5ac7996d..135da2498 100644 --- a/pkg/tcpip/link/fdbased/mmap_amd64_unsafe.go +++ b/pkg/tcpip/link/fdbased/mmap_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/loopback/loopback.go b/pkg/tcpip/link/loopback/loopback.go index 2dc4bcfda..2c1148123 100644 --- a/pkg/tcpip/link/loopback/loopback.go +++ b/pkg/tcpip/link/loopback/loopback.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/muxed/injectable.go b/pkg/tcpip/link/muxed/injectable.go index b3e71c7fc..be07b7c29 100644 --- a/pkg/tcpip/link/muxed/injectable.go +++ b/pkg/tcpip/link/muxed/injectable.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/muxed/injectable_test.go b/pkg/tcpip/link/muxed/injectable_test.go index 031449a05..5d40dfacc 100644 --- a/pkg/tcpip/link/muxed/injectable_test.go +++ b/pkg/tcpip/link/muxed/injectable_test.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/rawfile/blockingpoll_amd64.s b/pkg/tcpip/link/rawfile/blockingpoll_amd64.s index 9dade5421..b54131573 100644 --- a/pkg/tcpip/link/rawfile/blockingpoll_amd64.s +++ b/pkg/tcpip/link/rawfile/blockingpoll_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/rawfile/blockingpoll_amd64_unsafe.go b/pkg/tcpip/link/rawfile/blockingpoll_amd64_unsafe.go index 3ba96a123..0b51982c6 100644 --- a/pkg/tcpip/link/rawfile/blockingpoll_amd64_unsafe.go +++ b/pkg/tcpip/link/rawfile/blockingpoll_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/rawfile/blockingpoll_unsafe.go b/pkg/tcpip/link/rawfile/blockingpoll_unsafe.go index 94ddad8ea..4eab77c74 100644 --- a/pkg/tcpip/link/rawfile/blockingpoll_unsafe.go +++ b/pkg/tcpip/link/rawfile/blockingpoll_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/rawfile/errors.go b/pkg/tcpip/link/rawfile/errors.go index 7359849b1..8bde41637 100644 --- a/pkg/tcpip/link/rawfile/errors.go +++ b/pkg/tcpip/link/rawfile/errors.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/rawfile/rawfile_unsafe.go b/pkg/tcpip/link/rawfile/rawfile_unsafe.go index fe2779125..86db7a487 100644 --- a/pkg/tcpip/link/rawfile/rawfile_unsafe.go +++ b/pkg/tcpip/link/rawfile/rawfile_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/pipe/pipe.go b/pkg/tcpip/link/sharedmem/pipe/pipe.go index e014324cc..74c9f0311 100644 --- a/pkg/tcpip/link/sharedmem/pipe/pipe.go +++ b/pkg/tcpip/link/sharedmem/pipe/pipe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/pipe/pipe_test.go b/pkg/tcpip/link/sharedmem/pipe/pipe_test.go index 30742ccb1..59ef69a8b 100644 --- a/pkg/tcpip/link/sharedmem/pipe/pipe_test.go +++ b/pkg/tcpip/link/sharedmem/pipe/pipe_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go b/pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go index f491d74a2..62d17029e 100644 --- a/pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go +++ b/pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/pipe/rx.go b/pkg/tcpip/link/sharedmem/pipe/rx.go index 8d641c76f..f22e533ac 100644 --- a/pkg/tcpip/link/sharedmem/pipe/rx.go +++ b/pkg/tcpip/link/sharedmem/pipe/rx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/pipe/tx.go b/pkg/tcpip/link/sharedmem/pipe/tx.go index e75175d98..9841eb231 100644 --- a/pkg/tcpip/link/sharedmem/pipe/tx.go +++ b/pkg/tcpip/link/sharedmem/pipe/tx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/queue/queue_test.go b/pkg/tcpip/link/sharedmem/queue/queue_test.go index 391165bc3..d3f8f4b8b 100644 --- a/pkg/tcpip/link/sharedmem/queue/queue_test.go +++ b/pkg/tcpip/link/sharedmem/queue/queue_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/queue/rx.go b/pkg/tcpip/link/sharedmem/queue/rx.go index d3a5da08a..d9aecf2d9 100644 --- a/pkg/tcpip/link/sharedmem/queue/rx.go +++ b/pkg/tcpip/link/sharedmem/queue/rx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/queue/tx.go b/pkg/tcpip/link/sharedmem/queue/tx.go index 845108db1..a24dccd11 100644 --- a/pkg/tcpip/link/sharedmem/queue/tx.go +++ b/pkg/tcpip/link/sharedmem/queue/tx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/rx.go b/pkg/tcpip/link/sharedmem/rx.go index 3eeab769e..215cb607f 100644 --- a/pkg/tcpip/link/sharedmem/rx.go +++ b/pkg/tcpip/link/sharedmem/rx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/sharedmem.go b/pkg/tcpip/link/sharedmem/sharedmem.go index 6e6aa5a13..e34b780f8 100644 --- a/pkg/tcpip/link/sharedmem/sharedmem.go +++ b/pkg/tcpip/link/sharedmem/sharedmem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/sharedmem_test.go b/pkg/tcpip/link/sharedmem/sharedmem_test.go index 1f44e224c..65b9d7085 100644 --- a/pkg/tcpip/link/sharedmem/sharedmem_test.go +++ b/pkg/tcpip/link/sharedmem/sharedmem_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/sharedmem_unsafe.go b/pkg/tcpip/link/sharedmem/sharedmem_unsafe.go index b91adbaf7..f7e816a41 100644 --- a/pkg/tcpip/link/sharedmem/sharedmem_unsafe.go +++ b/pkg/tcpip/link/sharedmem/sharedmem_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/tx.go b/pkg/tcpip/link/sharedmem/tx.go index 37da34831..ac3577aa6 100644 --- a/pkg/tcpip/link/sharedmem/tx.go +++ b/pkg/tcpip/link/sharedmem/tx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sniffer/pcap.go b/pkg/tcpip/link/sniffer/pcap.go index 3d0d8d852..c16c19647 100644 --- a/pkg/tcpip/link/sniffer/pcap.go +++ b/pkg/tcpip/link/sniffer/pcap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sniffer/sniffer.go b/pkg/tcpip/link/sniffer/sniffer.go index 462a6e3a3..e87ae07d7 100644 --- a/pkg/tcpip/link/sniffer/sniffer.go +++ b/pkg/tcpip/link/sniffer/sniffer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/tun/tun_unsafe.go b/pkg/tcpip/link/tun/tun_unsafe.go index e4c589dda..09ca9b527 100644 --- a/pkg/tcpip/link/tun/tun_unsafe.go +++ b/pkg/tcpip/link/tun/tun_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/waitable/waitable.go b/pkg/tcpip/link/waitable/waitable.go index bd9f9845b..21690a226 100644 --- a/pkg/tcpip/link/waitable/waitable.go +++ b/pkg/tcpip/link/waitable/waitable.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/waitable/waitable_test.go b/pkg/tcpip/link/waitable/waitable_test.go index a2df6be95..62054fb7f 100644 --- a/pkg/tcpip/link/waitable/waitable_test.go +++ b/pkg/tcpip/link/waitable/waitable_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/arp/arp.go b/pkg/tcpip/network/arp/arp.go index 975919e80..a3f2bce3e 100644 --- a/pkg/tcpip/network/arp/arp.go +++ b/pkg/tcpip/network/arp/arp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/arp/arp_test.go b/pkg/tcpip/network/arp/arp_test.go index 14b9cb8b6..1b971b1a3 100644 --- a/pkg/tcpip/network/arp/arp_test.go +++ b/pkg/tcpip/network/arp/arp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/frag_heap.go b/pkg/tcpip/network/fragmentation/frag_heap.go index 55615c8e6..9ad3e5a8a 100644 --- a/pkg/tcpip/network/fragmentation/frag_heap.go +++ b/pkg/tcpip/network/fragmentation/frag_heap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/frag_heap_test.go b/pkg/tcpip/network/fragmentation/frag_heap_test.go index 1b1b72e88..3a2486ba8 100644 --- a/pkg/tcpip/network/fragmentation/frag_heap_test.go +++ b/pkg/tcpip/network/fragmentation/frag_heap_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/fragmentation.go b/pkg/tcpip/network/fragmentation/fragmentation.go index a5dda0398..e90edb375 100644 --- a/pkg/tcpip/network/fragmentation/fragmentation.go +++ b/pkg/tcpip/network/fragmentation/fragmentation.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/fragmentation_test.go b/pkg/tcpip/network/fragmentation/fragmentation_test.go index 5bf3463a9..99ded68a3 100644 --- a/pkg/tcpip/network/fragmentation/fragmentation_test.go +++ b/pkg/tcpip/network/fragmentation/fragmentation_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/reassembler.go b/pkg/tcpip/network/fragmentation/reassembler.go index c9ad2bef6..04f9ab964 100644 --- a/pkg/tcpip/network/fragmentation/reassembler.go +++ b/pkg/tcpip/network/fragmentation/reassembler.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/reassembler_test.go b/pkg/tcpip/network/fragmentation/reassembler_test.go index a2bc9707a..7eee0710d 100644 --- a/pkg/tcpip/network/fragmentation/reassembler_test.go +++ b/pkg/tcpip/network/fragmentation/reassembler_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/hash/hash.go b/pkg/tcpip/network/hash/hash.go index 07960ddf0..0c91905dc 100644 --- a/pkg/tcpip/network/hash/hash.go +++ b/pkg/tcpip/network/hash/hash.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ip_test.go b/pkg/tcpip/network/ip_test.go index 522009fac..4b822e2c6 100644 --- a/pkg/tcpip/network/ip_test.go +++ b/pkg/tcpip/network/ip_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv4/icmp.go b/pkg/tcpip/network/ipv4/icmp.go index 1c3acda4b..9cb81245a 100644 --- a/pkg/tcpip/network/ipv4/icmp.go +++ b/pkg/tcpip/network/ipv4/icmp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv4/ipv4.go b/pkg/tcpip/network/ipv4/ipv4.go index cbdca98a5..c6af0db79 100644 --- a/pkg/tcpip/network/ipv4/ipv4.go +++ b/pkg/tcpip/network/ipv4/ipv4.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv4/ipv4_test.go b/pkg/tcpip/network/ipv4/ipv4_test.go index 42e85564e..146143ab3 100644 --- a/pkg/tcpip/network/ipv4/ipv4_test.go +++ b/pkg/tcpip/network/ipv4/ipv4_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv6/icmp.go b/pkg/tcpip/network/ipv6/icmp.go index be28be36d..9c011e107 100644 --- a/pkg/tcpip/network/ipv6/icmp.go +++ b/pkg/tcpip/network/ipv6/icmp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv6/icmp_test.go b/pkg/tcpip/network/ipv6/icmp_test.go index 8b57a0641..d8737a616 100644 --- a/pkg/tcpip/network/ipv6/icmp_test.go +++ b/pkg/tcpip/network/ipv6/icmp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv6/ipv6.go b/pkg/tcpip/network/ipv6/ipv6.go index 9a743ea80..4b8cd496b 100644 --- a/pkg/tcpip/network/ipv6/ipv6.go +++ b/pkg/tcpip/network/ipv6/ipv6.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/ports/ports.go b/pkg/tcpip/ports/ports.go index d212a5792..a1712b590 100644 --- a/pkg/tcpip/ports/ports.go +++ b/pkg/tcpip/ports/ports.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/ports/ports_test.go b/pkg/tcpip/ports/ports_test.go index 01e7320b4..8466c661b 100644 --- a/pkg/tcpip/ports/ports_test.go +++ b/pkg/tcpip/ports/ports_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/sample/tun_tcp_connect/main.go b/pkg/tcpip/sample/tun_tcp_connect/main.go index cf8900c4d..1681de56e 100644 --- a/pkg/tcpip/sample/tun_tcp_connect/main.go +++ b/pkg/tcpip/sample/tun_tcp_connect/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/sample/tun_tcp_echo/main.go b/pkg/tcpip/sample/tun_tcp_echo/main.go index da6202f97..642607f83 100644 --- a/pkg/tcpip/sample/tun_tcp_echo/main.go +++ b/pkg/tcpip/sample/tun_tcp_echo/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/seqnum/seqnum.go b/pkg/tcpip/seqnum/seqnum.go index f2b988839..b40a3c212 100644 --- a/pkg/tcpip/seqnum/seqnum.go +++ b/pkg/tcpip/seqnum/seqnum.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/linkaddrcache.go b/pkg/tcpip/stack/linkaddrcache.go index 40e4bdb4a..42b9768ae 100644 --- a/pkg/tcpip/stack/linkaddrcache.go +++ b/pkg/tcpip/stack/linkaddrcache.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/linkaddrcache_test.go b/pkg/tcpip/stack/linkaddrcache_test.go index 77a09ca86..91b2ffea8 100644 --- a/pkg/tcpip/stack/linkaddrcache_test.go +++ b/pkg/tcpip/stack/linkaddrcache_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/nic.go b/pkg/tcpip/stack/nic.go index c18571b0f..8008d9870 100644 --- a/pkg/tcpip/stack/nic.go +++ b/pkg/tcpip/stack/nic.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/registration.go b/pkg/tcpip/stack/registration.go index 6e1660051..c70533a35 100644 --- a/pkg/tcpip/stack/registration.go +++ b/pkg/tcpip/stack/registration.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/route.go b/pkg/tcpip/stack/route.go index 8ae562dcd..3d4c282a9 100644 --- a/pkg/tcpip/stack/route.go +++ b/pkg/tcpip/stack/route.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/stack.go b/pkg/tcpip/stack/stack.go index cb9ffe9c2..f204ca790 100644 --- a/pkg/tcpip/stack/stack.go +++ b/pkg/tcpip/stack/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/stack_global_state.go b/pkg/tcpip/stack/stack_global_state.go index 3d7e4b719..dfec4258a 100644 --- a/pkg/tcpip/stack/stack_global_state.go +++ b/pkg/tcpip/stack/stack_global_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/stack_test.go b/pkg/tcpip/stack/stack_test.go index b5375df3c..351f63221 100644 --- a/pkg/tcpip/stack/stack_test.go +++ b/pkg/tcpip/stack/stack_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/transport_demuxer.go b/pkg/tcpip/stack/transport_demuxer.go index a8ac18e72..e8b562ad9 100644 --- a/pkg/tcpip/stack/transport_demuxer.go +++ b/pkg/tcpip/stack/transport_demuxer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/transport_test.go b/pkg/tcpip/stack/transport_test.go index 2df974bf2..8d74f1543 100644 --- a/pkg/tcpip/stack/transport_test.go +++ b/pkg/tcpip/stack/transport_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/tcpip.go b/pkg/tcpip/tcpip.go index b09137f08..9367c8c02 100644 --- a/pkg/tcpip/tcpip.go +++ b/pkg/tcpip/tcpip.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/tcpip_test.go b/pkg/tcpip/tcpip_test.go index 1f7b04398..ebb1c1b56 100644 --- a/pkg/tcpip/tcpip_test.go +++ b/pkg/tcpip/tcpip_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/time.s b/pkg/tcpip/time.s index 85d52cd9c..fb37360ac 100644 --- a/pkg/tcpip/time.s +++ b/pkg/tcpip/time.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/time_unsafe.go b/pkg/tcpip/time_unsafe.go index 7ec5741af..1a307483b 100644 --- a/pkg/tcpip/time_unsafe.go +++ b/pkg/tcpip/time_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/icmp/endpoint.go b/pkg/tcpip/transport/icmp/endpoint.go index 8f2e3aa20..00840cfcf 100644 --- a/pkg/tcpip/transport/icmp/endpoint.go +++ b/pkg/tcpip/transport/icmp/endpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/icmp/endpoint_state.go b/pkg/tcpip/transport/icmp/endpoint_state.go index 8a7909246..332b3cd33 100644 --- a/pkg/tcpip/transport/icmp/endpoint_state.go +++ b/pkg/tcpip/transport/icmp/endpoint_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/icmp/protocol.go b/pkg/tcpip/transport/icmp/protocol.go index 09ee2f892..954fde9d8 100644 --- a/pkg/tcpip/transport/icmp/protocol.go +++ b/pkg/tcpip/transport/icmp/protocol.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/raw/raw.go b/pkg/tcpip/transport/raw/raw.go index f0f60ce91..7004c7ff4 100644 --- a/pkg/tcpip/transport/raw/raw.go +++ b/pkg/tcpip/transport/raw/raw.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/raw/state.go b/pkg/tcpip/transport/raw/state.go index e3891a8b8..e8907ebb1 100644 --- a/pkg/tcpip/transport/raw/state.go +++ b/pkg/tcpip/transport/raw/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/accept.go b/pkg/tcpip/transport/tcp/accept.go index a3894ed8f..e506d7133 100644 --- a/pkg/tcpip/transport/tcp/accept.go +++ b/pkg/tcpip/transport/tcp/accept.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/connect.go b/pkg/tcpip/transport/tcp/connect.go index 6c4a4d95e..eaa67aeb7 100644 --- a/pkg/tcpip/transport/tcp/connect.go +++ b/pkg/tcpip/transport/tcp/connect.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/cubic.go b/pkg/tcpip/transport/tcp/cubic.go index 003525d86..e618cd2b9 100644 --- a/pkg/tcpip/transport/tcp/cubic.go +++ b/pkg/tcpip/transport/tcp/cubic.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/dual_stack_test.go b/pkg/tcpip/transport/tcp/dual_stack_test.go index 2886cc707..43bcfa070 100644 --- a/pkg/tcpip/transport/tcp/dual_stack_test.go +++ b/pkg/tcpip/transport/tcp/dual_stack_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/endpoint.go b/pkg/tcpip/transport/tcp/endpoint.go index 09eff5be1..982f491cc 100644 --- a/pkg/tcpip/transport/tcp/endpoint.go +++ b/pkg/tcpip/transport/tcp/endpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/endpoint_state.go b/pkg/tcpip/transport/tcp/endpoint_state.go index 7f9dabb4d..27b0be046 100644 --- a/pkg/tcpip/transport/tcp/endpoint_state.go +++ b/pkg/tcpip/transport/tcp/endpoint_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/forwarder.go b/pkg/tcpip/transport/tcp/forwarder.go index 6a7efaf1d..e088e24cb 100644 --- a/pkg/tcpip/transport/tcp/forwarder.go +++ b/pkg/tcpip/transport/tcp/forwarder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/protocol.go b/pkg/tcpip/transport/tcp/protocol.go index b5fb160bc..b86473891 100644 --- a/pkg/tcpip/transport/tcp/protocol.go +++ b/pkg/tcpip/transport/tcp/protocol.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/rcv.go b/pkg/tcpip/transport/tcp/rcv.go index fa6bdddba..b08a0e356 100644 --- a/pkg/tcpip/transport/tcp/rcv.go +++ b/pkg/tcpip/transport/tcp/rcv.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/reno.go b/pkg/tcpip/transport/tcp/reno.go index e4f8b7d5a..f83ebc717 100644 --- a/pkg/tcpip/transport/tcp/reno.go +++ b/pkg/tcpip/transport/tcp/reno.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/sack.go b/pkg/tcpip/transport/tcp/sack.go index 24e48fe7b..6a013d99b 100644 --- a/pkg/tcpip/transport/tcp/sack.go +++ b/pkg/tcpip/transport/tcp/sack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/sack_scoreboard.go b/pkg/tcpip/transport/tcp/sack_scoreboard.go index 21878ad82..99560d5b4 100644 --- a/pkg/tcpip/transport/tcp/sack_scoreboard.go +++ b/pkg/tcpip/transport/tcp/sack_scoreboard.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/sack_scoreboard_test.go b/pkg/tcpip/transport/tcp/sack_scoreboard_test.go index 3cf2ff451..8f6890cdf 100644 --- a/pkg/tcpip/transport/tcp/sack_scoreboard_test.go +++ b/pkg/tcpip/transport/tcp/sack_scoreboard_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/segment.go b/pkg/tcpip/transport/tcp/segment.go index c603fe713..187effb6b 100644 --- a/pkg/tcpip/transport/tcp/segment.go +++ b/pkg/tcpip/transport/tcp/segment.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/segment_heap.go b/pkg/tcpip/transport/tcp/segment_heap.go index 98422fadf..9fd061d7d 100644 --- a/pkg/tcpip/transport/tcp/segment_heap.go +++ b/pkg/tcpip/transport/tcp/segment_heap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/segment_queue.go b/pkg/tcpip/transport/tcp/segment_queue.go index 0c637d7ad..3b020e580 100644 --- a/pkg/tcpip/transport/tcp/segment_queue.go +++ b/pkg/tcpip/transport/tcp/segment_queue.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/segment_state.go b/pkg/tcpip/transport/tcp/segment_state.go index 68b049f06..dd7e14aa6 100644 --- a/pkg/tcpip/transport/tcp/segment_state.go +++ b/pkg/tcpip/transport/tcp/segment_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/snd.go b/pkg/tcpip/transport/tcp/snd.go index 6317748cf..50743670e 100644 --- a/pkg/tcpip/transport/tcp/snd.go +++ b/pkg/tcpip/transport/tcp/snd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/snd_state.go b/pkg/tcpip/transport/tcp/snd_state.go index 86bbd643f..12eff8afc 100644 --- a/pkg/tcpip/transport/tcp/snd_state.go +++ b/pkg/tcpip/transport/tcp/snd_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/tcp_sack_test.go b/pkg/tcpip/transport/tcp/tcp_sack_test.go index 06b0702c5..dbfbd5c4f 100644 --- a/pkg/tcpip/transport/tcp/tcp_sack_test.go +++ b/pkg/tcpip/transport/tcp/tcp_sack_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/tcp_test.go b/pkg/tcpip/transport/tcp/tcp_test.go index c5732ad1c..a8b290dae 100644 --- a/pkg/tcpip/transport/tcp/tcp_test.go +++ b/pkg/tcpip/transport/tcp/tcp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/tcp_timestamp_test.go b/pkg/tcpip/transport/tcp/tcp_timestamp_test.go index 87c640967..039bbcfba 100644 --- a/pkg/tcpip/transport/tcp/tcp_timestamp_test.go +++ b/pkg/tcpip/transport/tcp/tcp_timestamp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/testing/context/context.go b/pkg/tcpip/transport/tcp/testing/context/context.go index 6e2fed880..fa721a7f8 100644 --- a/pkg/tcpip/transport/tcp/testing/context/context.go +++ b/pkg/tcpip/transport/tcp/testing/context/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/timer.go b/pkg/tcpip/transport/tcp/timer.go index 38240d2d5..fc1c7cbd2 100644 --- a/pkg/tcpip/transport/tcp/timer.go +++ b/pkg/tcpip/transport/tcp/timer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go b/pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go index b94568fb1..f1dcd36d5 100644 --- a/pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go +++ b/pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go b/pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go index aaeae9b18..435e136de 100644 --- a/pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go +++ b/pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/udp/endpoint.go b/pkg/tcpip/transport/udp/endpoint.go index 1f9251de3..db65a4e88 100644 --- a/pkg/tcpip/transport/udp/endpoint.go +++ b/pkg/tcpip/transport/udp/endpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/udp/endpoint_state.go b/pkg/tcpip/transport/udp/endpoint_state.go index b2daaf751..163dcbc13 100644 --- a/pkg/tcpip/transport/udp/endpoint_state.go +++ b/pkg/tcpip/transport/udp/endpoint_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/udp/forwarder.go b/pkg/tcpip/transport/udp/forwarder.go index d80c47e34..25bdd2929 100644 --- a/pkg/tcpip/transport/udp/forwarder.go +++ b/pkg/tcpip/transport/udp/forwarder.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/udp/protocol.go b/pkg/tcpip/transport/udp/protocol.go index 616a9f388..8b47cce17 100644 --- a/pkg/tcpip/transport/udp/protocol.go +++ b/pkg/tcpip/transport/udp/protocol.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/udp/udp_test.go b/pkg/tcpip/transport/udp/udp_test.go index 2f4e94c58..86a8fa19b 100644 --- a/pkg/tcpip/transport/udp/udp_test.go +++ b/pkg/tcpip/transport/udp/udp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tmutex/tmutex.go b/pkg/tmutex/tmutex.go index df61d89f5..c4685020d 100644 --- a/pkg/tmutex/tmutex.go +++ b/pkg/tmutex/tmutex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tmutex/tmutex_test.go b/pkg/tmutex/tmutex_test.go index a4537cb3b..ce34c7962 100644 --- a/pkg/tmutex/tmutex_test.go +++ b/pkg/tmutex/tmutex_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/unet/unet.go b/pkg/unet/unet.go index 114fb8c5b..2aa1af4ff 100644 --- a/pkg/unet/unet.go +++ b/pkg/unet/unet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/unet/unet_test.go b/pkg/unet/unet_test.go index db5485539..763b23c7c 100644 --- a/pkg/unet/unet_test.go +++ b/pkg/unet/unet_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/unet/unet_unsafe.go b/pkg/unet/unet_unsafe.go index 1d6ec286c..fa0916439 100644 --- a/pkg/unet/unet_unsafe.go +++ b/pkg/unet/unet_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/urpc/urpc.go b/pkg/urpc/urpc.go index 719f0e92f..0f155ec74 100644 --- a/pkg/urpc/urpc.go +++ b/pkg/urpc/urpc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/urpc/urpc_test.go b/pkg/urpc/urpc_test.go index f1b9a85ca..5bf2c5ed2 100644 --- a/pkg/urpc/urpc_test.go +++ b/pkg/urpc/urpc_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/waiter/waiter.go b/pkg/waiter/waiter.go index a6c9dff3c..8a65ed164 100644 --- a/pkg/waiter/waiter.go +++ b/pkg/waiter/waiter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/waiter/waiter_test.go b/pkg/waiter/waiter_test.go index 60853f9c1..c1b94a4f3 100644 --- a/pkg/waiter/waiter_test.go +++ b/pkg/waiter/waiter_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/compat.go b/runsc/boot/compat.go index b3499bcde..c1b33c551 100644 --- a/runsc/boot/compat.go +++ b/runsc/boot/compat.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/compat_amd64.go b/runsc/boot/compat_amd64.go index 0c9472f18..99df5e614 100644 --- a/runsc/boot/compat_amd64.go +++ b/runsc/boot/compat_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/compat_test.go b/runsc/boot/compat_test.go index f1940dd72..ccec3d20c 100644 --- a/runsc/boot/compat_test.go +++ b/runsc/boot/compat_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/config.go b/runsc/boot/config.go index ba47effc1..b6771de30 100644 --- a/runsc/boot/config.go +++ b/runsc/boot/config.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/controller.go b/runsc/boot/controller.go index 712c50ee9..ab7c58838 100644 --- a/runsc/boot/controller.go +++ b/runsc/boot/controller.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/debug.go b/runsc/boot/debug.go index d224d08b7..79f7387ac 100644 --- a/runsc/boot/debug.go +++ b/runsc/boot/debug.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/events.go b/runsc/boot/events.go index 717adfedd..ffd99f5e9 100644 --- a/runsc/boot/events.go +++ b/runsc/boot/events.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/fds.go b/runsc/boot/fds.go index a3d21d963..4e428b49c 100644 --- a/runsc/boot/fds.go +++ b/runsc/boot/fds.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/filter/config.go b/runsc/boot/filter/config.go index 9c72e3b1a..652da1cef 100644 --- a/runsc/boot/filter/config.go +++ b/runsc/boot/filter/config.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/filter/extra_filters.go b/runsc/boot/filter/extra_filters.go index 67f3101fe..5c5ec4e06 100644 --- a/runsc/boot/filter/extra_filters.go +++ b/runsc/boot/filter/extra_filters.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/filter/extra_filters_msan.go b/runsc/boot/filter/extra_filters_msan.go index fb95283ab..ac5a0f1aa 100644 --- a/runsc/boot/filter/extra_filters_msan.go +++ b/runsc/boot/filter/extra_filters_msan.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/filter/extra_filters_race.go b/runsc/boot/filter/extra_filters_race.go index 02a122c95..ba3c1ce87 100644 --- a/runsc/boot/filter/extra_filters_race.go +++ b/runsc/boot/filter/extra_filters_race.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/filter/filter.go b/runsc/boot/filter/filter.go index fb197f9b1..17479e0dd 100644 --- a/runsc/boot/filter/filter.go +++ b/runsc/boot/filter/filter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/fs.go b/runsc/boot/fs.go index 07061b9b3..aeb1c52cc 100644 --- a/runsc/boot/fs.go +++ b/runsc/boot/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/limits.go b/runsc/boot/limits.go index 32e62cdf7..3364aa5e6 100644 --- a/runsc/boot/limits.go +++ b/runsc/boot/limits.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/loader.go b/runsc/boot/loader.go index 75ec19c32..0b5be0a42 100644 --- a/runsc/boot/loader.go +++ b/runsc/boot/loader.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/loader_test.go b/runsc/boot/loader_test.go index 01578cfc5..9a864ad3f 100644 --- a/runsc/boot/loader_test.go +++ b/runsc/boot/loader_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/network.go b/runsc/boot/network.go index 35baa36ad..598ec969e 100644 --- a/runsc/boot/network.go +++ b/runsc/boot/network.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/strace.go b/runsc/boot/strace.go index 028bcc1f4..19c7f8fbd 100644 --- a/runsc/boot/strace.go +++ b/runsc/boot/strace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cgroup/cgroup.go b/runsc/cgroup/cgroup.go index 2b338b6c6..7431b17d6 100644 --- a/runsc/cgroup/cgroup.go +++ b/runsc/cgroup/cgroup.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cgroup/cgroup_test.go b/runsc/cgroup/cgroup_test.go index ecc184f74..548c80e9a 100644 --- a/runsc/cgroup/cgroup_test.go +++ b/runsc/cgroup/cgroup_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/boot.go b/runsc/cmd/boot.go index ff2fa2fb9..ac937f7bc 100644 --- a/runsc/cmd/boot.go +++ b/runsc/cmd/boot.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/capability.go b/runsc/cmd/capability.go index e5da021e5..312e5b471 100644 --- a/runsc/cmd/capability.go +++ b/runsc/cmd/capability.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/capability_test.go b/runsc/cmd/capability_test.go index dd278b32d..ee74d33d8 100644 --- a/runsc/cmd/capability_test.go +++ b/runsc/cmd/capability_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/checkpoint.go b/runsc/cmd/checkpoint.go index f722df055..96d3c3378 100644 --- a/runsc/cmd/checkpoint.go +++ b/runsc/cmd/checkpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/chroot.go b/runsc/cmd/chroot.go index ed1dafef1..1a774db04 100644 --- a/runsc/cmd/chroot.go +++ b/runsc/cmd/chroot.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/cmd.go b/runsc/cmd/cmd.go index 208cf5304..aa7b1a636 100644 --- a/runsc/cmd/cmd.go +++ b/runsc/cmd/cmd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/create.go b/runsc/cmd/create.go index 30c8fa283..629c198fd 100644 --- a/runsc/cmd/create.go +++ b/runsc/cmd/create.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/debug.go b/runsc/cmd/debug.go index 3ee9a9b49..000f694c7 100644 --- a/runsc/cmd/debug.go +++ b/runsc/cmd/debug.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/delete.go b/runsc/cmd/delete.go index 3206b267a..9039723e9 100644 --- a/runsc/cmd/delete.go +++ b/runsc/cmd/delete.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/delete_test.go b/runsc/cmd/delete_test.go index 4a5b4774a..45fc91016 100644 --- a/runsc/cmd/delete_test.go +++ b/runsc/cmd/delete_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/do.go b/runsc/cmd/do.go index 343461130..67d415733 100644 --- a/runsc/cmd/do.go +++ b/runsc/cmd/do.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/events.go b/runsc/cmd/events.go index 208d2f74b..c6bc8fc3a 100644 --- a/runsc/cmd/events.go +++ b/runsc/cmd/events.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/exec.go b/runsc/cmd/exec.go index 718d01067..ad2508405 100644 --- a/runsc/cmd/exec.go +++ b/runsc/cmd/exec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/exec_test.go b/runsc/cmd/exec_test.go index 686c5e150..6f0f258c0 100644 --- a/runsc/cmd/exec_test.go +++ b/runsc/cmd/exec_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/gofer.go b/runsc/cmd/gofer.go index 82487887c..bccb29397 100644 --- a/runsc/cmd/gofer.go +++ b/runsc/cmd/gofer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/gofer_test.go b/runsc/cmd/gofer_test.go index 8e692feb9..cbea7f127 100644 --- a/runsc/cmd/gofer_test.go +++ b/runsc/cmd/gofer_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/kill.go b/runsc/cmd/kill.go index e67f82473..aed5f3291 100644 --- a/runsc/cmd/kill.go +++ b/runsc/cmd/kill.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/list.go b/runsc/cmd/list.go index 1dcea2af0..1f5ca2473 100644 --- a/runsc/cmd/list.go +++ b/runsc/cmd/list.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/path.go b/runsc/cmd/path.go index 1276f0dbd..0e9ef7fa5 100644 --- a/runsc/cmd/path.go +++ b/runsc/cmd/path.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/pause.go b/runsc/cmd/pause.go index 2c93e5f3e..11b36aa10 100644 --- a/runsc/cmd/pause.go +++ b/runsc/cmd/pause.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/ps.go b/runsc/cmd/ps.go index 060d796f2..3a3e6f17a 100644 --- a/runsc/cmd/ps.go +++ b/runsc/cmd/ps.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/restore.go b/runsc/cmd/restore.go index 66b23c38e..27b06713a 100644 --- a/runsc/cmd/restore.go +++ b/runsc/cmd/restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/resume.go b/runsc/cmd/resume.go index 5551d1450..9a2ade41e 100644 --- a/runsc/cmd/resume.go +++ b/runsc/cmd/resume.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/run.go b/runsc/cmd/run.go index be1c1b678..4d5f5c139 100644 --- a/runsc/cmd/run.go +++ b/runsc/cmd/run.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/spec.go b/runsc/cmd/spec.go index 063bd39c5..344da13ba 100644 --- a/runsc/cmd/spec.go +++ b/runsc/cmd/spec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/start.go b/runsc/cmd/start.go index 9e2e0c11d..657726251 100644 --- a/runsc/cmd/start.go +++ b/runsc/cmd/start.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/state.go b/runsc/cmd/state.go index c3ef65ab5..f0d449b19 100644 --- a/runsc/cmd/state.go +++ b/runsc/cmd/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/wait.go b/runsc/cmd/wait.go index 6498dd15c..a55a682f3 100644 --- a/runsc/cmd/wait.go +++ b/runsc/cmd/wait.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/console/console.go b/runsc/console/console.go index 2eb9a8807..64b23639a 100644 --- a/runsc/console/console.go +++ b/runsc/console/console.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/console_test.go b/runsc/container/console_test.go index 0b0dfb4cb..b8af27c15 100644 --- a/runsc/container/console_test.go +++ b/runsc/container/console_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/container.go b/runsc/container/container.go index a30c217f7..884bbc0fb 100644 --- a/runsc/container/container.go +++ b/runsc/container/container.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/container_test.go b/runsc/container/container_test.go index 603c4d929..9458dbb90 100644 --- a/runsc/container/container_test.go +++ b/runsc/container/container_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/hook.go b/runsc/container/hook.go index 6b9e5550a..acae6781e 100644 --- a/runsc/container/hook.go +++ b/runsc/container/hook.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/multi_container_test.go b/runsc/container/multi_container_test.go index 8922e6dbe..e554237cf 100644 --- a/runsc/container/multi_container_test.go +++ b/runsc/container/multi_container_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/shared_volume_test.go b/runsc/container/shared_volume_test.go index 8f81ed630..9d5a592a5 100644 --- a/runsc/container/shared_volume_test.go +++ b/runsc/container/shared_volume_test.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/status.go b/runsc/container/status.go index 234ffb0dd..91d9112f1 100644 --- a/runsc/container/status.go +++ b/runsc/container/status.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/test_app.go b/runsc/container/test_app.go index b5071ada6..62923f1ef 100644 --- a/runsc/container/test_app.go +++ b/runsc/container/test_app.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/filter/config.go b/runsc/fsgofer/filter/config.go index 75a087848..a1ad49fb2 100644 --- a/runsc/fsgofer/filter/config.go +++ b/runsc/fsgofer/filter/config.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/filter/extra_filters.go b/runsc/fsgofer/filter/extra_filters.go index 67f3101fe..5c5ec4e06 100644 --- a/runsc/fsgofer/filter/extra_filters.go +++ b/runsc/fsgofer/filter/extra_filters.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/filter/extra_filters_msan.go b/runsc/fsgofer/filter/extra_filters_msan.go index 7e142b790..553060bc3 100644 --- a/runsc/fsgofer/filter/extra_filters_msan.go +++ b/runsc/fsgofer/filter/extra_filters_msan.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/filter/extra_filters_race.go b/runsc/fsgofer/filter/extra_filters_race.go index 3cd29472a..28555f898 100644 --- a/runsc/fsgofer/filter/extra_filters_race.go +++ b/runsc/fsgofer/filter/extra_filters_race.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/filter/filter.go b/runsc/fsgofer/filter/filter.go index c120d57a6..ff8154369 100644 --- a/runsc/fsgofer/filter/filter.go +++ b/runsc/fsgofer/filter/filter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/fsgofer.go b/runsc/fsgofer/fsgofer.go index c964a2a3b..158f22ddc 100644 --- a/runsc/fsgofer/fsgofer.go +++ b/runsc/fsgofer/fsgofer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/fsgofer_test.go b/runsc/fsgofer/fsgofer_test.go index e74df7ede..695836927 100644 --- a/runsc/fsgofer/fsgofer_test.go +++ b/runsc/fsgofer/fsgofer_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/fsgofer_unsafe.go b/runsc/fsgofer/fsgofer_unsafe.go index 94413db86..58af5e44d 100644 --- a/runsc/fsgofer/fsgofer_unsafe.go +++ b/runsc/fsgofer/fsgofer_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/main.go b/runsc/main.go index b35726a74..11bc73f75 100644 --- a/runsc/main.go +++ b/runsc/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/sandbox/network.go b/runsc/sandbox/network.go index 6c6b665a0..2a68d7043 100644 --- a/runsc/sandbox/network.go +++ b/runsc/sandbox/network.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/sandbox/network_unsafe.go b/runsc/sandbox/network_unsafe.go index f7447f002..2a2a0fb7e 100644 --- a/runsc/sandbox/network_unsafe.go +++ b/runsc/sandbox/network_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/sandbox/sandbox.go b/runsc/sandbox/sandbox.go index 48a0dafe2..dac35ca0b 100644 --- a/runsc/sandbox/sandbox.go +++ b/runsc/sandbox/sandbox.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/specutils/fs.go b/runsc/specutils/fs.go index 98c3b19c0..1f3afb4e4 100644 --- a/runsc/specutils/fs.go +++ b/runsc/specutils/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/specutils/namespace.go b/runsc/specutils/namespace.go index 35da789f4..7d194335c 100644 --- a/runsc/specutils/namespace.go +++ b/runsc/specutils/namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/specutils/specutils.go b/runsc/specutils/specutils.go index ac85bec71..c72207fb4 100644 --- a/runsc/specutils/specutils.go +++ b/runsc/specutils/specutils.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/specutils/specutils_test.go b/runsc/specutils/specutils_test.go index 02af6e6ad..2c86fffe8 100644 --- a/runsc/specutils/specutils_test.go +++ b/runsc/specutils/specutils_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/image/image.go b/runsc/test/image/image.go index bcb6f876f..297f1ab92 100644 --- a/runsc/test/image/image.go +++ b/runsc/test/image/image.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/image/image_test.go b/runsc/test/image/image_test.go index f7e750d71..0c45602f9 100644 --- a/runsc/test/image/image_test.go +++ b/runsc/test/image/image_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/image/mysql.sql b/runsc/test/image/mysql.sql index c1271e719..51554b98d 100644 --- a/runsc/test/image/mysql.sql +++ b/runsc/test/image/mysql.sql @@ -1,4 +1,4 @@ -# Copyright 2018 Google LLC +# Copyright 2018 The gVisor Authors. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/runsc/test/image/ruby.rb b/runsc/test/image/ruby.rb index 25d1ac129..aced49c6d 100644 --- a/runsc/test/image/ruby.rb +++ b/runsc/test/image/ruby.rb @@ -1,4 +1,4 @@ -# Copyright 2018 Google LLC +# Copyright 2018 The gVisor Authors. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/runsc/test/image/ruby.sh b/runsc/test/image/ruby.sh index d3a9b5656..ebe8d5b0e 100644 --- a/runsc/test/image/ruby.sh +++ b/runsc/test/image/ruby.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google LLC +# Copyright 2018 The gVisor Authors. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/runsc/test/install.sh b/runsc/test/install.sh index 32e1e884e..457df2d26 100755 --- a/runsc/test/install.sh +++ b/runsc/test/install.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google LLC +# Copyright 2018 The gVisor Authors. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/runsc/test/integration/exec_test.go b/runsc/test/integration/exec_test.go index d87957e2d..7af064d79 100644 --- a/runsc/test/integration/exec_test.go +++ b/runsc/test/integration/exec_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/integration/integration.go b/runsc/test/integration/integration.go index e15321c87..4cd5f6c24 100644 --- a/runsc/test/integration/integration.go +++ b/runsc/test/integration/integration.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/integration/integration_test.go b/runsc/test/integration/integration_test.go index 4a2770d48..b2e86aacc 100644 --- a/runsc/test/integration/integration_test.go +++ b/runsc/test/integration/integration_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/root/cgroup_test.go b/runsc/test/root/cgroup_test.go index 91839048c..edb6dee1d 100644 --- a/runsc/test/root/cgroup_test.go +++ b/runsc/test/root/cgroup_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/root/chroot_test.go b/runsc/test/root/chroot_test.go index 0deca0532..da2f473b9 100644 --- a/runsc/test/root/chroot_test.go +++ b/runsc/test/root/chroot_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/root/crictl_test.go b/runsc/test/root/crictl_test.go index 37fe53ba3..3cc176104 100644 --- a/runsc/test/root/crictl_test.go +++ b/runsc/test/root/crictl_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/root/root.go b/runsc/test/root/root.go index 586ea0fe3..349c752cc 100644 --- a/runsc/test/root/root.go +++ b/runsc/test/root/root.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/root/testdata/busybox.go b/runsc/test/root/testdata/busybox.go index 544571c63..e4dbd2843 100644 --- a/runsc/test/root/testdata/busybox.go +++ b/runsc/test/root/testdata/busybox.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/root/testdata/containerd_config.go b/runsc/test/root/testdata/containerd_config.go index 949354987..e12f1ec88 100644 --- a/runsc/test/root/testdata/containerd_config.go +++ b/runsc/test/root/testdata/containerd_config.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/root/testdata/httpd.go b/runsc/test/root/testdata/httpd.go index f65b1da5d..45d5e33d4 100644 --- a/runsc/test/root/testdata/httpd.go +++ b/runsc/test/root/testdata/httpd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/root/testdata/httpd_mount_paths.go b/runsc/test/root/testdata/httpd_mount_paths.go index 5ca14340e..ac3f4446a 100644 --- a/runsc/test/root/testdata/httpd_mount_paths.go +++ b/runsc/test/root/testdata/httpd_mount_paths.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/root/testdata/sandbox.go b/runsc/test/root/testdata/sandbox.go index 194242a27..0db210370 100644 --- a/runsc/test/root/testdata/sandbox.go +++ b/runsc/test/root/testdata/sandbox.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/testutil/crictl.go b/runsc/test/testutil/crictl.go index 84bb4475a..4f9ee0c05 100644 --- a/runsc/test/testutil/crictl.go +++ b/runsc/test/testutil/crictl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/testutil/docker.go b/runsc/test/testutil/docker.go index b651319ed..29ef505b4 100644 --- a/runsc/test/testutil/docker.go +++ b/runsc/test/testutil/docker.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/testutil/testutil.go b/runsc/test/testutil/testutil.go index 79f0a8b6b..6a4c045a8 100644 --- a/runsc/test/testutil/testutil.go +++ b/runsc/test/testutil/testutil.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/testutil/testutil_race.go b/runsc/test/testutil/testutil_race.go index 9267af150..86db6ffa1 100644 --- a/runsc/test/testutil/testutil_race.go +++ b/runsc/test/testutil/testutil_race.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/tools/dockercfg/dockercfg.go b/runsc/tools/dockercfg/dockercfg.go index cc7a67816..6fb134558 100644 --- a/runsc/tools/dockercfg/dockercfg.go +++ b/runsc/tools/dockercfg/dockercfg.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/version.go b/runsc/version.go index 4894f2de6..ce0573a9b 100644 --- a/runsc/version.go +++ b/runsc/version.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/gtest/gtest.go b/test/syscalls/gtest/gtest.go index dfe5037cd..bdec8eb07 100644 --- a/test/syscalls/gtest/gtest.go +++ b/test/syscalls/gtest/gtest.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/32bit.cc b/test/syscalls/linux/32bit.cc index 78baf548e..a7cbee06b 100644 --- a/test/syscalls/linux/32bit.cc +++ b/test/syscalls/linux/32bit.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/accept_bind.cc b/test/syscalls/linux/accept_bind.cc index c2bb4a7ce..56377feab 100644 --- a/test/syscalls/linux/accept_bind.cc +++ b/test/syscalls/linux/accept_bind.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/accept_bind_stream.cc b/test/syscalls/linux/accept_bind_stream.cc index 1501e526e..b6cdb3f4f 100644 --- a/test/syscalls/linux/accept_bind_stream.cc +++ b/test/syscalls/linux/accept_bind_stream.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/access.cc b/test/syscalls/linux/access.cc index 6ea070a5d..bcc25cef4 100644 --- a/test/syscalls/linux/access.cc +++ b/test/syscalls/linux/access.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/affinity.cc b/test/syscalls/linux/affinity.cc index 81bd9bcb5..f2d8375b6 100644 --- a/test/syscalls/linux/affinity.cc +++ b/test/syscalls/linux/affinity.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/aio.cc b/test/syscalls/linux/aio.cc index b96aab9b9..68dc05417 100644 --- a/test/syscalls/linux/aio.cc +++ b/test/syscalls/linux/aio.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/alarm.cc b/test/syscalls/linux/alarm.cc index e0ddbb415..d89269985 100644 --- a/test/syscalls/linux/alarm.cc +++ b/test/syscalls/linux/alarm.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/arch_prctl.cc b/test/syscalls/linux/arch_prctl.cc index 5687ceb86..81bf5a775 100644 --- a/test/syscalls/linux/arch_prctl.cc +++ b/test/syscalls/linux/arch_prctl.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/bad.cc b/test/syscalls/linux/bad.cc index a2634a8bf..f246a799e 100644 --- a/test/syscalls/linux/bad.cc +++ b/test/syscalls/linux/bad.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/base_poll_test.cc b/test/syscalls/linux/base_poll_test.cc index bba0108ea..ab7a19dd0 100644 --- a/test/syscalls/linux/base_poll_test.cc +++ b/test/syscalls/linux/base_poll_test.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/base_poll_test.h b/test/syscalls/linux/base_poll_test.h index 9b9b81933..088831f9f 100644 --- a/test/syscalls/linux/base_poll_test.h +++ b/test/syscalls/linux/base_poll_test.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/bind.cc b/test/syscalls/linux/bind.cc index f5aa9c500..de8cca53b 100644 --- a/test/syscalls/linux/bind.cc +++ b/test/syscalls/linux/bind.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/brk.cc b/test/syscalls/linux/brk.cc index 33d353959..a03a44465 100644 --- a/test/syscalls/linux/brk.cc +++ b/test/syscalls/linux/brk.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/chdir.cc b/test/syscalls/linux/chdir.cc index a4b54f0ee..3182c228b 100644 --- a/test/syscalls/linux/chdir.cc +++ b/test/syscalls/linux/chdir.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/chmod.cc b/test/syscalls/linux/chmod.cc index 2f42fe326..79e98597f 100644 --- a/test/syscalls/linux/chmod.cc +++ b/test/syscalls/linux/chmod.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/chown.cc b/test/syscalls/linux/chown.cc index ad892cf6a..eb1762ddf 100644 --- a/test/syscalls/linux/chown.cc +++ b/test/syscalls/linux/chown.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/chroot.cc b/test/syscalls/linux/chroot.cc index 6c200f63e..a4354ff62 100644 --- a/test/syscalls/linux/chroot.cc +++ b/test/syscalls/linux/chroot.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/clock_getres.cc b/test/syscalls/linux/clock_getres.cc index 8f8842299..c408b936c 100644 --- a/test/syscalls/linux/clock_getres.cc +++ b/test/syscalls/linux/clock_getres.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/clock_gettime.cc b/test/syscalls/linux/clock_gettime.cc index 4ecb5f5b1..082ae1c39 100644 --- a/test/syscalls/linux/clock_gettime.cc +++ b/test/syscalls/linux/clock_gettime.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/clock_nanosleep.cc b/test/syscalls/linux/clock_nanosleep.cc index 61c67a5ff..52a69d230 100644 --- a/test/syscalls/linux/clock_nanosleep.cc +++ b/test/syscalls/linux/clock_nanosleep.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/concurrency.cc b/test/syscalls/linux/concurrency.cc index 7978845c1..4e0a13f8b 100644 --- a/test/syscalls/linux/concurrency.cc +++ b/test/syscalls/linux/concurrency.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/creat.cc b/test/syscalls/linux/creat.cc index df2cc0d5c..3c270d6da 100644 --- a/test/syscalls/linux/creat.cc +++ b/test/syscalls/linux/creat.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/dev.cc b/test/syscalls/linux/dev.cc index a140d3b30..b86ebe233 100644 --- a/test/syscalls/linux/dev.cc +++ b/test/syscalls/linux/dev.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/dup.cc b/test/syscalls/linux/dup.cc index e8de2f4c4..4f773bc75 100644 --- a/test/syscalls/linux/dup.cc +++ b/test/syscalls/linux/dup.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/epoll.cc b/test/syscalls/linux/epoll.cc index b4a3bfcba..a4f8f3cec 100644 --- a/test/syscalls/linux/epoll.cc +++ b/test/syscalls/linux/epoll.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/eventfd.cc b/test/syscalls/linux/eventfd.cc index 8111da30e..5e5c39d44 100644 --- a/test/syscalls/linux/eventfd.cc +++ b/test/syscalls/linux/eventfd.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/exceptions.cc b/test/syscalls/linux/exceptions.cc index 3f0aa8bf1..0da4c817d 100644 --- a/test/syscalls/linux/exceptions.cc +++ b/test/syscalls/linux/exceptions.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/exec.cc b/test/syscalls/linux/exec.cc index 30bc4b608..06c322a99 100644 --- a/test/syscalls/linux/exec.cc +++ b/test/syscalls/linux/exec.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/exec.h b/test/syscalls/linux/exec.h index b82bfffd1..5c0f7e654 100644 --- a/test/syscalls/linux/exec.h +++ b/test/syscalls/linux/exec.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/exec_assert_closed_workload.cc b/test/syscalls/linux/exec_assert_closed_workload.cc index 4448431e1..95643618d 100644 --- a/test/syscalls/linux/exec_assert_closed_workload.cc +++ b/test/syscalls/linux/exec_assert_closed_workload.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/exec_basic_workload.cc b/test/syscalls/linux/exec_basic_workload.cc index d4bdf511f..1bbd6437e 100644 --- a/test/syscalls/linux/exec_basic_workload.cc +++ b/test/syscalls/linux/exec_basic_workload.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/exec_binary.cc b/test/syscalls/linux/exec_binary.cc index c10d85398..bdd6eb10b 100644 --- a/test/syscalls/linux/exec_binary.cc +++ b/test/syscalls/linux/exec_binary.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/exec_proc_exe_workload.cc b/test/syscalls/linux/exec_proc_exe_workload.cc index b9a4ac749..b3fbd5042 100644 --- a/test/syscalls/linux/exec_proc_exe_workload.cc +++ b/test/syscalls/linux/exec_proc_exe_workload.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/exec_state_workload.cc b/test/syscalls/linux/exec_state_workload.cc index b66e22565..725c2977f 100644 --- a/test/syscalls/linux/exec_state_workload.cc +++ b/test/syscalls/linux/exec_state_workload.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/exit.cc b/test/syscalls/linux/exit.cc index 7246a7b3b..99de2b376 100644 --- a/test/syscalls/linux/exit.cc +++ b/test/syscalls/linux/exit.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/exit_script.sh b/test/syscalls/linux/exit_script.sh index f014fcf99..527518e06 100755 --- a/test/syscalls/linux/exit_script.sh +++ b/test/syscalls/linux/exit_script.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google LLC +# Copyright 2018 The gVisor Authors. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/fadvise64.cc b/test/syscalls/linux/fadvise64.cc index 041e8b7b6..2af7aa6d9 100644 --- a/test/syscalls/linux/fadvise64.cc +++ b/test/syscalls/linux/fadvise64.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/fallocate.cc b/test/syscalls/linux/fallocate.cc index e51538734..61b8acc7a 100644 --- a/test/syscalls/linux/fallocate.cc +++ b/test/syscalls/linux/fallocate.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/fault.cc b/test/syscalls/linux/fault.cc index cfa7d0d1f..f6e19026f 100644 --- a/test/syscalls/linux/fault.cc +++ b/test/syscalls/linux/fault.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/fchdir.cc b/test/syscalls/linux/fchdir.cc index 2b13e36c3..08bcae1e8 100644 --- a/test/syscalls/linux/fchdir.cc +++ b/test/syscalls/linux/fchdir.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/fcntl.cc b/test/syscalls/linux/fcntl.cc index 32a90a163..2f8e7c9dd 100644 --- a/test/syscalls/linux/fcntl.cc +++ b/test/syscalls/linux/fcntl.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/file_base.h b/test/syscalls/linux/file_base.h index 43f568111..b5b972c07 100644 --- a/test/syscalls/linux/file_base.h +++ b/test/syscalls/linux/file_base.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/flock.cc b/test/syscalls/linux/flock.cc index 1388d3839..d89cfcbd7 100644 --- a/test/syscalls/linux/flock.cc +++ b/test/syscalls/linux/flock.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/fork.cc b/test/syscalls/linux/fork.cc index 73ac885b5..dd6e1a422 100644 --- a/test/syscalls/linux/fork.cc +++ b/test/syscalls/linux/fork.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/fpsig_fork.cc b/test/syscalls/linux/fpsig_fork.cc index e8f1dfa8a..e7e9f06a1 100644 --- a/test/syscalls/linux/fpsig_fork.cc +++ b/test/syscalls/linux/fpsig_fork.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/fpsig_nested.cc b/test/syscalls/linux/fpsig_nested.cc index 2fa40b42d..395463aed 100644 --- a/test/syscalls/linux/fpsig_nested.cc +++ b/test/syscalls/linux/fpsig_nested.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/fsync.cc b/test/syscalls/linux/fsync.cc index b34229248..e7e057f06 100644 --- a/test/syscalls/linux/fsync.cc +++ b/test/syscalls/linux/fsync.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/futex.cc b/test/syscalls/linux/futex.cc index c7a709a0a..bfec95466 100644 --- a/test/syscalls/linux/futex.cc +++ b/test/syscalls/linux/futex.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/getcpu.cc b/test/syscalls/linux/getcpu.cc index 3a52b25fa..f4d94bd6a 100644 --- a/test/syscalls/linux/getcpu.cc +++ b/test/syscalls/linux/getcpu.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/getdents.cc b/test/syscalls/linux/getdents.cc index e8a7bcd43..d146c8db7 100644 --- a/test/syscalls/linux/getdents.cc +++ b/test/syscalls/linux/getdents.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/getrandom.cc b/test/syscalls/linux/getrandom.cc index be5325497..f97f60029 100644 --- a/test/syscalls/linux/getrandom.cc +++ b/test/syscalls/linux/getrandom.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/getrusage.cc b/test/syscalls/linux/getrusage.cc index 1ae603858..9bdb1e4cd 100644 --- a/test/syscalls/linux/getrusage.cc +++ b/test/syscalls/linux/getrusage.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/inotify.cc b/test/syscalls/linux/inotify.cc index b99d339e5..6a3539e22 100644 --- a/test/syscalls/linux/inotify.cc +++ b/test/syscalls/linux/inotify.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/ioctl.cc b/test/syscalls/linux/ioctl.cc index c7741a177..c525d41d2 100644 --- a/test/syscalls/linux/ioctl.cc +++ b/test/syscalls/linux/ioctl.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/ip_socket_test_util.cc b/test/syscalls/linux/ip_socket_test_util.cc index 0a149c2e5..7612919d4 100644 --- a/test/syscalls/linux/ip_socket_test_util.cc +++ b/test/syscalls/linux/ip_socket_test_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/ip_socket_test_util.h b/test/syscalls/linux/ip_socket_test_util.h index cac790e64..6898effb8 100644 --- a/test/syscalls/linux/ip_socket_test_util.h +++ b/test/syscalls/linux/ip_socket_test_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/itimer.cc b/test/syscalls/linux/itimer.cc index ddfbc28fc..57ffd1595 100644 --- a/test/syscalls/linux/itimer.cc +++ b/test/syscalls/linux/itimer.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/kill.cc b/test/syscalls/linux/kill.cc index cd98de41f..18ad923b8 100644 --- a/test/syscalls/linux/kill.cc +++ b/test/syscalls/linux/kill.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/link.cc b/test/syscalls/linux/link.cc index ed74437bc..a91703070 100644 --- a/test/syscalls/linux/link.cc +++ b/test/syscalls/linux/link.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/lseek.cc b/test/syscalls/linux/lseek.cc index 6a4f1423c..a8af8e545 100644 --- a/test/syscalls/linux/lseek.cc +++ b/test/syscalls/linux/lseek.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/madvise.cc b/test/syscalls/linux/madvise.cc index a79c8c75d..f6ad4d18b 100644 --- a/test/syscalls/linux/madvise.cc +++ b/test/syscalls/linux/madvise.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/memfd.cc b/test/syscalls/linux/memfd.cc index c2513682d..7e103124b 100644 --- a/test/syscalls/linux/memfd.cc +++ b/test/syscalls/linux/memfd.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/memory_accounting.cc b/test/syscalls/linux/memory_accounting.cc index b4b680c34..a6e20f9c3 100644 --- a/test/syscalls/linux/memory_accounting.cc +++ b/test/syscalls/linux/memory_accounting.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/mempolicy.cc b/test/syscalls/linux/mempolicy.cc index 9f8033bdf..4ac4cb88f 100644 --- a/test/syscalls/linux/mempolicy.cc +++ b/test/syscalls/linux/mempolicy.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/mincore.cc b/test/syscalls/linux/mincore.cc index c572bf5ec..5c1240c89 100644 --- a/test/syscalls/linux/mincore.cc +++ b/test/syscalls/linux/mincore.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/mkdir.cc b/test/syscalls/linux/mkdir.cc index 50807b68f..cf138d328 100644 --- a/test/syscalls/linux/mkdir.cc +++ b/test/syscalls/linux/mkdir.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/mknod.cc b/test/syscalls/linux/mknod.cc index 361ca299b..b1675b9c7 100644 --- a/test/syscalls/linux/mknod.cc +++ b/test/syscalls/linux/mknod.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/mlock.cc b/test/syscalls/linux/mlock.cc index a492b2404..aee4f7d1a 100644 --- a/test/syscalls/linux/mlock.cc +++ b/test/syscalls/linux/mlock.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/mmap.cc b/test/syscalls/linux/mmap.cc index a4fb9d1e0..5b5b4c2e8 100644 --- a/test/syscalls/linux/mmap.cc +++ b/test/syscalls/linux/mmap.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/mount.cc b/test/syscalls/linux/mount.cc index 201b83e87..3a17672aa 100644 --- a/test/syscalls/linux/mount.cc +++ b/test/syscalls/linux/mount.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/mremap.cc b/test/syscalls/linux/mremap.cc index 01116c1ab..7298d4ca8 100644 --- a/test/syscalls/linux/mremap.cc +++ b/test/syscalls/linux/mremap.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/msync.cc b/test/syscalls/linux/msync.cc index 5afbfce72..ac7146017 100644 --- a/test/syscalls/linux/msync.cc +++ b/test/syscalls/linux/msync.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/munmap.cc b/test/syscalls/linux/munmap.cc index e20039950..067241f4d 100644 --- a/test/syscalls/linux/munmap.cc +++ b/test/syscalls/linux/munmap.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/open.cc b/test/syscalls/linux/open.cc index 22e4666c2..42646bb02 100644 --- a/test/syscalls/linux/open.cc +++ b/test/syscalls/linux/open.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/open_create.cc b/test/syscalls/linux/open_create.cc index b2cbd63d1..e5a85ef9d 100644 --- a/test/syscalls/linux/open_create.cc +++ b/test/syscalls/linux/open_create.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/partial_bad_buffer.cc b/test/syscalls/linux/partial_bad_buffer.cc index 71288ebc4..83b1ad4e4 100644 --- a/test/syscalls/linux/partial_bad_buffer.cc +++ b/test/syscalls/linux/partial_bad_buffer.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/pause.cc b/test/syscalls/linux/pause.cc index 4e1148c24..8c05efd6f 100644 --- a/test/syscalls/linux/pause.cc +++ b/test/syscalls/linux/pause.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/pipe.cc b/test/syscalls/linux/pipe.cc index abd10b11b..8698295b3 100644 --- a/test/syscalls/linux/pipe.cc +++ b/test/syscalls/linux/pipe.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/poll.cc b/test/syscalls/linux/poll.cc index cd2161bb1..9e5aa7fd0 100644 --- a/test/syscalls/linux/poll.cc +++ b/test/syscalls/linux/poll.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/ppoll.cc b/test/syscalls/linux/ppoll.cc index f8c388c00..8245a11e8 100644 --- a/test/syscalls/linux/ppoll.cc +++ b/test/syscalls/linux/ppoll.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/prctl.cc b/test/syscalls/linux/prctl.cc index 854dec714..bce42dc74 100644 --- a/test/syscalls/linux/prctl.cc +++ b/test/syscalls/linux/prctl.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/prctl_setuid.cc b/test/syscalls/linux/prctl_setuid.cc index c1b561464..00dd6523e 100644 --- a/test/syscalls/linux/prctl_setuid.cc +++ b/test/syscalls/linux/prctl_setuid.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/pread64.cc b/test/syscalls/linux/pread64.cc index 4e5bcfcde..5e3eb1735 100644 --- a/test/syscalls/linux/pread64.cc +++ b/test/syscalls/linux/pread64.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/preadv.cc b/test/syscalls/linux/preadv.cc index 4a31123d8..eebd129f2 100644 --- a/test/syscalls/linux/preadv.cc +++ b/test/syscalls/linux/preadv.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/preadv2.cc b/test/syscalls/linux/preadv2.cc index 58a4f9224..aac960130 100644 --- a/test/syscalls/linux/preadv2.cc +++ b/test/syscalls/linux/preadv2.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/priority.cc b/test/syscalls/linux/priority.cc index 3906c7132..1d9bdfa70 100644 --- a/test/syscalls/linux/priority.cc +++ b/test/syscalls/linux/priority.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/priority_execve.cc b/test/syscalls/linux/priority_execve.cc index 5604bd3d0..5cb343bad 100644 --- a/test/syscalls/linux/priority_execve.cc +++ b/test/syscalls/linux/priority_execve.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/proc.cc b/test/syscalls/linux/proc.cc index 7ba274226..654f26242 100644 --- a/test/syscalls/linux/proc.cc +++ b/test/syscalls/linux/proc.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/proc_net.cc b/test/syscalls/linux/proc_net.cc index 6060d0644..03d0665eb 100644 --- a/test/syscalls/linux/proc_net.cc +++ b/test/syscalls/linux/proc_net.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/proc_net_unix.cc b/test/syscalls/linux/proc_net_unix.cc index ea7c93012..6d745f728 100644 --- a/test/syscalls/linux/proc_net_unix.cc +++ b/test/syscalls/linux/proc_net_unix.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/proc_pid_smaps.cc b/test/syscalls/linux/proc_pid_smaps.cc index cf5c462f3..7f2e8f203 100644 --- a/test/syscalls/linux/proc_pid_smaps.cc +++ b/test/syscalls/linux/proc_pid_smaps.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/proc_pid_uid_gid_map.cc b/test/syscalls/linux/proc_pid_uid_gid_map.cc index 96c58c564..df70b7eb9 100644 --- a/test/syscalls/linux/proc_pid_uid_gid_map.cc +++ b/test/syscalls/linux/proc_pid_uid_gid_map.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/pselect.cc b/test/syscalls/linux/pselect.cc index 3294f6c14..4e43c4d7f 100644 --- a/test/syscalls/linux/pselect.cc +++ b/test/syscalls/linux/pselect.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/ptrace.cc b/test/syscalls/linux/ptrace.cc index e0c56f1fc..4c212836c 100644 --- a/test/syscalls/linux/ptrace.cc +++ b/test/syscalls/linux/ptrace.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/pty.cc b/test/syscalls/linux/pty.cc index 5b2dc9ccb..0485d187c 100644 --- a/test/syscalls/linux/pty.cc +++ b/test/syscalls/linux/pty.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/pwrite64.cc b/test/syscalls/linux/pwrite64.cc index 485b1e48d..e1603fc2d 100644 --- a/test/syscalls/linux/pwrite64.cc +++ b/test/syscalls/linux/pwrite64.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/pwritev2.cc b/test/syscalls/linux/pwritev2.cc index a6949f08e..db519f4e0 100644 --- a/test/syscalls/linux/pwritev2.cc +++ b/test/syscalls/linux/pwritev2.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/raw_socket_ipv4.cc b/test/syscalls/linux/raw_socket_ipv4.cc index 8b8d032cb..e20b5cb50 100644 --- a/test/syscalls/linux/raw_socket_ipv4.cc +++ b/test/syscalls/linux/raw_socket_ipv4.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/read.cc b/test/syscalls/linux/read.cc index eb1b5bc10..4430fa3c2 100644 --- a/test/syscalls/linux/read.cc +++ b/test/syscalls/linux/read.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/readv.cc b/test/syscalls/linux/readv.cc index 0b933673a..f327ec3a9 100644 --- a/test/syscalls/linux/readv.cc +++ b/test/syscalls/linux/readv.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/readv_common.cc b/test/syscalls/linux/readv_common.cc index 349b80d7f..35d2dd9e3 100644 --- a/test/syscalls/linux/readv_common.cc +++ b/test/syscalls/linux/readv_common.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/readv_common.h b/test/syscalls/linux/readv_common.h index e261d545a..b16179fca 100644 --- a/test/syscalls/linux/readv_common.h +++ b/test/syscalls/linux/readv_common.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/readv_socket.cc b/test/syscalls/linux/readv_socket.cc index cf22c395e..3c315cc02 100644 --- a/test/syscalls/linux/readv_socket.cc +++ b/test/syscalls/linux/readv_socket.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/rename.cc b/test/syscalls/linux/rename.cc index c0cbc7cd9..c9d76c2e2 100644 --- a/test/syscalls/linux/rename.cc +++ b/test/syscalls/linux/rename.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/rlimits.cc b/test/syscalls/linux/rlimits.cc index 7b255d0f6..860f0f688 100644 --- a/test/syscalls/linux/rlimits.cc +++ b/test/syscalls/linux/rlimits.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/rtsignal.cc b/test/syscalls/linux/rtsignal.cc index ff948f9d5..81d193ffd 100644 --- a/test/syscalls/linux/rtsignal.cc +++ b/test/syscalls/linux/rtsignal.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/sched.cc b/test/syscalls/linux/sched.cc index 60cb6c443..735e99411 100644 --- a/test/syscalls/linux/sched.cc +++ b/test/syscalls/linux/sched.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/sched_yield.cc b/test/syscalls/linux/sched_yield.cc index fc45aa5c2..5d24f5b58 100644 --- a/test/syscalls/linux/sched_yield.cc +++ b/test/syscalls/linux/sched_yield.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/seccomp.cc b/test/syscalls/linux/seccomp.cc index 27740d7ef..e77586852 100644 --- a/test/syscalls/linux/seccomp.cc +++ b/test/syscalls/linux/seccomp.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/select.cc b/test/syscalls/linux/select.cc index 41e6043cc..88c010aec 100644 --- a/test/syscalls/linux/select.cc +++ b/test/syscalls/linux/select.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/semaphore.cc b/test/syscalls/linux/semaphore.cc index 1c47b6851..421318fcb 100644 --- a/test/syscalls/linux/semaphore.cc +++ b/test/syscalls/linux/semaphore.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/sendfile.cc b/test/syscalls/linux/sendfile.cc index 15fd01ff0..2fbb3f4ef 100644 --- a/test/syscalls/linux/sendfile.cc +++ b/test/syscalls/linux/sendfile.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/sendfile_socket.cc b/test/syscalls/linux/sendfile_socket.cc index e2ccf17ce..66adda515 100644 --- a/test/syscalls/linux/sendfile_socket.cc +++ b/test/syscalls/linux/sendfile_socket.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/shm.cc b/test/syscalls/linux/shm.cc index 2c0f9b04a..eb7a3966f 100644 --- a/test/syscalls/linux/shm.cc +++ b/test/syscalls/linux/shm.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/sigaction.cc b/test/syscalls/linux/sigaction.cc index cdd2dbf31..9a53fd3e0 100644 --- a/test/syscalls/linux/sigaction.cc +++ b/test/syscalls/linux/sigaction.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/sigaltstack.cc b/test/syscalls/linux/sigaltstack.cc index 5741720f4..7d4a12c1d 100644 --- a/test/syscalls/linux/sigaltstack.cc +++ b/test/syscalls/linux/sigaltstack.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/sigaltstack_check.cc b/test/syscalls/linux/sigaltstack_check.cc index b71f812a8..5ac1b661d 100644 --- a/test/syscalls/linux/sigaltstack_check.cc +++ b/test/syscalls/linux/sigaltstack_check.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/sigiret.cc b/test/syscalls/linux/sigiret.cc index 1b7cecccb..a47c781ea 100644 --- a/test/syscalls/linux/sigiret.cc +++ b/test/syscalls/linux/sigiret.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/sigprocmask.cc b/test/syscalls/linux/sigprocmask.cc index 1aea1ecb8..654c6a47f 100644 --- a/test/syscalls/linux/sigprocmask.cc +++ b/test/syscalls/linux/sigprocmask.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/sigstop.cc b/test/syscalls/linux/sigstop.cc index e21d23d51..9c7210e17 100644 --- a/test/syscalls/linux/sigstop.cc +++ b/test/syscalls/linux/sigstop.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/sigtimedwait.cc b/test/syscalls/linux/sigtimedwait.cc index 1df9c013f..1e5bf5942 100644 --- a/test/syscalls/linux/sigtimedwait.cc +++ b/test/syscalls/linux/sigtimedwait.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_abstract.cc b/test/syscalls/linux/socket_abstract.cc index 639cd4e59..2faf678f7 100644 --- a/test/syscalls/linux/socket_abstract.cc +++ b/test/syscalls/linux/socket_abstract.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_blocking.cc b/test/syscalls/linux/socket_blocking.cc index c1bca467f..00c50d1bf 100644 --- a/test/syscalls/linux/socket_blocking.cc +++ b/test/syscalls/linux/socket_blocking.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_blocking.h b/test/syscalls/linux/socket_blocking.h index 5cddee54b..db26e5ef5 100644 --- a/test/syscalls/linux/socket_blocking.h +++ b/test/syscalls/linux/socket_blocking.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_filesystem.cc b/test/syscalls/linux/socket_filesystem.cc index 2653be158..f7cb72df4 100644 --- a/test/syscalls/linux/socket_filesystem.cc +++ b/test/syscalls/linux/socket_filesystem.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_generic.cc b/test/syscalls/linux/socket_generic.cc index d04d5abe0..f99f3fe62 100644 --- a/test/syscalls/linux/socket_generic.cc +++ b/test/syscalls/linux/socket_generic.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_generic.h b/test/syscalls/linux/socket_generic.h index cd826abcf..00ae7bfc3 100644 --- a/test/syscalls/linux/socket_generic.h +++ b/test/syscalls/linux/socket_generic.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_inet_loopback.cc b/test/syscalls/linux/socket_inet_loopback.cc index 14d7827c2..f86a0f30c 100644 --- a/test/syscalls/linux/socket_inet_loopback.cc +++ b/test/syscalls/linux/socket_inet_loopback.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ip_loopback_blocking.cc b/test/syscalls/linux/socket_ip_loopback_blocking.cc index 9cec7a71d..d7fc20aad 100644 --- a/test/syscalls/linux/socket_ip_loopback_blocking.cc +++ b/test/syscalls/linux/socket_ip_loopback_blocking.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ip_tcp_generic.cc b/test/syscalls/linux/socket_ip_tcp_generic.cc index 54f00cd9b..5b198f49d 100644 --- a/test/syscalls/linux/socket_ip_tcp_generic.cc +++ b/test/syscalls/linux/socket_ip_tcp_generic.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ip_tcp_generic.h b/test/syscalls/linux/socket_ip_tcp_generic.h index f38500d14..a3eff3c73 100644 --- a/test/syscalls/linux/socket_ip_tcp_generic.h +++ b/test/syscalls/linux/socket_ip_tcp_generic.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ip_tcp_generic_loopback.cc b/test/syscalls/linux/socket_ip_tcp_generic_loopback.cc index 1963d5deb..2c6ae17bf 100644 --- a/test/syscalls/linux/socket_ip_tcp_generic_loopback.cc +++ b/test/syscalls/linux/socket_ip_tcp_generic_loopback.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ip_tcp_loopback.cc b/test/syscalls/linux/socket_ip_tcp_loopback.cc index 7e36c35d2..831de53b8 100644 --- a/test/syscalls/linux/socket_ip_tcp_loopback.cc +++ b/test/syscalls/linux/socket_ip_tcp_loopback.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ip_tcp_loopback_blocking.cc b/test/syscalls/linux/socket_ip_tcp_loopback_blocking.cc index 9e2a18d3e..d1ea8ef12 100644 --- a/test/syscalls/linux/socket_ip_tcp_loopback_blocking.cc +++ b/test/syscalls/linux/socket_ip_tcp_loopback_blocking.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ip_tcp_loopback_nonblock.cc b/test/syscalls/linux/socket_ip_tcp_loopback_nonblock.cc index 54053360f..96c1b3b3d 100644 --- a/test/syscalls/linux/socket_ip_tcp_loopback_nonblock.cc +++ b/test/syscalls/linux/socket_ip_tcp_loopback_nonblock.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ip_tcp_udp_generic.cc b/test/syscalls/linux/socket_ip_tcp_udp_generic.cc index 5bf1de7c6..251817a9f 100644 --- a/test/syscalls/linux/socket_ip_tcp_udp_generic.cc +++ b/test/syscalls/linux/socket_ip_tcp_udp_generic.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ip_udp_generic.cc b/test/syscalls/linux/socket_ip_udp_generic.cc index ac15154f2..044394ba7 100644 --- a/test/syscalls/linux/socket_ip_udp_generic.cc +++ b/test/syscalls/linux/socket_ip_udp_generic.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ip_udp_generic.h b/test/syscalls/linux/socket_ip_udp_generic.h index 8b8fc7c6e..106c54e9f 100644 --- a/test/syscalls/linux/socket_ip_udp_generic.h +++ b/test/syscalls/linux/socket_ip_udp_generic.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ip_udp_loopback.cc b/test/syscalls/linux/socket_ip_udp_loopback.cc index 0e4463649..fc124e9ef 100644 --- a/test/syscalls/linux/socket_ip_udp_loopback.cc +++ b/test/syscalls/linux/socket_ip_udp_loopback.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ip_udp_loopback_blocking.cc b/test/syscalls/linux/socket_ip_udp_loopback_blocking.cc index 0c3b669bf..1c3d1c0ad 100644 --- a/test/syscalls/linux/socket_ip_udp_loopback_blocking.cc +++ b/test/syscalls/linux/socket_ip_udp_loopback_blocking.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ip_udp_loopback_nonblock.cc b/test/syscalls/linux/socket_ip_udp_loopback_nonblock.cc index 7bf8597fe..7554b08d5 100644 --- a/test/syscalls/linux/socket_ip_udp_loopback_nonblock.cc +++ b/test/syscalls/linux/socket_ip_udp_loopback_nonblock.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.cc b/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.cc index 8e1c13ff4..3a068aacf 100644 --- a/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.cc +++ b/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.h b/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.h index b23de08d1..fb582b224 100644 --- a/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.h +++ b/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.h @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking_test.cc b/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking_test.cc index 773d84b13..040bb176e 100644 --- a/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking_test.cc +++ b/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking_test.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_udp_unbound.cc b/test/syscalls/linux/socket_ipv4_udp_unbound.cc index c99958ed5..709172580 100644 --- a/test/syscalls/linux/socket_ipv4_udp_unbound.cc +++ b/test/syscalls/linux/socket_ipv4_udp_unbound.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_udp_unbound.h b/test/syscalls/linux/socket_ipv4_udp_unbound.h index a780c0144..8e07bfbbf 100644 --- a/test/syscalls/linux/socket_ipv4_udp_unbound.h +++ b/test/syscalls/linux/socket_ipv4_udp_unbound.h @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.cc b/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.cc index 9dd9e1bd6..53dcd58cd 100644 --- a/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.cc +++ b/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.h b/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.h index 5cf9fa8eb..45e1d37ea 100644 --- a/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.h +++ b/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.h @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking_test.cc b/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking_test.cc index 535a5fa10..ffbb8e6eb 100644 --- a/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking_test.cc +++ b/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking_test.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_udp_unbound_loopback.cc b/test/syscalls/linux/socket_ipv4_udp_unbound_loopback.cc index d6a8e428c..cb0105471 100644 --- a/test/syscalls/linux/socket_ipv4_udp_unbound_loopback.cc +++ b/test/syscalls/linux/socket_ipv4_udp_unbound_loopback.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_netdevice.cc b/test/syscalls/linux/socket_netdevice.cc index b4e9fe51b..6a5fa8965 100644 --- a/test/syscalls/linux/socket_netdevice.cc +++ b/test/syscalls/linux/socket_netdevice.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_netlink_route.cc b/test/syscalls/linux/socket_netlink_route.cc index ed4ae1c71..c8693225f 100644 --- a/test/syscalls/linux/socket_netlink_route.cc +++ b/test/syscalls/linux/socket_netlink_route.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_netlink_util.cc b/test/syscalls/linux/socket_netlink_util.cc index edf549544..728d25434 100644 --- a/test/syscalls/linux/socket_netlink_util.cc +++ b/test/syscalls/linux/socket_netlink_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_netlink_util.h b/test/syscalls/linux/socket_netlink_util.h index 44b1f148c..bea449107 100644 --- a/test/syscalls/linux/socket_netlink_util.h +++ b/test/syscalls/linux/socket_netlink_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_non_blocking.cc b/test/syscalls/linux/socket_non_blocking.cc index 1bcc6fb7f..73e6dc618 100644 --- a/test/syscalls/linux/socket_non_blocking.cc +++ b/test/syscalls/linux/socket_non_blocking.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_non_blocking.h b/test/syscalls/linux/socket_non_blocking.h index 287e096bb..bd3e02fd2 100644 --- a/test/syscalls/linux/socket_non_blocking.h +++ b/test/syscalls/linux/socket_non_blocking.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_non_stream.cc b/test/syscalls/linux/socket_non_stream.cc index d170008a4..3c599b6e8 100644 --- a/test/syscalls/linux/socket_non_stream.cc +++ b/test/syscalls/linux/socket_non_stream.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_non_stream.h b/test/syscalls/linux/socket_non_stream.h index 02dd2a958..469fbe6a2 100644 --- a/test/syscalls/linux/socket_non_stream.h +++ b/test/syscalls/linux/socket_non_stream.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_non_stream_blocking.cc b/test/syscalls/linux/socket_non_stream_blocking.cc index 9e92628c3..76127d181 100644 --- a/test/syscalls/linux/socket_non_stream_blocking.cc +++ b/test/syscalls/linux/socket_non_stream_blocking.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_non_stream_blocking.h b/test/syscalls/linux/socket_non_stream_blocking.h index bde355452..6e205a039 100644 --- a/test/syscalls/linux/socket_non_stream_blocking.h +++ b/test/syscalls/linux/socket_non_stream_blocking.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_stream.cc b/test/syscalls/linux/socket_stream.cc index c8a8ad0f6..0417dd347 100644 --- a/test/syscalls/linux/socket_stream.cc +++ b/test/syscalls/linux/socket_stream.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_stream.h b/test/syscalls/linux/socket_stream.h index 35e591e17..b837b8f8c 100644 --- a/test/syscalls/linux/socket_stream.h +++ b/test/syscalls/linux/socket_stream.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_stream_blocking.cc b/test/syscalls/linux/socket_stream_blocking.cc index f0f86c01c..8367460d2 100644 --- a/test/syscalls/linux/socket_stream_blocking.cc +++ b/test/syscalls/linux/socket_stream_blocking.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_stream_blocking.h b/test/syscalls/linux/socket_stream_blocking.h index 06113ad03..9fd19ff90 100644 --- a/test/syscalls/linux/socket_stream_blocking.h +++ b/test/syscalls/linux/socket_stream_blocking.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_stream_nonblock.cc b/test/syscalls/linux/socket_stream_nonblock.cc index a3202ffe4..b00748b97 100644 --- a/test/syscalls/linux/socket_stream_nonblock.cc +++ b/test/syscalls/linux/socket_stream_nonblock.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_stream_nonblock.h b/test/syscalls/linux/socket_stream_nonblock.h index 491f53848..c3b7fad91 100644 --- a/test/syscalls/linux/socket_stream_nonblock.h +++ b/test/syscalls/linux/socket_stream_nonblock.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_test_util.cc b/test/syscalls/linux/socket_test_util.cc index 0be23e541..da69de37c 100644 --- a/test/syscalls/linux/socket_test_util.cc +++ b/test/syscalls/linux/socket_test_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_test_util.h b/test/syscalls/linux/socket_test_util.h index dfabdf179..058313986 100644 --- a/test/syscalls/linux/socket_test_util.h +++ b/test/syscalls/linux/socket_test_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix.cc b/test/syscalls/linux/socket_unix.cc index fafb23ad1..bb3397fa2 100644 --- a/test/syscalls/linux/socket_unix.cc +++ b/test/syscalls/linux/socket_unix.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix.h b/test/syscalls/linux/socket_unix.h index d2a16afb2..3625cc404 100644 --- a/test/syscalls/linux/socket_unix.h +++ b/test/syscalls/linux/socket_unix.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_abstract.cc b/test/syscalls/linux/socket_unix_abstract.cc index c4a3c889c..8241bf997 100644 --- a/test/syscalls/linux/socket_unix_abstract.cc +++ b/test/syscalls/linux/socket_unix_abstract.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_abstract_nonblock.cc b/test/syscalls/linux/socket_unix_abstract_nonblock.cc index a69ee027e..9de0f6dfe 100644 --- a/test/syscalls/linux/socket_unix_abstract_nonblock.cc +++ b/test/syscalls/linux/socket_unix_abstract_nonblock.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_blocking_local.cc b/test/syscalls/linux/socket_unix_blocking_local.cc index 57af118c5..320915b0f 100644 --- a/test/syscalls/linux/socket_unix_blocking_local.cc +++ b/test/syscalls/linux/socket_unix_blocking_local.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_dgram.cc b/test/syscalls/linux/socket_unix_dgram.cc index 5dd5e6d77..3e0f611d2 100644 --- a/test/syscalls/linux/socket_unix_dgram.cc +++ b/test/syscalls/linux/socket_unix_dgram.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_dgram.h b/test/syscalls/linux/socket_unix_dgram.h index 722a3d8e6..0764ef85b 100644 --- a/test/syscalls/linux/socket_unix_dgram.h +++ b/test/syscalls/linux/socket_unix_dgram.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_dgram_local.cc b/test/syscalls/linux/socket_unix_dgram_local.cc index da8f59704..4ba2c80ae 100644 --- a/test/syscalls/linux/socket_unix_dgram_local.cc +++ b/test/syscalls/linux/socket_unix_dgram_local.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_dgram_non_blocking.cc b/test/syscalls/linux/socket_unix_dgram_non_blocking.cc index 3becb513d..9fe86cee8 100644 --- a/test/syscalls/linux/socket_unix_dgram_non_blocking.cc +++ b/test/syscalls/linux/socket_unix_dgram_non_blocking.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_domain.cc b/test/syscalls/linux/socket_unix_domain.cc index f081c601f..fa3efc7f8 100644 --- a/test/syscalls/linux/socket_unix_domain.cc +++ b/test/syscalls/linux/socket_unix_domain.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_filesystem.cc b/test/syscalls/linux/socket_unix_filesystem.cc index 6a67da75f..5dbe67773 100644 --- a/test/syscalls/linux/socket_unix_filesystem.cc +++ b/test/syscalls/linux/socket_unix_filesystem.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_filesystem_nonblock.cc b/test/syscalls/linux/socket_unix_filesystem_nonblock.cc index c13a1e564..137db53c4 100644 --- a/test/syscalls/linux/socket_unix_filesystem_nonblock.cc +++ b/test/syscalls/linux/socket_unix_filesystem_nonblock.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_non_stream.cc b/test/syscalls/linux/socket_unix_non_stream.cc index a565978f9..dafe82494 100644 --- a/test/syscalls/linux/socket_unix_non_stream.cc +++ b/test/syscalls/linux/socket_unix_non_stream.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_non_stream.h b/test/syscalls/linux/socket_unix_non_stream.h index e4214d949..7478ab172 100644 --- a/test/syscalls/linux/socket_unix_non_stream.h +++ b/test/syscalls/linux/socket_unix_non_stream.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_non_stream_blocking_local.cc b/test/syscalls/linux/socket_unix_non_stream_blocking_local.cc index 6c435669b..98cf1fe8a 100644 --- a/test/syscalls/linux/socket_unix_non_stream_blocking_local.cc +++ b/test/syscalls/linux/socket_unix_non_stream_blocking_local.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_pair.cc b/test/syscalls/linux/socket_unix_pair.cc index c575fdcb2..bacfc11e4 100644 --- a/test/syscalls/linux/socket_unix_pair.cc +++ b/test/syscalls/linux/socket_unix_pair.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_pair_nonblock.cc b/test/syscalls/linux/socket_unix_pair_nonblock.cc index 1ae7f9b5e..583506f08 100644 --- a/test/syscalls/linux/socket_unix_pair_nonblock.cc +++ b/test/syscalls/linux/socket_unix_pair_nonblock.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_seqpacket.cc b/test/syscalls/linux/socket_unix_seqpacket.cc index ad0af77e9..6f6367dd5 100644 --- a/test/syscalls/linux/socket_unix_seqpacket.cc +++ b/test/syscalls/linux/socket_unix_seqpacket.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_seqpacket.h b/test/syscalls/linux/socket_unix_seqpacket.h index da8eb2b2b..30d9b9edf 100644 --- a/test/syscalls/linux/socket_unix_seqpacket.h +++ b/test/syscalls/linux/socket_unix_seqpacket.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_seqpacket_local.cc b/test/syscalls/linux/socket_unix_seqpacket_local.cc index e6484d9b4..b903a9e8f 100644 --- a/test/syscalls/linux/socket_unix_seqpacket_local.cc +++ b/test/syscalls/linux/socket_unix_seqpacket_local.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_stream.cc b/test/syscalls/linux/socket_unix_stream.cc index 95f454251..659c93945 100644 --- a/test/syscalls/linux/socket_unix_stream.cc +++ b/test/syscalls/linux/socket_unix_stream.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_stream_blocking_local.cc b/test/syscalls/linux/socket_unix_stream_blocking_local.cc index ec0fc6955..ce0f1e50d 100644 --- a/test/syscalls/linux/socket_unix_stream_blocking_local.cc +++ b/test/syscalls/linux/socket_unix_stream_blocking_local.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_stream_local.cc b/test/syscalls/linux/socket_unix_stream_local.cc index bf4c5f2eb..6b840189c 100644 --- a/test/syscalls/linux/socket_unix_stream_local.cc +++ b/test/syscalls/linux/socket_unix_stream_local.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_stream_nonblock_local.cc b/test/syscalls/linux/socket_unix_stream_nonblock_local.cc index df80b105a..ebec4e0ec 100644 --- a/test/syscalls/linux/socket_unix_stream_nonblock_local.cc +++ b/test/syscalls/linux/socket_unix_stream_nonblock_local.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_unbound_abstract.cc b/test/syscalls/linux/socket_unix_unbound_abstract.cc index b6fe7a9ce..4b5832de8 100644 --- a/test/syscalls/linux/socket_unix_unbound_abstract.cc +++ b/test/syscalls/linux/socket_unix_unbound_abstract.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_unbound_dgram.cc b/test/syscalls/linux/socket_unix_unbound_dgram.cc index 1ec11a08d..2ddc5c11f 100644 --- a/test/syscalls/linux/socket_unix_unbound_dgram.cc +++ b/test/syscalls/linux/socket_unix_unbound_dgram.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_unbound_filesystem.cc b/test/syscalls/linux/socket_unix_unbound_filesystem.cc index d09142aa6..8cb03c450 100644 --- a/test/syscalls/linux/socket_unix_unbound_filesystem.cc +++ b/test/syscalls/linux/socket_unix_unbound_filesystem.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_unbound_seqpacket.cc b/test/syscalls/linux/socket_unix_unbound_seqpacket.cc index 21209b244..0575f2e1d 100644 --- a/test/syscalls/linux/socket_unix_unbound_seqpacket.cc +++ b/test/syscalls/linux/socket_unix_unbound_seqpacket.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_unix_unbound_stream.cc b/test/syscalls/linux/socket_unix_unbound_stream.cc index b95f9569e..091d546b3 100644 --- a/test/syscalls/linux/socket_unix_unbound_stream.cc +++ b/test/syscalls/linux/socket_unix_unbound_stream.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/stat.cc b/test/syscalls/linux/stat.cc index 746318d09..80ba67496 100644 --- a/test/syscalls/linux/stat.cc +++ b/test/syscalls/linux/stat.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/stat_times.cc b/test/syscalls/linux/stat_times.cc index 8346e9a8e..9b53739a0 100644 --- a/test/syscalls/linux/stat_times.cc +++ b/test/syscalls/linux/stat_times.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/statfs.cc b/test/syscalls/linux/statfs.cc index e1e7fc707..aca51d30f 100644 --- a/test/syscalls/linux/statfs.cc +++ b/test/syscalls/linux/statfs.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/sticky.cc b/test/syscalls/linux/sticky.cc index 58cf0d014..59fb5dfe6 100644 --- a/test/syscalls/linux/sticky.cc +++ b/test/syscalls/linux/sticky.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/symlink.cc b/test/syscalls/linux/symlink.cc index 318917f4b..494072a9b 100644 --- a/test/syscalls/linux/symlink.cc +++ b/test/syscalls/linux/symlink.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/sync.cc b/test/syscalls/linux/sync.cc index 5b777b6eb..fe479390d 100644 --- a/test/syscalls/linux/sync.cc +++ b/test/syscalls/linux/sync.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/sync_file_range.cc b/test/syscalls/linux/sync_file_range.cc index d11f58481..36cc42043 100644 --- a/test/syscalls/linux/sync_file_range.cc +++ b/test/syscalls/linux/sync_file_range.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/sysinfo.cc b/test/syscalls/linux/sysinfo.cc index a0dd82640..1a71256da 100644 --- a/test/syscalls/linux/sysinfo.cc +++ b/test/syscalls/linux/sysinfo.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/syslog.cc b/test/syscalls/linux/syslog.cc index 5bd0d1cc3..9a7407d96 100644 --- a/test/syscalls/linux/syslog.cc +++ b/test/syscalls/linux/syslog.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/sysret.cc b/test/syscalls/linux/sysret.cc index 8e10220eb..819fa655a 100644 --- a/test/syscalls/linux/sysret.cc +++ b/test/syscalls/linux/sysret.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/tcp_socket.cc b/test/syscalls/linux/tcp_socket.cc index 33620a874..e3f9f9f9d 100644 --- a/test/syscalls/linux/tcp_socket.cc +++ b/test/syscalls/linux/tcp_socket.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/temp_umask.h b/test/syscalls/linux/temp_umask.h index f202dfa59..81a25440c 100644 --- a/test/syscalls/linux/temp_umask.h +++ b/test/syscalls/linux/temp_umask.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/tgkill.cc b/test/syscalls/linux/tgkill.cc index 2d258ef11..80acae5de 100644 --- a/test/syscalls/linux/tgkill.cc +++ b/test/syscalls/linux/tgkill.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/time.cc b/test/syscalls/linux/time.cc index 5a3dfd026..c7eead17e 100644 --- a/test/syscalls/linux/time.cc +++ b/test/syscalls/linux/time.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/timerfd.cc b/test/syscalls/linux/timerfd.cc index b85321795..9df53612f 100644 --- a/test/syscalls/linux/timerfd.cc +++ b/test/syscalls/linux/timerfd.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/timers.cc b/test/syscalls/linux/timers.cc index 14506eb12..fd42e81e1 100644 --- a/test/syscalls/linux/timers.cc +++ b/test/syscalls/linux/timers.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/tkill.cc b/test/syscalls/linux/tkill.cc index 3e8ce5327..bae377c69 100644 --- a/test/syscalls/linux/tkill.cc +++ b/test/syscalls/linux/tkill.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/truncate.cc b/test/syscalls/linux/truncate.cc index 2616a9147..e5cc5d97c 100644 --- a/test/syscalls/linux/truncate.cc +++ b/test/syscalls/linux/truncate.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/udp_bind.cc b/test/syscalls/linux/udp_bind.cc index 547eb2a6c..6d92bdbeb 100644 --- a/test/syscalls/linux/udp_bind.cc +++ b/test/syscalls/linux/udp_bind.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/udp_socket.cc b/test/syscalls/linux/udp_socket.cc index f39281d5c..31db8a2ad 100644 --- a/test/syscalls/linux/udp_socket.cc +++ b/test/syscalls/linux/udp_socket.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/uidgid.cc b/test/syscalls/linux/uidgid.cc index d78a09b1e..bf1ca8679 100644 --- a/test/syscalls/linux/uidgid.cc +++ b/test/syscalls/linux/uidgid.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/uname.cc b/test/syscalls/linux/uname.cc index d22a34bd7..0a5d91017 100644 --- a/test/syscalls/linux/uname.cc +++ b/test/syscalls/linux/uname.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/unix_domain_socket_test_util.cc b/test/syscalls/linux/unix_domain_socket_test_util.cc index 2d7a530b9..6f49e3660 100644 --- a/test/syscalls/linux/unix_domain_socket_test_util.cc +++ b/test/syscalls/linux/unix_domain_socket_test_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/unix_domain_socket_test_util.h b/test/syscalls/linux/unix_domain_socket_test_util.h index 1b09aeae7..aae990245 100644 --- a/test/syscalls/linux/unix_domain_socket_test_util.h +++ b/test/syscalls/linux/unix_domain_socket_test_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/unlink.cc b/test/syscalls/linux/unlink.cc index b10aae025..b6f65e027 100644 --- a/test/syscalls/linux/unlink.cc +++ b/test/syscalls/linux/unlink.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/unshare.cc b/test/syscalls/linux/unshare.cc index 9dd6ec4b6..e32619efe 100644 --- a/test/syscalls/linux/unshare.cc +++ b/test/syscalls/linux/unshare.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/utimes.cc b/test/syscalls/linux/utimes.cc index bf776cd93..80716859a 100644 --- a/test/syscalls/linux/utimes.cc +++ b/test/syscalls/linux/utimes.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/vdso.cc b/test/syscalls/linux/vdso.cc index 0f6e1c7c6..19c80add8 100644 --- a/test/syscalls/linux/vdso.cc +++ b/test/syscalls/linux/vdso.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/vdso_clock_gettime.cc b/test/syscalls/linux/vdso_clock_gettime.cc index 0e936594b..759a50569 100644 --- a/test/syscalls/linux/vdso_clock_gettime.cc +++ b/test/syscalls/linux/vdso_clock_gettime.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/vfork.cc b/test/syscalls/linux/vfork.cc index 9999a909e..631a53654 100644 --- a/test/syscalls/linux/vfork.cc +++ b/test/syscalls/linux/vfork.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/vsyscall.cc b/test/syscalls/linux/vsyscall.cc index cb6840cc6..2c2303358 100644 --- a/test/syscalls/linux/vsyscall.cc +++ b/test/syscalls/linux/vsyscall.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/wait.cc b/test/syscalls/linux/wait.cc index fcd606bec..50d0725a7 100644 --- a/test/syscalls/linux/wait.cc +++ b/test/syscalls/linux/wait.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/write.cc b/test/syscalls/linux/write.cc index 7f80b2fa8..9b219cfd6 100644 --- a/test/syscalls/linux/write.cc +++ b/test/syscalls/linux/write.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/syscall_test_runner.go b/test/syscalls/syscall_test_runner.go index c4af28103..28f312b8b 100644 --- a/test/syscalls/syscall_test_runner.go +++ b/test/syscalls/syscall_test_runner.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/syscall_test_runner.sh b/test/syscalls/syscall_test_runner.sh index 87d62786b..864bb2de4 100755 --- a/test/syscalls/syscall_test_runner.sh +++ b/test/syscalls/syscall_test_runner.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google LLC +# Copyright 2018 The gVisor Authors. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/test/util/capability_util.cc b/test/util/capability_util.cc index d1dd95e76..5d733887b 100644 --- a/test/util/capability_util.cc +++ b/test/util/capability_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/capability_util.h b/test/util/capability_util.h index 8708f5e69..e968a2583 100644 --- a/test/util/capability_util.h +++ b/test/util/capability_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/cleanup.h b/test/util/cleanup.h index fb4724f97..c76482ef4 100644 --- a/test/util/cleanup.h +++ b/test/util/cleanup.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/epoll_util.cc b/test/util/epoll_util.cc index 0b95aa8cd..2e5051468 100644 --- a/test/util/epoll_util.cc +++ b/test/util/epoll_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/epoll_util.h b/test/util/epoll_util.h index 521e7a3d3..f233b37d5 100644 --- a/test/util/epoll_util.h +++ b/test/util/epoll_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/eventfd_util.h b/test/util/eventfd_util.h index 1fdb07d3b..cb9ce829c 100644 --- a/test/util/eventfd_util.h +++ b/test/util/eventfd_util.h @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/file_descriptor.h b/test/util/file_descriptor.h index be8812d01..fc5caa55b 100644 --- a/test/util/file_descriptor.h +++ b/test/util/file_descriptor.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/fs_util.cc b/test/util/fs_util.cc index 6bd424417..bc90bd78e 100644 --- a/test/util/fs_util.cc +++ b/test/util/fs_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/fs_util.h b/test/util/fs_util.h index 9412b2f71..eb7cdaa24 100644 --- a/test/util/fs_util.h +++ b/test/util/fs_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/fs_util_test.cc b/test/util/fs_util_test.cc index ce70d58aa..4e12076a1 100644 --- a/test/util/fs_util_test.cc +++ b/test/util/fs_util_test.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/logging.cc b/test/util/logging.cc index 86ea71df3..cc71d77b0 100644 --- a/test/util/logging.cc +++ b/test/util/logging.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/logging.h b/test/util/logging.h index 6e957b172..589166fab 100644 --- a/test/util/logging.h +++ b/test/util/logging.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/memory_util.h b/test/util/memory_util.h index 8f6e99ba6..8c77778ea 100644 --- a/test/util/memory_util.h +++ b/test/util/memory_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/mount_util.h b/test/util/mount_util.h index 468170646..7782e6bf2 100644 --- a/test/util/mount_util.h +++ b/test/util/mount_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/multiprocess_util.cc b/test/util/multiprocess_util.cc index 12637db8c..95f5f3b4f 100644 --- a/test/util/multiprocess_util.cc +++ b/test/util/multiprocess_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/multiprocess_util.h b/test/util/multiprocess_util.h index ba5f2601f..0aecd3439 100644 --- a/test/util/multiprocess_util.h +++ b/test/util/multiprocess_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/posix_error.cc b/test/util/posix_error.cc index ead9ede16..cebf7e0ac 100644 --- a/test/util/posix_error.cc +++ b/test/util/posix_error.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/posix_error.h b/test/util/posix_error.h index 2a66e2e94..b604f4f8f 100644 --- a/test/util/posix_error.h +++ b/test/util/posix_error.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/posix_error_test.cc b/test/util/posix_error_test.cc index c5427b8e5..d67270842 100644 --- a/test/util/posix_error_test.cc +++ b/test/util/posix_error_test.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/proc_util.cc b/test/util/proc_util.cc index 2d9eb1986..9d4db37c3 100644 --- a/test/util/proc_util.cc +++ b/test/util/proc_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/proc_util.h b/test/util/proc_util.h index e1ee2db9c..af209a51e 100644 --- a/test/util/proc_util.h +++ b/test/util/proc_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/proc_util_test.cc b/test/util/proc_util_test.cc index 75335415a..71dd2355e 100644 --- a/test/util/proc_util_test.cc +++ b/test/util/proc_util_test.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/rlimit_util.cc b/test/util/rlimit_util.cc index a9912c372..684253f78 100644 --- a/test/util/rlimit_util.cc +++ b/test/util/rlimit_util.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/rlimit_util.h b/test/util/rlimit_util.h index fa5cc70dc..873252a32 100644 --- a/test/util/rlimit_util.h +++ b/test/util/rlimit_util.h @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/save_util.cc b/test/util/save_util.cc index 5540e2146..05f52b80d 100644 --- a/test/util/save_util.cc +++ b/test/util/save_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/save_util.h b/test/util/save_util.h index 919e4af3d..90460701e 100644 --- a/test/util/save_util.h +++ b/test/util/save_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/signal_util.cc b/test/util/signal_util.cc index 3e2df32a6..26738864f 100644 --- a/test/util/signal_util.cc +++ b/test/util/signal_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/signal_util.h b/test/util/signal_util.h index 80f1808f6..7fd2af015 100644 --- a/test/util/signal_util.h +++ b/test/util/signal_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/temp_path.cc b/test/util/temp_path.cc index 48ce82d20..c5d8fc635 100644 --- a/test/util/temp_path.cc +++ b/test/util/temp_path.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/temp_path.h b/test/util/temp_path.h index 33eb6a72c..89302e0fd 100644 --- a/test/util/temp_path.h +++ b/test/util/temp_path.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/test_main.cc b/test/util/test_main.cc index 4c6b5e860..5c7ee0064 100644 --- a/test/util/test_main.cc +++ b/test/util/test_main.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/test_util.cc b/test/util/test_util.cc index 9b7cfa4dc..c52fd9a4a 100644 --- a/test/util/test_util.cc +++ b/test/util/test_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/test_util.h b/test/util/test_util.h index 905412b24..8f5eb5089 100644 --- a/test/util/test_util.h +++ b/test/util/test_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/test_util_test.cc b/test/util/test_util_test.cc index 5889651d1..b7300d9e5 100644 --- a/test/util/test_util_test.cc +++ b/test/util/test_util_test.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/thread_util.h b/test/util/thread_util.h index df09ac8cf..860e77531 100644 --- a/test/util/thread_util.h +++ b/test/util/thread_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/timer_util.cc b/test/util/timer_util.cc index 681fafb69..43a26b0d3 100644 --- a/test/util/timer_util.cc +++ b/test/util/timer_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/timer_util.h b/test/util/timer_util.h index 9bdc51a57..2cebfa5d1 100644 --- a/test/util/timer_util.h +++ b/test/util/timer_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/third_party/gvsync/atomicptr_unsafe.go b/third_party/gvsync/atomicptr_unsafe.go index da9f16240..53a943282 100644 --- a/third_party/gvsync/atomicptr_unsafe.go +++ b/third_party/gvsync/atomicptr_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/atomicptrtest/atomicptr_test.go b/third_party/gvsync/atomicptrtest/atomicptr_test.go index 15d0936d4..8fdc5112e 100644 --- a/third_party/gvsync/atomicptrtest/atomicptr_test.go +++ b/third_party/gvsync/atomicptrtest/atomicptr_test.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/downgradable_rwmutex_test.go b/third_party/gvsync/downgradable_rwmutex_test.go index 6517dd5dc..40c384b8b 100644 --- a/third_party/gvsync/downgradable_rwmutex_test.go +++ b/third_party/gvsync/downgradable_rwmutex_test.go @@ -1,5 +1,5 @@ // Copyright 2009 The Go Authors. All rights reserved. -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/downgradable_rwmutex_unsafe.go b/third_party/gvsync/downgradable_rwmutex_unsafe.go index 131f0a2ba..4d43eb765 100644 --- a/third_party/gvsync/downgradable_rwmutex_unsafe.go +++ b/third_party/gvsync/downgradable_rwmutex_unsafe.go @@ -1,5 +1,5 @@ // Copyright 2009 The Go Authors. All rights reserved. -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/gvsync.go b/third_party/gvsync/gvsync.go index 46a2565fd..3bbef13c3 100644 --- a/third_party/gvsync/gvsync.go +++ b/third_party/gvsync/gvsync.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/memmove_unsafe.go b/third_party/gvsync/memmove_unsafe.go index d483fc739..4c8aa9ab6 100644 --- a/third_party/gvsync/memmove_unsafe.go +++ b/third_party/gvsync/memmove_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/norace_unsafe.go b/third_party/gvsync/norace_unsafe.go index f9c88d13f..e3852db8c 100644 --- a/third_party/gvsync/norace_unsafe.go +++ b/third_party/gvsync/norace_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/race_unsafe.go b/third_party/gvsync/race_unsafe.go index 2cdcdf7f7..13c02a830 100644 --- a/third_party/gvsync/race_unsafe.go +++ b/third_party/gvsync/race_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/seqatomic_unsafe.go b/third_party/gvsync/seqatomic_unsafe.go index ef61503e2..c52d378f1 100644 --- a/third_party/gvsync/seqatomic_unsafe.go +++ b/third_party/gvsync/seqatomic_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/seqatomictest/seqatomic_test.go b/third_party/gvsync/seqatomictest/seqatomic_test.go index d0c373bae..2da73cf96 100644 --- a/third_party/gvsync/seqatomictest/seqatomic_test.go +++ b/third_party/gvsync/seqatomictest/seqatomic_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/third_party/gvsync/seqcount.go b/third_party/gvsync/seqcount.go index c7ae91cfa..2c9c2c3d6 100644 --- a/third_party/gvsync/seqcount.go +++ b/third_party/gvsync/seqcount.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/seqcount_test.go b/third_party/gvsync/seqcount_test.go index ee6579ed8..085e574b3 100644 --- a/third_party/gvsync/seqcount_test.go +++ b/third_party/gvsync/seqcount_test.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/tools/go_generics/generics.go b/tools/go_generics/generics.go index eaf5c4970..ca414d8cb 100644 --- a/tools/go_generics/generics.go +++ b/tools/go_generics/generics.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/all_stmts/input.go b/tools/go_generics/generics_tests/all_stmts/input.go index 19184a3fe..4791d1ff1 100644 --- a/tools/go_generics/generics_tests/all_stmts/input.go +++ b/tools/go_generics/generics_tests/all_stmts/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/all_stmts/output/output.go b/tools/go_generics/generics_tests/all_stmts/output/output.go index 51582346c..a53d84535 100644 --- a/tools/go_generics/generics_tests/all_stmts/output/output.go +++ b/tools/go_generics/generics_tests/all_stmts/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/all_types/input.go b/tools/go_generics/generics_tests/all_types/input.go index ed6e97c29..3575d02ec 100644 --- a/tools/go_generics/generics_tests/all_types/input.go +++ b/tools/go_generics/generics_tests/all_types/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/all_types/lib/lib.go b/tools/go_generics/generics_tests/all_types/lib/lib.go index 7e73e678e..988786496 100644 --- a/tools/go_generics/generics_tests/all_types/lib/lib.go +++ b/tools/go_generics/generics_tests/all_types/lib/lib.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/all_types/output/output.go b/tools/go_generics/generics_tests/all_types/output/output.go index ec09a6be4..41fd147a1 100644 --- a/tools/go_generics/generics_tests/all_types/output/output.go +++ b/tools/go_generics/generics_tests/all_types/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/consts/input.go b/tools/go_generics/generics_tests/consts/input.go index 394bcc262..04b95fcc6 100644 --- a/tools/go_generics/generics_tests/consts/input.go +++ b/tools/go_generics/generics_tests/consts/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/consts/output/output.go b/tools/go_generics/generics_tests/consts/output/output.go index 91a07fdc2..18d316cc9 100644 --- a/tools/go_generics/generics_tests/consts/output/output.go +++ b/tools/go_generics/generics_tests/consts/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/imports/input.go b/tools/go_generics/generics_tests/imports/input.go index 22e6641a6..0f032c2a1 100644 --- a/tools/go_generics/generics_tests/imports/input.go +++ b/tools/go_generics/generics_tests/imports/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/imports/output/output.go b/tools/go_generics/generics_tests/imports/output/output.go index 2555c0004..2488ca58c 100644 --- a/tools/go_generics/generics_tests/imports/output/output.go +++ b/tools/go_generics/generics_tests/imports/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/remove_typedef/input.go b/tools/go_generics/generics_tests/remove_typedef/input.go index d9c9b8530..cf632bae7 100644 --- a/tools/go_generics/generics_tests/remove_typedef/input.go +++ b/tools/go_generics/generics_tests/remove_typedef/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/remove_typedef/output/output.go b/tools/go_generics/generics_tests/remove_typedef/output/output.go index f111a9426..d44fd8e1c 100644 --- a/tools/go_generics/generics_tests/remove_typedef/output/output.go +++ b/tools/go_generics/generics_tests/remove_typedef/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/simple/input.go b/tools/go_generics/generics_tests/simple/input.go index 711687cf5..2a917f16c 100644 --- a/tools/go_generics/generics_tests/simple/input.go +++ b/tools/go_generics/generics_tests/simple/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/simple/output/output.go b/tools/go_generics/generics_tests/simple/output/output.go index 139c9bf9d..6bfa0b25b 100644 --- a/tools/go_generics/generics_tests/simple/output/output.go +++ b/tools/go_generics/generics_tests/simple/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/globals/globals_visitor.go b/tools/go_generics/globals/globals_visitor.go index daaa17b1d..7ae48c662 100644 --- a/tools/go_generics/globals/globals_visitor.go +++ b/tools/go_generics/globals/globals_visitor.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/globals/scope.go b/tools/go_generics/globals/scope.go index b75a91689..96c965ea2 100644 --- a/tools/go_generics/globals/scope.go +++ b/tools/go_generics/globals/scope.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/go_generics_unittest.sh b/tools/go_generics/go_generics_unittest.sh index e7553a071..44b22db91 100755 --- a/tools/go_generics/go_generics_unittest.sh +++ b/tools/go_generics/go_generics_unittest.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google LLC +# Copyright 2018 The gVisor Authors. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/tools/go_generics/go_merge/main.go b/tools/go_generics/go_merge/main.go index 2f83facf8..f6a331123 100644 --- a/tools/go_generics/go_merge/main.go +++ b/tools/go_generics/go_merge/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/imports.go b/tools/go_generics/imports.go index 57f7c3dce..3a7230c97 100644 --- a/tools/go_generics/imports.go +++ b/tools/go_generics/imports.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/remove.go b/tools/go_generics/remove.go index 139d03955..568a6bbd3 100644 --- a/tools/go_generics/remove.go +++ b/tools/go_generics/remove.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/rules_tests/template.go b/tools/go_generics/rules_tests/template.go index f3f31ae8e..aace61da1 100644 --- a/tools/go_generics/rules_tests/template.go +++ b/tools/go_generics/rules_tests/template.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/rules_tests/template_test.go b/tools/go_generics/rules_tests/template_test.go index 3a38c8629..b2a3446ef 100644 --- a/tools/go_generics/rules_tests/template_test.go +++ b/tools/go_generics/rules_tests/template_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_stateify/main.go b/tools/go_stateify/main.go index 9e2c8e106..db7a7107b 100644 --- a/tools/go_stateify/main.go +++ b/tools/go_stateify/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/tag_release.sh b/tools/tag_release.sh index 6906a952f..02a49cdf1 100755 --- a/tools/tag_release.sh +++ b/tools/tag_release.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2019 Google LLC +# Copyright 2019 The gVisor Authors. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/tools/workspace_status.sh b/tools/workspace_status.sh index a0e646e45..64a905fc9 100755 --- a/tools/workspace_status.sh +++ b/tools/workspace_status.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google LLC +# Copyright 2018 The gVisor Authors. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/vdso/barrier.h b/vdso/barrier.h index 5b6c763f6..edba4afb5 100644 --- a/vdso/barrier.h +++ b/vdso/barrier.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/check_vdso.py b/vdso/check_vdso.py index 6f7d7e7ec..e41b09709 100644 --- a/vdso/check_vdso.py +++ b/vdso/check_vdso.py @@ -1,4 +1,4 @@ -# Copyright 2018 Google LLC +# Copyright 2018 The gVisor Authors. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/vdso/compiler.h b/vdso/compiler.h index d65f148fb..54a510000 100644 --- a/vdso/compiler.h +++ b/vdso/compiler.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/cycle_clock.h b/vdso/cycle_clock.h index 309e07a3f..5d3fbb257 100644 --- a/vdso/cycle_clock.h +++ b/vdso/cycle_clock.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/seqlock.h b/vdso/seqlock.h index ab2f3fda3..7a173174b 100644 --- a/vdso/seqlock.h +++ b/vdso/seqlock.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/syscalls.h b/vdso/syscalls.h index 90fb424ce..f5865bb72 100644 --- a/vdso/syscalls.h +++ b/vdso/syscalls.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/vdso.cc b/vdso/vdso.cc index 550729035..6265ad217 100644 --- a/vdso/vdso.cc +++ b/vdso/vdso.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/vdso_time.cc b/vdso/vdso_time.cc index 9fc262f60..1bb4bb86b 100644 --- a/vdso/vdso_time.cc +++ b/vdso/vdso_time.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/vdso_time.h b/vdso/vdso_time.h index 464dadff2..70d079efc 100644 --- a/vdso/vdso_time.h +++ b/vdso/vdso_time.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// Copyright 2018 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. -- cgit v1.2.3