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

Introduce translog generation rolling #23606

Merged
merged 29 commits into from
Mar 27, 2017

Conversation

jasontedor
Copy link
Member

@jasontedor jasontedor commented Mar 16, 2017

This commit introduces a maximum size for a translog generation and automatically rolls the translog when a generation exceeds the threshold into a new generation. This threshold is configurable per index and defaults to sixty-four megabytes. We introduce this constraint as sequence numbers will require keeping around more than the current generation (to ensure that we can rollback to the global checkpoint). Without keeping the size of generations under control, having to keep old generations around could consume excessive disk space. A follow-up will enable commits to trim previous generations based on the global checkpoint.

Relates #10708

This commit introduces a maximum size for a translog generation and
automatically folds the translog when a generation exceeds the threshold
into a new generation. This threshold is configurable per index and
defaults to sixty-four megabytes. We introduce this constraint as
sequence numbers will require keeping around more than the current
generation (to ensure that we can rollback to the global
checkpoint). Without keeping the size of generations under control,
having to keep old generations around could consume excessive disk
space. A follow-up will enable commits to trim previous generations
based on the global checkpoint.
Copy link
Member

@nik9000 nik9000 left a comment

Choose a reason for hiding this comment

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

I'm not sure about the word "fold" for this. "fold" sounds like you are combining existing translog generations. You are really artificially segmenting the translog for convenient slicing later. But if you like "fold" I'm fine with it.

@@ -399,6 +399,8 @@ TranslogWriter createWriter(long fileGeneration) throws IOException {
return newFile;
}

final AtomicBoolean foldingGeneration = new AtomicBoolean();
Copy link
Member

Choose a reason for hiding this comment

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

Putting this down here rather than up with the rest of the members will make it much harder to find. Even if this is the one place it is used.

Copy link
Member Author

Choose a reason for hiding this comment

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

I pushed 0d5e6e2.

}
try (ReleasableLock ignored = writeLock.acquire()) {
if (shouldFoldGeneration(this) && foldingGeneration.compareAndSet(false, true)) {
// we have to check the condition again lest we could fold twice in a race
Copy link
Member

Choose a reason for hiding this comment

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

If you already have the writeLock can this race?

Copy link
Member Author

Choose a reason for hiding this comment

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

My fingers did not do what they suppose to do here. 😉 I pushed 152cae7.

* @param translog the translog
* @return {@code true} if the current generation should be folded into a new generation
*/
private static boolean shouldFoldGeneration(final Translog translog) {
Copy link
Member

Choose a reason for hiding this comment

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

I'd wouldn't have declared this as static. I'm really curious why you did. I expect I have something to learn from this....

Copy link
Member Author

Choose a reason for hiding this comment

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

I pushed c88e788.

* @param generation the current translog generation
* @throws IOException if an I/O exception occurred during any file operations
*/
void foldGeneration(final long generation) throws IOException {
Copy link
Member

Choose a reason for hiding this comment

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

s/generation/currentGeneration/?

Copy link
Member Author

Choose a reason for hiding this comment

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

I pushed 7e69ae8.

final long generation = current.getGeneration();
currentCommittingGeneration = generation;
foldGeneration(generation);
} catch (final Exception e) {
Copy link
Member

Choose a reason for hiding this comment

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

Is this catch block still needed? foldGeneration has one just like it. Probably worth adding a comment if it is.

Copy link
Member Author

Choose a reason for hiding this comment

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

I pushed fb00dd3.

@jasontedor
Copy link
Member Author

jasontedor commented Mar 16, 2017

I'm not sure about the word "fold" for this. "fold" sounds like you are combining existing translog generations. You are really artificially segmenting the translog for convenient slicing later. But if you like "fold" I'm fine with it.

I was thinking of folding like folding cards, like "I'm done with this generation." I'm not tied to the name, "roll" would be an option as well.

@jasontedor
Copy link
Member Author

Thanks for your review @nik9000; I have responded to your comments and pushed commits.

Copy link
Contributor

@bleskes bleskes left a comment

Choose a reason for hiding this comment

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

Thx @jasontedor . I left some comments . +1 on my end to move from fold to roll.

return current.add(bytes, operation.seqNo());
location = current.add(bytes, operation.seqNo());
}
if (shouldFoldGeneration() && foldingGeneration.compareAndSet(false, true)) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I think he shouldFoldGeneration check should be done under the read lock to make sure things don't change on it inflight - this is similar to other access to current.

Copy link
Member Author

Choose a reason for hiding this comment

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

I pushed 8f6b609.

}
if (shouldFoldGeneration() && foldingGeneration.compareAndSet(false, true)) {
// we have to check the condition again lest we could fold twice in a race
if (shouldFoldGeneration()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

we should check again under the write lock when no one is writing.

Copy link
Member Author

Choose a reason for hiding this comment

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

I pushed 8f6b609.

try (ReleasableLock ignored = writeLock.acquire()) {
this.foldGeneration();
}
final boolean wasFoldingGeneration = foldingGeneration.getAndSet(false);
Copy link
Contributor

Choose a reason for hiding this comment

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

I would feel better if this was in a finally clause (although if we have any exception here, we will fail the translog anyway). Thinking about this more - shall we move this logic into it's own "foldIfNeeded" method?

Copy link
Member Author

Choose a reason for hiding this comment

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

I pushed 8f6b609.

for (int i = 0; i <= folds; i++) {
assertFileIsPresent(translog, generation + i);
}
translog.commit();
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 check that we can also fold between precommit and commit?

Copy link
Member Author

Choose a reason for hiding this comment

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

I pushed d9fbc42.

Copy link
Member

@nik9000 nik9000 left a comment

Choose a reason for hiding this comment

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

LGTM

if (randomBoolean()) {
bufferSize = TranslogConfig.DEFAULT_BUFFER_SIZE;
} else {
bufferSize = new ByteSizeValue(10 + randomInt(128 * 1024), ByteSizeUnit.BYTES);
Copy link
Member

Choose a reason for hiding this comment

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

This value interacts with the test for generation folding, right? Would making this too low break other tests? Can you leave a comment about the size that you set here?

Copy link
Member Author

Choose a reason for hiding this comment

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

This is only a refactoring of some existing logic.

@jasontedor
Copy link
Member Author

+1 on my end to move from fold to roll.

I pushed bbab126.

* master:
  Clear the interrupt flag before joining
  Migrate to max line length of 100
  Docs: Corrected path to elasticsearch-plugin (elastic#23622)
  Docs: Add comma to reverse nested agg snippet
  Fix third-party audit task for Gradle 3.4 (elastic#23612)
  Adapter action future should restore interrupts
  Update scripting.asciidoc
  Unmark reindex as experimental
  CompletionSuggestionContext#toQuery() should also consider text if prefix/regex missing (elastic#23451)
  Docs: Specify that byte units use powers of 1024 (elastic#23574)
  Remove Settings.settingsBuilder (elastic#23575)
* master:
  Eclipse: move print margin to 100 columns
  Add support for fragment_length in the unified highlighter (elastic#23431)
@jasontedor
Copy link
Member Author

Thanks for the feedback @nik9000 and @bleskes. I've pushed commits addressing your feedback. Would you please take another look?

Copy link
Contributor

@bleskes bleskes left a comment

Choose a reason for hiding this comment

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

Thx @jasontedor looks very good. I left some nitting but one important comment in the translog commit code.

*/
public static final Setting<ByteSizeValue> INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING =
Setting.byteSizeSetting(
"index.translog.generation_threshold_size",
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: wondering if this should just be index.translog.generation_size

Copy link
Member Author

Choose a reason for hiding this comment

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

I picked index.translog.generation_threshold_size to be consistent with index.translog.flush_threshold_size. Do you still wonder if it should be changed?

Copy link
Contributor

Choose a reason for hiding this comment

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

We can keep as is then. I don't mind much.

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 maybe document what happens if that size is exceeded or add a link to the explain?

Copy link
Contributor

Choose a reason for hiding this comment

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

I wanna understand why this is 64MB why can't we use INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING or it's default instead rather than an arbitrary value? I would love to get some insight that we also can document? Then there is also the question why we can't simply flush this from the outside via IndexShard#maybeFlush, that would simplify the translog potentially, one property that I liked about the Translog#add() operation was that it would never acquire a write lock. We are now entering potentially dangerous territory here, locking can be a beast.

Copy link
Contributor

Choose a reason for hiding this comment

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

I wanna understand why this is 64MB

Jason can say why he chose 64MB - I think any value we choose now will be arbitrary so I was good with it for now and tweak later if needed.

why can't we use INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING or it's default instead rather than an arbitrary value?

We want a number that's much smaller than the flush size. We're heading towards keeping the generations that are needed to recover all seq# ops after a certain point. That means that we won't always clean previous generation when flushing. If we use the flush size for generations we can end up in a poisonous where we repeatedly try to flush but the translog is not trimmed.

why we can't simply flush this from the outside via IndexShard#maybeFlush

Do you mean doing both flushing and potentially opening a new generation from the same method that is called after indexing? i.e., call it maybeFlushAndRoll ( :) ) ? I would be good with that (though prefer the current approach).

Copy link
Contributor

Choose a reason for hiding this comment

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

Do you mean doing both flushing and potentially opening a new generation from the same method that is called after indexing? i.e., call it maybeFlushAndRoll ( :) ) ? I would be good with that (though prefer the current approach).

I was looking into having a Translog#rollover() method we can from the maybeFlush we can rename maybeFlush to onAfterOpteration to not necessarily yield impl details. I am a bit concerned about the write lock, which might not be a problem today but maybe tomorrow.

We want a number that's much smaller than the flush size. We're heading towards keeping the generations that are needed to recover all seq# ops after a certain point.

makes sense. lets document this. it's not clear from reviewing the PR

Copy link
Member Author

Choose a reason for hiding this comment

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

I pushed dd14ba8 to move the control of rolling to index shard. Would @nik9000, @bleskes, and @s1monw please take another look?

assert readers.stream().anyMatch(r -> r.getGeneration() == currentCommittingGeneration)
: "readers missing committing generation [" + currentCommittingGeneration + "]";
// set the last committed generation otherwise old files will not be cleaned up
lastCommittedTranslogFileGeneration = current.getGeneration();
Copy link
Contributor

Choose a reason for hiding this comment

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

I think this is wrong - if we end up rolling during a commit we need to set lastCommittedTranslogFileGeneration to currentCommittingGeneration

Copy link
Member Author

Choose a reason for hiding this comment

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

Good catch (adding multiple rolls between pre-commit and commit catches it too)! I pushed 1669224.

@@ -370,6 +370,27 @@ public void testTranslogFlushSizeThreshold() {
assertEquals(actualNewTranslogFlushThresholdSize, settings.getFlushThresholdSize());
}

public void testTranslogGenerationSizeThreshold() {
final ByteSizeValue size = new ByteSizeValue(Math.abs(randomInt()));
Copy link
Contributor

Choose a reason for hiding this comment

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

should we protect against 0? Thinking about it more, we should at least fit one op in there. So I wonder if we should have a decent lower bound. The problem in making it too big is testing - how about making it 1kb?

Copy link
Member Author

Choose a reason for hiding this comment

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

Is it necessary? Even at zero one operation can squeeze into the generation before rolling (we do not check until after an operation is added to the translog). I like it small for testing, it's easier to think about. If you feel strongly that we should lower bound it, I'm fine.

Copy link
Contributor

Choose a reason for hiding this comment

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

I'm on the fence. Ideally we should protect from abusing a production system. Let's keep things as they are and not over engineer this. Thanks for pointing out that a single op will always go in there.

}

try (ReleasableLock ignored = translog.writeLock.acquire()) {
translog.rollGeneration();
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 do multiple of this?

Copy link
Member Author

Choose a reason for hiding this comment

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

I pushed 1669224.

Copy link
Contributor

@s1monw s1monw left a comment

Choose a reason for hiding this comment

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

I left some comments

*/
public static final Setting<ByteSizeValue> INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING =
Setting.byteSizeSetting(
"index.translog.generation_threshold_size",
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 maybe document what happens if that size is exceeded or add a link to the explain?

@@ -461,6 +479,10 @@ public TimeValue getGlobalCheckpointInterval() {
*/
public ByteSizeValue getFlushThresholdSize() { return flushThresholdSize; }

public ByteSizeValue getGenerationThresholdSize() {
Copy link
Contributor

Choose a reason for hiding this comment

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

javadocs please

Copy link
Member Author

Choose a reason for hiding this comment

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

I pushed 35d0119.

this.rollGeneration();
}
} finally {
final boolean wasRolling = rollingGeneration.getAndSet(false);
Copy link
Contributor

Choose a reason for hiding this comment

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

we have a similar logic in IndexShard#maybeFlush() which I think we should extract into a utility class and we can test separately and reuse in both places. The logic looks pretty much the same and given how error prone that is we should really make sure we isolate and reuse.

* @throws IOException if an I/O exception occurred during any file operations
*/
void rollGeneration() throws IOException {
assert writeLock.isHeldByCurrentThread();
Copy link
Contributor

Choose a reason for hiding this comment

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

maybe allow this assert to give us a message?

Copy link
Member Author

Choose a reason for hiding this comment

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

I pushed 32afd9e.

*/
public static final Setting<ByteSizeValue> INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING =
Setting.byteSizeSetting(
"index.translog.generation_threshold_size",
Copy link
Contributor

Choose a reason for hiding this comment

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

I wanna understand why this is 64MB why can't we use INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING or it's default instead rather than an arbitrary value? I would love to get some insight that we also can document? Then there is also the question why we can't simply flush this from the outside via IndexShard#maybeFlush, that would simplify the translog potentially, one property that I liked about the Translog#add() operation was that it would never acquire a write lock. We are now entering potentially dangerous territory here, locking can be a beast.

* master:
  Fix typo in allocation explain API docs
  Add unit tests for ReverseNestedAggregator (elastic#23651)
  Revert "Revert "Build: Upgrade min gradle to 3.3 (elastic#23544)""
  Revert "Build: Upgrade min gradle to 3.3 (elastic#23544)"
  Build: Upgrade min gradle to 3.3 (elastic#23544)
  Fix took assertion in response filter test
  Search took time should use a relative clock
  Adds toString() to snapshot operations in progress
  Docs: fix a typo in transport client's put-mapping.asciidoc (elastic#23607)
  Use include-tagged macro for high level client docs (elastic#23438)
  Update fill-column in .dir-locals.el to 100 characters
  Setup keystore during integration tests (elastic#22966)
  Fix typo 'Elastisearch' -> 'Elasticsearch' (elastic#23633)
  Comment and blank line cleanups (elastic#23647)
  docs: guidelines for students and teachers (elastic#23648)
  Fix MapperService StackOverflowError (elastic#23605)
* master:
  [API] change wait_for_completion defaults according to docs (elastic#23672)
  Share XContent rendering code in terms aggs (elastic#23680)
  Update ingest-node.asciidoc
  [DOCS] Update the docs about the fact that global ordinals for _parent field are loaded eagerly instead of lazily by default.
  Build: remove progress logger hack for gradle 2.13 (elastic#23679)
  Test: Add dump of integ test cluster logs on failure (elastic#23688)
  Plugins: Add plugin cli specific exit codes (elastic#23599)
  Plugins: Output better error message when existing plugin is incompatible (elastic#23562)
  Reindex: wait for cleanup before responding (elastic#23677)
  Packaging: Remove classpath ordering hack (elastic#23596)
  Docs: Add note about updating plugins requiring removal and reinstallation (elastic#23597)
  Build: Make plugin list for smoke tester dynamic (elastic#23601)
  [TEST] Propertly cleans up failing restore test
* master:
  Docs: Fix language on a few snippets
  Painless: Fix regex lexer and error messages (elastic#23634)
  Skip 5.4 bwc test for new name for now
  Count through the primary in list of strings test
  Skip testing new name if it isn't known
  Wait for all shards in list of strings test
  Deprecate request_cache for clear-cache (elastic#23638)
This reverts commit 192a7ce.
Copy link
Contributor

@s1monw s1monw left a comment

Choose a reason for hiding this comment

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

I left a couple of minors LGTM otherwise

maybeFlushOrRollTranslogGeneration();
}
};
threadPool.executor(ThreadPool.Names.FETCH_SHARD_STARTED).execute(roll);
Copy link
Contributor

Choose a reason for hiding this comment

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

this threadppol name makes not much sense

Copy link
Member Author

Choose a reason for hiding this comment

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

Oops, that looks like an autocomplete failure.

// in that situation we have an extra unexpected flush.
asyncFlushRunning.compareAndSet(true, false);
} else {
public void maybeFlushOrRollTranslogGeneration() {
Copy link
Contributor

Choose a reason for hiding this comment

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

maybe we name this onAfterOperation or something like this? no need to expose the implemenation details here?

@jasontedor jasontedor merged commit b54a9e9 into elastic:master Mar 27, 2017
@jasontedor jasontedor changed the title Introduce translog generation folding Introduce translog generation rolling Mar 27, 2017
@jasontedor jasontedor deleted the translog-generation branch March 27, 2017 20:44
dnhatn added a commit to dnhatn/elasticsearch that referenced this pull request Jan 25, 2018
If the translog flush threshold is too small (eg. smaller than the
translog header), we may repeatedly flush even there is no uncommitted
operation because the shouldFlush condition can still be true after
flushing. This is currently avoided by adding an extra guard against the
uncommitted operations. However, this extra guard makes the shouldFlush
complicated. This commit replaces that extra guard by a lower bound for
translog flush threshold. We keep the lower bound small for convenience
in testing.

Relates elastic#28350
Relates elastic#23606
dnhatn added a commit that referenced this pull request Feb 1, 2018
If the translog flush threshold is too small (eg. smaller than the
translog header), we may repeatedly flush even there is no uncommitted
operation because the shouldFlush condition can still be true after
flushing. This is currently avoided by adding an extra guard against the
uncommitted operations. However, this extra guard makes the shouldFlush
complicated. This commit replaces that extra guard by a lower bound for
translog flush threshold. We keep the lower bound small for convenience
in testing.

Relates #28350
Relates #23606
dnhatn added a commit that referenced this pull request Feb 1, 2018
If the translog flush threshold is too small (eg. smaller than the
translog header), we may repeatedly flush even there is no uncommitted
operation because the shouldFlush condition can still be true after
flushing. This is currently avoided by adding an extra guard against the
uncommitted operations. However, this extra guard makes the shouldFlush
complicated. This commit replaces that extra guard by a lower bound for
translog flush threshold. We keep the lower bound small for convenience
in testing.

Relates #28350
Relates #23606
@clintongormley clintongormley added :Distributed Indexing/Distributed A catch all label for anything in the Distributed Area. Please avoid if you can. :Distributed Indexing/Engine Anything around managing Lucene and the Translog in an open shard. and removed :Translog :Distributed Indexing/Distributed A catch all label for anything in the Distributed Area. Please avoid if you can. labels Feb 13, 2018
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
:Distributed Indexing/Engine Anything around managing Lucene and the Translog in an open shard. >enhancement v6.0.0-alpha1
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants