Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[dbnode] Aggregate() using only FSTs where possible #1545

Merged
merged 1 commit into from
Apr 23, 2019

Conversation

prateek
Copy link
Collaborator

@prateek prateek commented Apr 11, 2019

  • When we're not filtering on a query, we can compose the results of Aggregation from the FSTs directly. This avoids the code path from postings lists -> documents, thereby saving a lot of CPU.

field, term []byte,
includeTerms bool,
) []AggregateResultsEntry {
// NB(prateek): we make a copy of the (field, term) entries returned
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

highlight this bit as a potential contentious choice. could simplify to living with higher contention if people feel strongly.

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I like this approach fwiw :)

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Aye, me too.

aggValues = r.valuesPool.Get()
// we can avoid the copy because we assume ownership of the passed ident.ID,
// but still need to finalize it.
r.resultsMap.set(f, aggValues, _AggregateResultsMapKeyOptions{
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

using an internal method of codegen'd type feels sketch but we don't have the equivalent exported so no way around

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can't this use

SetUnsafe(f, aggValues, AggregateResultsMapSetUnsafeOptions{
 NoCopyKey: true,
 NoFinalizeKey: false,
})

to avoid the internal method?

Copy link
Collaborator Author

@prateek prateek Apr 12, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good call, will do.

@codecov
Copy link

codecov bot commented Apr 11, 2019

Codecov Report

Merging #1545 into master will decrease coverage by 0.1%.
The diff coverage is 39.6%.

Impacted file tree graph

@@           Coverage Diff            @@
##           master   #1545     +/-   ##
========================================
- Coverage    71.7%   71.6%   -0.2%     
========================================
  Files         947     948      +1     
  Lines       77981   78230    +249     
========================================
+ Hits        55960   56026     +66     
- Misses      18381   18548    +167     
- Partials     3640    3656     +16
Flag Coverage Δ
#aggregator 82.3% <ø> (ø) ⬆️
#cluster 85.7% <ø> (ø) ⬆️
#collector 63.7% <ø> (ø) ⬆️
#dbnode 79.6% <38.7%> (-0.5%) ⬇️
#m3em 73.2% <ø> (ø) ⬆️
#m3ninx 73.9% <100%> (ø) ⬆️
#m3nsch 51.1% <ø> (ø) ⬆️
#metrics 17.5% <ø> (ø) ⬆️
#msg 74.9% <ø> (ø) ⬆️
#query 66.5% <ø> (ø) ⬆️
#x 85.7% <ø> (-0.2%) ⬇️

Continue to review full report at Codecov.

Legend - Click here to learn more
Δ = absolute <relative> (impact), ø = not affected, ? = missing data
Powered by Codecov. Last update 0a395df...313d21f. Read the comment docs.

@prateek prateek force-pushed the prateek/dbnode/index-agg-fst branch 4 times, most recently from b143a0b to 501c20d Compare April 11, 2019 02:10
@@ -181,6 +181,16 @@ type AggregateResults interface {
aggregateQueryOpts AggregateResultsOptions,
)

// AggregateResultsOptions returns the set AggregateResultsOptions.
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

returns the options for this AggregateResult?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure

opts QueryOptions,
results AggregateResults,
) (exhaustive bool, err error)

// AddResults adds bootstrap results to the block, if c.
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I know it's not part of this pr but can you fix this comment?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@@ -273,6 +290,16 @@ type Block interface {
results BaseResults,
) (exhaustive bool, err error)

// Aggregate aggregates known tag names/values.
// NB(prateek): this is different from Query, as we can
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit; different from AggregateQuery?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nah, methods on this interface are Query() and Aggregate(). Unless you mean to say rename?

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

different from aggregating by means of Query maybe? Otherwise it kinda reads like this aggregates tag names/values which you don't really expect Query to do

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lol sure

@@ -304,6 +322,16 @@ func (o *opts) DocumentArrayPool() doc.DocumentArrayPool {
return o.docArrayPool
}

func (o *opts) SetAggregateResultsEntryArrayPool(value AggregateResultsEntryArrayPool) Options {
opts := *o
opts.aggResultsEntryArrayPool = value
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should this pool be added to the validate function?

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah nvm looks like that's only for pools which need to be added rather than the ones created by NewOptions

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

meh, i'll add for sanity.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

aggValues = r.valuesPool.Get()
// we can avoid the copy because we assume ownership of the passed ident.ID,
// but still need to finalize it.
r.resultsMap.set(f, aggValues, _AggregateResultsMapKeyOptions{
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can't this use

SetUnsafe(f, aggValues, AggregateResultsMapSetUnsafeOptions{
 NoCopyKey: true,
 NoFinalizeKey: false,
})

to avoid the internal method?

return false, err
}

if err := iter.Close(); err != nil {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: I don't think this is necessary; you reset every loop and close the iterator explicitly below already

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hm it's necessary cause we want to ensure we're releasing any internal state (via Close()). Reset() just clears that state, doesn't release it.

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fair; in that case it's a little weird that we double close below

For this particular iterator it's not really a problem, but it's plausible for an iter implementatiopn to panic if it's double closed here

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Definitely a fair point. This implementation doesn't have that issue intentionally for this reason - so as to ensure we free resources. I'll make a note about this property in a comment.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Still feels a little sketch to me because I think some of our Close() methods return objects to pools

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

that's a fair point. I'll rework to not rely on this behaviour.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done


// Add last batch to results if remaining.
if len(batch) > 0 {
batch, size, err = b.addAggregateResults(cancellable, results, batch)
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does this need to append aggregate results too?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nah, cause we're already done that for the elements in the batch in a previous loop iteration.

field, term []byte,
includeTerms bool,
) []AggregateResultsEntry {
// NB(prateek): we make a copy of the (field, term) entries returned
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I like this approach fwiw :)

var (
entry AggregateResultsEntry
lastField []byte
lastFieldIsValid bool
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: I don't think this is necessary, bytes.Equal should handle the lastField == nil case

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hm not sure we can. Consider the case when we're trying to distinguish the first element of a batch (i.e. no prior elements in the batch), from a batch with the last element having a nil field. I don't think it'll happen in practice but this way makes fewer assumptions about the data so tend to prefer it.

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the nil field will break this in general even if it's somewhere in the middle (since you'd be trying to call .Bytes() on it); otherwise is there any real difference between?

Could do something like

if len(batch) > 0 {
 last := batch[len(batch)-1]
 if last != nil {
  lastField = last.Bytes()
 }
}

 if bytes.Equal(lastField, field) {
  entry = batch[len(batch)-1] 
 } else {
 entry.Field = b.pooledID(field) 
} 
...

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we're guaranteed it'd only be the first element (cause its the first lexicographic string). It wouldn't break anything cause we'd still allocate an ident.ID backed by an empty slice for it.

return exhaustive, nil
}

func (b *block) appendAggregateResults(
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: the name was really confusing (as was addAggregateResults followed by addAggregateResults), maybe rename this to appendFieldAndTermToBatch

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure thing


func (fti *fieldsAndTermsIter) setNext() bool {
// if only need to iterate fields
if !fti.opts.iterateTerms {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: I think this might flow a little better if the branch is moved to the Next() function instead of calling setNext then instantly calling setNextField

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fair, done.

return false
}
fti.termIter = termsIter

Copy link
Collaborator

@arnikola arnikola Apr 15, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm this may need to check if !fti.opts.allow(field) and move onto the next fieldIter if so

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nope, cause setNextField() does that check already.

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍 misread that this was calling Next instead

slice := toSlice(t, iter)
requireSlicesEqual(t, []pair{}, slice)
}

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you add a few tests with allowFn set, maybe one which goes through a couple of segments

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

re: allowFn tests - does the existing TestFieldsTermsIteratorSimpleSkip not capture stuff you're worried about?

Copy link
Collaborator Author

@prateek prateek Apr 15, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

re: multiple segments, do you mean Reset()? if so, done

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also, re allowFn - the prop tests using fieldsTermsIteratorPropInput are generating test situations with those cases too.

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My bad yeah, was looking at a partial diff so the tests seemed lacking

@prateek prateek force-pushed the prateek/dbnode/index-agg-fst branch from 313d21f to 39a94c1 Compare April 15, 2019 19:24
exhaustive, err := i.query(ctx, query, results, opts.QueryOptions)
// use appropriate fn to query underlying blocks.
fn := i.execBlockQueryFn
if query.Equal(idx.NewAllQuery()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we just compare against a global one? Its miniscule but technically this allocates because it creates an interface internally

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lol it's not going to have an impact on perf but sure.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

return
}

if blockExhaustive {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Might be simpler to just do:

state.exhaustive = blockExhaustive
return

Would make adding more logic later easier, although technically it might be slower cause you're forcing a memory write back 🤷‍♂️

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

setting to state.exhaustive = state.exhaustive && blockExhaustive always works so doing that instead

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

return
}

if blockExhaustive {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same comment

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same as above.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

f := entry.Field
aggValues, ok := r.resultsMap.Get(f)
if !ok {
aggValues = r.valuesPool.Get()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How do these get reset? I don't see it after any Get() calls nor in the generated code on put. Are we just trusting that they've been properly reset here?

Copy link
Collaborator Author

@prateek prateek Apr 18, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The way it's supposed to work: we register a Finalizer on the incoming context to ensure the Finalize() method on the object is called (which in turn calls Reset(nil, ...) which does the actual releasing).

That said, the current code doesn't look to be registering a Finalizer on the context in either Query/Aggregate. Will put up another PR for this.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

#1567 as the follow up.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is it cheap to do a reset after pulling out the pool? I generally prefer that pattern over trust that every Put does a proper reset but I trust you understand this lifecycle better than I do

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yeah Reset is O(items in the map); so should be free considering the Put is doing the right thing. Don't need to do it tho, we ensure to cleanup before the Put

I prefer this approach (assume object returned from pool is valid) cause it puts the cleanup penalty on the last callsite to use the object (as opposed to the new callsite to receive the object). Seems like a "fairer" way to tax users

return r.aggregateOpts
}

func (r *aggregatedResults) AddFields(batch []AggregateResultsEntry) (int, error) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is the error just future proofing the API? Doesnt look like its currently possible to return one

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fair, will simplify

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

data := b.bytesPool.Get(len(id))
data.IncRef()
data.AppendAll(id)
data.DecRef()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmm, seems like this ID will have no refs for awhile until we add it to the results :/ Nothing we can do about that I assume? adding more accounting probably not worth it since these are really expensive

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The ref is only the bytes backing this ident, which the ident will take a ref to right after this line when we transfer the bytes to it (https://github.com/m3db/m3/blob/master/src/x/ident/identifier_pool.go#L105)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

oh duh I misunderstood this. 👍

results AggregateResults,
batch []AggregateResultsEntry,
) ([]AggregateResultsEntry, int, error) {
// Checkout the lifetime of the query before adding results
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: period at the end of all these comments

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

parameters.Rng = rand.New(rand.NewSource(seed))
properties := gopter.NewProperties(parameters)

properties.Property("Fields Terms Iteration doesn't blow up", prop.ForAll(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you add a comment about the motivation for why we need a separate test just to make sure there are no panics when we already have a correctness test (that presumably might also catch panics)

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

tl;dr - correctness prop test ensures we behave correctly on the happy path. this prop tests ensures we don't panic unless the underlying itself iterator panics.

will add a comment.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

itself -> iterator

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure thing

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

fieldsAndTermsIterZeroed fieldsAndTermsIter
)

var _ fieldsAndTermsIterator = &fieldsAndTermsIter{}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this only happens at package initialization anyways right? who cares haha

// Err returns any errors encountered during iteration.
Err() error

// Close releases any resources held by the iterator.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe add a comment explicitly saying this will not return the iter to the pool and that anything implementing this interface should explicitly support double closes

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

as mentioned earlier, reworked to avoid this dependency.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

Copy link
Contributor

@richardartoul richardartoul left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

:stamp: to unblock

@prateek prateek force-pushed the prateek/dbnode/index-agg-fst branch 6 times, most recently from b8fb0fc to 2ce7b21 Compare April 18, 2019 04:56
@prateek prateek force-pushed the prateek/dbnode/index-agg-fst branch from 22dd0c9 to c4fb8f4 Compare April 18, 2019 17:42
f := entry.Field
aggValues, ok := r.resultsMap.Get(f)
if !ok {
aggValues = r.valuesPool.Get()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is it cheap to do a reset after pulling out the pool? I generally prefer that pattern over trust that every Put does a proper reset but I trust you understand this lifecycle better than I do

@@ -823,30 +852,235 @@ func (b *block) addQueryResults(
results BaseResults,
batch []doc.Document,
) ([]doc.Document, int, error) {
// Checkout the lifetime of the query before adding results
// checkout the lifetime of the query before adding results.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why are you lower casing these? I've been commenting on all P.Rs to ensure they're capital!

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

oh really? I've been trying to stay all lower case for a while. Don't feel strongly about which one but we should pick a convention and stick to it.

size = results.Size()
batch = b.opts.AggregateResultsEntryArrayPool().Get()
batchSize = cap(batch)
iterClosed = false // tracking whether we need to free the iterator at the end.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this just a guard against future maintainers? Cause it seems like you always close it now. If so maybe just mention in the comment its an extra precaution so people dont spend time chasing why its needed (unless I missed something)

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nah so it's like the standard cleanup pattern we use in a few places

object := ...
cleanedUp := false

defer func() {
  if !cleanedUp {
    object.Cleanup()
  }
}()

// do thing 1 for object
// if it fails, just early `return`

// so on

// finally, 
cleanedUp = true
if err :=object.Cleanup(); err != nil {
  return err
}

this way we can be sure all exit paths from the function cleanup the object; either because we manually do it; or the defer takes care of it.

without the defer (and the var). i'd have to interlace the cleanup calls at every exit point from the function. which seems brittle at best.

data := b.bytesPool.Get(len(id))
data.IncRef()
data.AppendAll(id)
data.DecRef()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

oh duh I misunderstood this. 👍

parameters.Rng = rand.New(rand.NewSource(seed))
properties := gopter.NewProperties(parameters)

properties.Property("Fields Terms Iteration doesn't blow up", prop.ForAll(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

itself -> iterator

@prateek prateek force-pushed the prateek/dbnode/index-agg-fst branch from e13bfc0 to 40070bc Compare April 19, 2019 16:13
@prateek prateek force-pushed the prateek/dbnode/index-agg-fst branch from 40070bc to a42a09c Compare April 23, 2019 23:00
@prateek prateek merged commit 0185c0e into master Apr 23, 2019
@prateek prateek deleted the prateek/dbnode/index-agg-fst branch April 23, 2019 23:15
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants