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

moves finding split points into FATE #4178

Merged
merged 11 commits into from
Feb 21, 2024
Merged
58 changes: 41 additions & 17 deletions core/src/main/java/org/apache/accumulo/core/fate/Fate.java
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import com.google.common.base.Preconditions;

/**
* Fault tolerant executor
*/
Expand Down Expand Up @@ -334,30 +336,52 @@ public FateId startTransaction() {
return store.create();
}

public Optional<FateId> seedTransaction(String txName, FateKey fateKey, Repo<T> repo,
boolean autoCleanUp, String goalMessage) {

Optional<FateTxStore<T>> optTxStore = store.createAndReserve(fateKey);

return optTxStore.map(txStore -> {
var fateId = txStore.getID();
try {
Preconditions.checkState(txStore.getStatus() == NEW);
seedTransaction(txName, fateId, repo, autoCleanUp, goalMessage, txStore);
} finally {
txStore.unreserve(0, MILLISECONDS);
}
return fateId;
});
}

private void seedTransaction(String txName, FateId fateId, Repo<T> repo, boolean autoCleanUp,
String goalMessage, FateTxStore<T> txStore) {
if (txStore.top() == null) {
try {
log.info("Seeding {} {}", fateId, goalMessage);
txStore.push(repo);
} catch (StackOverflowException e) {
// this should not happen
throw new IllegalStateException(e);
}
}

if (autoCleanUp) {
txStore.setTransactionInfo(TxInfo.AUTO_CLEAN, autoCleanUp);
}

txStore.setTransactionInfo(TxInfo.TX_NAME, txName);

txStore.setStatus(SUBMITTED);
}

// start work in the transaction.. it is safe to call this
// multiple times for a transaction... but it will only seed once
public void seedTransaction(String txName, FateId fateId, Repo<T> repo, boolean autoCleanUp,
String goalMessage) {
FateTxStore<T> txStore = store.reserve(fateId);
try {
if (txStore.getStatus() == NEW) {
if (txStore.top() == null) {
try {
log.info("Seeding {} {}", fateId, goalMessage);
txStore.push(repo);
} catch (StackOverflowException e) {
// this should not happen
throw new IllegalStateException(e);
}
}

if (autoCleanUp) {
txStore.setTransactionInfo(TxInfo.AUTO_CLEAN, autoCleanUp);
}

txStore.setTransactionInfo(TxInfo.TX_NAME, txName);

txStore.setStatus(SUBMITTED);
seedTransaction(txName, fateId, repo, autoCleanUp, goalMessage, txStore);
}
} finally {
txStore.unreserve(0, TimeUnit.MILLISECONDS);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,6 @@ public enum CacheName {
COMPRESSION_ALGORITHM,
CRYPT_PASSWORDS,
HOST_REGEX_BALANCER_TABLE_REGEX,
HOSTING_REQUEST_CACHE,
INSTANCE_ID,
NAMESPACE_ID,
PROP_CACHE,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -280,9 +280,6 @@ public ThreadPoolExecutor createExecutorService(final AccumuloConfiguration conf
"summary partition", emitThreadPoolMetrics);
case GC_DELETE_THREADS:
return createFixedThreadPool(conf.getCount(p), "deleting", emitThreadPoolMetrics);
case MANAGER_SPLIT_WORKER_THREADS:
return createFixedThreadPool(conf.getCount(p), "tablet split inspection",
emitThreadPoolMetrics);
default:
throw new IllegalArgumentException("Unhandled thread pool property: " + p);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@
import org.apache.accumulo.core.util.threads.Threads;
import org.apache.accumulo.core.util.threads.Threads.AccumuloDaemonThread;
import org.apache.accumulo.manager.metrics.ManagerMetrics;
import org.apache.accumulo.manager.split.SplitTask;
import org.apache.accumulo.manager.split.SeedSplitTask;
import org.apache.accumulo.manager.state.TableCounts;
import org.apache.accumulo.manager.state.TableStats;
import org.apache.accumulo.manager.upgrade.UpgradeCoordinator;
Expand Down Expand Up @@ -521,17 +521,7 @@ private TableMgmtStats manageTablets(Iterator<TabletManagement> iter,

if (actions.contains(ManagementAction.NEEDS_SPLITTING)) {
LOG.debug("{} may need splitting.", tm.getExtent());
if (manager.getSplitter().isSplittable(tm)) {
if (manager.getSplitter().addSplitStarting(tm.getExtent())) {
LOG.debug("submitting tablet {} for split", tm.getExtent());
manager.getSplitter().executeSplit(new SplitTask(manager.getContext(), tm, manager));
}
} else {
LOG.debug("{} is not splittable.", tm.getExtent());
}
// ELASITICITY_TODO: See #3605. Merge is non-functional. Left this commented out code to
// show where merge used to make a call to split a tablet.
// sendSplitRequest(mergeStats.getMergeInfo(), state, tm);
manager.getSplitter().initiateSplit(new SeedSplitTask(manager, tm.getExtent()));
}

if (actions.contains(ManagementAction.NEEDS_COMPACTING)) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* 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.manager.split;

import java.util.Optional;

import org.apache.accumulo.core.dataImpl.KeyExtent;
import org.apache.accumulo.core.fate.FateId;
import org.apache.accumulo.core.fate.FateInstanceType;
import org.apache.accumulo.core.fate.FateKey;
import org.apache.accumulo.manager.Manager;
import org.apache.accumulo.manager.tableOps.split.FindSplits;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class SeedSplitTask implements Runnable {

private static final Logger log = LoggerFactory.getLogger(SeedSplitTask.class);
private final Manager manager;
private KeyExtent extent;

public SeedSplitTask(Manager manager, KeyExtent extent) {
this.manager = manager;
this.extent = extent;
}

@Override
public void run() {
try {
var fateInstanceType = FateInstanceType.fromTableId((extent.tableId()));

Optional<FateId> optFateId =
manager.fate(fateInstanceType).seedTransaction("SYSTEM_SPLIT", FateKey.forSplit(extent),
new FindSplits(extent), true, "System initiated split of tablet " + extent);

optFateId.ifPresentOrElse(fateId -> {
log.trace("System initiated a split for : {} {}", extent, fateId);
}, () -> {
log.trace("System attempted to initiate a split but one was in progress : {}", extent);
});

} catch (Exception e) {
log.error("Failed to split {}", extent, e);
}
}
}

This file was deleted.

Loading