Skip to content

Commit

Permalink
Makes seeding a fate transaction more efficient (apache#5122)
Browse files Browse the repository at this point in the history
Modified fate to seed fate transaction in single conditional mutation
instead of multiple.

fixes apache#5097


Co-authored-by: Kevin Rathbun <[email protected]>
  • Loading branch information
keith-turner and kevinrr888 authored Dec 10, 2024
1 parent 6dc52bc commit 6407c31
Show file tree
Hide file tree
Showing 16 changed files with 529 additions and 235 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,11 @@ public FateId fromTypeAndKey(FateInstanceType instanceType, FateKey fateKey) {
UUID txUUID = UUID.nameUUIDFromBytes(fateKey.getSerialized());
return FateId.from(instanceType, txUUID);
}

@Override
public FateId newRandomId(FateInstanceType instanceType) {
return FateId.from(instanceType, UUID.randomUUID());
}
};

// The ZooKeeper lock for the process that's running this store instance
Expand Down Expand Up @@ -402,6 +407,12 @@ public FateId getID() {

public interface FateIdGenerator {
FateId fromTypeAndKey(FateInstanceType instanceType, FateKey fateKey);

FateId newRandomId(FateInstanceType instanceType);
}

protected void seededTx() {
unreservedRunnableCount.increment();
}

protected byte[] serializeTxInfo(Serializable so) {
Expand Down
51 changes: 4 additions & 47 deletions core/src/main/java/org/apache/accumulo/core/fate/Fate.java
Original file line number Diff line number Diff line change
Expand Up @@ -63,8 +63,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import com.google.common.base.Preconditions;

/**
* Fault tolerant executor
*/
Expand Down Expand Up @@ -439,57 +437,16 @@ 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(Duration.ZERO);
}
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);
public void seedTransaction(String txName, FateKey fateKey, Repo<T> repo, boolean autoCleanUp) {
store.seedTransaction(txName, fateKey, repo, autoCleanUp);
}

// 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) {
seedTransaction(txName, fateId, repo, autoCleanUp, goalMessage, txStore);
}
} finally {
txStore.unreserve(Duration.ZERO);
}

log.info("Seeding {} {}", fateId, goalMessage);
store.seedTransaction(txName, fateId, repo, autoCleanUp);
}

// check on the transaction
Expand Down
42 changes: 32 additions & 10 deletions core/src/main/java/org/apache/accumulo/core/fate/FateStore.java
Original file line number Diff line number Diff line change
Expand Up @@ -50,19 +50,41 @@ public interface FateStore<T> extends ReadOnlyFateStore<T> {
FateId create();

/**
* Creates and reserves a transaction using the given key. If something is already running for the
* given key, then Optional.empty() will be returned. When this returns a non-empty id, it will be
* in the new state.
* Seeds a transaction with the given repo if it does not exists. A fateId will be derived from
* the fateKey. If seeded, sets the following data for the fateId in the store.
*
* <p>
* In the case where a process dies in the middle of a call to this. If later, another call is
* made with the same key and its in the new state then the FateId for that key will be returned.
* </p>
* <ul>
* <li>Set the tx name</li>
* <li>Set the status to SUBMITTED</li>
* <li>Set the fate key</li>
* <li>Sets autocleanup only if true</li>
* <li>Sets the creation time</li>
* </ul>
*
* @throws IllegalStateException when there is an unexpected collision. This can occur if two key
* hash to the same FateId or if a random FateId already exists.
* @return The return type is only intended for testing it may not be correct in the face of
* failures. When there are no failures returns optional w/ the fate id set if seeded and
* empty optional otherwise. If there was a failure this could return an empty optional
* when it actually succeeded.
*/
Optional<FateTxStore<T>> createAndReserve(FateKey fateKey);
Optional<FateId> seedTransaction(String txName, FateKey fateKey, Repo<T> repo,
boolean autoCleanUp);

/**
* Seeds a transaction with the given repo if its current status is NEW and it is currently
* unreserved. If seeded, sets the following data for the fateId in the store.
*
* <ul>
* <li>Set the tx name</li>
* <li>Set the status to SUBMITTED</li>
* <li>Sets autocleanup only if true</li>
* <li>Sets the creation time</li>
* </ul>
*
* @return The return type is only intended for testing it may not be correct in the face of
* failures. When there are no failures returns true if seeded and false otherwise. If
* there was a failure this could return false when it actually succeeded.
*/
boolean seedTransaction(String txName, FateId fateId, Repo<T> repo, boolean autoCleanUp);

/**
* An interface that allows read/write access to the data related to a single fate operation.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,29 @@ public interface FateMutator<T> {

FateMutator<T> putCreateTime(long ctime);

/**
* Requires that nothing exists for this fate mutation.
*/
FateMutator<T> requireAbsent();

/**
* Require that the transaction status is one of the given statuses. If no statuses are provided,
* require that the status column is absent.
*
* @param statuses The statuses to check against.
*/
FateMutator<T> requireStatus(TStatus... statuses);

/**
* Require the transaction has no reservation.
*/
FateMutator<T> requireUnreserved();

/**
* Require the transaction has no fate key set.
*/
FateMutator<T> requireAbsentKey();

/**
* Add a conditional mutation to {@link FateSchema.TxColumnFamily#RESERVATION_COLUMN} that will
* put the reservation if there is not already a reservation present
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.accumulo.core.client.AccumuloSecurityException;
import org.apache.accumulo.core.client.BatchWriter;
import org.apache.accumulo.core.client.ConditionalWriter;
import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.accumulo.core.client.MutationsRejectedException;
import org.apache.accumulo.core.client.Scanner;
import org.apache.accumulo.core.client.TableNotFoundException;
Expand All @@ -48,12 +49,16 @@
import org.apache.accumulo.core.security.Authorizations;
import org.apache.hadoop.io.Text;

import com.google.common.base.Preconditions;

public class FateMutatorImpl<T> implements FateMutator<T> {

private final ClientContext context;
private final String tableName;
private final FateId fateId;
private final ConditionalMutation mutation;
private boolean requiredUnreserved = false;
public static final int INITIAL_ITERATOR_PRIO = 1000000;

public FateMutatorImpl(ClientContext context, String tableName, FateId fateId) {
this.context = Objects.requireNonNull(context);
Expand Down Expand Up @@ -81,10 +86,34 @@ public FateMutator<T> putCreateTime(long ctime) {
}

@Override
public FateMutator<T> putReservedTx(FateStore.FateReservation reservation) {
public FateMutator<T> requireAbsent() {
IteratorSetting is = new IteratorSetting(INITIAL_ITERATOR_PRIO, RowExistsIterator.class);
Condition c = new Condition("", "").setIterators(is);
mutation.addCondition(c);
return this;
}

@Override
public FateMutator<T> requireUnreserved() {
Preconditions.checkState(!requiredUnreserved);
Condition condition = new Condition(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(),
TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier());
mutation.addCondition(condition);
requiredUnreserved = true;
return this;
}

@Override
public FateMutator<T> requireAbsentKey() {
Condition condition = new Condition(TxColumnFamily.TX_KEY_COLUMN.getColumnFamily(),
TxColumnFamily.TX_KEY_COLUMN.getColumnQualifier());
mutation.addCondition(condition);
return this;
}

@Override
public FateMutator<T> putReservedTx(FateStore.FateReservation reservation) {
requireUnreserved();
TxColumnFamily.RESERVATION_COLUMN.put(mutation, new Value(reservation.getSerialized()));
return this;
}
Expand Down Expand Up @@ -179,12 +208,7 @@ public FateMutator<T> delete() {
return this;
}

/**
* Require that the transaction status is one of the given statuses. If no statuses are provided,
* require that the status column is absent.
*
* @param statuses The statuses to check against.
*/
@Override
public FateMutator<T> requireStatus(TStatus... statuses) {
Condition condition = StatusMappingIterator.createCondition(statuses);
mutation.addCondition(condition);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* 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.core.fate.user;

import java.io.IOException;
import java.util.Collection;
import java.util.Set;

import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.iterators.WrappingIterator;

import com.google.common.base.Preconditions;

/**
* Iterator is used by conditional mutations to check if row exists.
*/
public class RowExistsIterator extends WrappingIterator {

@Override
public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive)
throws IOException {
Preconditions.checkState(range.getStartKey() != null && range.getEndKey() != null);
var startRow = range.getStartKey().getRow();
var endRow = range.getEndKey().getRow();
Preconditions.checkState(startRow.equals(endRow));
Range r = new Range(startRow);
super.seek(r, Set.of(), false);
}
}
Loading

0 comments on commit 6407c31

Please sign in to comment.