1
0

[HUDI-4065] Add FileBasedLockProvider (#6071)

This commit is contained in:
冯健
2022-07-19 07:52:47 +08:00
committed by GitHub
parent 1959b843b7
commit 382d19e85b
5 changed files with 349 additions and 36 deletions

View File

@@ -0,0 +1,135 @@
/*
* 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;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hudi.client.transaction.lock.FileSystemBasedLockProvider;
import org.apache.hudi.common.config.LockConfiguration;
import org.apache.hudi.common.testutils.minicluster.HdfsTestService;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieLockException;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_EXPIRE_PROP_KEY;
import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY;
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY;
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY;
public class TestFileBasedLockProvider {
private static HdfsTestService hdfsTestService;
private static MiniDFSCluster dfsCluster;
private static LockConfiguration lockConfiguration;
private static Configuration hadoopConf;
@BeforeAll
public static void setup() throws IOException {
hdfsTestService = new HdfsTestService();
dfsCluster = hdfsTestService.start(true);
hadoopConf = dfsCluster.getFileSystem().getConf();
Properties properties = new Properties();
properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, "/tmp/");
properties.setProperty(FILESYSTEM_LOCK_EXPIRE_PROP_KEY, "1");
properties.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "1000");
properties.setProperty(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "1000");
properties.setProperty(LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY, "3");
lockConfiguration = new LockConfiguration(properties);
}
@AfterAll
public static void cleanUpAfterAll() throws IOException {
Path workDir = dfsCluster.getFileSystem().getWorkingDirectory();
FileSystem fs = workDir.getFileSystem(hdfsTestService.getHadoopConf());
fs.delete(new Path("/tmp"), true);
if (hdfsTestService != null) {
hdfsTestService.stop();
hdfsTestService = null;
}
}
@AfterEach
public void cleanUpAfterEach() throws IOException {
Path workDir = dfsCluster.getFileSystem().getWorkingDirectory();
FileSystem fs = workDir.getFileSystem(hdfsTestService.getHadoopConf());
fs.delete(new Path("/tmp/lock"), true);
}
@Test
public void testAcquireLock() {
FileSystemBasedLockProvider fileBasedLockProvider = new FileSystemBasedLockProvider(lockConfiguration, hadoopConf);
Assertions.assertTrue(fileBasedLockProvider.tryLock(lockConfiguration.getConfig()
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS));
fileBasedLockProvider.unlock();
}
@Test
public void testAcquireLockWithDefaultPath() {
lockConfiguration.getConfig().remove(FILESYSTEM_LOCK_PATH_PROP_KEY);
lockConfiguration.getConfig().setProperty(HoodieWriteConfig.BASE_PATH.key(), "/tmp/");
FileSystemBasedLockProvider fileBasedLockProvider = new FileSystemBasedLockProvider(lockConfiguration, hadoopConf);
Assertions.assertTrue(fileBasedLockProvider.tryLock(lockConfiguration.getConfig()
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS));
fileBasedLockProvider.unlock();
lockConfiguration.getConfig().setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, "/tmp/");
}
@Test
public void testUnLock() {
FileSystemBasedLockProvider fileBasedLockProvider = new FileSystemBasedLockProvider(lockConfiguration, hadoopConf);
Assertions.assertTrue(fileBasedLockProvider.tryLock(lockConfiguration.getConfig()
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS));
fileBasedLockProvider.unlock();
Assertions.assertTrue(fileBasedLockProvider.tryLock(lockConfiguration.getConfig()
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS));
}
@Test
public void testReentrantLock() {
FileSystemBasedLockProvider fileBasedLockProvider = new FileSystemBasedLockProvider(lockConfiguration, hadoopConf);
Assertions.assertTrue(fileBasedLockProvider.tryLock(lockConfiguration.getConfig()
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS));
Assertions.assertFalse(fileBasedLockProvider.tryLock(lockConfiguration.getConfig()
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS));
fileBasedLockProvider.unlock();
}
@Test
public void testUnlockWithoutLock() {
try {
FileSystemBasedLockProvider fileBasedLockProvider = new FileSystemBasedLockProvider(lockConfiguration, hadoopConf);
fileBasedLockProvider.unlock();
} catch (HoodieLockException e) {
Assertions.fail();
}
}
}

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.client;
import org.apache.hudi.client.transaction.lock.FileSystemBasedLockProvider;
import org.apache.hudi.client.transaction.lock.InProcessLockProvider;
import org.apache.hudi.common.config.LockConfiguration;
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
@@ -45,9 +46,11 @@ import org.apache.hudi.testutils.HoodieClientTestBase;
import org.apache.hadoop.fs.Path;
import org.apache.spark.api.java.JavaRDD;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.EnumSource;
import org.junit.jupiter.params.provider.MethodSource;
import java.io.IOException;
import java.util.ArrayList;
@@ -66,7 +69,11 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_EXPIRE_PROP_KEY;
import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY;
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY;
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY;
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -76,6 +83,21 @@ import static org.junit.jupiter.api.Assertions.fail;
public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
private Properties lockProperties = null;
@BeforeEach
public void setup() throws IOException {
if (lockProperties == null) {
lockProperties = new Properties();
lockProperties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks");
lockProperties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000");
lockProperties.setProperty(FILESYSTEM_LOCK_EXPIRE_PROP_KEY, "1");
lockProperties.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "1000");
lockProperties.setProperty(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "1000");
lockProperties.setProperty(LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY, "3");
}
}
public void setUpMORTestTable() throws IOException {
cleanupResources();
initPath();
@@ -92,15 +114,27 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
cleanupResources();
}
private static final List<Class> LOCK_PROVIDER_CLASSES = Arrays.asList(
InProcessLockProvider.class,
FileSystemBasedLockProvider.class);
private static Iterable<Object[]> providerClassAndTableType() {
List<Object[]> opts = new ArrayList<>();
for (Object providerClass : LOCK_PROVIDER_CLASSES) {
opts.add(new Object[] {HoodieTableType.COPY_ON_WRITE, providerClass});
opts.add(new Object[] {HoodieTableType.MERGE_ON_READ, providerClass});
}
return opts;
}
@ParameterizedTest
@EnumSource(value = HoodieTableType.class, names = {"COPY_ON_WRITE", "MERGE_ON_READ"})
public void testHoodieClientBasicMultiWriter(HoodieTableType tableType) throws Exception {
@MethodSource("providerClassAndTableType")
public void testHoodieClientBasicMultiWriter(HoodieTableType tableType, Class providerClass) throws Exception {
if (tableType == HoodieTableType.MERGE_ON_READ) {
setUpMORTestTable();
}
Properties properties = new Properties();
properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks");
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000");
lockProperties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000");
HoodieWriteConfig writeConfig = getConfigBuilder()
.withCleanConfig(HoodieCleanConfig.newBuilder()
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY)
@@ -110,8 +144,8 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
// Timeline-server-based markers are not used for multi-writer tests
.withMarkersType(MarkerType.DIRECT.name())
.withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class)
.build()).withAutoCommit(false).withProperties(properties).build();
.withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(providerClass)
.build()).withAutoCommit(false).withProperties(lockProperties).build();
// Create the first commit
createCommitWithInserts(writeConfig, getHoodieWriteClient(writeConfig), "000", "001", 200, true);
@@ -172,16 +206,6 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
assertTrue(writer1Completed.get() && writer2Completed.get());
}
@Test
public void testMultiWriterWithAsyncTableServicesWithConflictCOW() throws Exception {
testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType.COPY_ON_WRITE);
}
@Test
public void testMultiWriterWithAsyncTableServicesWithConflictMOR() throws Exception {
testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType.MERGE_ON_READ);
}
@ParameterizedTest
@EnumSource(value = HoodieTableType.class, names = {"COPY_ON_WRITE", "MERGE_ON_READ"})
public void testMultiWriterWithInsertsToDistinctPartitions(HoodieTableType tableType) throws Exception {
@@ -189,11 +213,9 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
setUpMORTestTable();
}
Properties properties = new Properties();
properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks");
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000");
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "3000");
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "20");
lockProperties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000");
lockProperties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "3000");
lockProperties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "20");
HoodieWriteConfig cfg = getConfigBuilder()
.withCleanConfig(HoodieCleanConfig.newBuilder()
@@ -210,7 +232,7 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
.withAutoCommit(false)
// Timeline-server-based markers are not used for multi-writer tests
.withMarkersType(MarkerType.DIRECT.name())
.withProperties(properties)
.withProperties(lockProperties)
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.REMOTE_FIRST)
.withSecondaryStorageType(FileSystemViewStorageType.MEMORY).build())
.build();
@@ -260,15 +282,13 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
}
}
private void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType tableType) throws Exception {
@ParameterizedTest
@MethodSource("providerClassAndTableType")
public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType tableType, Class providerClass) throws Exception {
// create inserts X 1
if (tableType == HoodieTableType.MERGE_ON_READ) {
setUpMORTestTable();
}
Properties properties = new Properties();
properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks");
properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath);
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000");
// Disabling embedded timeline server, it doesn't work with multiwriter
HoodieWriteConfig.Builder writeConfigBuilder = getConfigBuilder()
.withCleanConfig(HoodieCleanConfig.newBuilder()
@@ -284,8 +304,8 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withStorageType(
FileSystemViewStorageType.MEMORY).build())
.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
.withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class)
.build()).withAutoCommit(false).withProperties(properties);
.withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(providerClass)
.build()).withAutoCommit(false).withProperties(lockProperties);
Set<String> validInstants = new HashSet<>();
// Create the first commit with inserts
HoodieWriteConfig cfg = writeConfigBuilder.build();
@@ -458,10 +478,7 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
@Test
public void testHoodieClientMultiWriterAutoCommitForConflict() throws Exception {
Properties properties = new Properties();
properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks");
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000");
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "100");
lockProperties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "100");
HoodieWriteConfig.Builder writeConfigBuilder = getConfigBuilder()
.withCleanConfig(HoodieCleanConfig.newBuilder()
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY)
@@ -470,7 +487,7 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
// Timeline-server-based markers are not used for multi-writer tests
.withMarkersType(MarkerType.DIRECT.name())
.withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class)
.build()).withAutoCommit(true).withProperties(properties);
.build()).withAutoCommit(true).withProperties(lockProperties);
HoodieWriteConfig cfg = writeConfigBuilder.build();
HoodieWriteConfig cfg2 = writeConfigBuilder.build();