[HUDI-2793] Fixing deltastreamer checkpoint fetch/copy over (#4034)
- Removed the copy over logic in transaction utils. Deltastreamer will go back to previous commits and get the checkpoint value.
This commit is contained in:
committed by
GitHub
parent
ff94d92980
commit
435ea1543c
@@ -18,11 +18,8 @@
|
||||
|
||||
package org.apache.hudi.client.utils;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import org.apache.hudi.client.transaction.ConflictResolutionStrategy;
|
||||
import org.apache.hudi.client.transaction.ConcurrentOperation;
|
||||
import org.apache.hudi.client.transaction.ConflictResolutionStrategy;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
@@ -34,9 +31,12 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieWriteConflictException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class TransactionUtils {
|
||||
@@ -45,6 +45,7 @@ public class TransactionUtils {
|
||||
|
||||
/**
|
||||
* Resolve any write conflicts when committing data.
|
||||
*
|
||||
* @param table
|
||||
* @param currentTxnOwnerInstant
|
||||
* @param thisCommitMetadata
|
||||
@@ -54,11 +55,11 @@ public class TransactionUtils {
|
||||
* @throws HoodieWriteConflictException
|
||||
*/
|
||||
public static Option<HoodieCommitMetadata> resolveWriteConflictIfAny(
|
||||
final HoodieTable table,
|
||||
final Option<HoodieInstant> currentTxnOwnerInstant,
|
||||
final Option<HoodieCommitMetadata> thisCommitMetadata,
|
||||
final HoodieWriteConfig config,
|
||||
Option<HoodieInstant> lastCompletedTxnOwnerInstant) throws HoodieWriteConflictException {
|
||||
final HoodieTable table,
|
||||
final Option<HoodieInstant> currentTxnOwnerInstant,
|
||||
final Option<HoodieCommitMetadata> thisCommitMetadata,
|
||||
final HoodieWriteConfig config,
|
||||
Option<HoodieInstant> lastCompletedTxnOwnerInstant) throws HoodieWriteConflictException {
|
||||
if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) {
|
||||
ConflictResolutionStrategy resolutionStrategy = config.getWriteConflictResolutionStrategy();
|
||||
Stream<HoodieInstant> instantStream = resolutionStrategy.getCandidateInstants(table.getActiveTimeline(), currentTxnOwnerInstant.get(), lastCompletedTxnOwnerInstant);
|
||||
@@ -68,7 +69,7 @@ public class TransactionUtils {
|
||||
ConcurrentOperation otherOperation = new ConcurrentOperation(instant, table.getMetaClient());
|
||||
if (resolutionStrategy.hasConflict(thisOperation, otherOperation)) {
|
||||
LOG.info("Conflict encountered between current instant = " + thisOperation + " and instant = "
|
||||
+ otherOperation + ", attempting to resolve it...");
|
||||
+ otherOperation + ", attempting to resolve it...");
|
||||
resolutionStrategy.resolveConflict(table, thisOperation, otherOperation);
|
||||
}
|
||||
} catch (IOException io) {
|
||||
@@ -77,10 +78,6 @@ public class TransactionUtils {
|
||||
});
|
||||
LOG.info("Successfully resolved conflicts, if any");
|
||||
|
||||
if (config.mergeDeltastreamerStateFromPreviousCommit()) {
|
||||
mergeCheckpointStateFromPreviousCommit(table.getMetaClient(), thisOperation.getCommitMetadataOption());
|
||||
}
|
||||
|
||||
return thisOperation.getCommitMetadataOption();
|
||||
}
|
||||
return thisCommitMetadata;
|
||||
@@ -88,6 +85,7 @@ public class TransactionUtils {
|
||||
|
||||
/**
|
||||
* Get the last completed transaction hoodie instant and {@link HoodieCommitMetadata#getExtraMetadata()}.
|
||||
*
|
||||
* @param metaClient
|
||||
* @return
|
||||
*/
|
||||
@@ -117,30 +115,4 @@ public class TransactionUtils {
|
||||
throw new HoodieIOException("Unable to read metadata for instant " + hoodieInstantOption.get(), io);
|
||||
}
|
||||
}
|
||||
|
||||
protected static void mergeCheckpointStateFromPreviousCommit(HoodieTableMetaClient metaClient, Option<HoodieCommitMetadata> thisMetadata) {
|
||||
overrideWithLatestCommitMetadata(metaClient, thisMetadata, Collections.singletonList(HoodieWriteConfig.DELTASTREAMER_CHECKPOINT_KEY));
|
||||
}
|
||||
|
||||
/**
|
||||
* Generic method allowing us to override the current metadata with the metadata from
|
||||
* the latest instant for the specified key prefixes.
|
||||
* @param metaClient
|
||||
* @param thisMetadata
|
||||
* @param keyPrefixes The key prefixes to merge from the previous commit
|
||||
*/
|
||||
private static void overrideWithLatestCommitMetadata(HoodieTableMetaClient metaClient,
|
||||
Option<HoodieCommitMetadata> thisMetadata,
|
||||
List<String> keyPrefixes) {
|
||||
if (keyPrefixes.size() == 1 && keyPrefixes.get(0).length() < 1) {
|
||||
return;
|
||||
}
|
||||
Option<Pair<HoodieInstant, Map<String, String>>> lastInstant = getLastCompletedTxnInstantAndMetadata(metaClient);
|
||||
if (lastInstant.isPresent() && thisMetadata.isPresent()) {
|
||||
Stream<String> lastCommitMetadataKeys = lastInstant.get().getRight().keySet().stream();
|
||||
keyPrefixes.stream().forEach(keyPrefix -> lastCommitMetadataKeys
|
||||
.filter(key -> key.startsWith(keyPrefix))
|
||||
.forEach(key -> thisMetadata.get().getExtraMetadata().put(key, lastInstant.get().getRight().get(key))));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -378,14 +378,6 @@ public class HoodieWriteConfig extends HoodieConfig {
|
||||
+ "OPTIMISTIC_CONCURRENCY_CONTROL: Multiple writers can operate on the table and exactly one of them succeed "
|
||||
+ "if a conflict (writes affect the same file group) is detected.");
|
||||
|
||||
public static final ConfigProperty<Boolean> WRITE_CONCURRENCY_MERGE_DELTASTREAMER_STATE = ConfigProperty
|
||||
.key("hoodie.write.concurrency.merge.deltastreamer.state")
|
||||
.defaultValue(false)
|
||||
.withAlternatives("hoodie.write.meta.key.prefixes")
|
||||
.withDocumentation("If enabled, this writer will merge Deltastreamer state from the previous checkpoint in order to allow both realtime "
|
||||
+ "and batch writers to ingest into a single table. This should not be enabled on Deltastreamer writers. Enabling this config means,"
|
||||
+ "for a spark writer, deltastreamer checkpoint will be copied over from previous commit to the current one.");
|
||||
|
||||
/**
|
||||
* Currently the use this to specify the write schema.
|
||||
*/
|
||||
@@ -1778,10 +1770,6 @@ public class HoodieWriteConfig extends HoodieConfig {
|
||||
return WriteConcurrencyMode.fromValue(getString(WRITE_CONCURRENCY_MODE));
|
||||
}
|
||||
|
||||
public Boolean mergeDeltastreamerStateFromPreviousCommit() {
|
||||
return getBoolean(HoodieWriteConfig.WRITE_CONCURRENCY_MERGE_DELTASTREAMER_STATE);
|
||||
}
|
||||
|
||||
public Boolean inlineTableServices() {
|
||||
return inlineClusteringEnabled() || inlineCompactionEnabled() || isAutoClean();
|
||||
}
|
||||
|
||||
@@ -1,98 +0,0 @@
|
||||
/*
|
||||
* 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
|
||||
*
|
||||
* http://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.hudi.utils;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.client.utils.TransactionUtils;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
public class TestTransactionUtils extends HoodieCommonTestHarness {
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
init();
|
||||
}
|
||||
|
||||
public void init() throws Exception {
|
||||
initPath();
|
||||
initMetaClient();
|
||||
metaClient.getFs().mkdirs(new Path(basePath));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckpointStateMerge() throws IOException {
|
||||
HoodieActiveTimeline timeline = new HoodieActiveTimeline(metaClient);
|
||||
|
||||
// Create completed commit with deltastreamer checkpoint state
|
||||
HoodieInstant commitInstantWithCheckpointState = new HoodieInstant(
|
||||
true,
|
||||
HoodieTimeline.COMMIT_ACTION,
|
||||
HoodieActiveTimeline.createNewInstantTime()
|
||||
);
|
||||
timeline.createNewInstant(commitInstantWithCheckpointState);
|
||||
|
||||
HoodieCommitMetadata metadataWithCheckpoint = new HoodieCommitMetadata();
|
||||
String checkpointVal = "00001";
|
||||
metadataWithCheckpoint.addMetadata(HoodieWriteConfig.DELTASTREAMER_CHECKPOINT_KEY, checkpointVal);
|
||||
timeline.saveAsComplete(
|
||||
commitInstantWithCheckpointState,
|
||||
Option.of(metadataWithCheckpoint.toJsonString().getBytes(StandardCharsets.UTF_8))
|
||||
);
|
||||
|
||||
// Inflight commit without checkpoint metadata
|
||||
HoodieInstant commitInstantWithoutCheckpointState = new HoodieInstant(
|
||||
true,
|
||||
HoodieTimeline.COMMIT_ACTION,
|
||||
HoodieActiveTimeline.createNewInstantTime()
|
||||
);
|
||||
timeline.createNewInstant(commitInstantWithoutCheckpointState);
|
||||
HoodieCommitMetadata metadataWithoutCheckpoint = new HoodieCommitMetadata();
|
||||
|
||||
// Ensure that checkpoint state is merged in from previous completed commit
|
||||
MockTransactionUtils.assertCheckpointStateWasMerged(metaClient, metadataWithoutCheckpoint, checkpointVal);
|
||||
}
|
||||
|
||||
private static class MockTransactionUtils extends TransactionUtils {
|
||||
|
||||
public static void assertCheckpointStateWasMerged(
|
||||
HoodieTableMetaClient metaClient,
|
||||
HoodieCommitMetadata currentMetadata,
|
||||
String expectedCheckpointState) {
|
||||
TransactionUtils.mergeCheckpointStateFromPreviousCommit(metaClient, Option.of(currentMetadata));
|
||||
assertEquals(
|
||||
expectedCheckpointState,
|
||||
currentMetadata.getExtraMetadata().get(HoodieWriteConfig.DELTASTREAMER_CHECKPOINT_KEY)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user