Skip to content
Merged
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@

package org.apache.hudi.client.utils;

import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import org.apache.hudi.client.transaction.ConflictResolutionStrategy;
Expand Down Expand Up @@ -53,8 +53,12 @@ public class TransactionUtils {
* @return
* @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 {
public static Option<HoodieCommitMetadata> resolveWriteConflictIfAny(
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);
Expand All @@ -72,8 +76,11 @@ public static Option<HoodieCommitMetadata> resolveWriteConflictIfAny(final Hoodi
}
});
LOG.info("Successfully resolved conflicts, if any");
// carry over necessary metadata from latest commit metadata
overrideWithLatestCommitMetadata(table.getMetaClient(), thisOperation.getCommitMetadataOption(), currentTxnOwnerInstant, Arrays.asList(config.getWriteMetaKeyPrefixes().split(",")));

if (config.mergeDeltastreamerStateFromPreviousCommit()) {
mergeCheckpointStateFromPreviousCommit(table.getMetaClient(), thisOperation.getCommitMetadataOption());
}

return thisOperation.getCommitMetadataOption();
}
return thisCommitMetadata;
Expand Down Expand Up @@ -111,16 +118,27 @@ public static Option<Pair<HoodieInstant, Map<String, String>>> getLastCompletedT
}
}

// override the current metadata with the metadata from the latest instant for the specified key prefixes
private static void overrideWithLatestCommitMetadata(HoodieTableMetaClient metaClient, Option<HoodieCommitMetadata> thisMetadata,
Option<HoodieInstant> thisInstant, List<String> keyPrefixes) {
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> keys = thisMetadata.get().getExtraMetadata().keySet().stream();
keyPrefixes.stream().forEach(keyPrefix -> keys
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))));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,10 @@ public class HoodieWriteConfig extends HoodieConfig {

private static final long serialVersionUID = 0L;

// This is a constant as is should never be changed via config (will invalidate previous commits)
// It is here so that both the client and deltastreamer use the same reference
public static final String DELTASTREAMER_CHECKPOINT_KEY = "deltastreamer.checkpoint.key";

public static final ConfigProperty<String> TBL_NAME = ConfigProperty
.key("hoodie.table.name")
.noDefaultValue()
Expand Down Expand Up @@ -368,11 +372,13 @@ 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<String> WRITE_META_KEY_PREFIXES = ConfigProperty
.key("hoodie.write.meta.key.prefixes")
.defaultValue("")
.withDocumentation("Comma separated metadata key prefixes to override from latest commit "
+ "during overlapping commits via multi writing");
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.
Expand Down Expand Up @@ -783,16 +789,6 @@ public class HoodieWriteConfig extends HoodieConfig {
*/
@Deprecated
public static final String DEFAULT_WRITE_CONCURRENCY_MODE = WRITE_CONCURRENCY_MODE.defaultValue();
/**
* @deprecated Use {@link #WRITE_META_KEY_PREFIXES} and its methods instead
*/
@Deprecated
public static final String WRITE_META_KEY_PREFIXES_PROP = WRITE_META_KEY_PREFIXES.key();
/**
* @deprecated Use {@link #WRITE_META_KEY_PREFIXES} and its methods instead
*/
@Deprecated
public static final String DEFAULT_WRITE_META_KEY_PREFIXES = WRITE_META_KEY_PREFIXES.defaultValue();
/**
* @deprecated Use {@link #ALLOW_MULTI_WRITE_ON_SAME_INSTANT_ENABLE} and its methods instead
*/
Expand Down Expand Up @@ -1764,12 +1760,12 @@ public WriteConcurrencyMode getWriteConcurrencyMode() {
return WriteConcurrencyMode.fromValue(getString(WRITE_CONCURRENCY_MODE));
}

public Boolean inlineTableServices() {
return inlineClusteringEnabled() || inlineCompactionEnabled() || isAutoClean();
public Boolean mergeDeltastreamerStateFromPreviousCommit() {
return getBoolean(HoodieWriteConfig.WRITE_CONCURRENCY_MERGE_DELTASTREAMER_STATE);
}

public String getWriteMetaKeyPrefixes() {
return getString(WRITE_META_KEY_PREFIXES);
public Boolean inlineTableServices() {
return inlineClusteringEnabled() || inlineCompactionEnabled() || isAutoClean();
}

public String getPreCommitValidators() {
Expand Down Expand Up @@ -2131,11 +2127,6 @@ public Builder withWriteConcurrencyMode(WriteConcurrencyMode concurrencyMode) {
return this;
}

public Builder withWriteMetaKeyPrefixes(String writeMetaKeyPrefixes) {
writeConfig.setValue(WRITE_META_KEY_PREFIXES, writeMetaKeyPrefixes);
return this;
}

public Builder withPopulateMetaFields(boolean populateMetaFields) {
writeConfig.setValue(HoodieTableConfig.POPULATE_META_FIELDS, Boolean.toString(populateMetaFields));
return this;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,98 @@
/*
* 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)
);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,7 @@
import static org.apache.hudi.config.HoodieWriteConfig.AUTO_COMMIT_ENABLE;
import static org.apache.hudi.config.HoodieWriteConfig.COMBINE_BEFORE_INSERT;
import static org.apache.hudi.config.HoodieWriteConfig.COMBINE_BEFORE_UPSERT;
import static org.apache.hudi.config.HoodieWriteConfig.WRITE_CONCURRENCY_MERGE_DELTASTREAMER_STATE;
import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY;
import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_RESET_KEY;
import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE;
Expand Down Expand Up @@ -715,6 +716,11 @@ private HoodieWriteConfig getHoodieClientConfig(Schema schema) {
String.format("%s should be set to %s", COMBINE_BEFORE_INSERT.key(), cfg.filterDupes));
ValidationUtils.checkArgument(config.shouldCombineBeforeUpsert(),
String.format("%s should be set to %s", COMBINE_BEFORE_UPSERT.key(), combineBeforeUpsert));
ValidationUtils.checkArgument(!config.mergeDeltastreamerStateFromPreviousCommit(),
String.format(
"Deltastreamer processes should not merge state from previous deltastreamer commits. Please unset '%s'",
WRITE_CONCURRENCY_MERGE_DELTASTREAMER_STATE.key())
);

return config;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieClusteringConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.hive.HiveSyncTool;
Expand Down Expand Up @@ -90,7 +91,7 @@ public class HoodieDeltaStreamer implements Serializable {
private static final long serialVersionUID = 1L;
private static final Logger LOG = LogManager.getLogger(HoodieDeltaStreamer.class);

public static final String CHECKPOINT_KEY = "deltastreamer.checkpoint.key";
public static final String CHECKPOINT_KEY = HoodieWriteConfig.DELTASTREAMER_CHECKPOINT_KEY;
public static final String CHECKPOINT_RESET_KEY = "deltastreamer.checkpoint.reset_key";

protected final transient Config cfg;
Expand Down
Loading