-
Notifications
You must be signed in to change notification settings - Fork 450
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
Add test for tserver.wal.max.referenced #5250
base: 2.1
Are you sure you want to change the base?
Changes from all commits
2feaa45
a176b85
3bc2502
449e457
bf484ec
97e9e38
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,135 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* https://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.accumulo.test; | ||
|
||
import java.time.Duration; | ||
import java.util.SortedSet; | ||
import java.util.TreeSet; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
|
||
import org.apache.accumulo.core.client.Accumulo; | ||
import org.apache.accumulo.core.client.AccumuloClient; | ||
import org.apache.accumulo.core.client.BatchWriter; | ||
import org.apache.accumulo.core.client.BatchWriterConfig; | ||
import org.apache.accumulo.core.client.admin.NewTableConfiguration; | ||
import org.apache.accumulo.core.conf.Property; | ||
import org.apache.accumulo.core.data.Mutation; | ||
import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; | ||
import org.apache.accumulo.server.ServerContext; | ||
import org.apache.accumulo.server.log.WalStateManager; | ||
import org.apache.accumulo.test.functional.ConfigurableMacBase; | ||
import org.apache.accumulo.test.util.Wait; | ||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.fs.RawLocalFileSystem; | ||
import org.apache.hadoop.io.Text; | ||
import org.junit.jupiter.api.Test; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
/** | ||
* Test that verifies the behavior of {@link Property#TSERV_WAL_MAX_REFERENCED}. | ||
* <p> | ||
* This test creates a table with splits and writes data in batches until the number of WALs in use | ||
* exceeds the configured limit. It then waits for minor compactions to reduce the WAL count. | ||
*/ | ||
public class MaxWalReferencedIT extends ConfigurableMacBase { | ||
private static final Logger log = LoggerFactory.getLogger(MaxWalReferencedIT.class); | ||
|
||
final int WAL_MAX_REFERENCED = 3; | ||
final int hdfsMinBlockSize = 1048576; | ||
|
||
@Override | ||
protected Duration defaultTimeout() { | ||
return Duration.ofMinutes(4); | ||
} | ||
|
||
@Override | ||
protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) { | ||
// Set a small WAL size so we roll frequently | ||
cfg.setProperty(Property.TSERV_WAL_MAX_SIZE, Integer.toString(hdfsMinBlockSize)); | ||
// Set the max number of WALs that can be referenced | ||
cfg.setProperty(Property.TSERV_WAL_MAX_REFERENCED, Integer.toString(WAL_MAX_REFERENCED)); | ||
cfg.setProperty(Property.TSERV_MAXMEM, "256M"); // avoid minor compactions via low memory | ||
cfg.setNumTservers(1); | ||
|
||
// Use raw local file system so WAL syncs and flushes work as expected | ||
hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName()); | ||
} | ||
|
||
@Test | ||
public void testWALMaxReferenced() throws Exception { | ||
try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) { | ||
String tableName = getUniqueNames(1)[0]; | ||
|
||
SortedSet<Text> splits = new TreeSet<>(); | ||
for (int i = 1; i <= 4; i++) { | ||
splits.add(new Text(Integer.toString(i))); | ||
} | ||
client.tableOperations().create(tableName, new NewTableConfiguration().withSplits(splits)); | ||
|
||
log.info("Created table {} with splits. Now writing data.", tableName); | ||
|
||
// Write data multiple times until we see the WAL count exceed WAL_MAX_REFERENCED | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The way this writes data it seems out of sync w/ the tablets. Would be good to push a lot of data to single tablet until a new non-open log has been produced and then move to the next tablet. |
||
AtomicInteger iteration = new AtomicInteger(0); | ||
Wait.waitFor(() -> { | ||
|
||
// Write data that should fill or partially fill the WAL | ||
writeData(client, tableName); | ||
|
||
// Check the current number of WALs in use | ||
long walCount = getWalCount(getServerContext()); | ||
log.info("After iteration {}, WAL count is {}", iteration, walCount); | ||
iteration.getAndIncrement(); | ||
|
||
if (walCount > WAL_MAX_REFERENCED) { | ||
log.info("Reached WAL count of {}, now wait for minor compactions to reduce WAL count", | ||
walCount); | ||
return true; | ||
} else { | ||
return false; | ||
} | ||
}, 60000, 10, "Expected to see WAL count exceed " + WAL_MAX_REFERENCED); | ||
|
||
// wait for minor compactions to reduce the WAL count | ||
Wait.waitFor(() -> getWalCount(getServerContext()) <= WAL_MAX_REFERENCED, 30000, 1000, | ||
"WAL count never dropped within 30 seconds"); | ||
} | ||
} | ||
|
||
/** | ||
* Writes data to a single tablet until the total written data size exceeds 2 * TSERV_WAL_MAX_SIZE | ||
*/ | ||
private void writeData(AccumuloClient client, String table) throws Exception { | ||
try (BatchWriter bw = client.createBatchWriter(table, new BatchWriterConfig())) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could modify this method to write 2XTSERV_WAL_MAX_SIZE data to a single tablet by generating rows that fall within a target tablet and using the Mutation.estimatedMemoryUsed() method. while(totalWritten < 2*hdfsMinBlockSize){
Mutation m = ...
bw.addMutation(m);
totalWritten+=m.estimateMemoryUsed();
} Could use the same row, it does not matter for the walog it will keep adding it even if it is the same row. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Added in 3bc2502 |
||
long totalWritten = 0; | ||
while (totalWritten < 2 * hdfsMinBlockSize) { | ||
Mutation m = new Mutation("target_row"); | ||
m.put("cf", "cq", "value"); | ||
bw.addMutation(m); | ||
totalWritten += m.estimatedMemoryUsed(); | ||
} | ||
} | ||
} | ||
|
||
private long getWalCount(ServerContext context) throws Exception { | ||
return new WalStateManager(context).getAllState().values().stream() | ||
.filter(walState -> walState != WalStateManager.WalState.OPEN).count(); | ||
} | ||
|
||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
May want to increase the amount of memory that tservers can use to hold new data in memory. If this is too low then tservers will minor compaction because memory is low and not because of walogs.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added in 449e457