diff --git a/.gitignore b/.gitignore index fa912d374..4d6f1a763 100644 --- a/.gitignore +++ b/.gitignore @@ -24,6 +24,9 @@ builds/ *.tfstate.backup .vscode +idk/testdata/idk*.out +idk/testenv/certs/* + # copy of .gitignore from archived idk repo # Compiled Object files, Static and Dynamic libs (Shared Objects) *.o diff --git a/Dockerfile b/Dockerfile index 992336ee0..59f906052 100644 --- a/Dockerfile +++ b/Dockerfile @@ -22,7 +22,7 @@ FROM golang:${GO_VERSION} as pilosa-builder ARG MAKE_FLAGS WORKDIR /pilosa -RUN go get github.com/rakyll/statik +RUN go install github.com/rakyll/statik@v0.1.7 COPY . ./ COPY --from=lattice-builder /lattice/build /lattice diff --git a/Makefile b/Makefile index 305cd549f..112791f83 100644 --- a/Makefile +++ b/Makefile @@ -205,12 +205,8 @@ generate-pql: require-peg generate-proto-grpc: require-protoc require-protoc-gen-go protoc -I proto proto/pilosa.proto --go_out=plugins=grpc:proto - protoc -I proto proto/vdsm/vdsm.proto --go_out=plugins=grpc:proto - # TODO: Modify above commands and remove the below mv if possible. - # See https://go-review.googlesource.com/c/protobuf/+/219298/ for info on --go-opt - # I couldn't get it to work during development - Cody - cp -r proto/github.com/featurebasedb/featurebase/v3/proto/ proto/ - rm -rf proto/github.com +# address re-generation here only if we need to +# protoc -I proto proto/vdsm.proto --go_out=plugins=grpc:proto # `go generate` all needed packages generate: generate-protoc generate-statik generate-stringer generate-pql diff --git a/api.go b/api.go index dacd90ef9..57f40419a 100644 --- a/api.go +++ b/api.go @@ -224,14 +224,23 @@ func (api *API) CreateIndex(ctx context.Context, indexName string, options Index span, _ := tracing.StartSpanFromContext(ctx, "API.CreateIndex") defer span.Finish() + // get the requestUserID from the context -- assumes the http handler has populated this from + // authN/Z info + requestUserID, ok := ctx.Value(ContextRequestUserIdKey).(string) + if !ok { + requestUserID = "" + } + if err := api.validate(apiCreateIndex); err != nil { return nil, errors.Wrap(err, "validating api method") } // Populate the create index message. + ts := timestamp() cim := &CreateIndexMessage{ Index: indexName, - CreatedAt: timestamp(), + CreatedAt: ts, + Owner: requestUserID, Meta: options, } @@ -307,6 +316,13 @@ func (api *API) CreateField(ctx context.Context, indexName string, fieldName str return nil, errors.Wrap(err, "validating api method") } + // get the requestUserID from the context -- assumes the http handler has populated this from + // authN/Z info + requestUserID, ok := ctx.Value(ContextRequestUserIdKey).(string) + if !ok { + requestUserID = "" + } + // Apply and validate functional options. fo, err := newFieldOptions(opts...) if err != nil { @@ -324,11 +340,12 @@ func (api *API) CreateField(ctx context.Context, indexName string, fieldName str Index: indexName, Field: fieldName, CreatedAt: timestamp(), + Owner: requestUserID, Meta: fo, } // Create field. - field, err := index.CreateField(fieldName, opts...) + field, err := index.CreateField(fieldName, requestUserID, opts...) if err != nil { return nil, errors.Wrap(err, "creating field") } @@ -358,7 +375,11 @@ func (api *API) UpdateField(ctx context.Context, indexName, fieldName string, up return newNotFoundError(ErrIndexNotFound, indexName) } - cfm, err := index.UpdateField(ctx, fieldName, update) + // get the requestUserID from the context -- assumes the http handler has populated this from + // authN/Z info + requestUserID, _ := ctx.Value(ContextRequestUserIdKey).(string) + + cfm, err := index.UpdateField(ctx, fieldName, requestUserID, update) if err != nil { return errors.Wrap(err, "updating field") } diff --git a/api_test.go b/api_test.go index 632768e37..f6179a6fc 100644 --- a/api_test.go +++ b/api_test.go @@ -1388,11 +1388,11 @@ func TestVariousApiTranslateCalls(t *testing.T) { // this should never actually get used because we're testing for errors here r := strings.NewReader("") // test index - idx, err := api.Holder().CreateIndex(c.Idx(), pilosa.IndexOptions{}) + idx, err := api.Holder().CreateIndex(c.Idx(), "", pilosa.IndexOptions{}) if err != nil { t.Fatalf("%v: could not create test index", err) } - if _, err = idx.CreateFieldIfNotExistsWithOptions("field", &pilosa.FieldOptions{Keys: false}); err != nil { + if _, err = idx.CreateFieldIfNotExistsWithOptions("field", "", &pilosa.FieldOptions{Keys: false}); err != nil { t.Fatalf("creating field: %v", err) } t.Run("translateIndexDbOnNilIndex", diff --git a/client/batch.go b/client/batch.go index 265c52733..1827aad2a 100644 --- a/client/batch.go +++ b/client/batch.go @@ -87,6 +87,7 @@ type agedTranslation struct { // | uint64 | set | any | // | int64 | int | any | // | float64| decimal | scale | +// | bool | bool | any | // | nil | any | | // // nil values are ignored. @@ -124,6 +125,15 @@ type Batch struct { // values holds the values for each record of an int field values map[string][]int64 + // boolValues is a map[fieldName][idsIndex]bool, which holds the values for + // each record of a bool field. It is a map of maps in order to accomodate + // nil values (they just aren't recorded in the map[int]). + boolValues map[string]map[int]bool + + // boolNulls holds a slice of indices into b.ids for each bool field which + // has nil values. + boolNulls map[string][]uint64 + // times holds a time for each record. (if any of the fields are time fields) times []QuantizedTime @@ -233,6 +243,8 @@ func NewBatch(client *Client, size int, index *Index, fields []*Field, opts ...B headerMap := make(map[string]*Field, len(fields)) rowIDs := make(map[int][]uint64, len(fields)) values := make(map[string][]int64) + boolValues := make(map[string]map[int]bool) + boolNulls := make(map[string][]uint64) tt := make(map[int]map[string][]int, len(fields)) ttSets := make(map[string]map[string][]int) hasTime := false @@ -257,6 +269,8 @@ func NewBatch(client *Client, size int, index *Index, fields []*Field, opts ...B tt[i] = make(map[string][]int) } rowIDs[i] = make([]uint64, 0, size) + case FieldTypeBool: + boolValues[field.Name()] = make(map[int]bool) default: return nil, errors.Errorf("field type '%s' is not currently supported through Batch", typ) } @@ -273,6 +287,8 @@ func NewBatch(client *Client, size int, index *Index, fields []*Field, opts ...B clearRowIDs: make(map[int]map[int]uint64), rowIDSets: make(map[string][][]uint64), values: values, + boolValues: boolValues, + boolNulls: boolNulls, nullIndices: make(map[string][]uint64), toTranslate: tt, toTranslateClear: make(map[int]map[string][]int), @@ -480,28 +496,8 @@ func (b *Batch) Add(rec Row) error { field := b.header[i] switch val := rec.Values[i].(type) { case string: - if field.Opts().Type() != FieldTypeInt { - // nil-extend - for len(b.rowIDs[i]) < curPos { - b.rowIDs[i] = append(b.rowIDs[i], nilSentinel) - } - rowIDs := b.rowIDs[i] - // empty string is not a valid value at this point (Pilosa refuses to translate it) - if val == "" { // - b.rowIDs[i] = append(rowIDs, nilSentinel) - - } else if rowID, ok := b.getRowTranslation(field.Name(), val); ok { - b.rowIDs[i] = append(rowIDs, rowID) - } else { - ints, ok := b.toTranslate[i][val] - if !ok { - ints = make([]int, 0) - } - ints = append(ints, curPos) - b.toTranslate[i][val] = ints - b.rowIDs[i] = append(rowIDs, 0) - } - } else if field.Opts().Type() == FieldTypeInt { + switch field.Opts().Type() { + case FieldTypeInt: if val == "" { // copied from the `case nil:` section for ints and decimals b.values[field.Name()] = append(b.values[field.Name()], 0) @@ -522,6 +518,30 @@ func (b *Batch) Add(rec Row) error { b.toTranslate[i][val] = ints b.values[field.Name()] = append(b.values[field.Name()], 0) } + case FieldTypeBool: + // If we want to support bools as string values, we would do + // that here. + default: + // nil-extend + for len(b.rowIDs[i]) < curPos { + b.rowIDs[i] = append(b.rowIDs[i], nilSentinel) + } + rowIDs := b.rowIDs[i] + // empty string is not a valid value at this point (Pilosa refuses to translate it) + if val == "" { // + b.rowIDs[i] = append(rowIDs, nilSentinel) + + } else if rowID, ok := b.getRowTranslation(field.Name(), val); ok { + b.rowIDs[i] = append(rowIDs, rowID) + } else { + ints, ok := b.toTranslate[i][val] + if !ok { + ints = make([]int, 0) + } + ints = append(ints, curPos) + b.toTranslate[i][val] = ints + b.rowIDs[i] = append(rowIDs, 0) + } } case uint64: // nil-extend @@ -579,8 +599,8 @@ func (b *Batch) Add(rec Row) error { } b.rowIDSets[field.Name()] = append(rowIDSets, val) case nil: - t := field.Opts().Type() - if t == FieldTypeInt || t == FieldTypeDecimal || t == FieldTypeTimestamp { + switch field.Opts().Type() { + case FieldTypeInt, FieldTypeDecimal, FieldTypeTimestamp: b.values[field.Name()] = append(b.values[field.Name()], 0) nullIndices, ok := b.nullIndices[field.Name()] if !ok { @@ -589,7 +609,15 @@ func (b *Batch) Add(rec Row) error { nullIndices = append(nullIndices, uint64(curPos)) b.nullIndices[field.Name()] = nullIndices - } else { + case FieldTypeBool: + boolNulls, ok := b.boolNulls[field.Name()] + if !ok { + boolNulls = make([]uint64, 0) + } + boolNulls = append(boolNulls, uint64(curPos)) + b.boolNulls[field.Name()] = boolNulls + + default: // only append nil to rowIDs if this field already has // rowIDs. Otherwise, this could be a []string or // []uint64 field where we've only seen nil values so @@ -600,6 +628,10 @@ func (b *Batch) Add(rec Row) error { b.rowIDs[i] = append(rowIDs, nilSentinel) } } + + case bool: + b.boolValues[field.Name()][curPos] = val + default: return errors.Errorf("Val %v Type %[1]T is not currently supported. Use string, uint64 (row id), or int64 (integer value)", val) } @@ -637,7 +669,6 @@ func (b *Batch) Add(rec Row) error { } b.rowIDs[i][len(b.rowIDs[i])-1] = clearSentinel } - default: return errors.Errorf("Clearing a value '%v' Type %[1]T is not currently supported (field '%s')", val, field.Name()) } @@ -712,7 +743,6 @@ func (b *Batch) Import() error { return errors.Wrap(err, "making fragments (flush)") } if b.useShardTransactionalEndpoint { - // TODO handle bool? frags, clearFrags, err = b.makeSingleValFragments(frags, clearFrags) if err != nil { return errors.Wrap(err, "making single val fragments") @@ -1478,6 +1508,60 @@ func (b *Batch) makeSingleValFragments(frags, clearFrags fragments) (fragments, } } } + // ------------------------- + // Boolean fields + // ------------------------- + falseRowOffset := 0 * shardWidth // fragment row 0 + trueRowOffset := 1 * shardWidth // fragment row 1 + + // For bools which have been set to null, clear both the true and false + // values for the record. Because this ends up going through the + // API.ImportRoaringShard() method (which handles `bool` fields the same as + // `mutex` fields), we don't actually set the true and false rows of the + // boolean fragment; rather, we just set the first row to indicate which + // records (for all rows) to clear. + for fieldname, boolNulls := range b.boolNulls { + field := b.headerMap[fieldname] + if field.Opts().Type() != featurebase.FieldTypeBool { + continue + } + for _, pos := range boolNulls { + recID := b.ids[pos] + shard := recID / shardWidth + clearBM := clearFrags.GetOrCreate(shard, field.Name(), "standard") + + fragmentColumn := recID % shardWidth + clearBM.Add(fragmentColumn) + } + } + + // For bools which have been set to a non-nil value, set the appropriate + // value for the record, and unset the opposing values. For example, if the + // bool is set to `false`, then set the bit in the "false" row, and clear + // the bit in the "true" row. + for fieldname, boolMap := range b.boolValues { + field := b.headerMap[fieldname] + if field.Opts().Type() != featurebase.FieldTypeBool { + continue + } + + for pos, boolVal := range boolMap { + recID := b.ids[pos] + + shard := recID / shardWidth + bitmap := frags.GetOrCreate(shard, field.Name(), "standard") + clearBM := clearFrags.GetOrCreate(shard, field.Name(), "standard") + + fragmentColumn := recID % shardWidth + clearBM.Add(fragmentColumn) + + if boolVal { + bitmap.Add(trueRowOffset + fragmentColumn) + } else { + bitmap.Add(falseRowOffset + fragmentColumn) + } + } + } return frags, clearFrags, nil } @@ -1700,6 +1784,14 @@ func (b *Batch) reset() { delete(clearMap, k) } } + for _, boolsMap := range b.boolValues { + for k := range boolsMap { + delete(boolsMap, k) + } + } + for k := range b.boolNulls { + delete(b.boolNulls, k) // TODO pool these slices + } for i := range b.toTranslateID { b.toTranslateID[i] = "" } diff --git a/client/batch_test.go b/client/batch_test.go index b78f7a616..ed893963e 100644 --- a/client/batch_test.go +++ b/client/batch_test.go @@ -33,6 +33,7 @@ func TestAgainstCluster(t *testing.T) { client := NewTestClient(t, c) t.Run("string-slice-combos", func(t *testing.T) { testStringSliceCombos(t, c, client) }) t.Run("import-batch-ints", func(t *testing.T) { testImportBatchInts(t, c, client) }) + t.Run("import-batch-bools", func(t *testing.T) { testImportBatchBools(t, c, client) }) t.Run("import-batch-sorting", func(t *testing.T) { testImportBatchSorting(t, c, client) }) t.Run("test-trim-null", func(t *testing.T) { testTrimNull(t, c, client) }) t.Run("test-string-slice-empty-and-nil", func(t *testing.T) { testStringSliceEmptyAndNil(t, c, client) }) @@ -1842,3 +1843,45 @@ func mutexNilClearKey(t *testing.T, c *test.Cluster, client *Client) { } errorIfNotEqual(t, resp.Result().Row().Keys, []string{"0"}) } + +func testImportBatchBools(t *testing.T, c *test.Cluster, client *Client) { + schema := NewSchema() + idx := schema.Index("test-import-batch-bools") + field := idx.Field("boolcol", OptFieldTypeBool()) + err := client.SyncSchema(schema) + if err != nil { + t.Fatalf("syncing schema: %v", err) + } + + b, err := NewBatch(client, 3, idx, []*Field{field}, OptUseShardTransactionalEndpoint(true)) + if err != nil { + t.Fatalf("getting batch: %v", err) + } + r := Row{Values: make([]interface{}, 1)} + + r.ID = uint64(0) + r.Values[0] = bool(false) + err = b.Add(r) + if err != nil { + t.Fatalf("adding after import: %v", err) + } + r.ID = uint64(1) + r.Values[0] = bool(true) + err = b.Add(r) + if err != nil { + t.Fatalf("adding second after import: %v", err) + } + + err = b.Import() + if err != nil { + t.Fatalf("second import: %v", err) + } + + resp, err := client.Query(idx.RawQuery("Count(All())")) + if err != nil { + t.Fatalf("querying: %v", err) + } + if res := resp.Results()[0]; res.Count() != 2 { + t.Fatalf("unexpected result: %+v", res) + } +} diff --git a/cluster.go b/cluster.go index 40193415a..2864b43b6 100644 --- a/cluster.go +++ b/cluster.go @@ -879,7 +879,9 @@ type CreateShardMessage struct { type CreateIndexMessage struct { Index string CreatedAt int64 - Meta IndexOptions + Owner string + + Meta IndexOptions } // DeleteIndexMessage is an internal message indicating index deletion. @@ -892,6 +894,7 @@ type CreateFieldMessage struct { Index string Field string CreatedAt int64 + Owner string Meta *FieldOptions } diff --git a/dbshard_internal_test.go b/dbshard_internal_test.go index 208b8fe0d..a481817c0 100644 --- a/dbshard_internal_test.go +++ b/dbshard_internal_test.go @@ -188,7 +188,7 @@ func makeSampleRoaringDir(t *testing.T, root, index, backend string, minBytes in } func helperCreateDBShard(h *Holder, index string, shard uint64) *Index { - idx, err := h.CreateIndexIfNotExists(index, IndexOptions{}) + idx, err := h.CreateIndexIfNotExists(index, "", IndexOptions{}) PanicOn(err) // TODO: It's not clear that this is actually doing anything. dbs, err := h.txf.dbPerShard.GetDBShard(index, shard, idx) @@ -253,7 +253,7 @@ func Test_DBPerShard_GetFieldView2Shards_map_from_RBF(t *testing.T) { index := "rick" field := "f" - idx, err := holder.CreateIndex(index, IndexOptions{}) + idx, err := holder.CreateIndex(index, "", IndexOptions{}) if err != nil { t.Fatalf("creating index: %v", err) } diff --git a/disco/disco.go b/disco/disco.go index 4921f43c1..88933f002 100644 --- a/disco/disco.go +++ b/disco/disco.go @@ -101,8 +101,8 @@ type Schemator interface { CreateIndex(ctx context.Context, name string, val []byte) error DeleteIndex(ctx context.Context, name string) error Field(ctx context.Context, index, field string) ([]byte, error) - CreateField(ctx context.Context, index, field string, val []byte) error - UpdateField(ctx context.Context, index, field string, val []byte) error + CreateField(ctx context.Context, index, field string, fieldVal []byte) error + UpdateField(ctx context.Context, index, field string, fieldVal []byte) error DeleteField(ctx context.Context, index, field string) error View(ctx context.Context, index, field, view string) (bool, error) CreateView(ctx context.Context, index, field, view string) error @@ -186,23 +186,27 @@ func (*nopSchemator) Index(ctx context.Context, name string) ([]byte, error) { r func (*nopSchemator) CreateIndex(ctx context.Context, name string, val []byte) error { return nil } // DeleteIndex is a no-op implementation of the Schemator DeleteIndex method. -func (*nopSchemator) DeleteIndex(ctx context.Context, name string) error { return nil } +func (*nopSchemator) DeleteIndex(ctx context.Context, name string) error { + return nil +} // Field is a no-op implementation of the Schemator Field method. func (*nopSchemator) Field(ctx context.Context, index, field string) ([]byte, error) { return nil, nil } // CreateField is a no-op implementation of the Schemator CreateField method. -func (*nopSchemator) CreateField(ctx context.Context, index, field string, val []byte) error { +func (*nopSchemator) CreateField(ctx context.Context, index, field string, fieldVal []byte) error { return nil } // UpdateField is a no-op implementation of the Schemator UpdateField method. -func (*nopSchemator) UpdateField(ctx context.Context, index, field string, val []byte) error { +func (*nopSchemator) UpdateField(ctx context.Context, index, field string, fieldVal []byte) error { return nil } // DeleteField is a no-op implementation of the Schemator DeleteField method. -func (*nopSchemator) DeleteField(ctx context.Context, index, field string) error { return nil } +func (*nopSchemator) DeleteField(ctx context.Context, index, field string) error { + return nil +} // View is a no-op implementation of the Schemator View method. func (*nopSchemator) View(ctx context.Context, index, field, view string) (bool, error) { @@ -296,7 +300,7 @@ func (s *inMemSchemator) Field(ctx context.Context, index, field string) ([]byte } // CreateField is an in-memory implementation of the Schemator CreateField method. -func (s *inMemSchemator) CreateField(ctx context.Context, index, field string, val []byte) error { +func (s *inMemSchemator) CreateField(ctx context.Context, index, field string, fieldVal []byte) error { s.mu.Lock() defer s.mu.Unlock() idx, ok := s.schema[index] @@ -307,17 +311,17 @@ func (s *inMemSchemator) CreateField(ctx context.Context, index, field string, v // The current logic in pilosa doesn't allow us to return ErrFieldExists // here, so for now we just update the Data value if the field already // exists. - fld.Data = val + fld.Data = fieldVal return nil } idx.Fields[field] = &Field{ - Data: val, + Data: fieldVal, Views: make(map[string]struct{}), } return nil } -func (s *inMemSchemator) UpdateField(ctx context.Context, index, field string, val []byte) error { +func (s *inMemSchemator) UpdateField(ctx context.Context, index, field string, fieldVal []byte) error { s.mu.Lock() defer s.mu.Unlock() idx, ok := s.schema[index] @@ -328,7 +332,7 @@ func (s *inMemSchemator) UpdateField(ctx context.Context, index, field string, v // The current logic in pilosa doesn't allow us to return ErrFieldExists // here, so for now we just update the Data value if the field already // exists. - fld.Data = val + fld.Data = fieldVal return nil } else { return ErrFieldDoesNotExist diff --git a/encoding/proto/proto.go b/encoding/proto/proto.go index dad34da40..37766d8ad 100644 --- a/encoding/proto/proto.go +++ b/encoding/proto/proto.go @@ -681,12 +681,14 @@ func (s Serializer) encodeCreateIndexMessage(m *pilosa.CreateIndexMessage) *pb.C return &pb.CreateIndexMessage{ Index: m.Index, CreatedAt: m.CreatedAt, + Owner: m.Owner, Meta: s.encodeIndexMeta(&m.Meta), } } func (s Serializer) encodeIndexMeta(m *pilosa.IndexOptions) *pb.IndexMeta { return &pb.IndexMeta{ + Description: m.Description, Keys: m.Keys, TrackExistence: m.TrackExistence, } @@ -703,6 +705,7 @@ func (s Serializer) encodeCreateFieldMessage(m *pilosa.CreateFieldMessage) *pb.C Index: m.Index, Field: m.Field, CreatedAt: m.CreatedAt, + Owner: m.Owner, Meta: s.encodeFieldOptions(m.Meta), } } @@ -1057,12 +1060,14 @@ func (s Serializer) decodeCreateShardMessage(pb *pb.CreateShardMessage, m *pilos func (s Serializer) decodeCreateIndexMessage(pb *pb.CreateIndexMessage, m *pilosa.CreateIndexMessage) { m.Index = pb.Index m.CreatedAt = pb.CreatedAt + m.Owner = pb.Owner m.Meta = pilosa.IndexOptions{} s.decodeIndexMeta(pb.Meta, &m.Meta) } func (s Serializer) decodeIndexMeta(pb *pb.IndexMeta, m *pilosa.IndexOptions) { if pb != nil { + m.Description = pb.Description m.Keys = pb.Keys m.TrackExistence = pb.TrackExistence } @@ -1076,6 +1081,7 @@ func (s Serializer) decodeCreateFieldMessage(pb *pb.CreateFieldMessage, m *pilos m.Index = pb.Index m.Field = pb.Field m.CreatedAt = pb.CreatedAt + m.Owner = pb.Owner m.Meta = &pilosa.FieldOptions{} s.decodeFieldOptions(pb.Meta, m.Meta) } diff --git a/etcd/embed.go b/etcd/embed.go index e681f8090..11162fd3b 100644 --- a/etcd/embed.go +++ b/etcd/embed.go @@ -869,19 +869,20 @@ func (e *Etcd) Field(ctx context.Context, indexName string, name string) ([]byte return e.getKeyBytes(ctx, key) } -func (e *Etcd) CreateField(ctx context.Context, indexName string, name string, val []byte) error { +func (e *Etcd) CreateField(ctx context.Context, indexName string, name string, fieldVal []byte) error { key := schemaPrefix + indexName + "/" + name // Set up Op to write field value as bytes. op := clientv3.OpPut(key, "") - op.WithValueBytes(val) + op.WithValueBytes(fieldVal) // Check for key existence, and execute Op within a transaction. var resp *clientv3.TxnResponse err := e.retryClient(func(cli *clientv3.Client) (err error) { resp, err = cli.Txn(ctx). - If(clientv3util.KeyMissing(key)). + If( + clientv3util.KeyMissing(key)). Then(op). Commit() return err @@ -897,19 +898,20 @@ func (e *Etcd) CreateField(ctx context.Context, indexName string, name string, v return nil } -func (e *Etcd) UpdateField(ctx context.Context, indexName string, name string, val []byte) error { +func (e *Etcd) UpdateField(ctx context.Context, indexName string, name string, fieldVal []byte) error { key := schemaPrefix + indexName + "/" + name // Set up Op to write field value as bytes. op := clientv3.OpPut(key, "") - op.WithValueBytes(val) + op.WithValueBytes(fieldVal) // Check for key existence, and execute Op within a transaction. var resp *clientv3.TxnResponse err := e.retryClient(func(cli *clientv3.Client) (err error) { resp, err = cli.Txn(ctx). - If(clientv3util.KeyExists(key)). + If( + clientv3util.KeyExists(key)). Then(op). Commit() return err @@ -924,20 +926,31 @@ func (e *Etcd) UpdateField(ctx context.Context, indexName string, name string, v return nil } -func (e *Etcd) DeleteField(ctx context.Context, indexname string, name string) (err error) { - key := schemaPrefix + indexname + "/" + name +func (e *Etcd) DeleteField(ctx context.Context, indexName string, name string) (err error) { + key := schemaPrefix + indexName + "/" + name + + var resp *clientv3.TxnResponse + // Deleting field and views in one transaction. err = e.retryClient(func(cli *clientv3.Client) (err error) { - _, err = cli.Txn(ctx). - If(clientv3.Compare(clientv3.Version(key), ">", -1)). + resp, err = cli.Txn(ctx). + If( + clientv3.Compare(clientv3.Version(key), ">", -1)). Then( clientv3.OpDelete(key+"/", clientv3.WithPrefix()), // deleting field views clientv3.OpDelete(key), // deleting field ).Commit() return err }) + if err != nil { + return errors.Wrap(err, "executing transaction") + } - return errors.Wrap(err, "DeleteField") + if !resp.Succeeded { + return errors.New("deleting field from etcd failed") + } + + return nil } func (e *Etcd) View(ctx context.Context, indexName, fieldName, name string) (bool, error) { diff --git a/executor.go b/executor.go index a5026b565..9ccb0ee0e 100644 --- a/executor.go +++ b/executor.go @@ -6547,7 +6547,7 @@ func (e *executor) collectCallKeys(dst *keyCollector, c *pql.Call, index string) if keyed { opts = append(opts, OptFieldKeys()) } - if _, err := idx.CreateField(field, opts...); err != nil { + if _, err := idx.CreateField(field, "", opts...); err != nil { // We wrap these because we want to indicate that it wasn't found, // but also the problem we encountered trying to create it. return newNotFoundError(errors.Wrap(err, "creating field"), field) diff --git a/executor_internal_test.go b/executor_internal_test.go index 9db7d253c..2d20d553e 100644 --- a/executor_internal_test.go +++ b/executor_internal_test.go @@ -47,7 +47,7 @@ func TestExecutor_TranslateRowsOnBool(t *testing.T) { Cluster: NewTestCluster(t, 1), } - idx, err := e.Holder.CreateIndex("i", IndexOptions{}) + idx, err := e.Holder.CreateIndex("i", "", IndexOptions{}) if err != nil { t.Fatalf("creating index: %v", err) } @@ -55,8 +55,8 @@ func TestExecutor_TranslateRowsOnBool(t *testing.T) { qcx := holder.Txf().NewWritableQcx() defer qcx.Abort() - fb, errb := idx.CreateField("b", OptFieldTypeBool()) - _, errbk := idx.CreateField("bk", OptFieldTypeBool(), OptFieldKeys()) + fb, errb := idx.CreateField("b", "", OptFieldTypeBool()) + _, errbk := idx.CreateField("bk", "", OptFieldTypeBool(), OptFieldKeys()) if errb != nil || errbk != nil { t.Fatalf("creating fields %v, %v", errb, errbk) } @@ -563,12 +563,12 @@ func TestDistinctTimestampUnion(t *testing.T) { func TestExecutor_DeleteRows(t *testing.T) { holder := newTestHolder(t) - idx, err := holder.CreateIndex("i", IndexOptions{TrackExistence: true}) + idx, err := holder.CreateIndex("i", "", IndexOptions{TrackExistence: true}) if err != nil { t.Fatalf("creating index: %v", err) } - f, err := idx.CreateField("f") + f, err := idx.CreateField("f", "") if err != nil { t.Fatalf("creating field: %v", err) } diff --git a/executor_test.go b/executor_test.go index a620dda11..34dc612d9 100644 --- a/executor_test.go +++ b/executor_test.go @@ -1155,11 +1155,11 @@ func runCallTest(c *test.Cluster, t *testing.T, writeQuery string, readQueries [ } hldr := c.GetHolder(0) - index, err := hldr.CreateIndex(indexName, *indexOptions) + index, err := hldr.CreateIndex(indexName, "", *indexOptions) if err != nil { t.Fatal(err) } - _, err = index.CreateField("f", fieldOption...) + _, err = index.CreateField("f", "", fieldOption...) if err != nil { t.Fatal(err) } @@ -1445,7 +1445,7 @@ func TestExecutor_Execute_Set(t *testing.T) { t.Fatal(err) } - if _, err := idx.CreateField("f"); err != nil { + if _, err := idx.CreateField("f", ""); err != nil { t.Fatal(err) } @@ -1462,7 +1462,7 @@ func TestExecutor_Execute_Set(t *testing.T) { t.Run("ErrInvalidRowValueType", func(t *testing.T) { idx := hldr.MustCreateIndexIfNotExists(c.Idx("inokey"), pilosa.IndexOptions{}) - if _, err := idx.CreateField("f", pilosa.OptFieldKeys()); err != nil { + if _, err := idx.CreateField("f", "", pilosa.OptFieldKeys()); err != nil { t.Fatal(err) } if _, err := cmd.API.Query(context.Background(), &pilosa.QueryRequest{Index: c.Idx("inokey"), Query: `Set(2, f=1.2)`}); err == nil || !strings.Contains(err.Error(), "invalid value") { @@ -1488,7 +1488,7 @@ func TestExecutor_Execute_SetBool(t *testing.T) { // Create fields. index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{}) - if _, err := index.CreateFieldIfNotExists("f", pilosa.OptFieldTypeBool()); err != nil { + if _, err := index.CreateFieldIfNotExists("f", "", pilosa.OptFieldTypeBool()); err != nil { t.Fatal(err) } @@ -1534,7 +1534,7 @@ func TestExecutor_Execute_SetBool(t *testing.T) { // Create fields. index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{}) - if _, err := index.CreateFieldIfNotExists("f", pilosa.OptFieldTypeBool()); err != nil { + if _, err := index.CreateFieldIfNotExists("f", "", pilosa.OptFieldTypeBool()); err != nil { t.Fatal(err) } @@ -1560,7 +1560,7 @@ func TestExecutor_Execute_SetDecimal(t *testing.T) { // Create fields. index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{}) - if _, err := index.CreateFieldIfNotExists("f", pilosa.OptFieldTypeDecimal(2)); err != nil { + if _, err := index.CreateFieldIfNotExists("f", "", pilosa.OptFieldTypeDecimal(2)); err != nil { t.Fatal(err) } @@ -1597,7 +1597,7 @@ func TestExecutor_Execute_SetDecimal(t *testing.T) { // Create fields. index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{}) - if _, err := index.CreateFieldIfNotExists("f", pilosa.OptFieldTypeDecimal(2)); err != nil { + if _, err := index.CreateFieldIfNotExists("f", "", pilosa.OptFieldTypeDecimal(2)); err != nil { t.Fatal(err) } @@ -1631,9 +1631,9 @@ func TestExecutor_Execute_SetValue(t *testing.T) { // Create fields. index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{}) - if _, err := index.CreateFieldIfNotExists("f", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { + if _, err := index.CreateFieldIfNotExists("f", "", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { t.Fatal(err) - } else if _, err := index.CreateFieldIfNotExists("xxx"); err != nil { + } else if _, err := index.CreateFieldIfNotExists("xxx", ""); err != nil { t.Fatal(err) } @@ -1672,7 +1672,7 @@ func TestExecutor_Execute_SetValue(t *testing.T) { hldr := c.GetHolder(0) index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{}) - if _, err := index.CreateFieldIfNotExists("f", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { + if _, err := index.CreateFieldIfNotExists("f", "", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { t.Fatal(err) } @@ -1702,9 +1702,9 @@ func TestExecutor_Execute_SetValue(t *testing.T) { // Create fields. index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{}) - if _, err := index.CreateFieldIfNotExists("f", pilosa.OptFieldTypeTimestamp(pilosa.DefaultEpoch, pilosa.TimeUnitSeconds)); err != nil { + if _, err := index.CreateFieldIfNotExists("f", "", pilosa.OptFieldTypeTimestamp(pilosa.DefaultEpoch, pilosa.TimeUnitSeconds)); err != nil { t.Fatal(err) - } else if _, err := index.CreateFieldIfNotExists("xxx"); err != nil { + } else if _, err := index.CreateFieldIfNotExists("xxx", ""); err != nil { t.Fatal(err) } @@ -1841,11 +1841,11 @@ func TestExecutor_Execute_TopN(t *testing.T) { hldr := c.GetHolder(0) // Set columns for rows 0, 10, & 20 across two shards. - if idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{}); err != nil { + if idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{}); err != nil { t.Fatal(err) - } else if _, err := idx.CreateField("f"); err != nil { + } else if _, err := idx.CreateField("f", ""); err != nil { t.Fatal(err) - } else if _, err := idx.CreateField("other"); err != nil { + } else if _, err := idx.CreateField("other", ""); err != nil { t.Fatal(err) } else if _, err := c.GetNode(0).API.Query(context.Background(), &pilosa.QueryRequest{Index: c.Idx(), Query: ` Set(0, f=0) @@ -1885,11 +1885,11 @@ func TestExecutor_Execute_TopN(t *testing.T) { hldr := c.GetHolder(0) // Set columns for rows 0, 10, & 20 across two shards. - if idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{Keys: true}); err != nil { + if idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{Keys: true}); err != nil { t.Fatal(err) - } else if _, err := idx.CreateField("f"); err != nil { + } else if _, err := idx.CreateField("f", ""); err != nil { t.Fatal(err) - } else if _, err := idx.CreateField("other"); err != nil { + } else if _, err := idx.CreateField("other", ""); err != nil { t.Fatal(err) } else if _, err := c.GetNode(0).API.Query(context.Background(), &pilosa.QueryRequest{Index: c.Idx(), Query: ` Set("zero", f=0) @@ -1929,11 +1929,11 @@ func TestExecutor_Execute_TopN(t *testing.T) { hldr := c.GetHolder(0) // Set columns for rows 0, 10, & 20 across two shards. - if idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{Keys: true}); err != nil { + if idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{Keys: true}); err != nil { t.Fatal(err) - } else if _, err := idx.CreateField("f", pilosa.OptFieldKeys()); err != nil { + } else if _, err := idx.CreateField("f", "", pilosa.OptFieldKeys()); err != nil { t.Fatal(err) - } else if _, err := idx.CreateField("other", pilosa.OptFieldKeys()); err != nil { + } else if _, err := idx.CreateField("other", "", pilosa.OptFieldKeys()); err != nil { t.Fatal(err) } else if _, err := c.GetNode(0).API.Query(context.Background(), &pilosa.QueryRequest{Index: c.Idx(), Query: ` Set("zero", f="zero") @@ -1975,11 +1975,11 @@ func TestExecutor_Execute_TopN(t *testing.T) { hldr := c.GetHolder(0) // Set columns for rows 0, 10, & 20 across two shards. - if idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{Keys: true}); err != nil { + if idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{Keys: true}); err != nil { t.Fatal(err) - } else if _, err := idx.CreateField("f", pilosa.OptFieldKeys()); err != nil { + } else if _, err := idx.CreateField("f", "", pilosa.OptFieldKeys()); err != nil { t.Fatal(err) - } else if _, err := idx.CreateField("other", pilosa.OptFieldKeys()); err != nil { + } else if _, err := idx.CreateField("other", "", pilosa.OptFieldKeys()); err != nil { t.Fatal(err) } else if _, err := c.GetNode(0).API.Query(context.Background(), &pilosa.QueryRequest{Index: c.Idx(), Query: ` Set("a", f="foo") @@ -2021,9 +2021,9 @@ func TestExecutor_Execute_TopN(t *testing.T) { hldr := c.GetHolder(0) // Set data on the "f" field. - if idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{}); err != nil { + if idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{}); err != nil { t.Fatal(err) - } else if _, err := idx.CreateField("f"); err != nil { + } else if _, err := idx.CreateField("f", ""); err != nil { t.Fatal(err) } else if _, err := c.GetNode(0).API.Query(context.Background(), &pilosa.QueryRequest{Index: c.Idx(), Query: ` Set(0, f=0) @@ -2046,9 +2046,9 @@ func TestExecutor_Execute_TopN(t *testing.T) { hldr := c.GetHolder(0) // Create BSI "f" field. - if idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{}); err != nil { + if idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{}); err != nil { t.Fatal(err) - } else if _, err := idx.CreateField("f", pilosa.OptFieldTypeInt(0, 100)); err != nil { + } else if _, err := idx.CreateField("f", "", pilosa.OptFieldTypeInt(0, 100)); err != nil { t.Fatal(err) } else if _, err := c.GetNode(0).API.Query(context.Background(), &pilosa.QueryRequest{Index: c.Idx(), Query: `TopN(f, n=2)`}); err == nil || !strings.Contains(err.Error(), `finding top results: mapping on primary node: cannot compute TopN() on integer, decimal, or timestamp field: "f"`) { t.Fatalf("unexpected error: %v", err) @@ -2060,9 +2060,9 @@ func TestExecutor_Execute_TopN(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) - if idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{}); err != nil { + if idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{}); err != nil { t.Fatal(err) - } else if _, err := idx.CreateField("f", pilosa.OptFieldTypeSet(pilosa.CacheTypeNone, 0)); err != nil { + } else if _, err := idx.CreateField("f", "", pilosa.OptFieldTypeSet(pilosa.CacheTypeNone, 0)); err != nil { t.Fatal(err) } else if _, err := c.GetNode(0).API.Query(context.Background(), &pilosa.QueryRequest{Index: c.Idx(), Query: ` Set(0, f=0) @@ -2187,7 +2187,7 @@ func TestExecutor_Execute_MinMax(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) - idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{}) + idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{}) if err != nil { t.Fatal(err) } @@ -2205,7 +2205,7 @@ func TestExecutor_Execute_MinMax(t *testing.T) { for i, test := range tests { fld := fmt.Sprintf("f%d", i) t.Run("MinMaxField_"+fld, func(t *testing.T) { - if _, err := idx.CreateField(fld, pilosa.OptFieldTypeInt(test.min, test.max)); err != nil { + if _, err := idx.CreateField(fld, "", pilosa.OptFieldTypeInt(test.min, test.max)); err != nil { t.Fatal(err) } @@ -2279,7 +2279,7 @@ func TestExecutor_Execute_MinMax(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) - idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{}) + idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{}) if err != nil { t.Fatal(err) } @@ -2323,7 +2323,7 @@ func TestExecutor_Execute_MinMax(t *testing.T) { // This extra field exists to make there be shards which are present, // but have no decimal values set, to make sure they don't break // the results. - if _, err := idx.CreateFieldIfNotExists("z"); err != nil { + if _, err := idx.CreateFieldIfNotExists("z", ""); err != nil { t.Fatal(err) } if res, err := c.GetNode(0).API.Query(context.Background(), &pilosa.QueryRequest{Index: c.Idx(), Query: `Set(1, z=0)`}); err != nil { @@ -2355,7 +2355,7 @@ func TestExecutor_Execute_MinMax(t *testing.T) { for i, test := range tests { fld := fmt.Sprintf("f%d", i) t.Run("MinMaxField_"+fld, func(t *testing.T) { - if _, err := idx.CreateField(fld, pilosa.OptFieldTypeDecimal(test.scale, test.min, test.max)); err != nil { + if _, err := idx.CreateField(fld, "", pilosa.OptFieldTypeDecimal(test.scale, test.min, test.max)); err != nil { t.Fatal(err) } @@ -2411,7 +2411,7 @@ func TestExecutor_Execute_MinMax(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) - idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{}) + idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{}) if err != nil { t.Fatal(err) } @@ -2428,7 +2428,7 @@ func TestExecutor_Execute_MinMax(t *testing.T) { for i, test := range tests { fld := fmt.Sprintf("f%d", i) t.Run("MinMaxField_"+fld, func(t *testing.T) { - if _, err := idx.CreateField(fld, pilosa.OptFieldTypeTimestamp(test.epoch, pilosa.TimeUnitSeconds)); err != nil { + if _, err := idx.CreateField(fld, "", pilosa.OptFieldTypeTimestamp(test.epoch, pilosa.TimeUnitSeconds)); err != nil { t.Fatal(err) } else if _, err := c.GetNode(0).API.Query(context.Background(), &pilosa.QueryRequest{Index: c.Idx(), Query: fmt.Sprintf(`Set(10, %s="%s")`, fld, test.set.Format(time.RFC3339))}); err != nil { t.Fatal(err) @@ -2499,16 +2499,16 @@ func TestExecutor_Execute_MinMax(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) - idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{}) + idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{}) if err != nil { t.Fatal(err) } - if _, err := idx.CreateField("x"); err != nil { + if _, err := idx.CreateField("x", ""); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("f", pilosa.OptFieldTypeInt(-1100, 1000)); err != nil { + if _, err := idx.CreateField("f", "", pilosa.OptFieldTypeInt(-1100, 1000)); err != nil { t.Fatal(err) } @@ -2563,16 +2563,16 @@ func TestExecutor_Execute_MinMax(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) - idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{Keys: true}) + idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{Keys: true}) if err != nil { t.Fatal(err) } - if _, err := idx.CreateField("x"); err != nil { + if _, err := idx.CreateField("x", ""); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("f", pilosa.OptFieldTypeInt(-1110, 1000)); err != nil { + if _, err := idx.CreateField("f", "", pilosa.OptFieldTypeInt(-1110, 1000)); err != nil { t.Fatal(err) } @@ -2656,12 +2656,12 @@ func TestExecutor_Execute_MinMaxRow(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) - idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{}) + idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{}) if err != nil { t.Fatal(err) } - if _, err := idx.CreateField("f"); err != nil { + if _, err := idx.CreateField("f", ""); err != nil { t.Fatal(err) } @@ -2720,12 +2720,12 @@ func TestExecutor_Execute_MinMaxRow(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) - idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{}) + idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{}) if err != nil { t.Fatal(err) } - if _, err := idx.CreateField("f", pilosa.OptFieldKeys()); err != nil { + if _, err := idx.CreateField("f", "", pilosa.OptFieldKeys()); err != nil { t.Fatal(err) } @@ -2776,28 +2776,28 @@ func TestExecutor_Execute_Sum(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) - idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{}) + idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{}) if err != nil { t.Fatal(err) } - if _, err := idx.CreateField("x"); err != nil { + if _, err := idx.CreateField("x", ""); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("foo", pilosa.OptFieldTypeInt(-990, 1000)); err != nil { + if _, err := idx.CreateField("foo", "", pilosa.OptFieldTypeInt(-990, 1000)); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("bar", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { + if _, err := idx.CreateField("bar", "", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("other", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { + if _, err := idx.CreateField("other", "", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("dec", pilosa.OptFieldTypeDecimal(3)); err != nil { + if _, err := idx.CreateField("dec", "", pilosa.OptFieldTypeDecimal(3)); err != nil { t.Fatal(err) } @@ -2910,24 +2910,24 @@ func TestExecutor_Execute_Sum(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) - idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{Keys: true}) + idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{Keys: true}) if err != nil { t.Fatal(err) } - if _, err := idx.CreateField("x"); err != nil { + if _, err := idx.CreateField("x", ""); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("foo", pilosa.OptFieldTypeInt(-990, 1000)); err != nil { + if _, err := idx.CreateField("foo", "", pilosa.OptFieldTypeInt(-990, 1000)); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("bar", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { + if _, err := idx.CreateField("bar", "", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("other", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { + if _, err := idx.CreateField("other", "", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { t.Fatal(err) } @@ -2970,7 +2970,7 @@ func TestExecutor_DecimalArgs(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) - idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{}) + idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{}) if err != nil { t.Fatal(err) } @@ -2984,7 +2984,7 @@ func TestExecutor_DecimalArgs(t *testing.T) { t.Fatal(err) } - if _, err := idx.CreateField("f", pilosa.OptFieldTypeDecimal(2, min, max)); err != nil { + if _, err := idx.CreateField("f", "", pilosa.OptFieldTypeDecimal(2, min, max)); err != nil { t.Fatal(err) } @@ -3001,28 +3001,28 @@ func TestExecutor_Execute_Row_BSIGroup(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) - idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{TrackExistence: true}) + idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{TrackExistence: true}) if err != nil { t.Fatal(err) } - if _, err := idx.CreateField("f"); err != nil { + if _, err := idx.CreateField("f", ""); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("foo", pilosa.OptFieldTypeInt(-990, 1000)); err != nil { + if _, err := idx.CreateField("foo", "", pilosa.OptFieldTypeInt(-990, 1000)); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("bar", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { + if _, err := idx.CreateField("bar", "", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("other", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { + if _, err := idx.CreateField("other", "", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("edge", pilosa.OptFieldTypeInt(-900, 1000)); err != nil { + if _, err := idx.CreateField("edge", "", pilosa.OptFieldTypeInt(-900, 1000)); err != nil { t.Fatal(err) } @@ -3214,13 +3214,13 @@ func TestExecutor_Execute_Row_BSIGroupEdge(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) - idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{}) + idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{}) if err != nil { t.Fatal(err) } t.Run("LT", func(t *testing.T) { - if _, err := idx.CreateField("f1", pilosa.OptFieldTypeInt(-2000, 2000)); err != nil { + if _, err := idx.CreateField("f1", "", pilosa.OptFieldTypeInt(-2000, 2000)); err != nil { t.Fatal(err) } @@ -3241,7 +3241,7 @@ func TestExecutor_Execute_Row_BSIGroupEdge(t *testing.T) { }) t.Run("GT", func(t *testing.T) { - if _, err := idx.CreateField("f2", pilosa.OptFieldTypeInt(-2000, 2000)); err != nil { + if _, err := idx.CreateField("f2", "", pilosa.OptFieldTypeInt(-2000, 2000)); err != nil { t.Fatal(err) } @@ -3262,7 +3262,7 @@ func TestExecutor_Execute_Row_BSIGroupEdge(t *testing.T) { }) t.Run("BTWN_LT_LT", func(t *testing.T) { - if _, err := idx.CreateField("f3", pilosa.OptFieldTypeInt(-2000, 2000)); err != nil { + if _, err := idx.CreateField("f3", "", pilosa.OptFieldTypeInt(-2000, 2000)); err != nil { t.Fatal(err) } @@ -3301,28 +3301,28 @@ func TestExecutor_Execute_Range_BSIGroup_Deprecated(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) - idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{}) + idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{}) if err != nil { t.Fatal(err) } - if _, err := idx.CreateField("f"); err != nil { + if _, err := idx.CreateField("f", ""); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("foo", pilosa.OptFieldTypeInt(-990, 1000)); err != nil { + if _, err := idx.CreateField("foo", "", pilosa.OptFieldTypeInt(-990, 1000)); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("bar", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { + if _, err := idx.CreateField("bar", "", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("other", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { + if _, err := idx.CreateField("other", "", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("edge", pilosa.OptFieldTypeInt(-1100, 1000)); err != nil { + if _, err := idx.CreateField("edge", "", pilosa.OptFieldTypeInt(-1100, 1000)); err != nil { t.Fatal(err) } @@ -3849,7 +3849,7 @@ func TestExecutor_Time_Clear_Quantums(t *testing.T) { indexName := c.Idx(strings.ToLower(string(tt.quantum))) index := hldr.MustCreateIndexIfNotExists(indexName, pilosa.IndexOptions{}) // Create field. - if _, err := index.CreateFieldIfNotExists("f", pilosa.OptFieldTypeTime(tt.quantum, "0")); err != nil { + if _, err := index.CreateFieldIfNotExists("f", "", pilosa.OptFieldTypeTime(tt.quantum, "0")); err != nil { t.Fatal(err) } // Populate @@ -3933,7 +3933,7 @@ func TestExecutor_Execute_Existence(t *testing.T) { hldr := c.GetHolder(0) index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{TrackExistence: true}) - _, err := index.CreateField("f") + _, err := index.CreateField("f", "") if err != nil { t.Fatal(err) } @@ -4335,7 +4335,7 @@ func TestExecutor_Execute_All(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{TrackExistence: true}) - fld, err := index.CreateField("f") + fld, err := index.CreateField("f", "") if err != nil { t.Fatal(err) } @@ -4427,7 +4427,7 @@ func TestExecutor_Execute_All(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{TrackExistence: true, Keys: true}) - fld, err := index.CreateField("f") + fld, err := index.CreateField("f", "") if err != nil { t.Fatal(err) } @@ -4492,7 +4492,7 @@ func TestExecutor_Execute_All(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{TrackExistence: true}) - _, err := index.CreateField("f") + _, err := index.CreateField("f", "") if err != nil { t.Fatal(err) } @@ -4520,7 +4520,7 @@ func TestExecutor_Execute_ClearRow(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{TrackExistence: true}) - _, err := index.CreateField("f", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)) + _, err := index.CreateField("f", "", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)) if err != nil { t.Fatal(err) } @@ -4536,7 +4536,7 @@ func TestExecutor_Execute_ClearRow(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{TrackExistence: true}) - _, err := index.CreateField("f") + _, err := index.CreateField("f", "") if err != nil { t.Fatal(err) } @@ -4616,10 +4616,10 @@ func TestExecutor_Execute_SetRow(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{TrackExistence: true}) - if _, err := index.CreateField("f"); err != nil { + if _, err := index.CreateField("f", ""); err != nil { t.Fatal(err) } - if _, err := index.CreateField("tmp"); err != nil { + if _, err := index.CreateField("tmp", ""); err != nil { t.Fatal(err) } @@ -4671,7 +4671,7 @@ func TestExecutor_Execute_SetRow(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) idx := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{TrackExistence: true}) - _, err := idx.CreateField("f") + _, err := idx.CreateField("f", "") if err != nil { t.Fatal(err) } @@ -4724,7 +4724,7 @@ func TestExecutor_Execute_SetRow(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{TrackExistence: true}) - _, err := index.CreateField("f") + _, err := index.CreateField("f", "") if err != nil { t.Fatal(err) } @@ -4765,7 +4765,7 @@ func TestExecutor_Execute_SetRow(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{TrackExistence: true}) - if _, err := index.CreateField("f", pilosa.OptFieldKeys()); err != nil { + if _, err := index.CreateField("f", "", pilosa.OptFieldKeys()); err != nil { t.Fatal(err) } @@ -4829,7 +4829,7 @@ func benchmarkExistence(nn bool, b *testing.B) { index := hldr.MustCreateIndexIfNotExists(indexName, pilosa.IndexOptions{TrackExistence: nn}) // Create field. - if _, err := index.CreateFieldIfNotExists(fieldName); err != nil { + if _, err := index.CreateFieldIfNotExists(fieldName, ""); err != nil { b.Fatal(err) } @@ -6630,7 +6630,7 @@ func TestExecutor_Execute_IncludesColumn(t *testing.T) { cmd := c.GetNode(0) hldr := c.GetHolder(0) index := hldr.MustCreateIndexIfNotExists(c.Idx(), pilosa.IndexOptions{Keys: true}) - if _, err := index.CreateField("general", pilosa.OptFieldKeys()); err != nil { + if _, err := index.CreateField("general", "", pilosa.OptFieldKeys()); err != nil { t.Fatal(err) } @@ -6696,20 +6696,20 @@ func TestExecutor_Execute_MinMaxCountEqual(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) - idx, err := hldr.CreateIndex(c.Idx(), pilosa.IndexOptions{}) + idx, err := hldr.CreateIndex(c.Idx(), "", pilosa.IndexOptions{}) if err != nil { t.Fatal(err) } - if _, err := idx.CreateField("x"); err != nil { + if _, err := idx.CreateField("x", ""); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("f", pilosa.OptFieldTypeInt(-1100, 1000)); err != nil { + if _, err := idx.CreateField("f", "", pilosa.OptFieldTypeInt(-1100, 1000)); err != nil { t.Fatal(err) } - if _, err := idx.CreateField("dec", pilosa.OptFieldTypeDecimal(3)); err != nil { + if _, err := idx.CreateField("dec", "", pilosa.OptFieldTypeDecimal(3)); err != nil { t.Fatal(err) } @@ -6869,7 +6869,7 @@ func TestExecutor_Execute_NoIndex(t *testing.T) { defer c.Close() hldr := c.GetHolder(0) index := hldr.MustCreateIndexIfNotExists(c.Idx(), *indexOptions) - _, err := index.CreateField("f") + _, err := index.CreateField("f", "") if err != nil { t.Fatal("should work") } @@ -9684,11 +9684,11 @@ func TestExecutorTimeRange(t *testing.T) { } indexName := c.Idx(t.Name()) hldr := c.GetHolder(0) - index, err := hldr.CreateIndex(indexName, pilosa.IndexOptions{}) + index, err := hldr.CreateIndex(indexName, "", pilosa.IndexOptions{}) if err != nil { t.Fatal(err) } - _, err = index.CreateField("f") + _, err = index.CreateField("f", "") if err != nil { t.Fatal(err) } diff --git a/field.go b/field.go index 1037cf877..34e22ae95 100644 --- a/field.go +++ b/field.go @@ -73,6 +73,7 @@ var availableShardFileFlushDuration = &protected{ type Field struct { mu sync.RWMutex createdAt int64 + owner string path string index string name string @@ -1972,6 +1973,7 @@ func (p fieldSlice) Less(i, j int) bool { return p[i].Name() < p[j].Name() } type FieldInfo struct { Name string `json:"name"` CreatedAt int64 `json:"createdAt,omitempty"` + Owner string `json:"owner"` Options FieldOptions `json:"options"` Cardinality *uint64 `json:"cardinality,omitempty"` Views []*ViewInfo `json:"views,omitempty"` diff --git a/field_test.go b/field_test.go index 4e26d4902..e972e483a 100644 --- a/field_test.go +++ b/field_test.go @@ -22,7 +22,7 @@ func TestField_SetValue(t *testing.T) { t.Run("OK", func(t *testing.T) { h, idx := test.MustOpenIndex(t) - f, err := idx.CreateField("f", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)) + f, err := idx.CreateField("f", "", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)) if err != nil { t.Fatal(err) } @@ -63,7 +63,7 @@ func TestField_SetValue(t *testing.T) { t.Run("Overwrite", func(t *testing.T) { h, idx := test.MustOpenIndex(t) - f, err := idx.CreateField("f", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)) + f, err := idx.CreateField("f", "", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)) if err != nil { t.Fatal(err) } @@ -98,7 +98,7 @@ func TestField_SetValue(t *testing.T) { t.Run("ErrBSIGroupNotFound", func(t *testing.T) { h, idx := test.MustOpenIndex(t) - f, err := idx.CreateField("f") + f, err := idx.CreateField("f", "") if err != nil { t.Fatal(err) } @@ -115,7 +115,7 @@ func TestField_SetValue(t *testing.T) { t.Run("ErrBSIGroupValueTooLow", func(t *testing.T) { h, idx := test.MustOpenIndex(t) - f, err := idx.CreateField("f", pilosa.OptFieldTypeInt(20, 30)) + f, err := idx.CreateField("f", "", pilosa.OptFieldTypeInt(20, 30)) if err != nil { t.Fatal(err) } @@ -130,7 +130,7 @@ func TestField_SetValue(t *testing.T) { t.Run("ErrBSIGroupValueTooHigh", func(t *testing.T) { h, idx := test.MustOpenIndex(t) - f, err := idx.CreateField("f", pilosa.OptFieldTypeInt(20, 30)) + f, err := idx.CreateField("f", "", pilosa.OptFieldTypeInt(20, 30)) if err != nil { t.Fatal(err) } @@ -170,13 +170,13 @@ func TestField_NameValidation(t *testing.T) { _, idx := test.MustOpenIndex(t) for _, name := range validFieldNames { - _, err := idx.CreateField(name) + _, err := idx.CreateField(name, "") if err != nil { t.Fatalf("unexpected field name: %s %s", name, err) } } for _, name := range invalidFieldNames { - _, err := idx.CreateField(name) + _, err := idx.CreateField(name, "") if err == nil { t.Fatalf("expected error on field name: %s", name) } @@ -189,7 +189,7 @@ const includeRemote = false // for calls to Index.AvailableShards(localOnly bool func TestField_AvailableShards(t *testing.T) { h, idx := test.MustOpenIndex(t) - f, err := idx.CreateField("fld-shards") + f, err := idx.CreateField("fld-shards", "") if err != nil { t.Fatal(err) } @@ -230,7 +230,7 @@ func TestField_ClearValue(t *testing.T) { t.Run("OK", func(t *testing.T) { h, idx := test.MustOpenIndex(t) - f, err := idx.CreateField("f", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)) + f, err := idx.CreateField("f", "", pilosa.OptFieldTypeInt(math.MinInt64, math.MaxInt64)) if err != nil { t.Fatal(err) } @@ -291,7 +291,7 @@ func TestFieldInfoMarshal(t *testing.T) { if err != nil { t.Fatalf("unexpected error marshalling index info, %v", err) } - expected := []byte(`{"name":"timestamp","createdAt":1649270079233541000,"options":{"type":"timestamp","epoch":"1970-01-01T00:00:00Z","bitDepth":0,"min":-4294967296,"max":4294967296,"timeUnit":"s"}}`) + expected := []byte(`{"name":"timestamp","createdAt":1649270079233541000,"owner":"","options":{"type":"timestamp","epoch":"1970-01-01T00:00:00Z","bitDepth":0,"min":-4294967296,"max":4294967296,"timeUnit":"s"}}`) if !bytes.Equal(a, expected) { t.Fatalf("expected %s, got %s", expected, a) } diff --git a/fragment_internal_test.go b/fragment_internal_test.go index fa053bbb6..e39cfc0dc 100644 --- a/fragment_internal_test.go +++ b/fragment_internal_test.go @@ -1331,7 +1331,7 @@ func TestFragment_TopN_CacheSize(t *testing.T) { index := mustOpenIndex(t, IndexOptions{}) // Create field. - field, err := index.CreateFieldIfNotExists("f", OptFieldTypeSet(CacheTypeRanked, cacheSize)) + field, err := index.CreateFieldIfNotExists("f", "", OptFieldTypeSet(CacheTypeRanked, cacheSize)) if err != nil { t.Fatal(err) } @@ -3087,11 +3087,11 @@ func newTestField(tb testing.TB, fieldOpts ...FieldOption) (*Holder, *Index, *Fi fieldOpts = []FieldOption{OptFieldTypeDefault()} } h := newTestHolder(tb) - idx, err := h.CreateIndex("i", IndexOptions{}) + idx, err := h.CreateIndex("i", "", IndexOptions{}) if err != nil { tb.Fatalf("creating test index: %v", err) } - fld, err := idx.CreateField("f", fieldOpts...) + fld, err := idx.CreateField("f", "", fieldOpts...) if err != nil { tb.Fatalf("creating test field: %v", err) } diff --git a/go.mod b/go.mod index a82844842..dc38dafbb 100644 --- a/go.mod +++ b/go.mod @@ -82,6 +82,7 @@ require ( github.com/jaffee/commandeer v0.5.0 github.com/linkedin/goavro/v2 v2.11.1 google.golang.org/grpc v1.46.0 + google.golang.org/protobuf v1.28.0 ) require ( @@ -166,7 +167,6 @@ require ( golang.org/x/text v0.3.7 // indirect google.golang.org/appengine v1.6.7 // indirect google.golang.org/genproto v0.0.0-20220503193339-ba3ae3f07e29 // indirect - google.golang.org/protobuf v1.28.0 // indirect gopkg.in/ini.v1 v1.62.0 // indirect gopkg.in/natefinch/lumberjack.v2 v2.0.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/holder.go b/holder.go index c9993b081..78009e104 100644 --- a/holder.go +++ b/holder.go @@ -436,14 +436,17 @@ func (h *Holder) Open() error { return errors.Wrap(err, "opening index") } - // Since we don't have createdAt stored on disk within the data + // Since we don't have createdAt and the other metadata stored on disk within the data // directory, we need to populate it from the etcd schema data. + // TODO: we may no longer need the createdAt value stored in memory on // the index struct; it may only be needed in the schema return value // from the API, which already comes from etcd. In that case, this logic // could be removed, and the createdAt on the index struct could be // removed. index.createdAt = cim.CreatedAt + index.owner = cim.Owner + index.description = cim.Meta.Description err = index.OpenWithSchema(idx) if err != nil { @@ -678,10 +681,13 @@ func (h *Holder) schema(ctx context.Context, includeViews bool) ([]*IndexInfo, e di := &IndexInfo{ Name: cim.Index, CreatedAt: cim.CreatedAt, + Owner: cim.Owner, Options: cim.Meta, ShardWidth: ShardWidth, Fields: make([]*FieldInfo, 0, len(index.Fields)), } + updatedAt := cim.CreatedAt + lastUpdateUser := cim.Owner for fieldName, field := range index.Fields { if fieldName == existenceFieldName { continue @@ -690,9 +696,14 @@ func (h *Holder) schema(ctx context.Context, includeViews bool) ([]*IndexInfo, e if err != nil { return nil, errors.Wrap(err, "decoding CreateFieldMessage") } + if cfm.CreatedAt > updatedAt { + updatedAt = cfm.CreatedAt + lastUpdateUser = cfm.Owner + } fi := &FieldInfo{ Name: cfm.Field, CreatedAt: cfm.CreatedAt, + Owner: cfm.Owner, Options: *cfm.Meta, } if includeViews { @@ -703,6 +714,8 @@ func (h *Holder) schema(ctx context.Context, includeViews bool) ([]*IndexInfo, e } di.Fields = append(di.Fields, fi) } + di.UpdatedAt = updatedAt + di.LastUpdateUser = lastUpdateUser sort.Sort(fieldInfoSlice(di.Fields)) a = append(a, di) } @@ -716,14 +729,14 @@ func (h *Holder) applySchema(schema *Schema) error { // We use h.CreateIndex() instead of h.CreateIndexIfNotExists() because we // want to limit the use of this method for now to only new indexes. for _, i := range schema.Indexes { - idx, err := h.CreateIndex(i.Name, i.Options) + idx, err := h.CreateIndex(i.Name, i.Owner, i.Options) if err != nil { return errors.Wrap(err, "creating index") } // Create fields that don't exist. for _, f := range i.Fields { - fld, err := idx.CreateFieldIfNotExistsWithOptions(f.Name, &f.Options) + fld, err := idx.CreateFieldIfNotExistsWithOptions(f.Name, "", &f.Options) if err != nil { return errors.Wrap(err, "creating field") } @@ -775,7 +788,7 @@ func (h *Holder) Indexes() []*Index { // CreateIndex creates an index. // An error is returned if the index already exists. -func (h *Holder) CreateIndex(name string, opt IndexOptions) (*Index, error) { +func (h *Holder) CreateIndex(name string, requestUserID string, opt IndexOptions) (*Index, error) { h.mu.Lock() defer h.mu.Unlock() @@ -784,9 +797,11 @@ func (h *Holder) CreateIndex(name string, opt IndexOptions) (*Index, error) { return nil, newConflictError(ErrIndexExists) } + ts := timestamp() cim := &CreateIndexMessage{ Index: name, - CreatedAt: timestamp(), + CreatedAt: ts, + Owner: requestUserID, Meta: opt, } @@ -874,13 +889,15 @@ func (h *Holder) CreateIndexAndBroadcast(ctx context.Context, cim *CreateIndexMe // CreateIndexIfNotExists returns an index by name. // The index is created if it does not already exist. -func (h *Holder) CreateIndexIfNotExists(name string, opt IndexOptions) (*Index, error) { +func (h *Holder) CreateIndexIfNotExists(name string, requestUserID string, opt IndexOptions) (*Index, error) { h.mu.Lock() defer h.mu.Unlock() + ts := timestamp() cim := &CreateIndexMessage{ Index: name, - CreatedAt: timestamp(), + CreatedAt: ts, + Owner: requestUserID, Meta: opt, } @@ -931,6 +948,8 @@ func (h *Holder) createIndex(cim *CreateIndexMessage, broadcast bool) (*Index, e index.keys = cim.Meta.Keys index.trackExistence = cim.Meta.TrackExistence index.createdAt = cim.CreatedAt + index.owner = cim.Owner + index.description = cim.Meta.Description if err = index.Open(); err != nil { return nil, errors.Wrap(err, "opening") diff --git a/holder_internal_test.go b/holder_internal_test.go index e9faa3a12..314bffe61 100644 --- a/holder_internal_test.go +++ b/holder_internal_test.go @@ -7,11 +7,11 @@ import ( ) func setupTest(t *testing.T, h *Holder, rowCol []rowCols, indexName string) (*Index, *Field) { - idx, err := h.CreateIndexIfNotExists(indexName, IndexOptions{TrackExistence: true}) + idx, err := h.CreateIndexIfNotExists(indexName, "", IndexOptions{TrackExistence: true}) if err != nil { t.Fatalf("failed to create index %v: %v", indexName, err) } - f, err := idx.CreateFieldIfNotExists("f") + f, err := idx.CreateFieldIfNotExists("f", "") if err != nil { t.Fatalf("failed to create field in index %v: %v", indexName, err) } diff --git a/holder_test.go b/holder_test.go index ead88e2c8..0e99958ae 100644 --- a/holder_test.go +++ b/holder_test.go @@ -32,7 +32,7 @@ func TestHolder_Open(t *testing.T) { // no automatic close here, because we manually close this, and then // *fail* to reopen it. - if _, err := h.CreateIndex("test", pilosa.IndexOptions{}); err != nil { + if _, err := h.CreateIndex("test", "", pilosa.IndexOptions{}); err != nil { t.Fatal(err) } else if err := h.Close(); err != nil { t.Fatal(err) @@ -51,10 +51,10 @@ func TestHolder_Open(t *testing.T) { t.Run("ErrForeignIndexNotFound", func(t *testing.T) { h := test.MustOpenHolder(t) - if idx, err := h.CreateIndex("foo", pilosa.IndexOptions{}); err != nil { + if idx, err := h.CreateIndex("foo", "", pilosa.IndexOptions{}); err != nil { t.Fatal(err) } else { - _, err := idx.CreateField("bar", pilosa.OptFieldTypeInt(0, 100), pilosa.OptFieldForeignIndex("nonexistent")) + _, err := idx.CreateField("bar", "", pilosa.OptFieldTypeInt(0, 100), pilosa.OptFieldForeignIndex("nonexistent")) if err == nil { t.Fatalf("expected error: %s", pilosa.ErrForeignIndexNotFound) } else if errors.Cause(err) != pilosa.ErrForeignIndexNotFound { @@ -67,11 +67,11 @@ func TestHolder_Open(t *testing.T) { t.Run("ForeignIndexNotOpenYet", func(t *testing.T) { h := test.MustOpenHolder(t) - if _, err := h.CreateIndex("zzz", pilosa.IndexOptions{}); err != nil { + if _, err := h.CreateIndex("zzz", "", pilosa.IndexOptions{}); err != nil { t.Fatal(err) - } else if idx, err := h.CreateIndex("foo", pilosa.IndexOptions{}); err != nil { + } else if idx, err := h.CreateIndex("foo", "", pilosa.IndexOptions{}); err != nil { t.Fatal(err) - } else if _, err := idx.CreateField("bar", pilosa.OptFieldTypeInt(0, 100), pilosa.OptFieldForeignIndex("zzz")); err != nil { + } else if _, err := idx.CreateField("bar", "", pilosa.OptFieldTypeInt(0, 100), pilosa.OptFieldForeignIndex("zzz")); err != nil { t.Fatal(err) } else if err := h.Holder.Close(); err != nil { t.Fatal(err) @@ -86,11 +86,11 @@ func TestHolder_Open(t *testing.T) { t.Run("ForeignIndexIsOpen", func(t *testing.T) { h := test.MustOpenHolder(t) - if _, err := h.CreateIndex("aaa", pilosa.IndexOptions{}); err != nil { + if _, err := h.CreateIndex("aaa", "", pilosa.IndexOptions{}); err != nil { t.Fatal(err) - } else if idx, err := h.CreateIndex("foo", pilosa.IndexOptions{}); err != nil { + } else if idx, err := h.CreateIndex("foo", "", pilosa.IndexOptions{}); err != nil { t.Fatal(err) - } else if _, err := idx.CreateField("bar", pilosa.OptFieldTypeInt(0, 100), pilosa.OptFieldForeignIndex("aaa")); err != nil { + } else if _, err := idx.CreateField("bar", "", pilosa.OptFieldTypeInt(0, 100), pilosa.OptFieldForeignIndex("aaa")); err != nil { t.Fatal(err) } else if err := h.Holder.Close(); err != nil { t.Fatal(err) @@ -105,18 +105,18 @@ func TestHolder_Open(t *testing.T) { t.Run("CreateIndexIfNotExists", func(t *testing.T) { h := test.MustOpenHolder(t) - idx1, err := h.CreateIndexIfNotExists("aaa", pilosa.IndexOptions{}) + idx1, err := h.CreateIndexIfNotExists("aaa", "", pilosa.IndexOptions{}) if err != nil { t.Fatal(err) } - if _, err = h.CreateIndex("aaa", pilosa.IndexOptions{}); err == nil { + if _, err = h.CreateIndex("aaa", "", pilosa.IndexOptions{}); err == nil { t.Fatalf("expected: ConflictError, got: nil") } else if _, ok := err.(pilosa.ConflictError); !ok { t.Fatalf("expected: ConflictError, got: %s", err) } - idx2, err := h.CreateIndexIfNotExists("aaa", pilosa.IndexOptions{}) + idx2, err := h.CreateIndexIfNotExists("aaa", "", pilosa.IndexOptions{}) if err != nil { t.Fatal(err) } @@ -136,7 +136,7 @@ func TestHolder_HasData(t *testing.T) { t.Fatal("expected HasData to return false, no err, but", ok, err) } - if _, err := h.CreateIndex("test", pilosa.IndexOptions{}); err != nil { + if _, err := h.CreateIndex("test", "", pilosa.IndexOptions{}); err != nil { t.Fatal(err) } diff --git a/http_handler.go b/http_handler.go index 95e85f71e..b5261d9e8 100644 --- a/http_handler.go +++ b/http_handler.go @@ -49,6 +49,18 @@ import ( "github.com/zeebo/blake3" ) +type ContextRequestUserIdKeyType string + +const ( + // ContextRequestUserIdKey is request userid key for a request ctx + ContextRequestUserIdKey = ContextRequestUserIdKeyType("request-user-id") +) + +const ( + // HeaderRequestUserID is request userid header + HeaderRequestUserID = "X-Request-Userid" +) + // Handler represents an HTTP handler. type Handler struct { Handler http.Handler @@ -692,34 +704,52 @@ func (h *Handler) chkAllowedNetworks(r *http.Request) (bool, context.Context) { func (h *Handler) chkAuthN(handler http.HandlerFunc) http.HandlerFunc { return func(w http.ResponseWriter, r *http.Request) { ctx := r.Context() - if h.auth != nil { - // if IP is in allowed networks, then serve the request - allowedNetwork, ctx := h.chkAllowedNetworks(r) - if allowedNetwork { - handler.ServeHTTP(w, r.WithContext(ctx)) - return - } - access, refresh := getTokens(r) - uinfo, err := h.auth.Authenticate(access, refresh) - if err != nil { - http.Error(w, errors.Wrap(err, "authenticating").Error(), http.StatusUnauthorized) - return - } - // just in case it got refreshed - ctx = context.WithValue(ctx, authn.ContextValueAccessToken, "Bearer "+access) - ctx = context.WithValue(ctx, authn.ContextValueRefreshToken, refresh) - h.auth.SetCookie(w, uinfo.Token, uinfo.RefreshToken, uinfo.Expiry) + //if the request is unauthenticated and we have the appropriate header get the userid from the header + requestUserID := r.Header.Get(HeaderRequestUserID) + ctx = context.WithValue(ctx, ContextRequestUserIdKey, requestUserID) + + if h.auth == nil { + handler.ServeHTTP(w, r.WithContext(ctx)) + return + } + + // if IP is in allowed networks, then serve the request + allowedNetwork, ctx := h.chkAllowedNetworks(r) + if allowedNetwork { + handler.ServeHTTP(w, r.WithContext(ctx)) + return } + + access, refresh := getTokens(r) + uinfo, err := h.auth.Authenticate(access, refresh) + if err != nil { + http.Error(w, errors.Wrap(err, "authenticating").Error(), http.StatusUnauthorized) + return + } + // prefer the user id from an authenticated request over one in a header + ctx = context.WithValue(ctx, ContextRequestUserIdKey, uinfo.UserID) + + // just in case it got refreshed + ctx = context.WithValue(ctx, authn.ContextValueAccessToken, "Bearer "+access) + ctx = context.WithValue(ctx, authn.ContextValueRefreshToken, refresh) + h.auth.SetCookie(w, uinfo.Token, uinfo.RefreshToken, uinfo.Expiry) + handler.ServeHTTP(w, r.WithContext(ctx)) } } func (h *Handler) chkAuthZ(handler http.HandlerFunc, perm authz.Permission) http.HandlerFunc { return func(w http.ResponseWriter, r *http.Request) { - // if auth isn't turned on, just serve the request + ctx := r.Context() + + //if the request is unauthenticated and we have the appropriate header get the userid from the header + requestUserID := r.Header.Get(HeaderRequestUserID) + ctx = context.WithValue(ctx, ContextRequestUserIdKey, requestUserID) + + // handle the case when auth is not turned on if h.auth == nil { - handler.ServeHTTP(w, r) + handler.ServeHTTP(w, r.WithContext(ctx)) return } @@ -738,16 +768,22 @@ func (h *Handler) chkAuthZ(handler http.HandlerFunc, perm authz.Permission) http access, refresh := getTokens(r) uinfo, err := h.auth.Authenticate(access, refresh) - if err != nil { http.Error(w, errors.Wrap(err, "authenticating").Error(), http.StatusForbidden) return } + + // prefer the user id from an authenticated request over one in a header + ctx = context.WithValue(ctx, ContextRequestUserIdKey, uinfo.UserID) + + ctx = context.WithValue(ctx, authn.ContextValueAccessToken, "Bearer "+access) + ctx = context.WithValue(ctx, authn.ContextValueRefreshToken, refresh) + // just in case it got refreshed h.auth.SetCookie(w, uinfo.Token, uinfo.RefreshToken, uinfo.Expiry) // put the user's authN/Z info in the context - ctx = context.WithValue(r.Context(), contextKeyGroupMembership, uinfo.Groups) + ctx = context.WithValue(ctx, contextKeyGroupMembership, uinfo.Groups) ctx = context.WithValue(ctx, authn.ContextValueAccessToken, "Bearer "+uinfo.Token) ctx = context.WithValue(ctx, authn.ContextValueRefreshToken, uinfo.RefreshToken) // unlikely h.permissions will be nil, but we'll check to be safe @@ -825,7 +861,6 @@ func (h *Handler) chkAuthZ(handler http.HandlerFunc, perm authz.Permission) http } } handler.ServeHTTP(w, r.WithContext(ctx)) - } } @@ -1700,6 +1735,7 @@ func (p *postIndexRequest) UnmarshalJSON(b []byte) error { // Unmarshal expected values. _p := _postIndexRequest{ Options: IndexOptions{ + Description: "", Keys: false, TrackExistence: true, }, @@ -1785,6 +1821,7 @@ func (h *Handler) handlePostIndex(w http.ResponseWriter, r *http.Request) { // Decode request. req := postIndexRequest{ Options: IndexOptions{ + Description: "", Keys: false, TrackExistence: true, }, diff --git a/http_translator_test.go b/http_translator_test.go index 739bedef9..1b17d4dd4 100644 --- a/http_translator_test.go +++ b/http_translator_test.go @@ -26,7 +26,7 @@ func TestTranslateStore_EntryReader(t *testing.T) { hldr := test.Holder{Holder: primary.Server.Holder()} index := hldr.MustCreateIndexIfNotExists("i", pilosa.IndexOptions{Keys: true}) - _, err := index.CreateField("f") + _, err := index.CreateField("f", "") if err != nil { t.Fatal(err) } diff --git a/idk/ingest.go b/idk/ingest.go index f359f0c09..4d1faf65a 100644 --- a/idk/ingest.go +++ b/idk/ingest.go @@ -1531,6 +1531,19 @@ func (m *Main) batchFromSchema(schema []Field) ([]Recordizer, pilosaclient.Recor if hasMutex { //need to clear the mutex rec.Clears[valIdx] = nil //? maybe } + // else { //TODO(twg) set fields not supported + + // } + default: + rec.Values[valIdx], err = idkField.PilosafyVal(rawRec[i]) + } + return errors.Wrapf(err, "converting field %d:%+v, val:%+v", i, idkField, rawRec[i]) + }) + case BoolField: + recordizers = append(recordizers, func(rawRec []interface{}, rec *pilosaclient.Row) (err error) { + switch rawRec[i].(type) { + case DeleteSentinel: + rec.Values[valIdx] = nil default: rec.Values[valIdx], err = idkField.PilosafyVal(rawRec[i]) } @@ -1602,10 +1615,20 @@ func (m *Main) batchFromSchema(schema []Field) ([]Recordizer, pilosaclient.Recor fields = append(fields, m.index.Field(fld.DestName(), pilosaclient.OptFieldTypeBool())) valIdx := len(fields) - 1 recordizers = append(recordizers, func(rawRec []interface{}, rec *pilosaclient.Row) (err error) { - rec.Values[valIdx], err = idkField.PilosafyVal(rawRec[i]) + switch rawRec[i].(type) { + case DeleteSentinel: + rec.Values[valIdx] = nil + default: + val, err := idkField.PilosafyVal(rawRec[i]) + if err != nil { + return errors.Wrapf(err, "booling '%v' of %[1]T", val) + } + if b, ok := val.(bool); ok { + rec.Values[valIdx] = b + } + } return errors.Wrapf(err, "converting field %d:%+v, val:%+v", i, idkField, rawRec[i]) }) - // TODO, unpacked bools aren't actually supported by importbatch } else { fields = append(fields, boolField, boolFieldExists) fieldIdx := len(fields) - 2 diff --git a/idk/ingest_test.go b/idk/ingest_test.go index ae9eb0005..8a7aa1f79 100644 --- a/idk/ingest_test.go +++ b/idk/ingest_test.go @@ -20,6 +20,7 @@ import ( "github.com/featurebasedb/featurebase/v3/logger" "github.com/golang-jwt/jwt" "github.com/pkg/errors" + "github.com/stretchr/testify/assert" ) func configureTestFlags(main *Main) { @@ -269,6 +270,7 @@ func TestSingleBoolClear(t *testing.T) { ingester.Index = fmt.Sprintf("single_bool_clear%d", rand.Intn(100000)) ingester.BatchSize = 1 ingester.IDField = "id" + ingester.PackBools = "bools" if err := ingester.Run(); err != nil { t.Fatalf("%s: %v", idktest.ErrRunningIngest, err) @@ -300,6 +302,7 @@ func TestSingleBoolClear(t *testing.T) { ingester2.NewSource = func() (Source, error) { return ts2, nil } ingester2.Index = ingester.Index ingester2.IDField = "id" + ingester2.PackBools = "bools" if err := ingester2.Run(); err != nil { t.Fatalf("running ingester2: %v", err) @@ -566,6 +569,7 @@ func TestDelete(t *testing.T) { deleter := NewMain() configureTestFlags(deleter) deleter.Delete = true + deleter.PackBools = "bools" deleter.NewSource = func() (Source, error) { return tsDelete, nil } deleter.Index = indexName deleter.BatchSize = 5 @@ -579,6 +583,7 @@ func TestDelete(t *testing.T) { ingester := NewMain() configureTestFlags(ingester) + ingester.PackBools = "bools" ingester.NewSource = func() (Source, error) { return tsWrite, nil } ingester.PrimaryKeyFields = primaryKeyFields ingester.Index = indexName @@ -874,7 +879,6 @@ func TestBatchFromSchema(t *testing.T) { rawRec: []interface{}{true, uint64(7), false}, rowID: uint64(7), rowVals: []interface{}{true, false}, - err: "field type 'bool' is not currently supported through Batch", }, { name: "mutex field", @@ -1429,6 +1433,7 @@ func TestNilIngest(t *testing.T) { err string batchErr string rdzErrs []string + packBools string } runTest := func(t *testing.T, test testcase, removeIndex bool, server serverInfo, rawRec []interface{}, clearmap map[int]interface{}, values []interface{}) { m := NewMain() @@ -1437,6 +1442,7 @@ func TestNilIngest(t *testing.T) { m.PrimaryKeyFields = test.pkFields m.BatchSize = 2 m.Pprof = "" + m.PackBools = test.packBools m.NewSource = func() (Source, error) { return nil, nil } if server.AuthToken != "" { m.AuthToken = server.AuthToken @@ -1522,12 +1528,13 @@ func TestNilIngest(t *testing.T) { Vals2: []interface{}{nil, nil}, Vals3: []interface{}{nil, nil}, }, { - name: "bools null", - pkFields: []string{"user_id"}, - rawRec1: []interface{}{"1a", true}, // bool and bool-exists - rawRec2: []interface{}{"1a", DELETE_SENTINEL}, - rawRec3: []interface{}{"1a", nil}, - rowID: "1a", + name: "bools null", + packBools: "bools", + pkFields: []string{"user_id"}, + rawRec1: []interface{}{"1a", true}, // bool and bool-exists + rawRec2: []interface{}{"1a", DELETE_SENTINEL}, + rawRec3: []interface{}{"1a", nil}, + rowID: "1a", schema: []Field{ StringField{NameVal: "user_id"}, BoolField{NameVal: "bool_val_1"}, @@ -1587,3 +1594,128 @@ func TestNilIngest(t *testing.T) { } } + +func TestBoolIngest(t *testing.T) { + rand.Seed(time.Now().UTC().UnixNano()) + indexName := fmt.Sprintf("boolingest%d", rand.Intn(100000)) + primaryKeyFields := []string{"user_id"} + + tests := []struct { + src *testSource + expTrue []string + expFalse []string + expNull []string + }{ + { + src: newTestSource( + []Field{ + StringField{NameVal: "user_id"}, + BoolField{NameVal: "bool_val"}, + }, + [][]interface{}{ + {"a1", true}, + }, + ), + expTrue: []string{"a1"}, + expFalse: nil, + expNull: nil, + }, + { + src: newTestSource( + []Field{ + StringField{NameVal: "user_id"}, + BoolField{NameVal: "bool_val"}, + }, + [][]interface{}{ + {"a1", false}, + }, + ), + expTrue: nil, + expFalse: []string{"a1"}, + expNull: nil, + }, + { + src: newTestSource( + []Field{ + StringField{NameVal: "user_id"}, + BoolField{NameVal: "bool_val"}, + }, + [][]interface{}{ + {"a1", nil}, + }, + ), + expTrue: nil, + expFalse: nil, + expNull: []string{"a1"}, + }, + { + src: newTestSource( + []Field{ + StringField{NameVal: "user_id"}, + BoolField{NameVal: "bool_val"}, + }, + [][]interface{}{ + {"a1", DELETE_SENTINEL}, + }, + ), + expTrue: nil, + expFalse: nil, + expNull: []string{"a1"}, + }, + } + + var ing *Main + defer func() { + ing := ing + if err := ing.PilosaClient().DeleteIndexByName(ing.Index); err != nil { + t.Logf("%s for index %s: %v", idktest.ErrDeletingIndex, ing.Index, err) + } + }() + + for i, test := range tests { + t.Run(fmt.Sprintf("test-%d", i), func(t *testing.T) { + ingester := NewMain() + configureTestFlags(ingester) + ingester.PackBools = "" + ingester.NewSource = func() (Source, error) { return test.src, nil } + ingester.PrimaryKeyFields = primaryKeyFields + ingester.Index = indexName + ingester.BatchSize = 1 + ingester.UseShardTransactionalEndpoint = true + + // Set ing so the defer can do cleanup. + if i == 0 { + ing = ingester + } + + if err := ingester.Run(); err != nil { + t.Fatalf("%s: %v", idktest.ErrRunningIngest, err) + } + + client := ingester.PilosaClient() + idx := ingester.index + fld := ingester.index.Field("bool_val") + + // Check true. + { + resp, err := client.Query(fld.Row(true)) + assert.NoError(t, err) + assert.Equal(t, test.expTrue, resp.Result().Row().Keys) + } + + // Check false. + { + resp, err := client.Query(fld.Row(false)) + assert.NoError(t, err) + assert.Equal(t, test.expFalse, resp.Result().Row().Keys) + } + + // Check nil. This is used to test the ingestion of nil and null. + { + resp, err := client.Query(idx.Difference(idx.All(), idx.Union(fld.Row(true), fld.Row(false)))) + assert.NoError(t, err) + assert.Equal(t, test.expNull, resp.Result().Row().Keys) + } + }) + } +} diff --git a/idk/interfaces.go b/idk/interfaces.go index 82b4b6c46..c1454cfa4 100644 --- a/idk/interfaces.go +++ b/idk/interfaces.go @@ -1131,11 +1131,16 @@ func toBool(val interface{}) (bool, error) { } return vt != 0, nil case string: - switch strings.ToLower(vt) { + vt = strings.ToLower(vt) + vt = strings.TrimSpace(vt) + switch vt { case "", "0", "f", "false": return false, nil + case "1", "t", "true": + return true, nil } - return true, nil + return false, errors.Errorf("couldn't convert %v of %[1]T to bool", vt) + default: if vint, err := toInt64(val); err == nil { return vint != 0, nil diff --git a/idk/interfaces_test.go b/idk/interfaces_test.go index 069cc54ae..b7ab73362 100644 --- a/idk/interfaces_test.go +++ b/idk/interfaces_test.go @@ -15,6 +15,29 @@ func TestDecimalFieldPilosafy(t *testing.T) { } +func TestBoolFieldPilosafy(t *testing.T) { + f := BoolField{NameVal: "boolcol"} + validTrue := []interface{}{true, 1, "t", "true", " T ", " True"} + validFalse := []interface{}{false, 0, "f", "false", " F ", " False"} + invalid := []interface{}{"boat", " test "} + + for i, v := range validTrue { + if ret, err := f.PilosafyVal(v); err != nil || ret != true { + t.Errorf("test: %d, got: %v of %[1]T, err: %v", i, ret, err) + } + } + for i, v := range validFalse { + if ret, err := f.PilosafyVal(v); err != nil || ret != false { + t.Errorf("test: %d, got: %v of %[1]T, err: %v", i, ret, err) + } + } + for i, v := range invalid { + if ret, err := f.PilosafyVal(v); err == nil { + t.Errorf("test: %d, got: %v of %[1]T, err: %v", i, ret, err) + } + } +} + func TestPilosafyVal(t *testing.T) { tests := []struct { field Field diff --git a/idk/kafka/cmd_delete_test.go b/idk/kafka/cmd_delete_test.go index 7ca359f34..f2512addf 100644 --- a/idk/kafka/cmd_delete_test.go +++ b/idk/kafka/cmd_delete_test.go @@ -56,6 +56,7 @@ func TestDeleteConsumerCompoundStringKey(t *testing.T) { m.Index = fmt.Sprintf("cmd_del_comp_indexij%s", topic) m.PrimaryKeyFields = []string{"abc", "db", "user_id"} m.Topics = []string{topic} + m.PackBools = "bools" defer func() { // TODO: for some reason (which I didn't dig into), diff --git a/idk/kafka/source_test.go b/idk/kafka/source_test.go index 3f796a259..eaeccca3c 100644 --- a/idk/kafka/source_test.go +++ b/idk/kafka/source_test.go @@ -549,8 +549,6 @@ func TestKafkaSourceTimeout(t *testing.T) { // be, but I think it has something to do with kafka rebalancing // itself when a new client joins. func TestKafkaSourceIntegration(t *testing.T) { - t.Parallel() - if testing.Short() { t.Skip() } diff --git a/idk/testenv/certs/ca.crl b/idk/testenv/certs/ca.crl deleted file mode 100644 index 905918346..000000000 --- a/idk/testenv/certs/ca.crl +++ /dev/null @@ -1,16 +0,0 @@ ------BEGIN X509 CRL----- -MIICfjBoAgEBMA0GCSqGSIb3DQEBCwUAMA0xCzAJBgNVBAMTAmNhFw0yMjA3MjIx -OTUxNDVaGA8yMTIyMDcyMjE5NTE0MlowAKAjMCEwHwYDVR0jBBgwFoAU+NtDPKa1 -UzeelNYEuhD5snOV+aowDQYJKoZIhvcNAQELBQADggIBAIfRwTYSQNDwnFB3YlB/ -PoHhEWKdmyoFnN/Zax9FIa8/0F7K/mWwoCPH5xhdP9oo4MR3XR9R0A+iVX3Vohme -ku6XgL2YxGwYL5pZLNH1N34iRNF8R2TnLHMi8dF37JJcJ/HsuiGRqNnpnELlz1m8 -/sxJVl8w3sZBsCmMsDI/lyzk1du+mKJk3bIjapefyT/cQGd4r8kbDyM4ZRzORiks -g4X6nXZK1jhBo6/7nzLsa9egHUGA4rleBleimCXcnaVCyQgVOXorhHdGPNIJQRZg -JGIzV1FdSp2ecrzZ0ERALtz4VkOyWK9Nfhy6TJBiKvc4gDS7M/meNl4GdIu8eV6W -BdL/VlR1JsUSBejd2vjdd5X0SZoMi6dJzUoU6EhmyRFnZMdLAlRcn7eHQqXW5bqP -ZqnGAbb68tVLZI0vt0QyYlmlaWKFkOMGSS+5Gu04uoYNDufHzMb0Nh5vt5MEEkNE -ZYERuzT/PcrI1dEq6di9iK6dRoeTERMgWcZoKg7Yx5jReks0v+Pae4d+B4dHdB52 -LtmK9bn43/E2gbVhRYNDX7xi6rG9AxdshL3/IcLq11bWiDjPL7sUAcvJKk//KXEc -nA/p8H+ZHsD3E1Lln+/3Yf7nj/cxjbF46RRm0yzLdDuTVa+Ov8OB7DtGhT8wW+Ok -gEJXdXlcnzv51gGa2F35/DFi ------END X509 CRL----- diff --git a/idk/testenv/certs/ca.crt b/idk/testenv/certs/ca.crt deleted file mode 100644 index ab84e8414..000000000 --- a/idk/testenv/certs/ca.crt +++ /dev/null @@ -1,28 +0,0 @@ ------BEGIN CERTIFICATE----- -MIIE3DCCAsSgAwIBAgIBATANBgkqhkiG9w0BAQsFADANMQswCQYDVQQDEwJjYTAg -Fw0yMjA3MjIxOTUxNDVaGA8yMTIyMDcyMjE5NTE0MlowDTELMAkGA1UEAxMCY2Ew -ggIiMA0GCSqGSIb3DQEBAQUAA4ICDwAwggIKAoICAQD3qe+w4WaeFtroEqBpxKJY -/a/IpC12Bg4HAYqkK3FwWhOVA+AQ1w2AA/BnLa3u8eszVsNdsnmo+qWZmI9fiJiR -sylJNbnATuWYus7rNNXWnhot3P6CJjEMs5phC0S9zaizFVzL7GFnpjfGgPrbZoQJ -Blmxb/IuEaaKGjUw0MSs83SuauRTGBjVSX80JYI61Wc/KO8hhmLEMne9WSD+Ow3i -GEt5RGRtN6HLkmCBPHieBsWtqToltEV5AqgOXiskGK16q13osBK6mg17QDqg3g7z -w2XSGQx/HSS8e16SqBBt3Ia43ZL7r8qgNxXA9wQP86RKI55ncR3Q7MGVcIMobYbh -gB3nlgqSIYkTs5+5Ffq3na+Fv7yjhw/4gl9UKb84wpZdgHyL4HFrkwdZ3k/go319 -6n0gLpAsgruBF8MNRdQmyojFJX0NKgT7qW6FabpjZklLfzI8MWHX+2sCmICc8Ny5 -imbdz9sJGJi9vEU+AT6HK0pNHQx+4aQyiq/Md3XFNJmkvt9m9vfE7IcTlV/1Pf7W -7Sj06q55k9lO+nmv6qfu3nO1EL+MIHDvObkqNrKiA4ynuBWa+ZIwjLvc/zs6w6or -l7qGSBZZCHv/e/cPrq/EIxUAvD9yhUUbpJY2FCTGv5U87vxWmqHGnoTviRbkg9u7 -6mEwX84Xspg1bTgMbHs7OQIDAQABo0UwQzAOBgNVHQ8BAf8EBAMCAQYwEgYDVR0T -AQH/BAgwBgEB/wIBADAdBgNVHQ4EFgQU+NtDPKa1UzeelNYEuhD5snOV+aowDQYJ -KoZIhvcNAQELBQADggIBADLv2XnXmYCpHVjqVD4NHmg7iv1nitOPsgcqmIUlJ8ZA -iJKN7o5eZs1k26MQXFjcTtEqjV7+yzMfnH53bvD5oEzpxzK6IzQyhxOmz07h907w -/31GVy1VD9+hfUukRnwcVCabc8attyrSs1rI+4RmMdEBNrUEQkNnwWxwzgBMFaIM -P2uH8qen3EIJFxLMl5JJOYE2FzuXrTpvsA0jsCzxVI8bEHzBVaLiNqdMQedsdKAV -WLp64yW/ACnqadk1YIfb8S4sAx/HsLBsnz5+LHHoCzUcodSVODm/otBxJAB2vNUb -ulh3bSWZVdQ4SykMFLlDDd0l+gbgPCVsB8aKj5r6b4EuM6Bsq8GTG511f03vP96g -U4sUkZHssYxA7T5VzIHUtzSl0xC0JngVPVXAcUqGv0etNwlaL/zDotVC8mtPTF15 -cTwy9upE/MHXrYegMTdi2t5ZvM9vVHWemSJ1J0X0jsVmJtT9hYuzPyFR8qAEpd06 -XZD2upB8c3X+kyWdIV0oh2fk37glUU3JihZTRib24qBZ7TUlkGIM0K3PkUHlZ8+v -ljwouIa5Ghf+rYBA0ABJeYdFw6i9hRglWvBc/FHmpIBl9gLzB7+C9Yg20WXMZCvr -p7H/xSf5Dh8fpCO0yeGdpzfWy/3kpTmxg9/RPWyv8hTaB+4ur6CY3pVsVcshLhS3 ------END CERTIFICATE----- diff --git a/idk/testenv/certs/ca.key b/idk/testenv/certs/ca.key deleted file mode 100644 index d208e1915..000000000 --- a/idk/testenv/certs/ca.key +++ /dev/null @@ -1,51 +0,0 @@ ------BEGIN RSA PRIVATE KEY----- -MIIJKgIBAAKCAgEA96nvsOFmnhba6BKgacSiWP2vyKQtdgYOBwGKpCtxcFoTlQPg -ENcNgAPwZy2t7vHrM1bDXbJ5qPqlmZiPX4iYkbMpSTW5wE7lmLrO6zTV1p4aLdz+ -giYxDLOaYQtEvc2osxVcy+xhZ6Y3xoD622aECQZZsW/yLhGmiho1MNDErPN0rmrk -UxgY1Ul/NCWCOtVnPyjvIYZixDJ3vVkg/jsN4hhLeURkbTehy5JggTx4ngbFrak6 -JbRFeQKoDl4rJBiteqtd6LASupoNe0A6oN4O88Nl0hkMfx0kvHtekqgQbdyGuN2S -+6/KoDcVwPcED/OkSiOeZ3Ed0OzBlXCDKG2G4YAd55YKkiGJE7OfuRX6t52vhb+8 -o4cP+IJfVCm/OMKWXYB8i+Bxa5MHWd5P4KN9fep9IC6QLIK7gRfDDUXUJsqIxSV9 -DSoE+6luhWm6Y2ZJS38yPDFh1/trApiAnPDcuYpm3c/bCRiYvbxFPgE+hytKTR0M -fuGkMoqvzHd1xTSZpL7fZvb3xOyHE5Vf9T3+1u0o9OqueZPZTvp5r+qn7t5ztRC/ -jCBw7zm5KjayogOMp7gVmvmSMIy73P87OsOqK5e6hkgWWQh7/3v3D66vxCMVALw/ -coVFG6SWNhQkxr+VPO78Vpqhxp6E74kW5IPbu+phMF/OF7KYNW04DGx7OzkCAwEA -AQKCAgEA6bBo4f8dhAhO0HJa+NI42j2t82WvG9GExOmYd2YiqP/lwFMixuEU9PlC -iykYGQIei5fPyoaQs3imb3L7vgo25CwoxdKzDhmkHWQPwPJe4B7Y1vPTFt2QpMqH -3g/y8iGvkCOVJzpJgrLdqCmmCwYFijp6wam+2+d9vIwUKpajLgqey8FC5oEzVXX/ -WCYYZwlcXcVzNrAGW9i/EWSbdCgm5nNELA3zsPawbVUWSHvV2E7Nkhq/Kjqa3x85 -ki0bCflbIAe5GcmiMn8QP38QkyBU/YAfgIpwNzU5p7vFNh7tOay/VoqEmYZ7bIft -t+Gc7KxaFjbbzgF3P99WnnrG14vZmuy4w/hCKsqDPNRefBTs6YQnEbfmu6CApuiP -Sog69F9n3oEQFzYbePeUDe9pOv8LpTDDkZL/gPW83Pp3C8LFtYhuXtGBWAvBj9P6 -/ryG5pC9USk1gD6mJLBY+UbvEDsN7TKRLX8+Zx6M8YSoAMFceHNW91GGUd1I95s8 -ZQqCNSJ8fR2jscpB8bs9pUs0pa2/WLG8EVaQ10H6nYn0aC/ySQAm92nG5dVnd9Ze -mXkmSOnWVMfLhw7h8vI9hv3E39EAYpwvCS/c4STNJX3dl5fsBq63UWBkHBW83mXR -4w/AT8jvHNpmLRumSBGULFZI9pAajI0ZFODL8JSySEE1KqvTLc0CggEBAP/VAze7 -jAroP0N3icm1eZY0T2WaFqSOra3PO1Y/9NLi+wsteUnf1jMlvAO6qkHgV59RCjCu -X6fRCdiGGmUPS8ksPBzWshfUK9d+9ENIL3hj0gqNqpqDDqBNyqTICML1FWGLnVT7 -Z5o0cPvqh0w+/Xv9A/67ryz9QpjaN8VsP2P/X7gBIgTZrv/yRy8oBI1kCebVdwfK -NEW6B8+yB67kt9yyRELPhXd67PqRwjTTkJkAvaghOJl9lQ7wm4qRyA2DHEoFuKK9 -XPBWvQzBLtBPsVOPdyk9ENfYXIjoqkz4wyJKER4ltAEU+Eb2Daa1sjMUR94FneF6 -redWZrAFHUeav18CggEBAPfTjRwmd1ROhoETMKdx8A1Q40Edy7FNdeVGPA7Evagn -77hmXbQlBHP1jCvAOisGJ3jG7L6mkpDltdsRUqHFnr3u3xV1KAK0kxMlefYKh6da -3/vDOd7xODed7QsFvmJH3YS0Sbjs1/C5WwGdqRopPzX73aS8Y+hJzhfDUsDV/PA4 -PcHAT+Q86Rbr31XUURqP8+v5KEGkbHB0NBe/4t4sK8YY7KBbeLbB8zQR5pg4keKz -5M3lFIIpzcSsYVbObSTf1iBsMsGgTP26d+266qtmSwabBtD4t6Lye0uugLbAg6C+ -FHk7cv0n3NxeoBFxkN0fFN4r8sKttFlMNgdbdo99RGcCggEBAMdduKQ2A2x8kNCC -113Zjmb/XOBQtFSI29AXLqaIMMInIHwVc++d/87cGSjOzt+HdMEy5j4JpYGh5YpW -9zrKMMUVM8NFjIQFVCUbSj8kgdnP8F9JByph94P1G3ZObIWBeKgLpRAU2yfQtjub -CNTiQafeBL7+hAULWFrFs7CmKsBCwtUiDOC6wyWyKMcW6HVHAHBi6d+oNBQiZPP2 -SQaCNsZGJevqzJfPs92QuUdxl72kVigvu2vh8ccyugPCl6PwCJum2bv41fR1o93M -ri6n7AiyRKpebEHI2lPW/N/+/BI8phpYR26QaOrj1L+V2Mj1hQwAVIjikaM2Mc/6 -LFcgwQECggEBAPBw75dAOsYw6Y6Rb8MFaEGDGPk0T0vWO2wWmG47s0ZSeOIPbxo4 -T/mxYi2Qct8LOYCUf1z5f3nVZHsGc4VAlqLv/sByyj7Vs1wAIDP2Q2a7ZFR/NMZ4 -gnJzZmgrSmtQDjuSL3SbVWbYbtNCg64+FnXpx0DtBoF4Gdw0RTWSZ7Uo28H/M4Pd -FMiIUEaVl5qNz7pCIiyNuifglin0ocesU0v/IpTyKkwmIC+ErpVBTUkGABdDCoK+ -zETkegweewYrpTQPxa7WN9xz7adYkhV5SZE/F33n0ULmv3jN9VNZtFtgD70PUKEM -69ticyp5ZHMklLp5KS54GWIJ7uF0e7L7dkECggEAc2MBDPn0b5kb+7YTy5eayizM -n6OcUrTRpTpVGiRmeYkG25VNwH+P99dwbYV/+Im8TwNmlmfs4i1r0gx04lMDOiPr -eQ3c6LqI/N49QfFWOZLPjh5noZM7Rae/xkHTar7g49LUFGv1RiWNyIylzW931Ysc -/nwVY8ERIhNUp/7OTosIZDdF0TEnQYl2j3SvYd3SMqJpUBGX4ClEADzMil+hiD7w -DkLa9GFrGMAgGkcdp3bl1IUSipyI9zGdoUUMw/xXHdycm0MMbhhNn3M7j82bF6aP -OlirJSWDdOlclNXSpVTaNZ6zfBtaMfnZgUM2LF6U2ZtI0iC5ak/nDPo85Eiltw== ------END RSA PRIVATE KEY----- diff --git a/idk/testenv/certs/localhost.crt b/idk/testenv/certs/localhost.crt deleted file mode 100644 index 6708b2c7e..000000000 --- a/idk/testenv/certs/localhost.crt +++ /dev/null @@ -1,25 +0,0 @@ ------BEGIN CERTIFICATE----- -MIIENjCCAh6gAwIBAgIQLtKA8VN9DP2u3nQt1rcZFzANBgkqhkiG9w0BAQsFADAN -MQswCQYDVQQDEwJjYTAgFw0yMjA3MjIxOTUxNDVaGA8yMTIyMDcyMjE5NTE0Mlow -FDESMBAGA1UEAxMJbG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIB -CgKCAQEAsxpHn/bm5EwiYcKIKY3sKG5sByIJa6gG6MFnrBt2Urd9LlDEnyq6A38i -fhIwTqPUY4dvCXW6SNOqp/ugTCbp78F0GUPvJodK+IsrvvVjmQeXg/9ieN91VqY5 -xaVkkaLOEWrcoTAD23+eO0Q3afG5+AFplFyrGwsCFIgz0iB8DMuixsObCao3EWA+ -YwiFUTFE6gthq2rPPXzFtszlGFm6ZJAJOIfnVTgEjNMSc5KgkF39+1vL6URtybnk -3thhnsgSWQDoWgKiBxPt/IuTnJun5JdHkgKTgfO2PVczsFmMzg2e+LJjI8ds8GEY -T7P9jP5KjtzrZ8WMKw6BeeNzXCmjvQIDAQABo4GIMIGFMA4GA1UdDwEB/wQEAwID -uDAdBgNVHSUEFjAUBggrBgEFBQcDAQYIKwYBBQUHAwIwHQYDVR0OBBYEFB9m9jk7 -qfCfPnHUcWDTf7b40ZH0MB8GA1UdIwQYMBaAFPjbQzymtVM3npTWBLoQ+bJzlfmq -MBQGA1UdEQQNMAuCCWxvY2FsaG9zdDANBgkqhkiG9w0BAQsFAAOCAgEAwwXgkVYu -VBq6Uxm+DORby4LiFmLwKIJFuO1UUlAvqNAmUyy6h3S/azb9qvfNiMmHe0+C9j2M -t4eb1KGbZaNiMr5bATFGkJzQl4bEsKZqLv23D6LDRV8D/DtZzNb6e2xhIx1uQYc8 -J8oVlciTKVr5jI82ewvO4O4mu9MlFACva1Zvzk0CB4UZEq1QXpTAlNPG7e3PVT1u -wNZXJ/8kmE1/7OMfQ9PZBGtiYK3OAiTLc0GHFhNjnCFWvFq+sD70bSJCiy3UjMUq -GhcAVHsyzv+O2nmDX4G/QXn+An1wb6hfKYFsR4nWBXQn1HjaswXdw91O6t1UViTS -RittkL/jS3Pn1wzBl80zn+/oOcxSt/KKA07Wfi4JYk6KHWHMk6lTnvMn+csXiRpq -hvUd4gcr4fTYynaXWMP3Of+H1DyJ0edXHxrHjVRS7f9AOpQF1LaZSkWff9ety5Ir -omCvkmopsOeFxKvrbVvgkAi0r8rwfockSNxZI2tp2QFXu6FDywKtL/P/YBJujcsm -zm5XFrjnri/R+c9JQNbuctZtb3Y1WAXnz2RWeunNxxjyaYvEiWh6vXgu0QCXUbev -4wUOK+gFgyczHtbprph9a5iUjEA3P+wzp5u7zGpax07BXMEFSul0HEklc1YP3/ac -35SOBQlvo1oX0VJE0CZB8zlT7upPTuTjus0= ------END CERTIFICATE----- diff --git a/idk/testenv/certs/localhost.csr b/idk/testenv/certs/localhost.csr deleted file mode 100644 index 86fd59537..000000000 --- a/idk/testenv/certs/localhost.csr +++ /dev/null @@ -1,16 +0,0 @@ ------BEGIN CERTIFICATE REQUEST----- -MIICgDCCAWgCAQAwFDESMBAGA1UEAxMJbG9jYWxob3N0MIIBIjANBgkqhkiG9w0B -AQEFAAOCAQ8AMIIBCgKCAQEAsxpHn/bm5EwiYcKIKY3sKG5sByIJa6gG6MFnrBt2 -Urd9LlDEnyq6A38ifhIwTqPUY4dvCXW6SNOqp/ugTCbp78F0GUPvJodK+IsrvvVj -mQeXg/9ieN91VqY5xaVkkaLOEWrcoTAD23+eO0Q3afG5+AFplFyrGwsCFIgz0iB8 -DMuixsObCao3EWA+YwiFUTFE6gthq2rPPXzFtszlGFm6ZJAJOIfnVTgEjNMSc5Kg -kF39+1vL6URtybnk3thhnsgSWQDoWgKiBxPt/IuTnJun5JdHkgKTgfO2PVczsFmM -zg2e+LJjI8ds8GEYT7P9jP5KjtzrZ8WMKw6BeeNzXCmjvQIDAQABoCcwJQYJKoZI -hvcNAQkOMRgwFjAUBgNVHREEDTALgglsb2NhbGhvc3QwDQYJKoZIhvcNAQELBQAD -ggEBAItPe0ClBPlTpkfmbTE0JWsai6Nh12bakigQePI58HYbeV2g0Ld9vpUNGE+i -esTnvDKSYEHKyscNRsehXU8XArKEj6MzUgicPxhqr0eow9O2kGanUz5/M4Y8w0Oa -Q2wUCpqS08jCT0bwdCtFY/y19DXJmwovkGa5+yB+UV5FKkA31l6X7c0vzrdQexAc -FOQjG/9TpvSWXkvkZOv9041f9xeg2g23lfx5gnDvef0hgKhbOsN9tr6tVfJ4zGDv -qC+PuKMTLTfJWKgc2wNLBevA3lYFoHchD+hZMGNktsMraS2Re0FC6MCqH9UMlSGd -T4FN0g5TdI1o5/Hcxjmeac+T444= ------END CERTIFICATE REQUEST----- diff --git a/idk/testenv/certs/localhost.key b/idk/testenv/certs/localhost.key deleted file mode 100644 index 4db92c33f..000000000 --- a/idk/testenv/certs/localhost.key +++ /dev/null @@ -1,27 +0,0 @@ ------BEGIN RSA PRIVATE KEY----- -MIIEowIBAAKCAQEAsxpHn/bm5EwiYcKIKY3sKG5sByIJa6gG6MFnrBt2Urd9LlDE -nyq6A38ifhIwTqPUY4dvCXW6SNOqp/ugTCbp78F0GUPvJodK+IsrvvVjmQeXg/9i -eN91VqY5xaVkkaLOEWrcoTAD23+eO0Q3afG5+AFplFyrGwsCFIgz0iB8DMuixsOb -Cao3EWA+YwiFUTFE6gthq2rPPXzFtszlGFm6ZJAJOIfnVTgEjNMSc5KgkF39+1vL -6URtybnk3thhnsgSWQDoWgKiBxPt/IuTnJun5JdHkgKTgfO2PVczsFmMzg2e+LJj -I8ds8GEYT7P9jP5KjtzrZ8WMKw6BeeNzXCmjvQIDAQABAoIBAFuT5meusWSuYS4n -b8P/o28Q9v+2saZwZggBTGc+K4Qj+kgCWGciS7aZ/SMFXdn3/xNdHrNxlOzD/hCi -otYNV7SV8C0zBUdoCjeHwbiEdIa80QZXQRyY9cB2gjfDV+b3nfJd964bhP1pBZk8 -SjFLvlFHN5r6vyq2uDuTYRviLEvFwMqSf0avKTC5ZgNY5GqmL5tdHiFQHhKaCuTd -BdI1RJ4Hsmjo1GfOY2vQy1mzjeraHaDhmIN9v9mmyDr/aCVaKk7QVHvAiIG/E+T/ -mQImA+1z/G6+BFxvdYu2e42N3jZvl2YELTcb6FnwIswG/1rQG2Cm6OPe2AeW1kBY -2ffdrqkCgYEA28tl3bhVUsA70+A3QS7xWuWuYuwIP1GnqWCP881G+uowC1tqVr2X -wrOEB+gF2v4JZybi6/yIE78LPf7z1pm9Oi1ZT8y/D1T+MjLux0CCG5tGWX26flPm -hFhCg2XvSpiTRnbwsbIoa3czjRxgvI1zpkLNkfV366ILOfiEGlvC+pcCgYEA0Jru -MESdulirmWWJMmIQxsGqMEdln2idh4r3csYHzUfEuQ7DSUunE9uB3ZcnsEzTbpPR -OP4+VdwfSinFA4EXIZ/kgOJMmdgWRWPMBpMkbRNOMigvy8K/k97JH8v0fV3wMa+U -5qK0O4EWthZjaJZtD2WyKoiwZDXXrJj8QsPeQssCgYEAsP529nglRXEF/JW3CSer -Nj51ErXt4kd4E15uLa8ltJL3w32HAXjFiAWVkeRXKsWdftKCs/R1BCm4/OCdLTg6 -eGniXO+M/+S6mmQHgq7A10hP+2rSzj30CKkmTre8Xb7GAl1vcDj+caPZAI5UnCHw -enebISmhuz+PeMxWXP994NsCgYBM/0K3sjv90kjUf/PaEvRaSJWG6HOMGVM7+oFH -beznSnxESjyuyyvKWpVeszJFErZf9FOfk8hybKNn/m7n0jg1gw0WvXDEwEvIJQPj -0V/6msucG9U4gpOwAuV9xQxjUc8cp9BwKNIZvJYjc7QRoKR0sHbI2FczReRrorRJ -+H4LbQKBgDprPI55tyt15PivkyGJYjeHMr6bEk7bqHMDeloPwXsbZr4Esp/4PcYB -Js2BwC+PvXUG/37uk/8B9EdC/l0mbSWos0lhIkaORErnjwWOi1NBh8ZInbetUNIi -A+TeEEeSdnK7ou4s030krS/pG0ZsTUYVEMRFwHiIvoD6/G2wsJhi ------END RSA PRIVATE KEY----- diff --git a/idk/testenv/certs/pilosa-tls.crt b/idk/testenv/certs/pilosa-tls.crt deleted file mode 100644 index 9b1de2b33..000000000 --- a/idk/testenv/certs/pilosa-tls.crt +++ /dev/null @@ -1,25 +0,0 @@ ------BEGIN CERTIFICATE----- -MIIEODCCAiCgAwIBAgIQbk/XbQqd/JZWsL2MQ2BmpjANBgkqhkiG9w0BAQsFADAN -MQswCQYDVQQDEwJjYTAgFw0yMjA3MjIxOTUxNDVaGA8yMTIyMDcyMjE5NTE0Mlow -FTETMBEGA1UEAxMKcGlsb3NhLXRsczCCASIwDQYJKoZIhvcNAQEBBQADggEPADCC -AQoCggEBAN++mmirorKBY7HNe+obmh/1od/5INgjIMobQ6BILGzKeWIH0BpEVR9w -dTUtGM2z/kct8Oi6RibOl99Lj0d/nseKn9kjqMXdJEnoyooQBI++QrWwyYHZ76ag -9H1gmpCqUQILibFGUy9WLRuJbbLAGYfFsb2aiISEIFwbhFg2uFFD8W7VGzOHCXfA -S+oiYSFP3E9pddN4bkurmeuiL+lXxFBTL+RhssdBNreiS4pJ21vbPJw0xpwU7RVB -2aSNVqfdVzXIMwnjPX9zqeGUcvJAcI+9kcMLwGmynjaZ1idVPbpmShhAHJqR/R/1 -r20i32UE6l5SHBWeW+n/2Wb5ZXHT6lkCAwEAAaOBiTCBhjAOBgNVHQ8BAf8EBAMC -A7gwHQYDVR0lBBYwFAYIKwYBBQUHAwEGCCsGAQUFBwMCMB0GA1UdDgQWBBSpCzKw -Yt9vI7BsLAy43R9fGNLbjjAfBgNVHSMEGDAWgBT420M8prVTN56U1gS6EPmyc5X5 -qjAVBgNVHREEDjAMggpwaWxvc2EtdGxzMA0GCSqGSIb3DQEBCwUAA4ICAQBPLSKl -LU3BenjGhIoPfz43odk+XWtcBWx2xxI0+RKtYMXA689lk9UO8ghtPfbhvovVzORF -Sveh1GC6n6kIwmA6v8svQewqWiKlcOglQttRuCdKPLF3y0OnyxW4ki4/nHOmKWvo -lUS+iB37HNMMkvAp9t19k3d18NVDWPpa1R8tukWYdsbeze/lznXO4lHL5AoRZqJP -h5ihCdW07spDX4N0cx09vfqCBGEMClYvnBZzIjiDdtEXjh9Ve81+kxqak6lX0nL0 -p4ilqnWbz4zokTusJZCSfPslcq7V95+3EiBZjZ/NV9ZNjR9HCjfPCe5uQqYK5LJh -1/RG8gBUfcDLvm36c2cQI0ZrG9RMXKjBPlSv3OPd4vRlEMVHVtTyxP3FAMnhlYqp -kVmgoo0ambTxoYhdS3vKV/47KNnmWiJoPNZUZO+40zRfcCuBfuPQGlQgr3pctGZy -NQVp0eNz9xOH+UDrHhPPyDG0F+XcwgflBaIWSoyh4skESlzGqTBcsKGOkVCyrapq -a5jS1z8kdbunnwit8Di0OxXHxbjoIUAEpyP9fakGRcUD4LRZOUkHiR2Dci4hpzb3 -6Xd4BtL0Va9uj//vcRQ+5nnO1PshOde6sQ7Kup2B4OmwyDPcm8Af3nViYQOt1hCc -5vO61aRQN6YmLO5i4CD+kjaurBn8ddTzbGJaaA== ------END CERTIFICATE----- diff --git a/idk/testenv/certs/pilosa-tls.csr b/idk/testenv/certs/pilosa-tls.csr deleted file mode 100644 index 1392027c8..000000000 --- a/idk/testenv/certs/pilosa-tls.csr +++ /dev/null @@ -1,16 +0,0 @@ ------BEGIN CERTIFICATE REQUEST----- -MIICgjCCAWoCAQAwFTETMBEGA1UEAxMKcGlsb3NhLXRsczCCASIwDQYJKoZIhvcN -AQEBBQADggEPADCCAQoCggEBAN++mmirorKBY7HNe+obmh/1od/5INgjIMobQ6BI -LGzKeWIH0BpEVR9wdTUtGM2z/kct8Oi6RibOl99Lj0d/nseKn9kjqMXdJEnoyooQ -BI++QrWwyYHZ76ag9H1gmpCqUQILibFGUy9WLRuJbbLAGYfFsb2aiISEIFwbhFg2 -uFFD8W7VGzOHCXfAS+oiYSFP3E9pddN4bkurmeuiL+lXxFBTL+RhssdBNreiS4pJ -21vbPJw0xpwU7RVB2aSNVqfdVzXIMwnjPX9zqeGUcvJAcI+9kcMLwGmynjaZ1idV -PbpmShhAHJqR/R/1r20i32UE6l5SHBWeW+n/2Wb5ZXHT6lkCAwEAAaAoMCYGCSqG -SIb3DQEJDjEZMBcwFQYDVR0RBA4wDIIKcGlsb3NhLXRsczANBgkqhkiG9w0BAQsF -AAOCAQEAauULC7xyYwTWpeM7b4I3LXMHAR3Q9/kQqyvBYkMUacKZRftzGZ2TwTwZ -6PXa3TI6zFQSL7Nb+Y7ybFhKWZ6jy8TQ8iXXLERxlk//Mg7+0EKNFIACfF7+iiuB -kOyfRSIYYQ3YhdoDUHAxe44eStPlvbMK+DR1SI96gC75Q3fxN73Tj45VCvvglSYt -tx3xPx6kQjseXodjzElYPEnjTSBADY0Ep8QPCZH1Q0fCWfh4xm/nScxNvvdlhc2q -uBV5mmciUrIbJycIcYKnlcUiWz0JkEPjEVu4LORlReFrKsVlTt7JYAuFAhOHxxP9 -eroL8R2Fc5vQ/c/hM97oU2xdOrulTQ== ------END CERTIFICATE REQUEST----- diff --git a/idk/testenv/certs/pilosa-tls.key b/idk/testenv/certs/pilosa-tls.key deleted file mode 100644 index aa872e19f..000000000 --- a/idk/testenv/certs/pilosa-tls.key +++ /dev/null @@ -1,27 +0,0 @@ ------BEGIN RSA PRIVATE KEY----- -MIIEpAIBAAKCAQEA376aaKuisoFjsc176huaH/Wh3/kg2CMgyhtDoEgsbMp5YgfQ -GkRVH3B1NS0YzbP+Ry3w6LpGJs6X30uPR3+ex4qf2SOoxd0kSejKihAEj75CtbDJ -gdnvpqD0fWCakKpRAguJsUZTL1YtG4ltssAZh8WxvZqIhIQgXBuEWDa4UUPxbtUb -M4cJd8BL6iJhIU/cT2l103huS6uZ66Iv6VfEUFMv5GGyx0E2t6JLiknbW9s8nDTG -nBTtFUHZpI1Wp91XNcgzCeM9f3Op4ZRy8kBwj72RwwvAabKeNpnWJ1U9umZKGEAc -mpH9H/WvbSLfZQTqXlIcFZ5b6f/ZZvllcdPqWQIDAQABAoIBADNQ7OsqymL0iCAE -h/IWqI/B74GGCwFndSLFhAoj44SEH7jbH+CWYYuXaFps4G51ZNGAeOt3pZVK9sww -8UitvYX4hlbv9cqDwMnuyV1G9TTep0AyrtTIXk2yRsDmwiyB05iLeuYcwgFuW2Qg -bfd8VZ6tOenJopv6Dc3yRbVRv45rgAfcwRxhDa8xaSVf1jAlXT0+8iPQFX0OHsv6 -iWXk1c48BFEL4nQEtxcWquHqX676raDDizpeVUrnLhRzx0nYq8hiEdtm+O0VY77i -v4i9/vBuWEVVA3OkeufkkkiGuNYZyiDd1a8CZA07aJAMoSpK8pHBjT4N0j5z/lY9 -+AvGV3ECgYEA6V2qkvwjyn+aQgU/Ma2q+mOWa/GdA13vsCFcRfh57UYL95QhVezu -HPk8KskzA6W2CwhG6nhUCfHLlYKqGvozRiaNsnp5dqPMOuLdNlo5ehziIp2FhSM+ -2OmIdAHNzlWp0WaLBbNQQVddmQY1jJPmJitBmQuAs0k6g6uSaD++lfUCgYEA9XIL -d5dkKiK5w4pvAQ7tdszF2YRZvCY77J2cZKmNRO0OI10bU+m321xes+VuFap9VpRn -MTadYeZT9w3gyPX2rjU6bhKx1NKn8oqWOPmOsR+Rhc3OtEFt2fiMtIsNtH5Bd18f -VuSqri6NvGIJJjZ496gV7YtDr1VvxboaSpunoFUCgYEAgPdMnvJM7UnbfATeSZwK -U1nZ9UmPVh8BqTqmfdy1tRTy5B2u1oebh9ONFrAeSzO3CR3H4yggeWZw9e1CCKqg -z3Ha+PmKSrB220woqDLwdLWlmy+mbE9wGfvldwbQL+lTce/TgJD6F0Iq7bhqu4e4 -tvtMw0GCHJKpjb6xqH8h2kkCgYAJ+xY+Sbi4Aet96R40c8/yOcd93eTsQr3DFHGs -ojxtZhpiE8Itul5QnEP5msXMOb+ZCrXbiXYC6iPV8wmGAoKCeQWkPjxfBCXyNfJ/ -5J4J09fOEh1qtRJrf7DxEop/IXc2DHcwyGGsbZYz5+SpWiO3Jm2l2DjrY1UFPFkf -WY8KdQKBgQCtsRcM9Yhpid/wljem283Ea418nrIAY7zw4/JY4XRbaqxOW/YptQq3 -YHQP/jQb1eFeszLwNAorczdXbms/bie2u1+S+vG13z4IgmMtUVXMn8OMN9XYRGXk -UZOtUsJtD8jEEivLoeUN2vdq4oQXWimjVPlh7KS4Qu/W2r0CO2VPBg== ------END RSA PRIVATE KEY----- diff --git a/idk/testenv/certs/theclient.crt b/idk/testenv/certs/theclient.crt deleted file mode 100644 index c0387738f..000000000 --- a/idk/testenv/certs/theclient.crt +++ /dev/null @@ -1,25 +0,0 @@ ------BEGIN CERTIFICATE----- -MIIENjCCAh6gAwIBAgIQSrfnXrPnUTXzmeGaoP31NjANBgkqhkiG9w0BAQsFADAN -MQswCQYDVQQDEwJjYTAgFw0yMjA3MjIxOTUxNDVaGA8yMTIyMDcyMjE5NTE0Mlow -FDESMBAGA1UEAxMJdGhlY2xpZW50MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIB -CgKCAQEAudP6EFxKU3gcyd3NYIS7n5drg8Dwg6FyngyHsQM8l+EROAXep45iIJvu -QNiorUo+dUg5YlxRc1ElqpLi7oSrTZtRZ6EO7pyR44RJtZ9tJ6QjLSh9QaGFq0/2 -GbWQKJJlPT4ftftTdk8ioe3TTDlSmZEGBY1+6Z3c8xxk98VoYJkB86KsLtC9b8q9 -m4D3LWI4+J4echhM6x9YlHiq8HXsPHyBg51Kla3Q3jl1aDHfXuuvEiOVqxJEVx8e -11GPhYmFqQ561id9/S6uBc+u4DnNJUZ4xdA/uEo6PP4XgKBviAtaZvQGGBgtoKAr -tGBFF1QCDkq0QXvv65aArZnXyo+0ewIDAQABo4GIMIGFMA4GA1UdDwEB/wQEAwID -uDAdBgNVHSUEFjAUBggrBgEFBQcDAQYIKwYBBQUHAwIwHQYDVR0OBBYEFN7YKblI -oBtbbLrJaa4XNWrnykw3MB8GA1UdIwQYMBaAFPjbQzymtVM3npTWBLoQ+bJzlfmq -MBQGA1UdEQQNMAuCCXRoZWNsaWVudDANBgkqhkiG9w0BAQsFAAOCAgEA0usMDZ0b -Ktazcxfd7yDgmuIXy06R6fyt/wJjRdPo8oi96a0b9Zj0+BTP2HjkM8/lPO5F8HkX -Ct5tBvuVtwHiTK5fHVsAsSH9H/G9JbX+q7EF3JWx1DDYgXyhU25mcegEkyRZ+hUe -2T3nItwUYjntVoaw4zBS8awTKc78eDZQ3UV3GlGvY3nejYdhpMkGg9dJ4y4M/HGC -P1VuzEpvT7il0cCGMDpuXcdMKNvljmcZYUEC0LCAADTpgfBPGP6lZaB0kU0mpP+e -Lhl6gwZm/toT3u+MGBECUrzHhNvav9LHHBFkquhVopNEEfpBHPtMD8pYm8ULDd5w -2cy53B9m+p3a6jjB85SL9kYudrjMOCbKbABiLxUABfH+5cV9lITNm9GdwuqqtBqL -H4lBAe8wVTf+u4v14diIJXSIjw+mSf7+4htSTTB8BdBmNShmwlCfJA/PXmJqRkVl -I8KE0OTFxHM0rndawXJjFCeNpbHYlTmVW1jtgu3Oz3tqKNwZgHd8kFCwCEkJtno5 -OkKamF1AAzAQELOqzIksIxRPde7egqQ3BfsHkOBbAdybBnUKGRhKf5/rA9SZFfe4 -ncujFadlq+sUk60TTCZaCyMIxDgbb1vQk+7pf3RoB2BDLsfNCqo8rdAhcrB9/8sQ -04HBTyQF+FSJH4bStq5Zm+jMDcb9tx3EAVs= ------END CERTIFICATE----- diff --git a/idk/testenv/certs/theclient.csr b/idk/testenv/certs/theclient.csr deleted file mode 100644 index 66d84c022..000000000 --- a/idk/testenv/certs/theclient.csr +++ /dev/null @@ -1,16 +0,0 @@ ------BEGIN CERTIFICATE REQUEST----- -MIICgDCCAWgCAQAwFDESMBAGA1UEAxMJdGhlY2xpZW50MIIBIjANBgkqhkiG9w0B -AQEFAAOCAQ8AMIIBCgKCAQEAudP6EFxKU3gcyd3NYIS7n5drg8Dwg6FyngyHsQM8 -l+EROAXep45iIJvuQNiorUo+dUg5YlxRc1ElqpLi7oSrTZtRZ6EO7pyR44RJtZ9t -J6QjLSh9QaGFq0/2GbWQKJJlPT4ftftTdk8ioe3TTDlSmZEGBY1+6Z3c8xxk98Vo -YJkB86KsLtC9b8q9m4D3LWI4+J4echhM6x9YlHiq8HXsPHyBg51Kla3Q3jl1aDHf -XuuvEiOVqxJEVx8e11GPhYmFqQ561id9/S6uBc+u4DnNJUZ4xdA/uEo6PP4XgKBv -iAtaZvQGGBgtoKArtGBFF1QCDkq0QXvv65aArZnXyo+0ewIDAQABoCcwJQYJKoZI -hvcNAQkOMRgwFjAUBgNVHREEDTALggl0aGVjbGllbnQwDQYJKoZIhvcNAQELBQAD -ggEBAAqCJIhdEa+y7/KwjLWdHeG4+kE0ergbmIEab4rswrUxCNq/8wedAASXGJ/O -Fj3WgxnpNnIV8MazGDbiRke498EiTiUnGQ20t0o8NXYxYxkk9SWW+DzOTojRBRjx -0HLFCImymdu8B7S/pBdzLO3RcCpa6iRdwGd/9TJgxhKf0NBC/fyZJ+fHhYGZNdt9 -U4g60/hJAD1XGBC3Ry0p8DfTpC3beb6RP0cJNT8YX4qMjq27nuQuaRKcdIV4jZa1 -rms2VspHpOPt+w+XoGFul1lfUut/nFlu5X2aTgnFNHTPZ7eDBVTBCdX1puvTvslg -gzN8dqH97Qyg3lOLYtHUii9dm5M= ------END CERTIFICATE REQUEST----- diff --git a/idk/testenv/certs/theclient.key b/idk/testenv/certs/theclient.key deleted file mode 100644 index 4fa7c43c6..000000000 --- a/idk/testenv/certs/theclient.key +++ /dev/null @@ -1,27 +0,0 @@ ------BEGIN RSA PRIVATE KEY----- -MIIEogIBAAKCAQEAudP6EFxKU3gcyd3NYIS7n5drg8Dwg6FyngyHsQM8l+EROAXe -p45iIJvuQNiorUo+dUg5YlxRc1ElqpLi7oSrTZtRZ6EO7pyR44RJtZ9tJ6QjLSh9 -QaGFq0/2GbWQKJJlPT4ftftTdk8ioe3TTDlSmZEGBY1+6Z3c8xxk98VoYJkB86Ks -LtC9b8q9m4D3LWI4+J4echhM6x9YlHiq8HXsPHyBg51Kla3Q3jl1aDHfXuuvEiOV -qxJEVx8e11GPhYmFqQ561id9/S6uBc+u4DnNJUZ4xdA/uEo6PP4XgKBviAtaZvQG -GBgtoKArtGBFF1QCDkq0QXvv65aArZnXyo+0ewIDAQABAoIBAGXNQh46YSGPGL5i -O4QX0y2f52+bgGesR71cAe4wZr6RtzwowwyqDQnuQrzVtLqShNr9gSpohtqU5vym -UZn3plzyrMsHycVgku/AZh0QxNzFp9uyZcRRt4fdW2ZEOEBxyvKdGQQ6NkQHIGLz -hiEo+h5U+6aJGLDlIckxkgLG6+KWoSUH1sm+Xl9CiP4st/m7gHiqBHz6dQTDVbmC -2uJJut3rlMjK2dzTxB+5mGvzknJfaqHg8ImXSv9kUpYBqYiDNQv2qNB2AolAKKt6 -52Y8+T9iGrX78wrCfcal2cbhWe3lwnaIJ3ueeyt+5hPs8yvtq7vrrhG6d8KjiCZZ -oHPYVcECgYEA3D7jJ33oeX9/5iwBzMXAER6LFZlPDFN6LfwuvdrrO4UD4zw2NB9f -/yHXgpy1sgcSpweqEolTyQSUjqD5imJnIOujfYkrqmL6g1UlfKNwT/DtnCGAnnr2 -fwq8RB/5omuuWaQn3a61RG7lXWTE2akFd0+u7bH1qYqADmMJLOu22JsCgYEA1/69 -VnYETuETr8sCZe7Dj031w3Rc8VZWBgDO3G07tOPyadcAvAJWy/TxPQSDcKky2E0h -yO8Tw2mIxhYvLkyieok9hV4pckdaAgka8utOY0EzWTyJ64hQGRvbN6G+Tdqo2k/o -yTdhtRfYnRsZZcNdlQ0A4jkvzVAO8mU/lTSIoaECgYBDM9kh6rQtpm7DL7hY4BoL -3zL3IzFQhIQqVeF5Qrw2DA80neG4I76p3YyhqOlz6d6SIq+NOmPbWU9u+cVKqV3o -WJ4pJtqxAcSIoElw73nMmqoV0s8pDEH8X600aEFGgu7PIVycsrtSpzfLOH0sIPuu -/RxWIM/hrIYIXcWl9EJ1cwKBgGl9uOtpaGE9/Gd3NwyhuFFdOhLj2iDkkc8MxftS -Pj7ytYqLv4gTfGDQmKG8epWjjXmm8kE4wuZFvM9CNnjeZNyD+tAQVR3DS2qDF+5c -jkRbq8CBmLq4MKaSvFZHYLKVgLJfJMphig7wPpM8kQLqj+IkH7JdS8NyvloUqSEu -Ly0BAoGATwLAQ5bZ1zT+ul4zxR3vkIS7YgTcEHa5OzszKVQ6iDArRDa7NWC8R06j -1R2Ur+T3UVINVFsChDsbebb6AUoSCzM48ZDvnX6C12T0vx+wLziw29hN84gAa+1f -kgFJZ+uena+ZZc8MLyOcM6E1nW0BpAXE0yVF4Vn2Df/wNzm9cGY= ------END RSA PRIVATE KEY----- diff --git a/index.go b/index.go index 557983ec0..be377d6c5 100644 --- a/index.go +++ b/index.go @@ -24,8 +24,11 @@ import ( // Index represents a container for fields. type Index struct { - mu sync.RWMutex - createdAt int64 + mu sync.RWMutex + createdAt int64 + owner string + description string + path string name string qualifiedName string @@ -148,6 +151,7 @@ func (i *Index) Options() IndexOptions { func (i *Index) options() IndexOptions { return IndexOptions{ + Description: i.description, Keys: i.keys, TrackExistence: i.trackExistence, } @@ -348,6 +352,7 @@ func (i *Index) openExistenceField() error { cfm := &CreateFieldMessage{ Index: i.name, Field: existenceFieldName, + Owner: "", CreatedAt: 0, Meta: &FieldOptions{Type: FieldTypeSet, CacheType: CacheTypeNone, CacheSize: 0}, } @@ -524,7 +529,7 @@ func (i *Index) recalculateCaches() { } // CreateField creates a field. -func (i *Index) CreateField(name string, opts ...FieldOption) (*Field, error) { +func (i *Index) CreateField(name string, requestUserID string, opts ...FieldOption) (*Field, error) { err := ValidateName(name) if err != nil { return nil, errors.Wrap(err, "validating name") @@ -553,10 +558,12 @@ func (i *Index) CreateField(name string, opts ...FieldOption) (*Field, error) { return nil, errors.Wrap(err, "applying option") } + ts := timestamp() cfm := &CreateFieldMessage{ Index: i.name, Field: name, - CreatedAt: timestamp(), + CreatedAt: ts, + Owner: requestUserID, Meta: fo, } @@ -585,7 +592,7 @@ func (i *Index) CreateField(name string, opts ...FieldOption) (*Field, error) { } // CreateFieldIfNotExists creates a field with the given options if it doesn't exist. -func (i *Index) CreateFieldIfNotExists(name string, opts ...FieldOption) (*Field, error) { +func (i *Index) CreateFieldIfNotExists(name string, requestUserID string, opts ...FieldOption) (*Field, error) { err := ValidateName(name) if err != nil { return nil, errors.Wrap(err, "validating name") @@ -605,10 +612,12 @@ func (i *Index) CreateFieldIfNotExists(name string, opts ...FieldOption) (*Field return nil, errors.Wrap(err, "applying option") } + ts := timestamp() cfm := &CreateFieldMessage{ Index: i.name, Field: name, - CreatedAt: timestamp(), + CreatedAt: ts, + Owner: requestUserID, Meta: fo, } @@ -629,7 +638,7 @@ func (i *Index) CreateFieldIfNotExists(name string, opts ...FieldOption) (*Field // function options, taking a *FieldOptions struct. TODO: This should // definintely be refactored so we don't have these virtually equivalent // methods, but I'm puttin this here for now just to see if it works. -func (i *Index) CreateFieldIfNotExistsWithOptions(name string, opt *FieldOptions) (*Field, error) { +func (i *Index) CreateFieldIfNotExistsWithOptions(name string, requestUserID string, opt *FieldOptions) (*Field, error) { err := ValidateName(name) if err != nil { return nil, errors.Wrap(err, "validating name") @@ -680,10 +689,12 @@ func (i *Index) CreateFieldIfNotExistsWithOptions(name string, opt *FieldOptions } } + ts := timestamp() cfm := &CreateFieldMessage{ Index: i.name, Field: name, - CreatedAt: timestamp(), + CreatedAt: ts, + Owner: requestUserID, Meta: opt, } @@ -712,7 +723,7 @@ func (i *Index) persistField(ctx context.Context, cfm *CreateFieldMessage) error } if b, err := i.serializer.Marshal(cfm); err != nil { - return errors.Wrap(err, "marshaling") + return errors.Wrap(err, "marshaling field") } else if err := i.holder.Schemator.CreateField(ctx, cfm.Index, cfm.Field, b); errors.Cause(err) == disco.ErrFieldExists { return ErrFieldExists } else if err != nil { @@ -729,7 +740,7 @@ func (i *Index) persistUpdateField(ctx context.Context, cfm *CreateFieldMessage) } if b, err := i.serializer.Marshal(cfm); err != nil { - return errors.Wrap(err, "marshaling") + return errors.Wrap(err, "marshaling field") } else if err := i.holder.Schemator.UpdateField(ctx, cfm.Index, cfm.Field, b); errors.Cause(err) == disco.ErrFieldDoesNotExist { return ErrFieldNotFound } else if err != nil { @@ -738,7 +749,7 @@ func (i *Index) persistUpdateField(ctx context.Context, cfm *CreateFieldMessage) return nil } -func (i *Index) UpdateField(ctx context.Context, name string, update FieldUpdate) (*CreateFieldMessage, error) { +func (i *Index) UpdateField(ctx context.Context, name string, requestUserID string, update FieldUpdate) (*CreateFieldMessage, error) { // Get field from etcd buf, err := i.holder.Schemator.Field(ctx, i.name, name) if err != nil { @@ -783,6 +794,9 @@ func (i *Index) UpdateField(ctx context.Context, name string, update FieldUpdate return nil, errors.Wrap(err, "persisting updated field") } + i.mu.Lock() + defer i.mu.Unlock() + return cfm, nil } @@ -843,6 +857,7 @@ func (i *Index) createField(cfm *CreateFieldMessage) (*Field, error) { return nil, errors.Wrap(err, "initializing") } f.createdAt = cfm.CreatedAt + f.owner = cfm.Owner // Pass holder through to the field for use in looking // up a foreign index. @@ -928,11 +943,14 @@ func (p indexSlice) Less(i, j int) bool { return p[i].Name() < p[j].Name() } // IndexInfo represents schema information for an index. type IndexInfo struct { - Name string `json:"name"` - CreatedAt int64 `json:"createdAt,omitempty"` - Options IndexOptions `json:"options"` - Fields []*FieldInfo `json:"fields"` - ShardWidth uint64 `json:"shardWidth"` + Name string `json:"name"` + CreatedAt int64 `json:"createdAt,omitempty"` + UpdatedAt int64 `json:"updatedAt"` + Owner string `json:"owner"` + LastUpdateUser string `json:"lastUpdatedUser"` + Options IndexOptions `json:"options"` + Fields []*FieldInfo `json:"fields"` + ShardWidth uint64 `json:"shardWidth"` } // Field returns the FieldInfo the provided field name. If the field does not @@ -954,9 +972,10 @@ func (p indexInfoSlice) Less(i, j int) bool { return p[i].Name < p[j].Name } // IndexOptions represents options to set when initializing an index. type IndexOptions struct { - Keys bool `json:"keys"` - TrackExistence bool `json:"trackExistence"` - PartitionN int `json:"partitionN"` + Keys bool `json:"keys"` + TrackExistence bool `json:"trackExistence"` + PartitionN int `json:"partitionN"` + Description string `json:"description"` } type importData struct { diff --git a/index_internal_test.go b/index_internal_test.go index e275069cf..4919b1793 100644 --- a/index_internal_test.go +++ b/index_internal_test.go @@ -9,7 +9,7 @@ import ( // mustOpenIndex returns a new, opened index at a temporary path. Panic on error. func mustOpenIndex(tb testing.TB, opt IndexOptions) *Index { h := newTestHolder(tb) - index, err := h.CreateIndex("i", opt) + index, err := h.CreateIndex("i", "", opt) if err != nil { panic(err) diff --git a/index_test.go b/index_test.go index b291165e6..15addd338 100644 --- a/index_test.go +++ b/index_test.go @@ -26,7 +26,7 @@ func TestIndex_CreateFieldIfNotExists(t *testing.T) { _, index := test.MustOpenIndex(t) // Create field. - f, err := index.CreateFieldIfNotExists("f") + f, err := index.CreateFieldIfNotExists("f", "") if err != nil { t.Fatal(err) } else if f == nil { @@ -34,7 +34,7 @@ func TestIndex_CreateFieldIfNotExists(t *testing.T) { } // Retrieve existing field. - other, err := index.CreateFieldIfNotExists("f") + other, err := index.CreateFieldIfNotExists("f", "") if err != nil { t.Fatal(err) } else if f.Field != other.Field { @@ -53,7 +53,7 @@ func TestIndex_CreateField(t *testing.T) { _, index := test.MustOpenIndex(t) // Create field with explicit quantum. - f, err := index.CreateField("f", pilosa.OptFieldTypeTime(pilosa.TimeQuantum("YMDH"), "0")) + f, err := index.CreateField("f", "", pilosa.OptFieldTypeTime(pilosa.TimeQuantum("YMDH"), "0")) if err != nil { t.Fatal(err) } else if q := f.TimeQuantum(); q != pilosa.TimeQuantum("YMDH") { @@ -68,7 +68,7 @@ func TestIndex_CreateField(t *testing.T) { _, index := test.MustOpenIndex(t) // Create field with explicit quantum with no standard view - f, err := index.CreateField("f", pilosa.OptFieldTypeTime(pilosa.TimeQuantum("YMDH"), "0", true)) + f, err := index.CreateField("f", "", pilosa.OptFieldTypeTime(pilosa.TimeQuantum("YMDH"), "0", true)) if err != nil { t.Fatal(err) } else if q := f.TimeQuantum(); q != pilosa.TimeQuantum("YMDH") { @@ -83,7 +83,7 @@ func TestIndex_CreateField(t *testing.T) { _, index := test.MustOpenIndex(t) // Create field with schema and verify it exists. - if f, err := index.CreateField("f", pilosa.OptFieldTypeInt(-990, 1000)); err != nil { + if f, err := index.CreateField("f", "", pilosa.OptFieldTypeInt(-990, 1000)); err != nil { t.Fatal(err) } else if !reflect.DeepEqual(f.Type(), pilosa.FieldTypeInt) { t.Fatalf("unexpected type: %#v", f.Type()) @@ -101,7 +101,7 @@ func TestIndex_CreateField(t *testing.T) { _, index := test.MustOpenIndex(t) // Create field with schema and verify it exists. - if f, err := index.CreateField("f", pilosa.OptFieldTypeTimestamp(pilosa.DefaultEpoch, pilosa.TimeUnitSeconds)); err != nil { + if f, err := index.CreateField("f", "", pilosa.OptFieldTypeTimestamp(pilosa.DefaultEpoch, pilosa.TimeUnitSeconds)); err != nil { t.Fatal(err) } else if !reflect.DeepEqual(f.Type(), pilosa.FieldTypeTimestamp) { t.Fatalf("unexpected type: %#v", f.Type()) @@ -193,7 +193,7 @@ func TestIndex_CreateField(t *testing.T) { t.Run("IntField", func(t *testing.T) { _, index := test.MustOpenIndex(t) - _, err := index.CreateField("f", pilosa.OptFieldTypeInt(-1, 1), pilosa.OptFieldKeys()) + _, err := index.CreateField("f", "", pilosa.OptFieldTypeInt(-1, 1), pilosa.OptFieldKeys()) if errors.Cause(err) != pilosa.ErrIntFieldWithKeys { t.Fatal("int field cannot be created with keys=true") } @@ -203,7 +203,7 @@ func TestIndex_CreateField(t *testing.T) { t.Run("DecimalField", func(t *testing.T) { _, index := test.MustOpenIndex(t) - _, err := index.CreateField("f", pilosa.OptFieldTypeDecimal(1, pql.NewDecimal(-1, 0), pql.NewDecimal(1, 0)), pilosa.OptFieldKeys()) + _, err := index.CreateField("f", "", pilosa.OptFieldTypeDecimal(1, pql.NewDecimal(-1, 0), pql.NewDecimal(1, 0)), pilosa.OptFieldKeys()) if errors.Cause(err) != pilosa.ErrDecimalFieldWithKeys { t.Fatal("decimal field cannot be created with keys=true") } @@ -216,7 +216,7 @@ func TestIndex_DeleteField(t *testing.T) { _, index := test.MustOpenIndex(t) // Create field. - if _, err := index.CreateFieldIfNotExists("f"); err != nil { + if _, err := index.CreateFieldIfNotExists("f", ""); err != nil { t.Fatal(err) } @@ -263,7 +263,7 @@ func TestIndex_RecreateFieldOnRestart(t *testing.T) { // create index indexName := fmt.Sprintf("idx_%d", rand.Uint64()) holder := c.GetHolder(0) - _, err := holder.CreateIndex(indexName, pilosa.IndexOptions{ + _, err := holder.CreateIndex(indexName, "", pilosa.IndexOptions{ Keys: false, }) if err != nil { diff --git a/internal_client_test.go b/internal_client_test.go index 445a6d3fb..dc61b1881 100644 --- a/internal_client_test.go +++ b/internal_client_test.go @@ -854,7 +854,7 @@ func TestClient_ImportKeys(t *testing.T) { // Load bitmap into cache to ensure cache gets updated. index := hldr.MustCreateIndexIfNotExists(cluster.Idx(), pilosa.IndexOptions{Keys: true}) - _, err := index.CreateFieldIfNotExists(fldName, pilosa.OptFieldTypeInt(-100, 100)) + _, err := index.CreateFieldIfNotExists(fldName, "", pilosa.OptFieldTypeInt(-100, 100)) if err != nil { t.Fatal(err) } @@ -933,7 +933,7 @@ func TestClient_ImportIDs(t *testing.T) { // Load bitmap into cache to ensure cache gets updated. index := hldr.MustCreateIndexIfNotExists(idxName, pilosa.IndexOptions{Keys: false}) - _, err := index.CreateFieldIfNotExists(fldName, pilosa.OptFieldTypeInt(-10000, 10000)) + _, err := index.CreateFieldIfNotExists(fldName, "", pilosa.OptFieldTypeInt(-10000, 10000)) if err != nil { t.Fatal(err) } @@ -1001,7 +1001,7 @@ func TestClient_ImportValue(t *testing.T) { // Load bitmap into cache to ensure cache gets updated. index := hldr.MustCreateIndexIfNotExists(cluster.Idx(), pilosa.IndexOptions{}) - _, err := index.CreateFieldIfNotExists(fldName, pilosa.OptFieldTypeInt(-100, 100)) + _, err := index.CreateFieldIfNotExists(fldName, "", pilosa.OptFieldTypeInt(-100, 100)) if err != nil { t.Fatal(err) } @@ -1112,7 +1112,7 @@ func TestClient_ImportExistence(t *testing.T) { fldName := "fset" index := hldr.MustCreateIndexIfNotExists(idxName, pilosa.IndexOptions{TrackExistence: true}) - _, err := index.CreateFieldIfNotExists(fldName) + _, err := index.CreateFieldIfNotExists(fldName, "") if err != nil { t.Fatal(err) } @@ -1148,7 +1148,7 @@ func TestClient_ImportExistence(t *testing.T) { fldName := "fint" index := hldr.MustCreateIndexIfNotExists(idxName, pilosa.IndexOptions{TrackExistence: true}) - _, err := index.CreateFieldIfNotExists(fldName, pilosa.OptFieldTypeInt(-100, 100)) + _, err := index.CreateFieldIfNotExists(fldName, "", pilosa.OptFieldTypeInt(-100, 100)) if err != nil { t.Fatal(err) } diff --git a/lattice/public/favicon.ico b/lattice/public/favicon.ico old mode 100755 new mode 100644 index 2fc5595e1..7e53bed5e Binary files a/lattice/public/favicon.ico and b/lattice/public/favicon.ico differ diff --git a/lattice/public/favicon.png b/lattice/public/favicon.png index fca3ac8c0..8b26a3857 100644 Binary files a/lattice/public/favicon.png and b/lattice/public/favicon.png differ diff --git a/lattice/public/favicon.svg b/lattice/public/favicon.svg deleted file mode 100644 index c8f396a21..000000000 --- a/lattice/public/favicon.svg +++ /dev/null @@ -1,7 +0,0 @@ - - - - - - diff --git a/lattice/public/index.html b/lattice/public/index.html index 5a7acad02..32bbaaeb7 100644 --- a/lattice/public/index.html +++ b/lattice/public/index.html @@ -2,7 +2,6 @@ - - Molecula + FeatureBase diff --git a/lattice/public/manifest.json b/lattice/public/manifest.json index 1a608464d..9f1632193 100644 --- a/lattice/public/manifest.json +++ b/lattice/public/manifest.json @@ -1,6 +1,6 @@ { - "short_name": "Molecula", - "name": "Molecula Demo App", + "short_name": "FeatureBase", + "name": "FeatureBase Demo App", "icons": [ { "src": "favicon.ico", diff --git a/lattice/src/App/AuthFlow/AuthFlow.module.scss b/lattice/src/App/AuthFlow/AuthFlow.module.scss index 5ac275622..2ce23015e 100644 --- a/lattice/src/App/AuthFlow/AuthFlow.module.scss +++ b/lattice/src/App/AuthFlow/AuthFlow.module.scss @@ -16,7 +16,7 @@ } .logo { - height: 85px; + height: 50px; margin: 16px; } diff --git a/lattice/src/App/AuthFlow/Signin.tsx b/lattice/src/App/AuthFlow/Signin.tsx index 4a5bb8ce2..66e863900 100644 --- a/lattice/src/App/AuthFlow/Signin.tsx +++ b/lattice/src/App/AuthFlow/Signin.tsx @@ -2,7 +2,7 @@ import Card from '@material-ui/core/Card'; import CardContent from '@material-ui/core/CardContent'; import CardHeader from '@material-ui/core/CardHeader'; -import { ReactComponent as MLogo } from 'assets/m-bug-alt.svg'; +import { ReactComponent as FeatureBaseIcon } from 'assets/featurebase-icon.svg'; import css from './AuthFlow.module.scss'; import SignInButton from './SignInButton'; @@ -20,7 +20,7 @@ function Signin(props) {
- +
{renderLoginForm()}
diff --git a/lattice/src/App/Home/ClusterHealth/Metrics/Metrics.tsx b/lattice/src/App/Home/ClusterHealth/Metrics/Metrics.tsx index e2d3a5229..40df99384 100644 --- a/lattice/src/App/Home/ClusterHealth/Metrics/Metrics.tsx +++ b/lattice/src/App/Home/ClusterHealth/Metrics/Metrics.tsx @@ -239,7 +239,7 @@ export const Metrics: FC = ({ open, node, onClose }) => {
- Molecula provides metrics for use with Prometheus. Instantaneous + FeatureBase provides metrics for use with Prometheus. Instantaneous metrics are shown here for convenience; to take full advantage of this feature, consider using a Prometheus instance, perhaps with a{' '} ( Try FeatureBase Cloud at - - app.molecula.cloud + + cloud.featurebase.com diff --git a/lattice/src/App/QueryBuilder/QueryBuilderContainer.tsx b/lattice/src/App/QueryBuilder/QueryBuilderContainer.tsx index 3f7afb292..a2dacce41 100644 --- a/lattice/src/App/QueryBuilder/QueryBuilderContainer.tsx +++ b/lattice/src/App/QueryBuilder/QueryBuilderContainer.tsx @@ -113,7 +113,7 @@ export const QueryBuilderContainer = () => { type: 'text/plain;charset=utf-8', }); element.href = URL.createObjectURL(file); - element.download = `molecula-${results?.index}-${dateTime}.csv`; + element.download = `featurebase-${results?.index}-${dateTime}.csv`; document.body.appendChild(element); element.click(); exportRows = []; diff --git a/lattice/src/assets/darkTheme/MoleculaBug.svg b/lattice/src/assets/darkTheme/MoleculaBug.svg deleted file mode 100644 index b52c8e27d..000000000 --- a/lattice/src/assets/darkTheme/MoleculaBug.svg +++ /dev/null @@ -1 +0,0 @@ -MoleculaBug diff --git a/lattice/src/assets/darkTheme/MoleculaLogo.svg b/lattice/src/assets/darkTheme/MoleculaLogo.svg deleted file mode 100644 index ae3bdaac0..000000000 --- a/lattice/src/assets/darkTheme/MoleculaLogo.svg +++ /dev/null @@ -1 +0,0 @@ -Molecula Logo diff --git a/lattice/src/assets/darkTheme/featurebase-logo.svg b/lattice/src/assets/darkTheme/featurebase-logo.svg new file mode 100644 index 000000000..0f388dd15 --- /dev/null +++ b/lattice/src/assets/darkTheme/featurebase-logo.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/lattice/src/assets/featurebase-icon.svg b/lattice/src/assets/featurebase-icon.svg new file mode 100644 index 000000000..6510e5e7e --- /dev/null +++ b/lattice/src/assets/featurebase-icon.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/lattice/src/assets/lightTheme/MoleculaBug.svg b/lattice/src/assets/lightTheme/MoleculaBug.svg deleted file mode 100644 index e37ef93b4..000000000 --- a/lattice/src/assets/lightTheme/MoleculaBug.svg +++ /dev/null @@ -1 +0,0 @@ -MoleculaBug diff --git a/lattice/src/assets/lightTheme/MoleculaLogo.svg b/lattice/src/assets/lightTheme/MoleculaLogo.svg deleted file mode 100644 index 0a5159864..000000000 --- a/lattice/src/assets/lightTheme/MoleculaLogo.svg +++ /dev/null @@ -1 +0,0 @@ -Molecula Logo diff --git a/lattice/src/assets/lightTheme/featurebase-logo.svg b/lattice/src/assets/lightTheme/featurebase-logo.svg new file mode 100644 index 000000000..1d5da210c --- /dev/null +++ b/lattice/src/assets/lightTheme/featurebase-logo.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/lattice/src/assets/m-bug-alt.svg b/lattice/src/assets/m-bug-alt.svg deleted file mode 100644 index a0cc81bc1..000000000 --- a/lattice/src/assets/m-bug-alt.svg +++ /dev/null @@ -1,16 +0,0 @@ - - - - - - - - - - - - - - - - diff --git a/lattice/src/lattice.config.js b/lattice/src/lattice.config.js index 2ab3f6fa3..53b84a536 100644 --- a/lattice/src/lattice.config.js +++ b/lattice/src/lattice.config.js @@ -1,4 +1,4 @@ export const pilosaConfig = { - // hostname: 'localhost', - // httpPort: '10101' + hostname: 'localhost', + httpPort: '10101' }; diff --git a/lattice/src/shared/Header/Header.tsx b/lattice/src/shared/Header/Header.tsx index b32e72e63..e0a609140 100644 --- a/lattice/src/shared/Header/Header.tsx +++ b/lattice/src/shared/Header/Header.tsx @@ -1,17 +1,17 @@ -import SignOutButton from "App/AuthFlow/SignOutButton"; -import { ReactComponent as MoleculaLogoDark } from "assets/darkTheme/MoleculaLogo.svg"; -import { ReactComponent as MoleculaLogo } from "assets/lightTheme/MoleculaLogo.svg"; -import { FC } from "react"; -import { Link } from "react-router-dom"; -import { useAuth } from "services/useAuth"; -import { ThemeToggle } from "shared/ThemeToggle"; +import SignOutButton from 'App/AuthFlow/SignOutButton'; +import { ReactComponent as FeatureBaseLogoDark } from 'assets/darkTheme/featurebase-logo.svg'; +import { ReactComponent as FeatureBaseLogo } from 'assets/lightTheme/featurebase-logo.svg'; +import { FC } from 'react'; +import { Link } from 'react-router-dom'; +import { useAuth } from 'services/useAuth'; +import { ThemeToggle } from 'shared/ThemeToggle'; -import AppBar from "@material-ui/core/AppBar"; -import Button from "@material-ui/core/Button"; -import { useTheme } from "@material-ui/core/styles"; -import Toolbar from "@material-ui/core/Toolbar"; +import AppBar from '@material-ui/core/AppBar'; +import Button from '@material-ui/core/Button'; +import { useTheme } from '@material-ui/core/styles'; +import Toolbar from '@material-ui/core/Toolbar'; -import css from "./Header.module.scss"; +import css from './Header.module.scss'; type HeaderProps = { onToggleTheme: () => void; @@ -19,7 +19,7 @@ type HeaderProps = { export const Header: FC = ({ onToggleTheme }) => { const theme = useTheme(); - const isDark = theme.palette.type === "dark"; + const isDark = theme.palette.type === 'dark'; const auth = useAuth(); return ( @@ -32,8 +32,8 @@ export const Header: FC = ({ onToggleTheme }) => {
- {!isDark && } - {isDark && } + {!isDark && } + {isDark && }
@@ -50,8 +50,8 @@ export const Header: FC = ({ onToggleTheme }) => { {auth.user && (