[HUDI-845] Added locking capability to allow multiple writers (#2374)
* [HUDI-845] Added locking capability to allow multiple writers 1. Added LockProvider API for pluggable lock methodologies 2. Added Resolution Strategy API to allow for pluggable conflict resolution 3. Added TableService client API to schedule table services 4. Added Transaction Manager for wrapping actions within transactions
This commit is contained in:
@@ -0,0 +1,107 @@
|
||||
/*
|
||||
* 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.client.transaction;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.config.LockConfiguration;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.lock.LockProvider;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieLockException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP;
|
||||
|
||||
/**
|
||||
* This lock provider is used for testing purposes only. It provides a simple file system based lock using HDFS atomic
|
||||
* create operation. This lock does not support cleaning/expiring the lock after a failed write hence cannot be used
|
||||
* in production environments.
|
||||
*/
|
||||
public class FileSystemBasedLockProviderTestClass implements LockProvider<String> {
|
||||
|
||||
private static final String LOCK_NAME = "acquired";
|
||||
|
||||
private String lockPath;
|
||||
private FileSystem fs;
|
||||
protected LockConfiguration lockConfiguration;
|
||||
|
||||
public FileSystemBasedLockProviderTestClass(final LockConfiguration lockConfiguration, final Configuration configuration) {
|
||||
this.lockConfiguration = lockConfiguration;
|
||||
this.lockPath = lockConfiguration.getConfig().getString(FILESYSTEM_LOCK_PATH_PROP);
|
||||
this.fs = FSUtils.getFs(this.lockPath, configuration);
|
||||
}
|
||||
|
||||
public void acquireLock() {
|
||||
try {
|
||||
fs.create(new Path(lockPath + "/" + LOCK_NAME)).close();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to acquire lock", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
try {
|
||||
fs.delete(new Path(lockPath + "/" + LOCK_NAME), true);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieLockException("Unable to release lock", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean tryLock(long time, TimeUnit unit) {
|
||||
try {
|
||||
int numRetries = 0;
|
||||
while (fs.exists(new Path(lockPath + "/" + LOCK_NAME))
|
||||
&& (numRetries <= lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP))) {
|
||||
Thread.sleep(lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP));
|
||||
}
|
||||
acquireLock();
|
||||
return true;
|
||||
} catch (IOException | InterruptedException e) {
|
||||
throw new HoodieLockException("Failed to acquire lock", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void unlock() {
|
||||
try {
|
||||
if (fs.exists(new Path(lockPath + "/" + LOCK_NAME))) {
|
||||
fs.delete(new Path(lockPath + "/" + LOCK_NAME), true);
|
||||
}
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("Unable to delete lock on disk", io);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLock() {
|
||||
try {
|
||||
return fs.listStatus(new Path(lockPath))[0].getPath().toString();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieLockException("Failed to retrieve lock status from lock path " + lockPath);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,420 @@
|
||||
/*
|
||||
* 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.client.transaction;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hudi.avro.model.HoodieClusteringGroup;
|
||||
import org.apache.hudi.avro.model.HoodieClusteringPlan;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieSliceInfo;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieWriteConflictException;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
public class TestSimpleConcurrentFileWritesConflictResolutionStrategy extends HoodieCommonTestHarness {
|
||||
|
||||
@BeforeEach
|
||||
public void init() throws IOException {
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoConcurrentWrites() throws Exception {
|
||||
String newInstantTime = HoodieTestTable.makeNewCommitTime();
|
||||
createCommit(newInstantTime);
|
||||
// consider commits before this are all successful
|
||||
|
||||
Option<HoodieInstant> lastSuccessfulInstant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant();
|
||||
newInstantTime = HoodieTestTable.makeNewCommitTime();
|
||||
Option<HoodieInstant> currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime));
|
||||
|
||||
SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy();
|
||||
Stream<HoodieInstant> candidateInstants = strategy.getCandidateInstants(metaClient.getActiveTimeline(), currentInstant.get(), lastSuccessfulInstant);
|
||||
Assertions.assertTrue(candidateInstants.count() == 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConcurrentWrites() throws Exception {
|
||||
String newInstantTime = HoodieTestTable.makeNewCommitTime();
|
||||
createCommit(newInstantTime);
|
||||
// consider commits before this are all successful
|
||||
// writer 1
|
||||
createInflightCommit(HoodieTestTable.makeNewCommitTime());
|
||||
// writer 2
|
||||
createInflightCommit(HoodieTestTable.makeNewCommitTime());
|
||||
Option<HoodieInstant> lastSuccessfulInstant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant();
|
||||
newInstantTime = HoodieTestTable.makeNewCommitTime();
|
||||
Option<HoodieInstant> currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime));
|
||||
SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy();
|
||||
Stream<HoodieInstant> candidateInstants = strategy.getCandidateInstants(metaClient.getActiveTimeline(), currentInstant.get(), lastSuccessfulInstant);
|
||||
Assertions.assertTrue(candidateInstants.count() == 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConcurrentWritesWithInterleavingSuccesssfulCommit() throws Exception {
|
||||
createCommit(HoodieActiveTimeline.createNewInstantTime());
|
||||
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
|
||||
// consider commits before this are all successful
|
||||
Option<HoodieInstant> lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant();
|
||||
// writer 1 starts
|
||||
String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime();
|
||||
createInflightCommit(currentWriterInstant);
|
||||
// writer 2 starts and finishes
|
||||
String newInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
createCommit(newInstantTime);
|
||||
|
||||
Option<HoodieInstant> currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant));
|
||||
SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy();
|
||||
HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant);
|
||||
timeline = timeline.reload();
|
||||
List<HoodieInstant> candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect(
|
||||
Collectors.toList());
|
||||
// writer 1 conflicts with writer 2
|
||||
Assertions.assertTrue(candidateInstants.size() == 1);
|
||||
ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient);
|
||||
ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata);
|
||||
Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation));
|
||||
try {
|
||||
strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation);
|
||||
Assertions.fail("Cannot reach here, writer 1 and writer 2 should have thrown a conflict");
|
||||
} catch (HoodieWriteConflictException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConcurrentWritesWithInterleavingScheduledCompaction() throws Exception {
|
||||
createCommit(HoodieActiveTimeline.createNewInstantTime());
|
||||
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
|
||||
// consider commits before this are all successful
|
||||
Option<HoodieInstant> lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant();
|
||||
// writer 1 starts
|
||||
String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime();
|
||||
createInflightCommit(currentWriterInstant);
|
||||
// compaction 1 gets scheduled
|
||||
String newInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
createCompactionRequested(newInstantTime);
|
||||
|
||||
Option<HoodieInstant> currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant));
|
||||
SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy();
|
||||
HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant);
|
||||
timeline = timeline.reload();
|
||||
List<HoodieInstant> candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect(
|
||||
Collectors.toList());
|
||||
// writer 1 conflicts with scheduled compaction plan 1
|
||||
Assertions.assertTrue(candidateInstants.size() == 1);
|
||||
ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient);
|
||||
ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata);
|
||||
Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation));
|
||||
try {
|
||||
strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation);
|
||||
Assertions.fail("Cannot reach here, should have thrown a conflict");
|
||||
} catch (HoodieWriteConflictException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConcurrentWritesWithInterleavingSuccessfulCompaction() throws Exception {
|
||||
createCommit(HoodieActiveTimeline.createNewInstantTime());
|
||||
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
|
||||
// consider commits before this are all successful
|
||||
Option<HoodieInstant> lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant();
|
||||
// writer 1 starts
|
||||
String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime();
|
||||
createInflightCommit(currentWriterInstant);
|
||||
// compaction 1 gets scheduled and finishes
|
||||
String newInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
createCompaction(newInstantTime);
|
||||
|
||||
Option<HoodieInstant> currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant));
|
||||
SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy();
|
||||
HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant);
|
||||
timeline = timeline.reload();
|
||||
List<HoodieInstant> candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect(
|
||||
Collectors.toList());
|
||||
// writer 1 conflicts with compaction 1
|
||||
Assertions.assertTrue(candidateInstants.size() == 1);
|
||||
ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient);
|
||||
ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata);
|
||||
Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation));
|
||||
try {
|
||||
strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation);
|
||||
Assertions.fail("Cannot reach here, should have thrown a conflict");
|
||||
} catch (HoodieWriteConflictException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConcurrentWriteAndCompactionScheduledEarlier() throws Exception {
|
||||
createCommit(HoodieActiveTimeline.createNewInstantTime());
|
||||
// compaction 1 gets scheduled
|
||||
String newInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
createCompaction(newInstantTime);
|
||||
// consider commits before this are all successful
|
||||
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
|
||||
Option<HoodieInstant> lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant();
|
||||
// writer 1 starts
|
||||
String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime();
|
||||
createInflightCommit(currentWriterInstant);
|
||||
|
||||
Option<HoodieInstant> currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant));
|
||||
SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy();
|
||||
HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant);
|
||||
timeline = timeline.reload();
|
||||
List<HoodieInstant> candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect(
|
||||
Collectors.toList());
|
||||
// writer 1 should not conflict with an earlier scheduled compaction 1 with the same file ids
|
||||
Assertions.assertTrue(candidateInstants.size() == 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConcurrentWritesWithInterleavingScheduledCluster() throws Exception {
|
||||
createCommit(HoodieActiveTimeline.createNewInstantTime());
|
||||
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
|
||||
// consider commits before this are all successful
|
||||
Option<HoodieInstant> lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant();
|
||||
// writer 1 starts
|
||||
String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime();
|
||||
createInflightCommit(currentWriterInstant);
|
||||
// clustering 1 gets scheduled
|
||||
String newInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
createReplaceRequested(newInstantTime);
|
||||
|
||||
Option<HoodieInstant> currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant));
|
||||
SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy();
|
||||
HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant);
|
||||
timeline = timeline.reload();
|
||||
List<HoodieInstant> candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect(
|
||||
Collectors.toList());
|
||||
// writer 1 conflicts with scheduled compaction plan 1
|
||||
Assertions.assertTrue(candidateInstants.size() == 1);
|
||||
ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient);
|
||||
ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata);
|
||||
Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation));
|
||||
try {
|
||||
strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation);
|
||||
Assertions.fail("Cannot reach here, should have thrown a conflict");
|
||||
} catch (HoodieWriteConflictException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConcurrentWritesWithInterleavingSuccessfulCluster() throws Exception {
|
||||
createCommit(HoodieActiveTimeline.createNewInstantTime());
|
||||
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
|
||||
// consider commits before this are all successful
|
||||
Option<HoodieInstant> lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant();
|
||||
// writer 1 starts
|
||||
String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime();
|
||||
createInflightCommit(currentWriterInstant);
|
||||
// cluster 1 gets scheduled and finishes
|
||||
String newInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
createReplace(newInstantTime, WriteOperationType.CLUSTER);
|
||||
|
||||
Option<HoodieInstant> currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant));
|
||||
SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy();
|
||||
HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant);
|
||||
timeline = timeline.reload();
|
||||
List<HoodieInstant> candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect(
|
||||
Collectors.toList());
|
||||
// writer 1 conflicts with cluster 1
|
||||
Assertions.assertTrue(candidateInstants.size() == 1);
|
||||
ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient);
|
||||
ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata);
|
||||
Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation));
|
||||
try {
|
||||
strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation);
|
||||
Assertions.fail("Cannot reach here, should have thrown a conflict");
|
||||
} catch (HoodieWriteConflictException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConcurrentWritesWithInterleavingSuccessfulReplace() throws Exception {
|
||||
createCommit(HoodieActiveTimeline.createNewInstantTime());
|
||||
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
|
||||
// consider commits before this are all successful
|
||||
Option<HoodieInstant> lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant();
|
||||
// writer 1 starts
|
||||
String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime();
|
||||
createInflightCommit(currentWriterInstant);
|
||||
// replace 1 gets scheduled and finished
|
||||
String newInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
createReplace(newInstantTime, WriteOperationType.INSERT_OVERWRITE);
|
||||
|
||||
Option<HoodieInstant> currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant));
|
||||
SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy();
|
||||
HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant);
|
||||
timeline = timeline.reload();
|
||||
List<HoodieInstant> candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect(
|
||||
Collectors.toList());
|
||||
// writer 1 conflicts with replace 1
|
||||
Assertions.assertTrue(candidateInstants.size() == 1);
|
||||
ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient);
|
||||
ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata);
|
||||
Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation));
|
||||
try {
|
||||
strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation);
|
||||
Assertions.fail("Cannot reach here, should have thrown a conflict");
|
||||
} catch (HoodieWriteConflictException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
private void createCommit(String instantTime) throws Exception {
|
||||
String fileId1 = "file-1";
|
||||
String fileId2 = "file-2";
|
||||
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
commitMetadata.addMetadata("test", "test");
|
||||
HoodieWriteStat writeStat = new HoodieWriteStat();
|
||||
writeStat.setFileId("file-1");
|
||||
commitMetadata.addWriteStat(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, writeStat);
|
||||
commitMetadata.setOperationType(WriteOperationType.INSERT);
|
||||
HoodieTestTable.of(metaClient)
|
||||
.addCommit(instantTime, commitMetadata)
|
||||
.withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2);
|
||||
}
|
||||
|
||||
private HoodieCommitMetadata createCommitMetadata(String instantTime) {
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
commitMetadata.addMetadata("test", "test");
|
||||
HoodieWriteStat writeStat = new HoodieWriteStat();
|
||||
writeStat.setFileId("file-1");
|
||||
commitMetadata.addWriteStat(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, writeStat);
|
||||
commitMetadata.setOperationType(WriteOperationType.INSERT);
|
||||
return commitMetadata;
|
||||
}
|
||||
|
||||
private void createInflightCommit(String instantTime) throws Exception {
|
||||
String fileId1 = "file-" + instantTime + "-1";
|
||||
String fileId2 = "file-" + instantTime + "-2";
|
||||
HoodieTestTable.of(metaClient)
|
||||
.addInflightCommit(instantTime)
|
||||
.withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2);
|
||||
}
|
||||
|
||||
private void createCompactionRequested(String instantTime) throws Exception {
|
||||
String fileId1 = "file-1";
|
||||
HoodieCompactionPlan compactionPlan = new HoodieCompactionPlan();
|
||||
compactionPlan.setVersion(TimelineLayoutVersion.CURR_VERSION);
|
||||
HoodieCompactionOperation operation = new HoodieCompactionOperation();
|
||||
operation.setFileId(fileId1);
|
||||
operation.setPartitionPath(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH);
|
||||
operation.setDataFilePath("/file-1");
|
||||
operation.setDeltaFilePaths(Arrays.asList("/file-1"));
|
||||
compactionPlan.setOperations(Arrays.asList(operation));
|
||||
HoodieTestTable.of(metaClient)
|
||||
.addRequestedCompaction(instantTime, compactionPlan);
|
||||
}
|
||||
|
||||
private void createCompaction(String instantTime) throws Exception {
|
||||
String fileId1 = "file-1";
|
||||
String fileId2 = "file-2";
|
||||
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
commitMetadata.addMetadata("test", "test");
|
||||
commitMetadata.setOperationType(WriteOperationType.COMPACT);
|
||||
commitMetadata.setCompacted(true);
|
||||
HoodieWriteStat writeStat = new HoodieWriteStat();
|
||||
writeStat.setFileId("file-1");
|
||||
commitMetadata.addWriteStat(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, writeStat);
|
||||
HoodieTestTable.of(metaClient)
|
||||
.addCommit(instantTime, commitMetadata)
|
||||
.withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2);
|
||||
}
|
||||
|
||||
private void createReplaceRequested(String instantTime) throws Exception {
|
||||
String fileId1 = "file-1";
|
||||
String fileId2 = "file-2";
|
||||
|
||||
// create replace instant to mark fileId1 as deleted
|
||||
HoodieRequestedReplaceMetadata requestedReplaceMetadata = new HoodieRequestedReplaceMetadata();
|
||||
requestedReplaceMetadata.setOperationType(WriteOperationType.CLUSTER.name());
|
||||
HoodieClusteringPlan clusteringPlan = new HoodieClusteringPlan();
|
||||
HoodieClusteringGroup clusteringGroup = new HoodieClusteringGroup();
|
||||
HoodieSliceInfo sliceInfo = new HoodieSliceInfo();
|
||||
sliceInfo.setFileId(fileId1);
|
||||
sliceInfo.setPartitionPath(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH);
|
||||
clusteringGroup.setSlices(Arrays.asList(sliceInfo));
|
||||
clusteringPlan.setInputGroups(Arrays.asList(clusteringGroup));
|
||||
requestedReplaceMetadata.setClusteringPlan(clusteringPlan);
|
||||
requestedReplaceMetadata.setVersion(TimelineLayoutVersion.CURR_VERSION);
|
||||
HoodieTestTable.of(metaClient)
|
||||
.addRequestedReplace(instantTime, requestedReplaceMetadata)
|
||||
.withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2);
|
||||
}
|
||||
|
||||
private void createReplace(String instantTime, WriteOperationType writeOperationType) throws Exception {
|
||||
String fileId1 = "file-1";
|
||||
String fileId2 = "file-2";
|
||||
|
||||
// create replace instant to mark fileId1 as deleted
|
||||
HoodieReplaceCommitMetadata replaceMetadata = new HoodieReplaceCommitMetadata();
|
||||
Map<String, List<String>> partitionFileIds = new HashMap<>();
|
||||
partitionFileIds.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, Arrays.asList(fileId2));
|
||||
replaceMetadata.setPartitionToReplaceFileIds(partitionFileIds);
|
||||
HoodieWriteStat writeStat = new HoodieWriteStat();
|
||||
writeStat.setFileId("file-1");
|
||||
replaceMetadata.addWriteStat(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, writeStat);
|
||||
replaceMetadata.setOperationType(writeOperationType);
|
||||
// create replace instant to mark fileId1 as deleted
|
||||
HoodieRequestedReplaceMetadata requestedReplaceMetadata = new HoodieRequestedReplaceMetadata();
|
||||
requestedReplaceMetadata.setOperationType(WriteOperationType.CLUSTER.name());
|
||||
HoodieClusteringPlan clusteringPlan = new HoodieClusteringPlan();
|
||||
HoodieClusteringGroup clusteringGroup = new HoodieClusteringGroup();
|
||||
HoodieSliceInfo sliceInfo = new HoodieSliceInfo();
|
||||
sliceInfo.setFileId(fileId1);
|
||||
sliceInfo.setPartitionPath(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH);
|
||||
clusteringGroup.setSlices(Arrays.asList(sliceInfo));
|
||||
clusteringPlan.setInputGroups(Arrays.asList(clusteringGroup));
|
||||
requestedReplaceMetadata.setClusteringPlan(clusteringPlan);
|
||||
requestedReplaceMetadata.setVersion(TimelineLayoutVersion.CURR_VERSION);
|
||||
HoodieTestTable.of(metaClient)
|
||||
.addReplaceCommit(instantTime, requestedReplaceMetadata, replaceMetadata)
|
||||
.withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,116 @@
|
||||
/*
|
||||
* 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.client.transaction;
|
||||
|
||||
import org.apache.curator.framework.CuratorFramework;
|
||||
import org.apache.curator.framework.CuratorFrameworkFactory;
|
||||
import org.apache.curator.retry.RetryOneTime;
|
||||
import org.apache.curator.test.TestingServer;
|
||||
import org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider;
|
||||
import org.apache.hudi.common.config.LockConfiguration;
|
||||
import org.apache.hudi.exception.HoodieLockException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP;
|
||||
|
||||
public class TestZookeeperBasedLockProvider {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(TestZookeeperBasedLockProvider.class);
|
||||
|
||||
private static TestingServer server;
|
||||
private static CuratorFramework client;
|
||||
private static String basePath = "/hudi/test/lock";
|
||||
private static String key = "table1";
|
||||
private static LockConfiguration lockConfiguration;
|
||||
|
||||
@BeforeAll
|
||||
public static void setup() {
|
||||
while (server == null) {
|
||||
try {
|
||||
server = new TestingServer();
|
||||
CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder();
|
||||
client = builder.connectString(server.getConnectString()).retryPolicy(new RetryOneTime(1000)).build();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Getting bind exception - retrying to allocate server");
|
||||
server = null;
|
||||
}
|
||||
}
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty(ZK_BASE_PATH_PROP, basePath);
|
||||
properties.setProperty(ZK_LOCK_KEY_PROP, key);
|
||||
properties.setProperty(ZK_CONNECT_URL_PROP, server.getConnectString());
|
||||
properties.setProperty(ZK_BASE_PATH_PROP, server.getTempDirectory().getAbsolutePath());
|
||||
properties.setProperty(ZK_SESSION_TIMEOUT_MS_PROP, "10000");
|
||||
properties.setProperty(ZK_CONNECTION_TIMEOUT_MS_PROP, "10000");
|
||||
properties.setProperty(ZK_LOCK_KEY_PROP, "key");
|
||||
properties.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP, "1000");
|
||||
lockConfiguration = new LockConfiguration(properties);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAcquireLock() {
|
||||
ZookeeperBasedLockProvider zookeeperBasedLockProvider = new ZookeeperBasedLockProvider(lockConfiguration, client);
|
||||
Assertions.assertTrue(zookeeperBasedLockProvider.tryLock(lockConfiguration.getConfig()
|
||||
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS));
|
||||
zookeeperBasedLockProvider.unlock();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnLock() {
|
||||
ZookeeperBasedLockProvider zookeeperBasedLockProvider = new ZookeeperBasedLockProvider(lockConfiguration, client);
|
||||
Assertions.assertTrue(zookeeperBasedLockProvider.tryLock(lockConfiguration.getConfig()
|
||||
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS));
|
||||
zookeeperBasedLockProvider.unlock();
|
||||
zookeeperBasedLockProvider.tryLock(lockConfiguration.getConfig()
|
||||
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReentrantLock() {
|
||||
ZookeeperBasedLockProvider zookeeperBasedLockProvider = new ZookeeperBasedLockProvider(lockConfiguration, client);
|
||||
Assertions.assertTrue(zookeeperBasedLockProvider.tryLock(lockConfiguration.getConfig()
|
||||
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS));
|
||||
try {
|
||||
zookeeperBasedLockProvider.tryLock(lockConfiguration.getConfig()
|
||||
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS);
|
||||
Assertions.fail();
|
||||
} catch (HoodieLockException e) {
|
||||
// expected
|
||||
}
|
||||
zookeeperBasedLockProvider.unlock();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnlockWithoutLock() {
|
||||
ZookeeperBasedLockProvider zookeeperBasedLockProvider = new ZookeeperBasedLockProvider(lockConfiguration, client);
|
||||
zookeeperBasedLockProvider.unlock();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,213 @@
|
||||
/*
|
||||
* 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 static org.apache.hudi.common.util.CleanerUtils.convertCleanMetadata;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import org.apache.hudi.avro.model.HoodieActionInstant;
|
||||
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.avro.model.HoodieClusteringGroup;
|
||||
import org.apache.hudi.avro.model.HoodieClusteringPlan;
|
||||
import org.apache.hudi.avro.model.HoodieInstantInfo;
|
||||
import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackPartitionMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieSliceInfo;
|
||||
import org.apache.hudi.client.utils.MetadataConversionUtils;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV2MigrationHandler;
|
||||
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
public class TestMetadataConversionUtils extends HoodieCommonTestHarness {
|
||||
|
||||
@BeforeEach
|
||||
public void init() throws IOException {
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompletedClean() throws Exception {
|
||||
String newCommitTime = HoodieTestTable.makeNewCommitTime();
|
||||
createCleanMetadata(newCommitTime);
|
||||
HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper(
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, newCommitTime), metaClient);
|
||||
assertEquals(metaEntry.getActionState(), State.COMPLETED.toString());
|
||||
assertEquals(metaEntry.getHoodieCleanMetadata().getStartCleanTime(), newCommitTime);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompletedReplace() throws Exception {
|
||||
String newCommitTime = HoodieTestTable.makeNewCommitTime();
|
||||
createReplace(newCommitTime, WriteOperationType.INSERT_OVERWRITE);
|
||||
HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper(
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, newCommitTime), metaClient);
|
||||
assertEquals(metaEntry.getActionState(), State.COMPLETED.toString());
|
||||
assertEquals(metaEntry.getHoodieReplaceCommitMetadata().getOperationType(), WriteOperationType.INSERT_OVERWRITE.toString());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompletedCommitOrDeltaCommit() throws Exception {
|
||||
String newCommitTime = HoodieTestTable.makeNewCommitTime();
|
||||
createCommitMetadata(newCommitTime);
|
||||
HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper(
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, newCommitTime), metaClient);
|
||||
assertEquals(metaEntry.getActionState(), State.COMPLETED.toString());
|
||||
assertEquals(metaEntry.getHoodieCommitMetadata().getOperationType(), WriteOperationType.INSERT.toString());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompletedRollback() throws Exception {
|
||||
String newCommitTime = HoodieTestTable.makeNewCommitTime();
|
||||
createRollbackMetadata(newCommitTime);
|
||||
HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper(
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.ROLLBACK_ACTION, newCommitTime), metaClient);
|
||||
assertEquals(metaEntry.getActionState(), State.COMPLETED.toString());
|
||||
assertEquals(metaEntry.getHoodieRollbackMetadata().getStartRollbackTime(), newCommitTime);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompletedCompaction() throws Exception {
|
||||
String newCommitTime = HoodieTestTable.makeNewCommitTime();
|
||||
createCompactionMetadata(newCommitTime);
|
||||
HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper(
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, newCommitTime), metaClient);
|
||||
assertEquals(metaEntry.getActionState(), State.COMPLETED.toString());
|
||||
assertEquals(metaEntry.getHoodieCommitMetadata().getOperationType(), WriteOperationType.COMPACT.toString());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertCommitMetadata() {
|
||||
HoodieCommitMetadata hoodieCommitMetadata = new HoodieCommitMetadata();
|
||||
hoodieCommitMetadata.setOperationType(WriteOperationType.INSERT);
|
||||
org.apache.hudi.avro.model.HoodieCommitMetadata expectedCommitMetadata = MetadataConversionUtils
|
||||
.convertCommitMetadata(hoodieCommitMetadata);
|
||||
assertEquals(expectedCommitMetadata.getOperationType(), WriteOperationType.INSERT.toString());
|
||||
}
|
||||
|
||||
private void createCompactionMetadata(String instantTime) throws Exception {
|
||||
String fileId1 = "file-" + instantTime + "-1";
|
||||
String fileId2 = "file-" + instantTime + "-2";
|
||||
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
commitMetadata.addMetadata("test", "test");
|
||||
commitMetadata.setOperationType(WriteOperationType.COMPACT);
|
||||
commitMetadata.setCompacted(true);
|
||||
HoodieTestTable.of(metaClient)
|
||||
.addCommit(instantTime, commitMetadata)
|
||||
.withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2);
|
||||
}
|
||||
|
||||
private void createRollbackMetadata(String instantTime) throws Exception {
|
||||
HoodieRollbackMetadata rollbackMetadata = new HoodieRollbackMetadata();
|
||||
rollbackMetadata.setCommitsRollback(Arrays.asList(instantTime));
|
||||
rollbackMetadata.setStartRollbackTime(instantTime);
|
||||
HoodieRollbackPartitionMetadata rollbackPartitionMetadata = new HoodieRollbackPartitionMetadata();
|
||||
rollbackPartitionMetadata.setPartitionPath("p1");
|
||||
rollbackPartitionMetadata.setSuccessDeleteFiles(Arrays.asList("f1"));
|
||||
rollbackPartitionMetadata.setFailedDeleteFiles(new ArrayList<>());
|
||||
rollbackPartitionMetadata.setWrittenLogFiles(new HashMap<>());
|
||||
rollbackPartitionMetadata.setRollbackLogFiles(new HashMap<>());
|
||||
Map<String, HoodieRollbackPartitionMetadata> partitionMetadataMap = new HashMap<>();
|
||||
partitionMetadataMap.put("p1", rollbackPartitionMetadata);
|
||||
rollbackMetadata.setPartitionMetadata(partitionMetadataMap);
|
||||
rollbackMetadata.setInstantsRollback(Arrays.asList(new HoodieInstantInfo("1", HoodieTimeline.COMMIT_ACTION)));
|
||||
HoodieTestTable.of(metaClient)
|
||||
.addRollback(instantTime, rollbackMetadata);
|
||||
}
|
||||
|
||||
private void createCommitMetadata(String instantTime) throws Exception {
|
||||
String fileId1 = "file-" + instantTime + "-1";
|
||||
String fileId2 = "file-" + instantTime + "-2";
|
||||
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
commitMetadata.addMetadata("test", "test");
|
||||
commitMetadata.setOperationType(WriteOperationType.INSERT);
|
||||
HoodieTestTable.of(metaClient)
|
||||
.addCommit(instantTime, commitMetadata)
|
||||
.withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2);
|
||||
}
|
||||
|
||||
private void createReplace(String instantTime, WriteOperationType writeOperationType) throws Exception {
|
||||
String fileId1 = "file-1";
|
||||
String fileId2 = "file-2";
|
||||
|
||||
// create replace instant to mark fileId1 as deleted
|
||||
HoodieReplaceCommitMetadata replaceMetadata = new HoodieReplaceCommitMetadata();
|
||||
Map<String, List<String>> partitionFileIds = new HashMap<>();
|
||||
partitionFileIds.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, Arrays.asList(fileId2));
|
||||
replaceMetadata.setPartitionToReplaceFileIds(partitionFileIds);
|
||||
HoodieWriteStat writeStat = new HoodieWriteStat();
|
||||
writeStat.setFileId("file-1");
|
||||
replaceMetadata.addWriteStat(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, writeStat);
|
||||
replaceMetadata.setOperationType(writeOperationType);
|
||||
// create replace instant to mark fileId1 as deleted
|
||||
HoodieRequestedReplaceMetadata requestedReplaceMetadata = new HoodieRequestedReplaceMetadata();
|
||||
requestedReplaceMetadata.setOperationType(WriteOperationType.INSERT_OVERWRITE.name());
|
||||
HoodieClusteringPlan clusteringPlan = new HoodieClusteringPlan();
|
||||
HoodieClusteringGroup clusteringGroup = new HoodieClusteringGroup();
|
||||
HoodieSliceInfo sliceInfo = new HoodieSliceInfo();
|
||||
clusteringGroup.setSlices(Arrays.asList(sliceInfo));
|
||||
clusteringPlan.setInputGroups(Arrays.asList(clusteringGroup));
|
||||
requestedReplaceMetadata.setClusteringPlan(clusteringPlan);
|
||||
requestedReplaceMetadata.setVersion(TimelineLayoutVersion.CURR_VERSION);
|
||||
HoodieTestTable.of(metaClient)
|
||||
.addReplaceCommit(instantTime, requestedReplaceMetadata, replaceMetadata)
|
||||
.withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2);
|
||||
}
|
||||
|
||||
private void createCleanMetadata(String instantTime) throws IOException {
|
||||
HoodieCleanerPlan cleanerPlan = new HoodieCleanerPlan(new HoodieActionInstant("", "", ""), "", new HashMap<>(),
|
||||
CleanPlanV2MigrationHandler.VERSION, new HashMap<>());
|
||||
HoodieCleanStat cleanStats = new HoodieCleanStat(
|
||||
HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS,
|
||||
HoodieTestUtils.DEFAULT_PARTITION_PATHS[new Random().nextInt(HoodieTestUtils.DEFAULT_PARTITION_PATHS.length)],
|
||||
Collections.emptyList(),
|
||||
Collections.emptyList(),
|
||||
Collections.emptyList(),
|
||||
instantTime);
|
||||
HoodieCleanMetadata cleanMetadata = convertCleanMetadata(instantTime, Option.of(0L), Collections.singletonList(cleanStats));
|
||||
HoodieTestTable.of(metaClient).addClean(instantTime, cleanerPlan, cleanMetadata);
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user