Skip to content
This repository has been archived by the owner on Aug 23, 2023. It is now read-only.

Persist meta records to Cassandra index #1471

Merged
merged 9 commits into from
Oct 29, 2019
Merged

Persist meta records to Cassandra index #1471

merged 9 commits into from
Oct 29, 2019

Conversation

replay
Copy link
Contributor

@replay replay commented Sep 23, 2019

This implements persisting meta records into the persistent Cassandra index. It does not implement the same functionality for the persistent BigTable index yet.

@@ -49,6 +51,12 @@ var (
// metric idx.cassandra.save.skipped is how many saves have been skipped due to the writeQueue being full
statSaveSkipped = stats.NewCounter32("idx.cassandra.save.skipped")
errmetrics = cassandra.NewErrMetrics("idx.cassandra")

metaRecordRetryPolicy = gocql.ExponentialBackoffRetryPolicy{
Copy link
Member

Choose a reason for hiding this comment

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

This retry policy seems a bit aggressive. it will retry 10 times in less then 4seconds.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

What do you think would make more sense?

Copy link
Member

Choose a reason for hiding this comment

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

What failure modes are the retries there to defend against?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

network issues, the duration of those is not really predictable. if a cassandra pod restarts it can take over a minute for it to come back, so i guess the max backoff should be like 2 min then?

Copy link
Member

Choose a reason for hiding this comment

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

if a cassandra pod restarts, the request will just be retried on another pod.

I guess the real question is: how long can you afford to wait?
If you are protecting against network issues, then attempts will fail pretty quickly if the network is down. So you can work out the min and max that will allow retries up until your max desired execution time. eg, if these requests are originating from HTTP requests, then you could wait up to 60seconds (but probably best to target 30s or 45s).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good point, it should definitely be under the HTTP request timeout. These requests are originating from an HTTP request, so in case of a final error we want to return an error to the HTTP client.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

i explained the calculations in the comment:
88c309b

// schemaFile: file containing table definition
// entryName: identifier of the schema within the file
// tableName: name of the table in cassandra
func (c *CasIdx) EnsureTableExists(session *gocql.Session, schemaFile, entryName, tableName string) error {
Copy link
Member

Choose a reason for hiding this comment

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

instead of passing the schemaFile name, you should just pass an io.Reader that returns the schema file content. This will make it easier to do testing and enable other callers to be able to call the method without having to first write a schemaFile to disk

Copy link
Contributor Author

Choose a reason for hiding this comment

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

good idea, will do that

Copy link
Contributor Author

@replay replay Sep 24, 2019

Choose a reason for hiding this comment

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

Since we're using go-toml to read those schema files (https://github.com/grafana/metrictank/blob/master/util/template.go) , I'd first need to read the schema file's correct entry into a string which then gets returned by util.ReadEntry(), wrap it into a reader, pass that reader into EnsureTableExists, and then read it back into a string via a buffer.
That seems like a bit of unnecessarily complication, wouldn't it be better to just pass a string with the table schema into EnsureTableExists, instead of a reader? Then we still get the benefit of being able to unit test it, but we don't need to construct a reader and a buffer to read the reader into.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

What do you think about doing something like this:
0f0303e

I have pushed that into another branch, because I feel like it's getting out of scope of this PR. But with this we would have the advantages that initializing the cassandra index and stores becomes much more flexible because we can either provide a custom io.Reader to the <config>.ParseSchemas() functions, or we just directly set the schema we want in the according config struct.

@replay replay force-pushed the persist_meta_records branch 2 times, most recently from 88c309b to b75e1c5 Compare September 24, 2019 14:28
@fkaleo
Copy link
Contributor

fkaleo commented Sep 24, 2019

Looks like we will have to copy the meta records during migrations.

@replay
Copy link
Contributor Author

replay commented Sep 25, 2019

@fkaleo good point, I haven't thought about that. should be pretty simple though, as the amount of data in the meta_records table should be relatively small, so a simple dump, transfer, insert should be good enough.

}

func (c *CasIdx) MetaTagRecordSwap(orgId uint32, records []tagquery.MetaTagRecord, persist bool) (uint32, uint32, error) {
added, deleted, err := c.MemoryIndex.MetaTagRecordSwap(orgId, records, persist)
Copy link
Member

Choose a reason for hiding this comment

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

doesnt adding to the memoryIdx before adding to cassandra lead to things getting out of sync? eg the memoryIdx gets updated, but the batch update to cassandra fails.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

the plan is to reload from cassandra at some interval anyway, at this point the state of the memory index would then get reset to what it previously was.
the reason why the updating of the memory index should be done first is mainly because that way we will be able to detect whether anything has changed, or if the posted set of meta records is just the same as the old one. this is implemented in this subsequent PR:
https://github.com/grafana/metrictank/pull/1480/files#diff-c6ce4629577f4e064a1d2f636acc0568R496

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I guess it would also be reasonable to just always flush to cassandra first, even if there was no change. And if the query to cassandra failed we don't update the memory index. I'm going to do that.

Copy link
Contributor Author

@replay replay Sep 25, 2019

Choose a reason for hiding this comment

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

I forgot, there is one more reason to do the memory index update first. On upsert we want to be able to know whether a record has been created, or if it only updates an existing one. When one gets created then we set the createdat timestamp in cassandra, otherwise we only update the lastupdate timestamp.
We currently decide this based on the returned status from the Memory Index.
I don't think we really need the createdat column though, we could also just remove that one.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done: 6a73386

@replay
Copy link
Contributor Author

replay commented Oct 15, 2019

Going to push some more modifications, according to the plan defined here:

https://docs.google.com/document/d/1arRpSuoecqOV8rA0Bus0EphX15dZliQSlzWkRbNWDpo/edit?usp=sharing

@replay replay force-pushed the persist_meta_records branch 16 times, most recently from 5d8eee9 to ffae89c Compare October 17, 2019 18:14
@replay replay force-pushed the persist_meta_records branch 9 times, most recently from 168c91b to 86d6dd7 Compare October 21, 2019 11:56
@replay
Copy link
Contributor Author

replay commented Oct 22, 2019

FYI i deployed this branch in my QA instance to test it. I've done some upserts and some swaps, while doing them i checked to verify that cassandra gets updated as expected and I also queried multiple read pods to verify that they load the rules correctly.
AFAICT it looks fine.

Copy link
Contributor

@robert-milan robert-milan left a comment

Choose a reason for hiding this comment

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

LGTM

@replay replay merged commit 557eea9 into master Oct 29, 2019
@replay replay deleted the persist_meta_records branch October 29, 2019 21:25
Sign up for free to subscribe to this conversation on GitHub. Already have an account? Sign in.
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

4 participants