1
0

[HUDI-89] Add configOption & refactor all configs based on that (#2833)

Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
wenningd
2021-06-30 14:26:30 -07:00
committed by GitHub
parent 07e93de8b4
commit d412fb2fe6
173 changed files with 4277 additions and 3309 deletions

View File

@@ -74,7 +74,7 @@ public class DLASyncConfig implements Serializable {
public Boolean useFileListingFromMetadata = HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
@Parameter(names = {"--verify-metadata-file-listing"}, description = "Verify file listing from Hudi's metadata against file system")
public Boolean verifyMetadataFileListing = HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE;
public Boolean verifyMetadataFileListing = HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue();
@Parameter(names = {"--help", "-h"}, help = true)
public Boolean help = false;

View File

@@ -46,14 +46,14 @@ import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP;
import static org.apache.hudi.common.config.LockConfiguration.HIVE_METASTORE_URI_PROP;
import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_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;
import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP;
import static org.apache.hudi.common.config.LockConfiguration.ZK_PORT_PROP;
import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP;
import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP_KEY;
import static org.apache.hudi.common.config.LockConfiguration.HIVE_METASTORE_URI_PROP_KEY;
import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_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.ZK_CONNECT_URL_PROP_KEY;
import static org.apache.hudi.common.config.LockConfiguration.ZK_PORT_PROP_KEY;
import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP_KEY;
import static org.apache.hudi.common.lock.LockState.ACQUIRING;
import static org.apache.hudi.common.lock.LockState.ALREADY_ACQUIRED;
import static org.apache.hudi.common.lock.LockState.FAILED_TO_ACQUIRE;
@@ -100,9 +100,9 @@ public class HiveMetastoreBasedLockProvider implements LockProvider<LockResponse
HiveMetastoreBasedLockProvider(final LockConfiguration lockConfiguration) {
checkRequiredProps(lockConfiguration);
this.lockConfiguration = lockConfiguration;
this.databaseName = this.lockConfiguration.getConfig().getString(HIVE_DATABASE_NAME_PROP);
this.tableName = this.lockConfiguration.getConfig().getString(HIVE_TABLE_NAME_PROP);
this.hiveMetastoreUris = this.lockConfiguration.getConfig().getOrDefault(HIVE_METASTORE_URI_PROP, "").toString();
this.databaseName = this.lockConfiguration.getConfig().getString(HIVE_DATABASE_NAME_PROP_KEY);
this.tableName = this.lockConfiguration.getConfig().getString(HIVE_TABLE_NAME_PROP_KEY);
this.hiveMetastoreUris = this.lockConfiguration.getConfig().getOrDefault(HIVE_METASTORE_URI_PROP_KEY, "").toString();
}
@Override
@@ -196,8 +196,8 @@ public class HiveMetastoreBasedLockProvider implements LockProvider<LockResponse
}
private void checkRequiredProps(final LockConfiguration lockConfiguration) {
ValidationUtils.checkArgument(lockConfiguration.getConfig().getString(HIVE_DATABASE_NAME_PROP) != null);
ValidationUtils.checkArgument(lockConfiguration.getConfig().getString(HIVE_TABLE_NAME_PROP) != null);
ValidationUtils.checkArgument(lockConfiguration.getConfig().getString(HIVE_DATABASE_NAME_PROP_KEY) != null);
ValidationUtils.checkArgument(lockConfiguration.getConfig().getString(HIVE_TABLE_NAME_PROP_KEY) != null);
}
private void setHiveLockConfs(HiveConf hiveConf) {
@@ -206,18 +206,18 @@ public class HiveMetastoreBasedLockProvider implements LockProvider<LockResponse
}
hiveConf.set("hive.support.concurrency", "true");
hiveConf.set("hive.lock.manager", "org.apache.hadoop.hive.ql.lockmgr.zookeeper.ZooKeeperHiveLockManager");
hiveConf.set("hive.lock.numretries", lockConfiguration.getConfig().getString(LOCK_ACQUIRE_NUM_RETRIES_PROP));
hiveConf.set("hive.unlock.numretries", lockConfiguration.getConfig().getString(LOCK_ACQUIRE_NUM_RETRIES_PROP));
hiveConf.set("hive.lock.sleep.between.retries", lockConfiguration.getConfig().getString(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP));
String zkConnectUrl = lockConfiguration.getConfig().getOrDefault(ZK_CONNECT_URL_PROP, "").toString();
hiveConf.set("hive.lock.numretries", lockConfiguration.getConfig().getString(LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY));
hiveConf.set("hive.unlock.numretries", lockConfiguration.getConfig().getString(LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY));
hiveConf.set("hive.lock.sleep.between.retries", lockConfiguration.getConfig().getString(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY));
String zkConnectUrl = lockConfiguration.getConfig().getOrDefault(ZK_CONNECT_URL_PROP_KEY, "").toString();
if (zkConnectUrl.length() > 0) {
hiveConf.set("hive.zookeeper.quorum", zkConnectUrl);
}
String zkPort = lockConfiguration.getConfig().getOrDefault(ZK_PORT_PROP, "").toString();
String zkPort = lockConfiguration.getConfig().getOrDefault(ZK_PORT_PROP_KEY, "").toString();
if (zkPort.length() > 0) {
hiveConf.set("hive.zookeeper.client.port", zkPort);
}
String zkSessionTimeout = lockConfiguration.getConfig().getOrDefault(ZK_SESSION_TIMEOUT_MS_PROP, "").toString();
String zkSessionTimeout = lockConfiguration.getConfig().getOrDefault(ZK_SESSION_TIMEOUT_MS_PROP_KEY, "").toString();
if (zkSessionTimeout.length() > 0) {
hiveConf.set("hive.zookeeper.session.timeout", zkSessionTimeout);
}

View File

@@ -86,7 +86,7 @@ public class HiveSyncConfig implements Serializable {
public Boolean useFileListingFromMetadata = HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
@Parameter(names = {"--verify-metadata-file-listing"}, description = "Verify file listing from Hudi's metadata against file system")
public Boolean verifyMetadataFileListing = HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE;
public Boolean verifyMetadataFileListing = HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue();
@Parameter(names = {"--table-properties"}, description = "Table properties to hive table")
public String tableProperties;

View File

@@ -38,15 +38,15 @@ import java.util.concurrent.TimeUnit;
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_NUM_RETRIES;
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS;
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_CONNECTION_TIMEOUT_MS;
import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP;
import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_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;
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_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_PORT_PROP;
import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP;
import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP_KEY;
import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_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.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP_KEY;
import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP_KEY;
import static org.apache.hudi.common.config.LockConfiguration.ZK_PORT_PROP_KEY;
import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP_KEY;
/**
* For all tests, we need to set LockComponent.setOperationType(DataOperationType.NO_TXN).
@@ -65,15 +65,15 @@ public class TestHiveMetastoreBasedLockProvider extends HiveSyncFunctionalTestHa
@BeforeEach
public void init() throws Exception {
TypedProperties properties = new TypedProperties();
properties.setProperty(HIVE_DATABASE_NAME_PROP, TEST_DB_NAME);
properties.setProperty(HIVE_TABLE_NAME_PROP, TEST_TABLE_NAME);
properties.setProperty(LOCK_ACQUIRE_NUM_RETRIES_PROP, DEFAULT_LOCK_ACQUIRE_NUM_RETRIES);
properties.setProperty(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP, DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS);
properties.setProperty(ZK_CONNECT_URL_PROP, zkService().connectString());
properties.setProperty(ZK_PORT_PROP, hiveConf().get("hive.zookeeper.client.port"));
properties.setProperty(ZK_SESSION_TIMEOUT_MS_PROP, hiveConf().get("hive.zookeeper.session.timeout"));
properties.setProperty(ZK_CONNECTION_TIMEOUT_MS_PROP, String.valueOf(DEFAULT_ZK_CONNECTION_TIMEOUT_MS));
properties.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP, String.valueOf(1000));
properties.setProperty(HIVE_DATABASE_NAME_PROP_KEY, TEST_DB_NAME);
properties.setProperty(HIVE_TABLE_NAME_PROP_KEY, TEST_TABLE_NAME);
properties.setProperty(LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY, DEFAULT_LOCK_ACQUIRE_NUM_RETRIES);
properties.setProperty(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS);
properties.setProperty(ZK_CONNECT_URL_PROP_KEY, zkService().connectString());
properties.setProperty(ZK_PORT_PROP_KEY, hiveConf().get("hive.zookeeper.client.port"));
properties.setProperty(ZK_SESSION_TIMEOUT_MS_PROP_KEY, hiveConf().get("hive.zookeeper.session.timeout"));
properties.setProperty(ZK_CONNECTION_TIMEOUT_MS_PROP_KEY, String.valueOf(DEFAULT_ZK_CONNECTION_TIMEOUT_MS));
properties.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, String.valueOf(1000));
lockConfiguration = new LockConfiguration(properties);
lockComponent.setTablename(TEST_TABLE_NAME);
}
@@ -83,10 +83,10 @@ public class TestHiveMetastoreBasedLockProvider extends HiveSyncFunctionalTestHa
HiveMetastoreBasedLockProvider lockProvider = new HiveMetastoreBasedLockProvider(lockConfiguration, hiveConf());
lockComponent.setOperationType(DataOperationType.NO_TXN);
Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig()
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent));
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS, lockComponent));
try {
Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig()
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent));
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS, lockComponent));
Assertions.fail();
} catch (Exception e) {
// Expected since lock is already acquired
@@ -94,7 +94,7 @@ public class TestHiveMetastoreBasedLockProvider extends HiveSyncFunctionalTestHa
lockProvider.unlock();
// try to lock again after unlocking
Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig()
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent));
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS, lockComponent));
lockProvider.close();
}
@@ -103,11 +103,11 @@ public class TestHiveMetastoreBasedLockProvider extends HiveSyncFunctionalTestHa
HiveMetastoreBasedLockProvider lockProvider = new HiveMetastoreBasedLockProvider(lockConfiguration, hiveConf());
lockComponent.setOperationType(DataOperationType.NO_TXN);
Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig()
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent));
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS, lockComponent));
lockProvider.unlock();
// try to lock again after unlocking
Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig()
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent));
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS, lockComponent));
lockProvider.close();
}
@@ -116,10 +116,10 @@ public class TestHiveMetastoreBasedLockProvider extends HiveSyncFunctionalTestHa
HiveMetastoreBasedLockProvider lockProvider = new HiveMetastoreBasedLockProvider(lockConfiguration, hiveConf());
lockComponent.setOperationType(DataOperationType.NO_TXN);
Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig()
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent));
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS, lockComponent));
try {
lockProvider.acquireLock(lockConfiguration.getConfig()
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent);
.getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS, lockComponent);
Assertions.fail();
} catch (IllegalArgumentException e) {
// expected