Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Improvement] Use ConfigBuilder to rewrite the class RssSparkConfig #104

Merged
merged 23 commits into from
Aug 1, 2022
Merged
Show file tree
Hide file tree
Changes from 22 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -44,19 +44,16 @@ public static Configuration newHadoopConfiguration(SparkConf sparkConf) {
SparkHadoopUtil util = new SparkHadoopUtil();
Configuration conf = util.newConfiguration(sparkConf);

boolean useOdfs = sparkConf.getBoolean(RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE,
RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE_DEFAULT_VALUE);
boolean useOdfs = sparkConf.get(RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE);
if (useOdfs) {
final int OZONE_PREFIX_LEN = "spark.rss.ozone.".length();
conf.setBoolean(RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE.substring(OZONE_PREFIX_LEN), useOdfs);
conf.setBoolean(RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE.key().substring(OZONE_PREFIX_LEN), useOdfs);
conf.set(
RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.substring(OZONE_PREFIX_LEN),
sparkConf.get(RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL, RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL_DEFAULT_VALUE));
RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.key().substring(OZONE_PREFIX_LEN),
sparkConf.get(RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL));
conf.set(
RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.substring(OZONE_PREFIX_LEN),
sparkConf.get(
RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL,
RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL_DEFAULT_VALUE));
RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.key().substring(OZONE_PREFIX_LEN),
sparkConf.get(RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL));
}

return conf;
Expand All @@ -77,8 +74,7 @@ public static ShuffleManager loadShuffleManager(String name, SparkConf conf, boo
}

public static List<CoordinatorClient> createCoordinatorClients(SparkConf sparkConf) throws RuntimeException {
String clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE,
RssSparkConfig.RSS_CLIENT_TYPE_DEFAULT_VALUE);
String clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE);
String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM);
CoordinatorClientFactory coordinatorClientFactory = new CoordinatorClientFactory(clientType);
return coordinatorClientFactory.createCoordinatorClient(coordinators);
Expand Down Expand Up @@ -110,7 +106,7 @@ public static void applyDynamicClientConf(SparkConf sparkConf, Map<String, Strin

public static void validateRssClientConf(SparkConf sparkConf) {
String msgFormat = "%s must be set by the client or fetched from coordinators.";
if (!sparkConf.contains(RssSparkConfig.RSS_STORAGE_TYPE)) {
if (!sparkConf.contains(RssSparkConfig.RSS_STORAGE_TYPE.key())) {
String msg = String.format(msgFormat, "Storage type");
LOG.error(msg);
throw new IllegalArgumentException(msg);
Expand All @@ -131,7 +127,7 @@ public static Configuration getRemoteStorageHadoopConf(

public static Set<String> getAssignmentTags(SparkConf sparkConf) {
Set<String> assignmentTags = new HashSet<>();
String rawTags = sparkConf.get(RssSparkConfig.RSS_CLIENT_ASSIGNMENT_TAGS, "");
String rawTags = sparkConf.get(RssSparkConfig.RSS_CLIENT_ASSIGNMENT_TAGS.key(), "");
if (StringUtils.isNotEmpty(rawTags)) {
rawTags = rawTags.trim();
assignmentTags.addAll(Arrays.asList(rawTags.split(",")));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,38 +35,36 @@ public class BufferManagerOptions {
private int requireMemoryRetryMax;

public BufferManagerOptions(SparkConf sparkConf) {
bufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_BUFFER_SIZE,
RssSparkConfig.RSS_WRITER_BUFFER_SIZE_DEFAULT_VALUE);
serializerBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE,
RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE_DEFAULT_VALUE);
bufferSegmentSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE,
RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE_DEFAULT_VALUE);
bufferSpillThreshold = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE,
RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE_DEFAULT_VALUE);
preAllocatedBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE,
RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE_DEFAULT_VALUE);
requireMemoryInterval = sparkConf.getLong(RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_INTERVAL,
RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_INTERVAL_DEFAULT_VALUE);
requireMemoryRetryMax = sparkConf.getInt(RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX,
RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX_DEFAULT_VALUE);
LOG.info(RssSparkConfig.RSS_WRITER_BUFFER_SIZE + "=" + bufferSize);
LOG.info(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE + "=" + bufferSpillThreshold);
LOG.info(RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE + "=" + preAllocatedBufferSize);
bufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key(),
RssSparkConfig.RSS_WRITER_BUFFER_SIZE.defaultValue().get());
serializerBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key(),
RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.defaultValue().get());
bufferSegmentSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key(),
RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.defaultValue().get());
bufferSpillThreshold = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key(),
RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.defaultValue().get());
preAllocatedBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE.key(),
RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE.defaultValue().get());
requireMemoryInterval = sparkConf.get(RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_INTERVAL);
requireMemoryRetryMax = sparkConf.get(RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX);
LOG.info(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key() + "=" + bufferSize);
LOG.info(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key() + "=" + bufferSpillThreshold);
LOG.info(RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE.key() + "=" + preAllocatedBufferSize);
checkBufferSize();
}

private void checkBufferSize() {
if (bufferSize < 0) {
throw new RuntimeException("Unexpected value of " + RssSparkConfig.RSS_WRITER_BUFFER_SIZE
throw new RuntimeException("Unexpected value of " + RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key()
+ "=" + bufferSize);
}
if (bufferSpillThreshold < 0) {
throw new RuntimeException("Unexpected value of " + RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE
throw new RuntimeException("Unexpected value of " + RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key()
+ "=" + bufferSpillThreshold);
}
if (bufferSegmentSize > bufferSize) {
LOG.warn(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE + "[" + bufferSegmentSize + "] should be less than "
+ RssSparkConfig.RSS_WRITER_BUFFER_SIZE + "[" + bufferSize + "]");
LOG.warn(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key() + "[" + bufferSegmentSize + "] should be less than "
+ RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key() + "[" + bufferSize + "]");
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ public void testAssignmentTags() {
* Case2: set the multiple tags implicitly and will return the {@code Constants.SHUFFLE_SERVER_VERSION}
* and configured tags.
*/
conf.set(RssSparkConfig.RSS_CLIENT_ASSIGNMENT_TAGS, " a,b");
conf.set(RssSparkConfig.RSS_CLIENT_ASSIGNMENT_TAGS.key(), " a,b");
tags = RssSparkShuffleUtils.getAssignmentTags(conf);
assertEquals(3, tags.size());
Iterator<String> iterator = tags.iterator();
Expand All @@ -67,14 +67,14 @@ public void odfsConfigurationTest() {
assertFalse(conf1.getBoolean("dfs.namenode.odfs.enable", false));
assertEquals("org.apache.hadoop.fs.Hdfs", conf1.get("fs.AbstractFileSystem.hdfs.impl"));

conf.set(RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE, "true");
conf.set(RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE.key(), "true");
conf1 = RssSparkShuffleUtils.newHadoopConfiguration(conf);
assertTrue(conf1.getBoolean("dfs.namenode.odfs.enable", false));
assertEquals("org.apache.hadoop.odfs.HdfsOdfsFilesystem", conf1.get("fs.hdfs.impl"));
assertEquals("org.apache.hadoop.odfs.HdfsOdfs", conf1.get("fs.AbstractFileSystem.hdfs.impl"));

conf.set(RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL, "expect_odfs_impl");
conf.set(RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL, "expect_odfs_abstract_impl");
conf.set(RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.key(), "expect_odfs_impl");
conf.set(RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.key(), "expect_odfs_abstract_impl");
conf1 = RssSparkShuffleUtils.newHadoopConfiguration(conf);
assertEquals("expect_odfs_impl", conf1.get("fs.hdfs.impl"));
assertEquals("expect_odfs_abstract_impl", conf1.get("fs.AbstractFileSystem.hdfs.impl"));
Expand Down Expand Up @@ -116,47 +116,47 @@ public void applyDynamicClientConfTest() {
clientConf.put(mockKey, mockValue);

RssSparkShuffleUtils.applyDynamicClientConf(conf, clientConf);
assertEquals(remoteStoragePath, conf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH));
assertEquals(remoteStoragePath, conf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key()));
assertEquals(RssClientConfig.RSS_CLIENT_TYPE_DEFAULT_VALUE,
conf.get(RssSparkConfig.RSS_CLIENT_TYPE));
conf.get(RssSparkConfig.RSS_CLIENT_TYPE.key()));
assertEquals(Integer.toString(RssClientConfig.RSS_CLIENT_RETRY_MAX_DEFAULT_VALUE),
conf.get(RssSparkConfig.RSS_CLIENT_RETRY_MAX));
conf.get(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key()));
assertEquals(Long.toString(RssClientConfig.RSS_CLIENT_RETRY_INTERVAL_MAX_DEFAULT_VALUE),
conf.get(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX));
conf.get(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key()));
assertEquals(Integer.toString(RssClientConfig.RSS_DATA_REPLICA_DEFAULT_VALUE),
conf.get(RssSparkConfig.RSS_DATA_REPLICA));
conf.get(RssSparkConfig.RSS_DATA_REPLICA.key()));
assertEquals(Integer.toString(RssClientConfig.RSS_DATA_REPLICA_WRITE_DEFAULT_VALUE),
conf.get(RssSparkConfig.RSS_DATA_REPLICA_WRITE));
conf.get(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key()));
assertEquals(Integer.toString(RssClientConfig.RSS_DATA_REPLICA_READ_DEFAULT_VALUE),
conf.get(RssSparkConfig.RSS_DATA_REPLICA_READ));
conf.get(RssSparkConfig.RSS_DATA_REPLICA_READ.key()));
assertEquals(Long.toString(RssClientConfig.RSS_HEARTBEAT_INTERVAL_DEFAULT_VALUE),
conf.get(RssSparkConfig.RSS_HEARTBEAT_INTERVAL));
assertEquals(StorageType.MEMORY_LOCALFILE_HDFS.name(), conf.get(RssSparkConfig.RSS_STORAGE_TYPE));
conf.get(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key()));
assertEquals(StorageType.MEMORY_LOCALFILE_HDFS.name(), conf.get(RssSparkConfig.RSS_STORAGE_TYPE.key()));
assertEquals(Long.toString(RssClientConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE),
conf.get(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS));
conf.get(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key()));
assertEquals(Long.toString(RssClientConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE),
conf.get(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS));
conf.get(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key()));
assertEquals(Integer.toString(RssClientConfig.RSS_PARTITION_NUM_PER_RANGE_DEFAULT_VALUE),
conf.get(RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE));
conf.get(RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.key()));
assertEquals(Integer.toString(RssClientConfig.RSS_INDEX_READ_LIMIT_DEFAULT_VALUE),
conf.get(RssSparkConfig.RSS_INDEX_READ_LIMIT));
conf.get(RssSparkConfig.RSS_INDEX_READ_LIMIT.key()));
assertEquals(RssClientConfig.RSS_CLIENT_READ_BUFFER_SIZE_DEFAULT_VALUE,
conf.get(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE));
conf.get(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key()));
assertEquals(mockValue, conf.get(mockKey));

String remoteStoragePath2 = "hdfs://path2";
clientConf = Maps.newHashMap();
clientConf.put(RssClientConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_HDFS.name());
clientConf.put(RssSparkConfig.RSS_REMOTE_STORAGE_PATH, remoteStoragePath2);
clientConf.put(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key(), remoteStoragePath2);
clientConf.put(mockKey, "won't be rewrite");
clientConf.put(RssClientConfig.RSS_CLIENT_RETRY_MAX, "99999");
RssSparkShuffleUtils.applyDynamicClientConf(conf, clientConf);
// overwrite
assertEquals(remoteStoragePath2, conf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH));
assertEquals(StorageType.MEMORY_HDFS.name(), conf.get(RssSparkConfig.RSS_STORAGE_TYPE));
assertEquals(remoteStoragePath2, conf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key()));
assertEquals(StorageType.MEMORY_HDFS.name(), conf.get(RssSparkConfig.RSS_STORAGE_TYPE.key()));
// won't be overwrite
assertEquals(mockValue, conf.get(mockKey));
assertEquals(Integer.toString(RssClientConfig.RSS_CLIENT_RETRY_MAX_DEFAULT_VALUE),
conf.get(RssSparkConfig.RSS_CLIENT_RETRY_MAX));
conf.get(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -53,11 +53,11 @@ private WriteBufferManager createManager(SparkConf conf) {

private SparkConf getConf() {
SparkConf conf = new SparkConf(false);
conf.set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE, "64")
.set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE, "32")
.set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE, "128")
.set(RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE, "512")
.set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE, "190");
conf.set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key(), "64")
.set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key(), "32")
.set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key(), "128")
.set(RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE.key(), "512")
.set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key(), "190");
return conf;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,9 +46,7 @@ public class DelegationRssShuffleManager implements ShuffleManager {

public DelegationRssShuffleManager(SparkConf sparkConf, boolean isDriver) throws Exception {
this.sparkConf = sparkConf;
accessTimeoutMs = sparkConf.getInt(
RssSparkConfig.RSS_ACCESS_TIMEOUT_MS,
RssSparkConfig.RSS_ACCESS_TIMEOUT_MS_DEFAULT_VALUE);
accessTimeoutMs = sparkConf.get(RssSparkConfig.RSS_ACCESS_TIMEOUT_MS);
if (isDriver) {
coordinatorClients = RssSparkShuffleUtils.createCoordinatorClients(sparkConf);
delegate = createShuffleManagerInDriver();
Expand All @@ -69,7 +67,7 @@ private ShuffleManager createShuffleManagerInDriver() throws RssException {
if (canAccess) {
try {
shuffleManager = new RssShuffleManager(sparkConf, true);
sparkConf.set(RssSparkConfig.RSS_ENABLED, "true");
sparkConf.set(RssSparkConfig.RSS_ENABLED.key(), "true");
sparkConf.set("spark.shuffle.manager", RssShuffleManager.class.getCanonicalName());
LOG.info("Use RssShuffleManager");
return shuffleManager;
Expand All @@ -80,7 +78,7 @@ private ShuffleManager createShuffleManagerInDriver() throws RssException {

try {
shuffleManager = RssSparkShuffleUtils.loadShuffleManager(Constants.SORT_SHUFFLE_MANAGER_NAME, sparkConf, true);
sparkConf.set(RssSparkConfig.RSS_ENABLED, "false");
sparkConf.set(RssSparkConfig.RSS_ENABLED.key(), "false");
sparkConf.set("spark.shuffle.manager", "sort");
LOG.info("Use SortShuffleManager");
} catch (Exception e) {
Expand All @@ -92,7 +90,7 @@ private ShuffleManager createShuffleManagerInDriver() throws RssException {

private boolean tryAccessCluster() {
String accessId = sparkConf.get(
RssSparkConfig.RSS_ACCESS_ID, "").trim();
RssSparkConfig.RSS_ACCESS_ID.key(), "").trim();
if (StringUtils.isEmpty(accessId)) {
LOG.warn("Access id key is empty");
return false;
Expand Down Expand Up @@ -127,9 +125,7 @@ private boolean tryAccessCluster() {
private ShuffleManager createShuffleManagerInExecutor() throws RssException {
ShuffleManager shuffleManager;
// get useRSS from spark conf
boolean useRSS = sparkConf.getBoolean(
RssSparkConfig.RSS_ENABLED,
RssSparkConfig.RSS_USE_RSS_SHUFFLE_MANAGER_DEFAULT_VALUE);
boolean useRSS = sparkConf.get(RssSparkConfig.RSS_ENABLED);
if (useRSS) {
// Executor will not do any fallback
shuffleManager = new RssShuffleManager(sparkConf, false);
Expand Down