From e14ef60ea66ba7a25d4d4cf996ae3363147fc608 Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Fri, 29 Jul 2022 17:26:43 +0800 Subject: [PATCH 01/22] [Improvement] Use ConfigBuilder to rewrite the class RssSparkConfig --- .../apache/spark/shuffle/RssSparkConfig.java | 383 ++++++++++++------ .../spark/shuffle/RssSparkShuffleUtils.java | 27 +- .../shuffle/writer/BufferManagerOptions.java | 42 +- .../shuffle/RssSparkShuffleUtilsTest.java | 44 +- .../writer/WriteBufferManagerTest.java | 10 +- .../shuffle/DelegationRssShuffleManager.java | 12 +- .../shuffle/writer/RssShuffleWriter.java | 18 +- .../DelegationRssShuffleManagerTest.java | 24 +- .../shuffle/writer/RssShuffleWriterTest.java | 32 +- .../shuffle/DelegationRssShuffleManager.java | 14 +- .../shuffle/writer/RssShuffleWriter.java | 18 +- .../DelegationRssShuffleManagerTest.java | 24 +- .../shuffle/writer/RssShuffleWriterTest.java | 32 +- .../apache/uniffle/test/AutoAccessTest.java | 18 +- .../test/DynamicFetchClientConfTest.java | 16 +- ...epartitionWithHdfsMultiStorageRssTest.java | 2 +- .../test/RepartitionWithLocalFileRssTest.java | 2 +- ...artitionWithMemoryMultiStorageRssTest.java | 2 +- .../test/RepartitionWithMemoryRssTest.java | 2 +- .../apache/uniffle/test/SimpleTestBase.java | 2 +- .../test/SparkIntegrationTestBase.java | 24 +- .../SparkSQLWithDelegationShuffleManager.java | 4 +- ...LWithDelegationShuffleManagerFallback.java | 4 +- .../test/SparkSQLWithMemoryLocalTest.java | 2 +- .../apache/uniffle/test/GetReaderTest.java | 2 +- .../uniffle/test/AQERepartitionTest.java | 2 +- .../uniffle/test/AQESkewedJoinTest.java | 6 +- .../apache/uniffle/test/GetReaderTest.java | 2 +- 28 files changed, 459 insertions(+), 311 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index a9e845a8bc..459ee9078b 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -17,130 +17,277 @@ package org.apache.spark.shuffle; -import java.util.Set; - import com.google.common.collect.Sets; +import org.apache.spark.internal.config.ConfigBuilder; +import org.apache.spark.internal.config.ConfigEntry; -import org.apache.uniffle.client.util.RssClientConfig; +import java.util.Set; public class RssSparkConfig { + public static final class SparkConfigBuilder { + public static ConfigBuilder configBuilder; + + public static ConfigBuilder key(String key) { + configBuilder = new ConfigBuilder(key); + return configBuilder; + } + } + + public static class RssConfigEntry { + public ConfigEntry entry; + public String key; + + public RssConfigEntry(ConfigEntry entry) { + this.entry = entry; + this.key = entry.key(); + } + + public T getValue() { + return (T) entry.defaultValue(). get(); + } + } + + public static RssConfigEntry RSS_PARTITION_NUM_PER_RANGE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.partitionNum.per.range") + .intConf() + .createWithDefault(1)); + + public static RssConfigEntry RSS_WRITER_BUFFER_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.writer.buffer.size") + .doc("controls the buffer flushing size during shuffle write") + .stringConf() + .createWithDefault("3m")); + + public static RssConfigEntry RSS_WRITER_SERIALIZER_BUFFER_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.writer.serializer.buffer.size") + .stringConf() + .createWithDefault("3k")); + + public static RssConfigEntry RSS_WRITER_BUFFER_SEGMENT_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.writer.buffer.segment.size") + .stringConf() + .createWithDefault("3k")); + + public static RssConfigEntry RSS_WRITER_BUFFER_SPILL_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.writer.buffer.spill.size") + .stringConf() + .createWithDefault("128m")); + + public static RssConfigEntry RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.writer.pre.allocated.buffer.size") + .stringConf() + .createWithDefault("16m")); + + public static RssConfigEntry RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.writer.require.memory.retryMax") + .intConf() + .createWithDefault(1200)); + + public static RssConfigEntry RSS_WRITER_REQUIRE_MEMORY_INTERVAL = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.writer.require.memory.interval") + .longConf() + .createWithDefault(1000)); + + public static RssConfigEntry RSS_CLIENT_SEND_CHECK_TIMEOUT_MS = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.send.check.timeout.ms") + .longConf() + .createWithDefault(60 * 1000 * 10)); + + public static RssConfigEntry RSS_CLIENT_SEND_CHECK_INTERVAL_MS = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.send.check.interval.ms") + .longConf() + .createWithDefault(500)); + + public static RssConfigEntry RSS_TEST_FLAG = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.test") + .stringConf() + .createWithDefault("")); + + public static RssConfigEntry RSS_REMOTE_STORAGE_PATH = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.remote.storage.path") + .stringConf() + .createWithDefault("")); + + public static RssConfigEntry RSS_INDEX_READ_LIMIT = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.index.read.limit") + .intConf() + .createWithDefault(500)); + + public static RssConfigEntry RSS_CLIENT_TYPE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.type") + .stringConf() + .createWithDefault("GRPC")); + + public static RssConfigEntry RSS_STORAGE_TYPE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.storage.type") + .stringConf() + .createWithDefault("")); + + public static RssConfigEntry RSS_CLIENT_RETRY_MAX = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.retry.max") + .intConf() + .createWithDefault(100)); + + public static RssConfigEntry RSS_CLIENT_RETRY_INTERVAL_MAX = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.retry.interval.max") + .longConf() + .createWithDefault(10000)); + + public static RssConfigEntry RSS_CLIENT_HEARTBEAT_THREAD_NUM = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.heartBeat.threadNum") + .intConf() + .createWithDefault(4)); + + public static RssConfigEntry RSS_CLIENT_SEND_SIZE_LIMIT = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.send.size.limit") + .stringConf() + .createWithDefault("16m")); + + public static RssConfigEntry RSS_CLIENT_READ_BUFFER_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.read.buffer.size") + .stringConf() + .createWithDefault("14m")); + + public static RssConfigEntry RSS_HEARTBEAT_INTERVAL = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.heartbeat.interval") + .longConf() + .createWithDefault(10 * 1000L)); + + public static RssConfigEntry RSS_HEARTBEAT_TIMEOUT = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.heartbeat.timeout") + .longConf() + .createWithDefault(5 * 1000L)); + + public static RssConfigEntry RSS_CLIENT_SEND_THREAD_POOL_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.send.threadPool.size") + .intConf() + .createWithDefault(10)); + + public static RssConfigEntry RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.send.threadPool.keepalive") + .intConf() + .createWithDefault(60)); + + public static RssConfigEntry RSS_DATA_REPLICA = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.data.replica") + .intConf() + .createWithDefault(1)); + + public static RssConfigEntry RSS_DATA_REPLICA_WRITE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.data.replica.write") + .intConf() + .createWithDefault(1)); + + public static RssConfigEntry RSS_DATA_REPLICA_READ = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.data.replica.read") + .intConf() + .createWithDefault(1)); + + public static RssConfigEntry RSS_DATA_REPLICA_SKIP_ENABLED = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.data.replica.skip.enabled") + .booleanConf() + .createWithDefault(true)); + + public static RssConfigEntry RSS_DATA_TRANSFER_POOL_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.data.transfer.pool.size") + .intConf() + .createWithDefault(Runtime.getRuntime().availableProcessors())); + + public static RssConfigEntry RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.ozone.dfs.namenode.odfs.enable") + .booleanConf() + .createWithDefault(false)); + + public static RssConfigEntry RSS_OZONE_FS_HDFS_IMPL = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.ozone.fs.hdfs.impl") + .stringConf() + .createWithDefault("org.apache.hadoop.odfs.HdfsOdfsFilesystem")); + + public static RssConfigEntry RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.ozone.fs.AbstractFileSystem.hdfs.impl") + .stringConf() + .createWithDefault("org.apache.hadoop.odfs.HdfsOdfs")); + + public static RssConfigEntry RSS_CLIENT_BITMAP_SPLIT_NUM = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.bitmap.splitNum") + .intConf() + .createWithDefault(1)); + + public static RssConfigEntry RSS_ACCESS_ID = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.access.id") + .stringConf() + .createWithDefault("")); + + public static RssConfigEntry RSS_ACCESS_TIMEOUT_MS = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.access.timeout.ms") + .intConf() + .createWithDefault(10000)); + + public static RssConfigEntry RSS_ENABLED = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.enabled") + .booleanConf() + .createWithDefault(false)); + + public static RssConfigEntry RSS_DYNAMIC_CLIENT_CONF_ENABLED = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.dynamicClientConf.enabled") + .booleanConf() + .createWithDefault(true)); + + public static RssConfigEntry RSS_CLIENT_ASSIGNMENT_TAGS = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.assignment.tags") + .stringConf() + .createWithDefault("")); + + public static RssConfigEntry RSS_COORDINATOR_QUORUM = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.coordinator.quorum") + .stringConf() + .createWithDefault("")); + + public static final Set RSS_MANDATORY_CLUSTER_CONF = + Sets.newHashSet(RSS_STORAGE_TYPE.key, RSS_REMOTE_STORAGE_PATH.key); + public static final String SPARK_RSS_CONFIG_PREFIX = "spark."; - public static final String RSS_PARTITION_NUM_PER_RANGE = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_PARTITION_NUM_PER_RANGE; - public static final int RSS_PARTITION_NUM_PER_RANGE_DEFAULT_VALUE = - RssClientConfig.RSS_PARTITION_NUM_PER_RANGE_DEFAULT_VALUE; - public static final String RSS_WRITER_BUFFER_SIZE = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_WRITER_BUFFER_SIZE; - public static final String RSS_WRITER_BUFFER_SIZE_DEFAULT_VALUE = "3m"; - public static final String RSS_WRITER_SERIALIZER_BUFFER_SIZE = - SPARK_RSS_CONFIG_PREFIX + "rss.writer.serializer.buffer.size"; - public static final String RSS_WRITER_SERIALIZER_BUFFER_SIZE_DEFAULT_VALUE = "3k"; - public static final String RSS_WRITER_BUFFER_SEGMENT_SIZE = - SPARK_RSS_CONFIG_PREFIX + "rss.writer.buffer.segment.size"; - public static final String RSS_WRITER_BUFFER_SEGMENT_SIZE_DEFAULT_VALUE = "3k"; - public static final String RSS_WRITER_BUFFER_SPILL_SIZE = - SPARK_RSS_CONFIG_PREFIX + "rss.writer.buffer.spill.size"; - public static final String RSS_WRITER_BUFFER_SPILL_SIZE_DEFAULT_VALUE = "128m"; - public static final String RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE = - SPARK_RSS_CONFIG_PREFIX + "rss.writer.pre.allocated.buffer.size"; - public static final String RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE_DEFAULT_VALUE = "16m"; - public static final String RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX = - SPARK_RSS_CONFIG_PREFIX + "rss.writer.require.memory.retryMax"; - public static final int RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX_DEFAULT_VALUE = 1200; - public static final String RSS_WRITER_REQUIRE_MEMORY_INTERVAL = - SPARK_RSS_CONFIG_PREFIX + "rss.writer.require.memory.interval"; - public static final long RSS_WRITER_REQUIRE_MEMORY_INTERVAL_DEFAULT_VALUE = 1000; // 1s - public static final String RSS_COORDINATOR_QUORUM = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_COORDINATOR_QUORUM; - public static final String RSS_CLIENT_SEND_CHECK_TIMEOUT_MS = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS; - public static final long RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE = - RssClientConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE; // 10 min - public static final String RSS_CLIENT_SEND_CHECK_INTERVAL_MS = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS; - public static final long RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE = - RssClientConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE; - public static final String RSS_TEST_FLAG = SPARK_RSS_CONFIG_PREFIX + "rss.test"; - public static final String RSS_REMOTE_STORAGE_PATH = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_REMOTE_STORAGE_PATH; - public static final String RSS_INDEX_READ_LIMIT = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_INDEX_READ_LIMIT; - public static final int RSS_INDEX_READ_LIMIT_DEFAULT_VALUE = 500; - public static final String RSS_CLIENT_TYPE = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_TYPE; - public static final String RSS_CLIENT_TYPE_DEFAULT_VALUE = RssClientConfig.RSS_CLIENT_TYPE_DEFAULT_VALUE; - public static final String RSS_STORAGE_TYPE = SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_STORAGE_TYPE; - public static final String RSS_CLIENT_RETRY_MAX = SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_RETRY_MAX; - public static final int RSS_CLIENT_RETRY_MAX_DEFAULT_VALUE = RssClientConfig.RSS_CLIENT_RETRY_MAX_DEFAULT_VALUE; - public static final String RSS_CLIENT_RETRY_INTERVAL_MAX = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_RETRY_INTERVAL_MAX; - public static final long RSS_CLIENT_RETRY_INTERVAL_MAX_DEFAULT_VALUE = - RssClientConfig.RSS_CLIENT_RETRY_INTERVAL_MAX_DEFAULT_VALUE; - public static final String RSS_CLIENT_HEARTBEAT_THREAD_NUM = - SPARK_RSS_CONFIG_PREFIX + "rss.client.heartBeat.threadNum"; - public static final int RSS_CLIENT_HEARTBEAT_THREAD_NUM_DEFAULT_VALUE = 4; - public static final String RSS_CLIENT_SEND_SIZE_LIMIT = SPARK_RSS_CONFIG_PREFIX + "rss.client.send.size.limit"; - public static final String RSS_CLIENT_SEND_SIZE_LIMIT_DEFAULT_VALUE = "16m"; - public static final String RSS_CLIENT_READ_BUFFER_SIZE = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_READ_BUFFER_SIZE; - // When the size of read buffer reaches the half of JVM region (i.e., 32m), - // it will incur humongous allocation, so we set it to 14m. - public static final String RSS_CLIENT_READ_BUFFER_SIZE_DEFAULT_VALUE = - RssClientConfig.RSS_CLIENT_READ_BUFFER_SIZE_DEFAULT_VALUE; - public static final String RSS_HEARTBEAT_INTERVAL = SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_HEARTBEAT_INTERVAL; - public static final long RSS_HEARTBEAT_INTERVAL_DEFAULT_VALUE = RssClientConfig.RSS_HEARTBEAT_INTERVAL_DEFAULT_VALUE; - public static final String RSS_HEARTBEAT_TIMEOUT = SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_HEARTBEAT_TIMEOUT; - public static final String RSS_CLIENT_SEND_THREAD_POOL_SIZE = - SPARK_RSS_CONFIG_PREFIX + "rss.client.send.threadPool.size"; - public static final int RSS_CLIENT_SEND_THREAD_POOL_SIZE_DEFAULT_VALUE = 10; - public static final String RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE = - SPARK_RSS_CONFIG_PREFIX + "rss.client.send.threadPool.keepalive"; - public static final int RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE_DEFAULT_VALUE = 60; - public static final String RSS_DATA_REPLICA = SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA; - public static final int RSS_DATA_REPLICA_DEFAULT_VALUE = RssClientConfig.RSS_DATA_REPLICA_DEFAULT_VALUE; - public static final String RSS_DATA_REPLICA_WRITE = SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA_WRITE; - public static final int RSS_DATA_REPLICA_WRITE_DEFAULT_VALUE = RssClientConfig.RSS_DATA_REPLICA_WRITE_DEFAULT_VALUE; - public static final String RSS_DATA_REPLICA_READ = SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA_READ; - public static final int RSS_DATA_REPLICA_READ_DEFAULT_VALUE = RssClientConfig.RSS_DATA_REPLICA_READ_DEFAULT_VALUE; - public static final String RSS_DATA_REPLICA_SKIP_ENABLED = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA_SKIP_ENABLED; - public static final String RSS_DATA_TRANSFER_POOL_SIZE = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_TRANSFER_POOL_SIZE; - public static final int RSS_DATA_TRANSFER_POOL_SIZE_DEFAULT_VALUE = - RssClientConfig.RSS_DATA_TRANFER_POOL_SIZE_DEFAULT_VALUE; - public static final String RSS_DATA_COMMIT_POOL_SIZE = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_COMMIT_POOL_SIZE; - public static final int RSS_DATA_COMMIT_POOL_SIZE_DEFAULT_VALUE = - RssClientConfig.RSS_DATA_COMMIT_POOL_SIZE_DEFAULT_VALUE; - - public static final boolean RSS_DATA_REPLICA_SKIP_ENABLED_DEFAULT_VALUE = - RssClientConfig.RSS_DATA_REPLICA_SKIP_ENABLED_DEFAULT_VALUE; - public static final String RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE = - SPARK_RSS_CONFIG_PREFIX + "rss.ozone.dfs.namenode.odfs.enable"; + public static final boolean RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE_DEFAULT_VALUE = false; - public static final String RSS_OZONE_FS_HDFS_IMPL = SPARK_RSS_CONFIG_PREFIX + "rss.ozone.fs.hdfs.impl"; - public static final String RSS_OZONE_FS_HDFS_IMPL_DEFAULT_VALUE = "org.apache.hadoop.odfs.HdfsOdfsFilesystem"; - public static final String RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL = - SPARK_RSS_CONFIG_PREFIX + "rss.ozone.fs.AbstractFileSystem.hdfs.impl"; - public static final String RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL_DEFAULT_VALUE = - "org.apache.hadoop.odfs.HdfsOdfs"; - // todo: remove unnecessary configuration - public static final String RSS_CLIENT_BITMAP_SPLIT_NUM = - SPARK_RSS_CONFIG_PREFIX + "rss.client.bitmap.splitNum"; - public static final int RSS_CLIENT_BITMAP_SPLIT_NUM_DEFAULT_VALUE = 1; - public static final String RSS_ACCESS_ID = SPARK_RSS_CONFIG_PREFIX + "rss.access.id"; - public static final String RSS_ACCESS_TIMEOUT_MS = SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_ACCESS_TIMEOUT_MS; - public static final int RSS_ACCESS_TIMEOUT_MS_DEFAULT_VALUE = RssClientConfig.RSS_ACCESS_TIMEOUT_MS_DEFAULT_VALUE; - public static final String RSS_ENABLED = SPARK_RSS_CONFIG_PREFIX + "rss.enabled"; - public static final boolean RSS_USE_RSS_SHUFFLE_MANAGER_DEFAULT_VALUE = false; - public static final String RSS_DYNAMIC_CLIENT_CONF_ENABLED = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED; - public static final boolean RSS_DYNAMIC_CLIENT_CONF_ENABLED_DEFAULT_VALUE = - RssClientConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED_DEFAULT_VALUE; - public static final String RSS_CLIENT_ASSIGNMENT_TAGS = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_ASSIGNMENT_TAGS; - public static final Set RSS_MANDATORY_CLUSTER_CONF = - Sets.newHashSet(RSS_STORAGE_TYPE, RSS_REMOTE_STORAGE_PATH); } diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java index a4874f5129..9e5f99f038 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java @@ -44,19 +44,20 @@ 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.getBoolean(RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE.key, + RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE.getValue()); 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.key, + RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.getValue())); conf.set( - RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.substring(OZONE_PREFIX_LEN), + 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, - RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL_DEFAULT_VALUE)); + RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.key, + RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.getValue())); } return conf; @@ -77,9 +78,9 @@ public static ShuffleManager loadShuffleManager(String name, SparkConf conf, boo } public static List createCoordinatorClients(SparkConf sparkConf) throws RuntimeException { - String clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE, - RssSparkConfig.RSS_CLIENT_TYPE_DEFAULT_VALUE); - String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM); + String clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key, + RssSparkConfig.RSS_CLIENT_TYPE.getValue()); + String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM.key); CoordinatorClientFactory coordinatorClientFactory = new CoordinatorClientFactory(clientType); return coordinatorClientFactory.createCoordinatorClient(coordinators); } @@ -110,7 +111,7 @@ public static void applyDynamicClientConf(SparkConf sparkConf, Map getAssignmentTags(SparkConf sparkConf) { Set 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(","))); diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java index f817c00414..e027c1a1e9 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java @@ -35,38 +35,38 @@ 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.getValue()); + serializerBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key, + RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.getValue()); + bufferSegmentSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key, + RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.getValue()); + bufferSpillThreshold = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key, + RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.getValue()); + preAllocatedBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE.key, + RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE.getValue()); + requireMemoryInterval = sparkConf.getLong(RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_INTERVAL.key, + RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_INTERVAL.getValue()); + requireMemoryRetryMax = sparkConf.getInt(RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX.key, + RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX.getValue()); + 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 + "]"); } } diff --git a/client-spark/common/src/test/java/org/apache/spark/shuffle/RssSparkShuffleUtilsTest.java b/client-spark/common/src/test/java/org/apache/spark/shuffle/RssSparkShuffleUtilsTest.java index f15c4cfddc..8a5f6591b1 100644 --- a/client-spark/common/src/test/java/org/apache/spark/shuffle/RssSparkShuffleUtilsTest.java +++ b/client-spark/common/src/test/java/org/apache/spark/shuffle/RssSparkShuffleUtilsTest.java @@ -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 iterator = tags.iterator(); @@ -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")); @@ -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)); } } diff --git a/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/WriteBufferManagerTest.java b/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/WriteBufferManagerTest.java index 3e62f94926..0fc55f52bf 100644 --- a/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/WriteBufferManagerTest.java +++ b/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/WriteBufferManagerTest.java @@ -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; } diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java index 01c4cf6e22..45d2934820 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java @@ -47,8 +47,8 @@ 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); + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key, + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getValue()); if (isDriver) { coordinatorClients = RssSparkShuffleUtils.createCoordinatorClients(sparkConf); delegate = createShuffleManagerInDriver(); @@ -69,7 +69,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; @@ -80,7 +80,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) { @@ -92,7 +92,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; @@ -128,7 +128,7 @@ private ShuffleManager createShuffleManagerInExecutor() throws RssException { ShuffleManager shuffleManager; // get useRSS from spark conf boolean useRSS = sparkConf.getBoolean( - RssSparkConfig.RSS_ENABLED, + RssSparkConfig.RSS_ENABLED.key, RssSparkConfig.RSS_USE_RSS_SHUFFLE_MANAGER_DEFAULT_VALUE); if (useRSS) { // Executor will not do any fallback diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java index 19012584aa..4e51f3c606 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java @@ -105,20 +105,20 @@ public RssShuffleWriter( this.partitioner = shuffleDependency.partitioner(); this.shuffleManager = shuffleManager; this.shouldPartition = partitioner.numPartitions() > 1; - this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS, - RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE); - this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS, - RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE); - this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT, - RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT_DEFAULT_VALUE); - this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM, - RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM_DEFAULT_VALUE); + this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, + RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.getValue()); + this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, + RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.getValue()); + this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key, + RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.getValue()); + this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.key, + RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.getValue()); this.partitionToBlockIds = Maps.newConcurrentMap(); this.shuffleWriteClient = shuffleWriteClient; this.shuffleServersForData = rssHandle.getShuffleServersForData(); this.partitionToServers = rssHandle.getPartitionToServers(); this.isMemoryShuffleEnabled = isMemoryShuffleEnabled( - sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE)); + sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key)); } private boolean isMemoryShuffleEnabled(String storageType) { diff --git a/client-spark/spark2/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java b/client-spark/spark2/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java index 5936c9a4b7..c357bcbfa4 100644 --- a/client-spark/spark2/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java +++ b/client-spark/spark2/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java @@ -65,7 +65,7 @@ public void testCreateInDriverDenied() throws Exception { mockedStaticRssShuffleUtils.when(() -> RssSparkShuffleUtils.createCoordinatorClients(any())).thenReturn(coordinatorClients); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED, "false"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); assertCreateSortShuffleManager(conf); } @@ -81,15 +81,15 @@ public void testCreateInDriver() throws Exception { SparkConf conf = new SparkConf(); assertCreateSortShuffleManager(conf); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED, "false"); - conf.set(RssSparkConfig.RSS_ACCESS_ID, "mockId"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); + conf.set(RssSparkConfig.RSS_ACCESS_ID.key, "mockId"); conf.set("spark.rss.storage.type", StorageType.LOCALFILE.name()); assertCreateSortShuffleManager(conf); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); assertCreateRssShuffleManager(conf); conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); when(mockCoordinatorClient.accessCluster(any())).thenReturn( new RssAccessClusterResponse(SUCCESS, "")); assertCreateSortShuffleManager(conf); @@ -99,7 +99,7 @@ public void testCreateInDriver() throws Exception { public void testCreateInExecutor() throws Exception { DelegationRssShuffleManager delegationRssShuffleManager; SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); delegationRssShuffleManager = new DelegationRssShuffleManager(conf, false); assertFalse(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); assertTrue(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); @@ -115,15 +115,15 @@ public void testCreateFallback() throws Exception { RssSparkShuffleUtils.createCoordinatorClients(any())).thenReturn(coordinatorClients); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED, "false"); - conf.set(RssSparkConfig.RSS_ACCESS_ID, "mockId"); - conf.set(RssSparkConfig.RSS_ENABLED, "true"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); + conf.set(RssSparkConfig.RSS_ACCESS_ID.key, "mockId"); + conf.set(RssSparkConfig.RSS_ENABLED.key, "true"); // fall back to SortShuffleManager in driver assertCreateSortShuffleManager(conf); // No fall back in executor - conf.set(RssSparkConfig.RSS_ENABLED, "true"); + conf.set(RssSparkConfig.RSS_ENABLED.key, "true"); boolean hasException = false; try { new DelegationRssShuffleManager(conf, false); @@ -138,7 +138,7 @@ private DelegationRssShuffleManager assertCreateSortShuffleManager(SparkConf con DelegationRssShuffleManager delegationRssShuffleManager = new DelegationRssShuffleManager(conf, true); assertTrue(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); assertFalse(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); - assertFalse(conf.getBoolean(RssSparkConfig.RSS_ENABLED, false)); + assertFalse(conf.getBoolean(RssSparkConfig.RSS_ENABLED.key, false)); assertEquals("sort", conf.get("spark.shuffle.manager")); return delegationRssShuffleManager; } @@ -147,7 +147,7 @@ private DelegationRssShuffleManager assertCreateRssShuffleManager(SparkConf conf DelegationRssShuffleManager delegationRssShuffleManager = new DelegationRssShuffleManager(conf, true); assertFalse(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); assertTrue(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); - assertTrue(Boolean.parseBoolean(conf.get(RssSparkConfig.RSS_ENABLED))); + assertTrue(Boolean.parseBoolean(conf.get(RssSparkConfig.RSS_ENABLED.key))); assertEquals(RssShuffleManager.class.getCanonicalName(), conf.get("spark.shuffle.manager")); return delegationRssShuffleManager; } diff --git a/client-spark/spark2/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java b/client-spark/spark2/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java index 1ef7d2b99e..f67171850a 100644 --- a/client-spark/spark2/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java +++ b/client-spark/spark2/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java @@ -68,11 +68,11 @@ public void checkBlockSendResultTest() { String taskId = "checkBlockSendResultTest_taskId"; conf.setAppName("testApp") .setMaster("local[2]") - .set(RssSparkConfig.RSS_TEST_FLAG, "true") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS, "10000") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS, "1000") - .set(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.LOCALFILE.name()) - .set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "127.0.0.1:12345,127.0.0.1:12346"); + .set(RssSparkConfig.RSS_TEST_FLAG.key, "true") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, "10000") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, "1000") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()) + .set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "127.0.0.1:12345,127.0.0.1:12346"); // init SparkContext SparkContext sc = SparkContext.getOrCreate(conf); RssShuffleManager manager = new RssShuffleManager(conf, false); @@ -131,15 +131,15 @@ public void checkBlockSendResultTest() { public void writeTest() throws Exception { SparkConf conf = new SparkConf(); conf.setAppName("testApp").setMaster("local[2]") - .set(RssSparkConfig.RSS_TEST_FLAG, "true") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE, "32") - .set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE, "32") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE, "64") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE, "128") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS, "10000") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS, "1000") - .set(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.LOCALFILE.name()) - .set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "127.0.0.1:12345,127.0.0.1:12346"); + .set(RssSparkConfig.RSS_TEST_FLAG.key, "true") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key, "32") + .set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key, "32") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key, "64") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key, "128") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, "10000") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, "1000") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()) + .set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "127.0.0.1:12345,127.0.0.1:12346"); // init SparkContext SparkContext sc = SparkContext.getOrCreate(conf); RssShuffleManager manager = new RssShuffleManager(conf, false); @@ -273,8 +273,8 @@ public void onError(Throwable e) { when(mockHandle.getDependency()).thenReturn(mockDependency); ShuffleWriteClient mockWriteClient = mock(ShuffleWriteClient.class); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT, "64") - .set(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.LOCALFILE.name()); + conf.set(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key, "64") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()); RssShuffleWriter writer = new RssShuffleWriter("appId", 0, "taskId", 1L, mockBufferManager, mockMetrics, mockShuffleManager, conf, mockWriteClient, mockHandle); diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java index 6ec8901013..27d76a6b66 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java @@ -47,8 +47,8 @@ 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); + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key, + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getValue()); if (isDriver) { coordinatorClients = RssSparkShuffleUtils.createCoordinatorClients(sparkConf); delegate = createShuffleManagerInDriver(); @@ -69,7 +69,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; @@ -80,7 +80,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) { @@ -92,7 +92,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; @@ -128,8 +128,8 @@ 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); + RssSparkConfig.RSS_ENABLED.key, + RssSparkConfig.RSS_ENABLED.getValue()); if (useRSS) { // Executor will not do any fallback shuffleManager = new RssShuffleManager(sparkConf, false); diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java index bed94034ea..7d3b21f551 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java @@ -106,14 +106,14 @@ public RssShuffleWriter( this.shuffleDependency = rssHandle.getDependency(); this.partitioner = shuffleDependency.partitioner(); this.shouldPartition = partitioner.numPartitions() > 1; - this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS, - RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE); - this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS, - RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE); - this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT, - RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT_DEFAULT_VALUE); - this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM, - RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM_DEFAULT_VALUE); + this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, + RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.getValue()); + this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, + RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.getValue()); + this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key, + RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.getValue()); + this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.key, + RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.getValue()); this.partitionToBlockIds = Maps.newConcurrentMap(); this.shuffleWriteClient = shuffleWriteClient; this.shuffleServersForData = rssHandle.getShuffleServersForData(); @@ -121,7 +121,7 @@ public RssShuffleWriter( Arrays.fill(partitionLengths, 0); partitionToServers = rssHandle.getPartitionToServers(); this.isMemoryShuffleEnabled = isMemoryShuffleEnabled( - sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE)); + sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key)); } private boolean isMemoryShuffleEnabled(String storageType) { diff --git a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java index a96b52ac58..88999f4d7e 100644 --- a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java +++ b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java @@ -66,7 +66,7 @@ public void testCreateInDriverDenied() throws Exception { mockedStaticRssShuffleUtils.when(() -> RssSparkShuffleUtils.createCoordinatorClients(any())).thenReturn(coordinatorClients); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED, "false"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); assertCreateSortShuffleManager(conf); } @@ -82,15 +82,15 @@ public void testCreateInDriver() throws Exception { SparkConf conf = new SparkConf(); assertCreateSortShuffleManager(conf); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED, "false"); - conf.set(RssSparkConfig.RSS_ACCESS_ID, "mockId"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); + conf.set(RssSparkConfig.RSS_ACCESS_ID.key, "mockId"); assertCreateSortShuffleManager(conf); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); conf.set("spark.rss.storage.type", StorageType.LOCALFILE.name()); assertCreateRssShuffleManager(conf); conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); when(mockCoordinatorClient.accessCluster(any())).thenReturn( new RssAccessClusterResponse(SUCCESS, "")); assertCreateSortShuffleManager(conf); @@ -100,7 +100,7 @@ public void testCreateInDriver() throws Exception { public void testCreateInExecutor() throws Exception { DelegationRssShuffleManager delegationRssShuffleManager; SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); delegationRssShuffleManager = new DelegationRssShuffleManager(conf, false); assertFalse(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); assertTrue(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); @@ -116,15 +116,15 @@ public void testCreateFallback() throws Exception { RssSparkShuffleUtils.createCoordinatorClients(any())).thenReturn(coordinatorClients); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED, "false"); - conf.set(RssSparkConfig.RSS_ACCESS_ID, "mockId"); - conf.set(RssSparkConfig.RSS_ENABLED, "true"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); + conf.set(RssSparkConfig.RSS_ACCESS_ID.key, "mockId"); + conf.set(RssSparkConfig.RSS_ENABLED.key, "true"); // fall back to SortShuffleManager in driver assertCreateSortShuffleManager(conf); // No fall back in executor - conf.set(RssSparkConfig.RSS_ENABLED, "true"); + conf.set(RssSparkConfig.RSS_ENABLED.key, "true"); boolean hasException = false; try { new DelegationRssShuffleManager(conf, false); @@ -139,7 +139,7 @@ private DelegationRssShuffleManager assertCreateSortShuffleManager(SparkConf con DelegationRssShuffleManager delegationRssShuffleManager = new DelegationRssShuffleManager(conf, true); assertTrue(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); assertFalse(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); - assertFalse(conf.getBoolean(RssSparkConfig.RSS_ENABLED, false)); + assertFalse(conf.getBoolean(RssSparkConfig.RSS_ENABLED.key, false)); assertEquals("sort", conf.get("spark.shuffle.manager")); return delegationRssShuffleManager; } @@ -148,7 +148,7 @@ private DelegationRssShuffleManager assertCreateRssShuffleManager(SparkConf conf DelegationRssShuffleManager delegationRssShuffleManager = new DelegationRssShuffleManager(conf, true); assertFalse(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); assertTrue(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); - assertTrue(Boolean.parseBoolean(conf.get(RssSparkConfig.RSS_ENABLED))); + assertTrue(Boolean.parseBoolean(conf.get(RssSparkConfig.RSS_ENABLED.key))); assertEquals(RssShuffleManager.class.getCanonicalName(), conf.get("spark.shuffle.manager")); return delegationRssShuffleManager; } diff --git a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java index a943a453b5..8c2c03bd15 100644 --- a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java +++ b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java @@ -68,11 +68,11 @@ public void checkBlockSendResultTest() { SparkConf conf = new SparkConf(); conf.setAppName("testApp") .setMaster("local[2]") - .set(RssSparkConfig.RSS_TEST_FLAG, "true") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS, "10000") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS, "1000") - .set(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.LOCALFILE.name()) - .set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "127.0.0.1:12345,127.0.0.1:12346"); + .set(RssSparkConfig.RSS_TEST_FLAG.key, "true") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, "10000") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, "1000") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()) + .set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "127.0.0.1:12345,127.0.0.1:12346"); // init SparkContext SparkContext sc = SparkContext.getOrCreate(conf); Map> failBlocks = Maps.newConcurrentMap(); @@ -135,15 +135,15 @@ public void checkBlockSendResultTest() { public void writeTest() throws Exception { SparkConf conf = new SparkConf(); conf.setAppName("testApp").setMaster("local[2]") - .set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE, "32") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE, "32") - .set(RssSparkConfig.RSS_TEST_FLAG, "true") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE, "64") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS, "10000") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS, "1000") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE, "128") - .set(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.LOCALFILE.name()) - .set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "127.0.0.1:12345,127.0.0.1:12346"); + .set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key, "32") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key, "32") + .set(RssSparkConfig.RSS_TEST_FLAG.key, "true") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key, "64") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, "10000") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, "1000") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key, "128") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()) + .set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "127.0.0.1:12345,127.0.0.1:12346"); // init SparkContext List shuffleBlockInfos = Lists.newArrayList(); SparkContext sc = SparkContext.getOrCreate(conf); @@ -303,8 +303,8 @@ public void onError(Throwable e) { when(mockHandle.getDependency()).thenReturn(mockDependency); ShuffleWriteClient mockWriteClient = mock(ShuffleWriteClient.class); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT, "64") - .set(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE.name()); + conf.set(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key, "64") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE.name()); List shuffleBlockInfoList = createShuffleBlockList(1, 31); RssShuffleWriter writer = new RssShuffleWriter("appId", 0, "taskId", 1L, mockBufferManager, mockMetrics, mockShuffleManager, conf, mockWriteClient, mockHandle); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/AutoAccessTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/AutoAccessTest.java index 15523ba048..eedcce7e37 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/AutoAccessTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/AutoAccessTest.java @@ -48,9 +48,9 @@ public class AutoAccessTest extends IntegrationTestBase { public void test() throws Exception { SparkConf sparkConf = new SparkConf(); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.DelegationRssShuffleManager"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); sparkConf.set("spark.mock.2", "no-overwrite-conf"); - sparkConf.set(RssSparkConfig.RSS_REMOTE_STORAGE_PATH, "overwrite-path"); + sparkConf.set(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key, "overwrite-path"); sparkConf.set("spark.shuffle.service.enabled", "true"); String cfgFile = HDFS_URI + "/test/client_conf"; @@ -61,7 +61,7 @@ public void test() throws Exception { printWriter.println(" spark.mock.2 overwrite-conf "); printWriter.println(" spark.mock.3 true "); printWriter.println("spark.rss.storage.type " + StorageType.MEMORY_LOCALFILE_HDFS.name()); - printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH + " expectedPath"); + printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key + " expectedPath"); printWriter.flush(); printWriter.close(); @@ -100,7 +100,7 @@ public void test() throws Exception { ShuffleManager shuffleManager = delegationRssShuffleManager.getDelegate(); assertTrue(shuffleManager instanceof SortShuffleManager); assertTrue(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); - assertEquals("overwrite-path", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH)); + assertEquals("overwrite-path", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); assertFalse(sparkConf.contains("spark.rss.storage.type")); // wrong access id @@ -108,7 +108,7 @@ public void test() throws Exception { delegationRssShuffleManager = new DelegationRssShuffleManager(sparkConf, true); shuffleManager = delegationRssShuffleManager.getDelegate(); assertTrue(shuffleManager instanceof SortShuffleManager); - assertEquals("overwrite-path", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH)); + assertEquals("overwrite-path", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); assertTrue(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); assertFalse(sparkConf.contains("spark.rss.storage.type")); @@ -122,7 +122,7 @@ public void test() throws Exception { assertEquals("no-overwrite-conf", sparkConf.get("spark.mock.2")); assertTrue(sparkConf.getBoolean("spark.mock.3", false)); assertEquals(StorageType.MEMORY_LOCALFILE_HDFS.name(), sparkConf.get("spark.rss.storage.type")); - assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH)); + assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); assertFalse(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); // update candidates file @@ -149,7 +149,7 @@ public void test() throws Exception { assertEquals("no-overwrite-conf", sparkConf.get("spark.mock.2")); assertTrue(sparkConf.getBoolean("spark.mock.3", false)); assertEquals(StorageType.MEMORY_LOCALFILE_HDFS.name(), sparkConf.get("spark.rss.storage.type")); - assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH)); + assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); assertFalse(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); // update client conf file @@ -162,7 +162,7 @@ public void test() throws Exception { printWriter.println(" spark.mock.2 overwrite-conf "); printWriter.println(" spark.mock.3 false "); printWriter.println("spark.rss.storage.type " + StorageType.MEMORY_LOCALFILE_HDFS.name()); - printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH + " expectedPathNew"); + printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key + " expectedPathNew"); printWriter.flush(); printWriter.close(); fs.rename(tmpPath, path); @@ -177,7 +177,7 @@ public void test() throws Exception { assertEquals("overwrite-conf", sparkConf.get("spark.mock.2")); assertTrue(sparkConf.getBoolean("spark.mock.3", false)); assertEquals(StorageType.MEMORY_LOCALFILE_HDFS.name(), sparkConf.get("spark.rss.storage.type")); - assertEquals("expectedPathNew", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH)); + assertEquals("expectedPathNew", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); assertFalse(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); } } diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/DynamicFetchClientConfTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/DynamicFetchClientConfTest.java index e4f5375f95..e72f2fa20d 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/DynamicFetchClientConfTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/DynamicFetchClientConfTest.java @@ -42,7 +42,7 @@ public class DynamicFetchClientConfTest extends IntegrationTestBase { public void test() throws Exception { SparkConf sparkConf = new SparkConf(); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.RssShuffleManager"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); sparkConf.set("spark.mock.2", "no-overwrite-conf"); sparkConf.set("spark.shuffle.service.enabled", "true"); @@ -54,7 +54,7 @@ public void test() throws Exception { printWriter.println(" spark.mock.2 overwrite-conf "); printWriter.println(" spark.mock.3 true "); printWriter.println("spark.rss.storage.type " + StorageType.MEMORY_LOCALFILE_HDFS.name()); - printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH + " expectedPath"); + printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key + " expectedPath"); printWriter.flush(); printWriter.close(); for (String k : RssSparkConfig.RSS_MANDATORY_CLUSTER_CONF) { @@ -74,10 +74,10 @@ public void test() throws Exception { assertFalse(sparkConf.contains("spark.mock.1")); assertEquals("no-overwrite-conf", sparkConf.get("spark.mock.2")); assertFalse(sparkConf.contains("spark.mock.3")); - assertEquals("Dummy-" + RssSparkConfig.RSS_STORAGE_TYPE, - sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE)); - assertEquals("Dummy-" + RssSparkConfig.RSS_REMOTE_STORAGE_PATH, - sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH)); + assertEquals("Dummy-" + RssSparkConfig.RSS_STORAGE_TYPE.key, + sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key)); + assertEquals("Dummy-" + RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key, + sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); assertTrue(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); RssShuffleManager rssShuffleManager = new RssShuffleManager(sparkConf, true); @@ -86,14 +86,14 @@ public void test() throws Exception { assertEquals(1234, sparkConf1.getInt("spark.mock.1", 0)); assertEquals("no-overwrite-conf", sparkConf1.get("spark.mock.2")); assertEquals(StorageType.MEMORY_LOCALFILE_HDFS.name(), sparkConf.get("spark.rss.storage.type")); - assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH)); + assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); assertFalse(sparkConf1.getBoolean("spark.shuffle.service.enabled", true)); fs.delete(path, true); shutdownServers(); sparkConf = new SparkConf(); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.RssShuffleManager"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); sparkConf.set("spark.mock.2", "no-overwrite-conf"); sparkConf.set("spark.shuffle.service.enabled", "true"); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithHdfsMultiStorageRssTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithHdfsMultiStorageRssTest.java index 10fcd51218..b0ce89e536 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithHdfsMultiStorageRssTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithHdfsMultiStorageRssTest.java @@ -36,7 +36,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); dynamicConf.put(CoordinatorConf.COORDINATOR_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.LOCALFILE_HDFS.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE_HDFS.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithLocalFileRssTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithLocalFileRssTest.java index 06a529ee14..6f446a962d 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithLocalFileRssTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithLocalFileRssTest.java @@ -36,7 +36,7 @@ public class RepartitionWithLocalFileRssTest extends RepartitionTest { public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.LOCALFILE.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryMultiStorageRssTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryMultiStorageRssTest.java index 2c7933f0c8..b07aa1f368 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryMultiStorageRssTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryMultiStorageRssTest.java @@ -36,7 +36,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); dynamicConf.put(CoordinatorConf.COORDINATOR_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE_HDFS.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE_HDFS.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryRssTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryRssTest.java index 63a477a695..3397c61e4e 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryRssTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryRssTest.java @@ -38,7 +38,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); coordinatorConf.set(CoordinatorConf.COORDINATOR_APP_EXPIRED, 5000L); Map dynamicConf = Maps.newHashMap(); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SimpleTestBase.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SimpleTestBase.java index f08d286810..09a204b873 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SimpleTestBase.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SimpleTestBase.java @@ -34,7 +34,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); dynamicConf.put(CoordinatorConf.COORDINATOR_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE_HDFS.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE_HDFS.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkIntegrationTestBase.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkIntegrationTestBase.java index 1f94319112..da164004dc 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkIntegrationTestBase.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkIntegrationTestBase.java @@ -85,18 +85,18 @@ protected SparkConf createSparkConf() { public void updateSparkConfWithRss(SparkConf sparkConf) { sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.RssShuffleManager"); sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE, "4m"); - sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE, "32m"); - sparkConf.set(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE, "2m"); - sparkConf.set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE, "128k"); - sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE, "256k"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, COORDINATOR_QUORUM); - sparkConf.set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS, "30000"); - sparkConf.set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS, "1000"); - sparkConf.set(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX, "1000"); - sparkConf.set(RssSparkConfig.RSS_INDEX_READ_LIMIT, "100"); - sparkConf.set(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE, "1m"); - sparkConf.set(RssSparkConfig.RSS_HEARTBEAT_INTERVAL, "2000"); + sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key, "4m"); + sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key, "32m"); + sparkConf.set(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key, "2m"); + sparkConf.set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key, "128k"); + sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key, "256k"); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, "30000"); + sparkConf.set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, "1000"); + sparkConf.set(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key, "1000"); + sparkConf.set(RssSparkConfig.RSS_INDEX_READ_LIMIT.key, "100"); + sparkConf.set(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key, "1m"); + sparkConf.set(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key, "2000"); } private void verifyTestResult(Map expected, Map actual) { diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManager.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManager.java index 517bb20532..5da718f5cc 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManager.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManager.java @@ -47,7 +47,7 @@ public static void setupServers() throws Exception { coordinatorConf.set(CoordinatorConf.COORDINATOR_APP_EXPIRED, 5000L); coordinatorConf.set(CoordinatorConf.COORDINATOR_ACCESS_LOADCHECKER_SERVER_NUM_THRESHOLD, 1); Map dynamicConf = Maps.newHashMap(); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); @@ -67,7 +67,7 @@ public static void setupServers() throws Exception { @Override public void updateRssStorage(SparkConf sparkConf) { - sparkConf.set(RssSparkConfig.RSS_ACCESS_ID, "test_access_id"); + sparkConf.set(RssSparkConfig.RSS_ACCESS_ID.key, "test_access_id"); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.DelegationRssShuffleManager"); } diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManagerFallback.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManagerFallback.java index 6b60933ec1..8b47767444 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManagerFallback.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManagerFallback.java @@ -47,7 +47,7 @@ public static void setupServers() throws Exception { coordinatorConf.set(CoordinatorConf.COORDINATOR_APP_EXPIRED, 5000L); coordinatorConf.set(CoordinatorConf.COORDINATOR_ACCESS_LOADCHECKER_SERVER_NUM_THRESHOLD, 1); Map dynamicConf = Maps.newHashMap(); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); @@ -68,7 +68,7 @@ public static void setupServers() throws Exception { @Override public void updateRssStorage(SparkConf sparkConf) { - sparkConf.set(RssSparkConfig.RSS_ACCESS_ID, "wrong_id"); + sparkConf.set(RssSparkConfig.RSS_ACCESS_ID.key, "wrong_id"); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.DelegationRssShuffleManager"); } diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithMemoryLocalTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithMemoryLocalTest.java index 649b20a9e8..f4d0a0bf55 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithMemoryLocalTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithMemoryLocalTest.java @@ -41,7 +41,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); coordinatorConf.setLong("rss.coordinator.app.expired", 5000); Map dynamicConf = Maps.newHashMap(); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark2/src/test/java/org/apache/uniffle/test/GetReaderTest.java b/integration-test/spark2/src/test/java/org/apache/uniffle/test/GetReaderTest.java index 24edd03545..d15e601a06 100644 --- a/integration-test/spark2/src/test/java/org/apache/uniffle/test/GetReaderTest.java +++ b/integration-test/spark2/src/test/java/org/apache/uniffle/test/GetReaderTest.java @@ -55,7 +55,7 @@ public class GetReaderTest extends IntegrationTestBase { public void test() throws Exception { SparkConf sparkConf = new SparkConf(); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.RssShuffleManager"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); sparkConf.setMaster("local[4]"); final String remoteStorage1 = "hdfs://h1/p1"; final String remoteStorage2 = "hdfs://h2/p2"; diff --git a/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQERepartitionTest.java b/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQERepartitionTest.java index 58860b60d9..6a43606a86 100644 --- a/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQERepartitionTest.java +++ b/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQERepartitionTest.java @@ -47,7 +47,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); dynamicConf.put(CoordinatorConf.COORDINATOR_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE_HDFS.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE_HDFS.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQESkewedJoinTest.java b/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQESkewedJoinTest.java index dea7527681..ce215a12da 100644 --- a/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQESkewedJoinTest.java +++ b/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQESkewedJoinTest.java @@ -49,7 +49,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); dynamicConf.put(CoordinatorConf.COORDINATOR_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE_HDFS.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE_HDFS.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); @@ -69,8 +69,8 @@ public void updateCommonSparkConf(SparkConf sparkConf) { @Override public void updateSparkConfCustomer(SparkConf sparkConf) { - sparkConf.set(RssSparkConfig.RSS_STORAGE_TYPE, "HDFS"); - sparkConf.set(RssSparkConfig.RSS_REMOTE_STORAGE_PATH, HDFS_URI + "rss/test"); + sparkConf.set(RssSparkConfig.RSS_STORAGE_TYPE.key, "HDFS"); + sparkConf.set(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key, HDFS_URI + "rss/test"); } @Test diff --git a/integration-test/spark3/src/test/java/org/apache/uniffle/test/GetReaderTest.java b/integration-test/spark3/src/test/java/org/apache/uniffle/test/GetReaderTest.java index 286245cc12..5739598386 100644 --- a/integration-test/spark3/src/test/java/org/apache/uniffle/test/GetReaderTest.java +++ b/integration-test/spark3/src/test/java/org/apache/uniffle/test/GetReaderTest.java @@ -68,7 +68,7 @@ public class GetReaderTest extends IntegrationTestBase { public void test() throws Exception { SparkConf sparkConf = new SparkConf(); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.RssShuffleManager"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); sparkConf.setMaster("local[4]"); final String remoteStorage1 = "hdfs://h1/p1"; final String remoteStorage2 = "hdfs://h2/p2"; From 972bce02a9cfd591c1d8cf2ae6c53640be7ab835 Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Fri, 29 Jul 2022 18:10:18 +0800 Subject: [PATCH 02/22] [Improvement] Use ConfigBuilder to rewrite the class RssSparkConfig --- .../apache/spark/shuffle/RssSparkConfig.java | 383 ++++++++++++------ .../spark/shuffle/RssSparkShuffleUtils.java | 27 +- .../shuffle/writer/BufferManagerOptions.java | 42 +- .../shuffle/RssSparkShuffleUtilsTest.java | 44 +- .../writer/WriteBufferManagerTest.java | 10 +- .../shuffle/DelegationRssShuffleManager.java | 12 +- .../shuffle/writer/RssShuffleWriter.java | 18 +- .../DelegationRssShuffleManagerTest.java | 24 +- .../shuffle/writer/RssShuffleWriterTest.java | 32 +- .../shuffle/DelegationRssShuffleManager.java | 14 +- .../shuffle/writer/RssShuffleWriter.java | 18 +- .../DelegationRssShuffleManagerTest.java | 24 +- .../shuffle/writer/RssShuffleWriterTest.java | 32 +- .../apache/uniffle/test/AutoAccessTest.java | 18 +- .../test/DynamicFetchClientConfTest.java | 16 +- ...epartitionWithHdfsMultiStorageRssTest.java | 2 +- .../test/RepartitionWithLocalFileRssTest.java | 2 +- ...artitionWithMemoryMultiStorageRssTest.java | 2 +- .../test/RepartitionWithMemoryRssTest.java | 2 +- .../apache/uniffle/test/SimpleTestBase.java | 2 +- .../test/SparkIntegrationTestBase.java | 24 +- .../SparkSQLWithDelegationShuffleManager.java | 4 +- ...LWithDelegationShuffleManagerFallback.java | 4 +- .../test/SparkSQLWithMemoryLocalTest.java | 2 +- .../apache/uniffle/test/GetReaderTest.java | 2 +- .../uniffle/test/AQERepartitionTest.java | 2 +- .../uniffle/test/AQESkewedJoinTest.java | 6 +- .../apache/uniffle/test/GetReaderTest.java | 2 +- 28 files changed, 459 insertions(+), 311 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index a9e845a8bc..459ee9078b 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -17,130 +17,277 @@ package org.apache.spark.shuffle; -import java.util.Set; - import com.google.common.collect.Sets; +import org.apache.spark.internal.config.ConfigBuilder; +import org.apache.spark.internal.config.ConfigEntry; -import org.apache.uniffle.client.util.RssClientConfig; +import java.util.Set; public class RssSparkConfig { + public static final class SparkConfigBuilder { + public static ConfigBuilder configBuilder; + + public static ConfigBuilder key(String key) { + configBuilder = new ConfigBuilder(key); + return configBuilder; + } + } + + public static class RssConfigEntry { + public ConfigEntry entry; + public String key; + + public RssConfigEntry(ConfigEntry entry) { + this.entry = entry; + this.key = entry.key(); + } + + public T getValue() { + return (T) entry.defaultValue(). get(); + } + } + + public static RssConfigEntry RSS_PARTITION_NUM_PER_RANGE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.partitionNum.per.range") + .intConf() + .createWithDefault(1)); + + public static RssConfigEntry RSS_WRITER_BUFFER_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.writer.buffer.size") + .doc("controls the buffer flushing size during shuffle write") + .stringConf() + .createWithDefault("3m")); + + public static RssConfigEntry RSS_WRITER_SERIALIZER_BUFFER_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.writer.serializer.buffer.size") + .stringConf() + .createWithDefault("3k")); + + public static RssConfigEntry RSS_WRITER_BUFFER_SEGMENT_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.writer.buffer.segment.size") + .stringConf() + .createWithDefault("3k")); + + public static RssConfigEntry RSS_WRITER_BUFFER_SPILL_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.writer.buffer.spill.size") + .stringConf() + .createWithDefault("128m")); + + public static RssConfigEntry RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.writer.pre.allocated.buffer.size") + .stringConf() + .createWithDefault("16m")); + + public static RssConfigEntry RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.writer.require.memory.retryMax") + .intConf() + .createWithDefault(1200)); + + public static RssConfigEntry RSS_WRITER_REQUIRE_MEMORY_INTERVAL = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.writer.require.memory.interval") + .longConf() + .createWithDefault(1000)); + + public static RssConfigEntry RSS_CLIENT_SEND_CHECK_TIMEOUT_MS = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.send.check.timeout.ms") + .longConf() + .createWithDefault(60 * 1000 * 10)); + + public static RssConfigEntry RSS_CLIENT_SEND_CHECK_INTERVAL_MS = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.send.check.interval.ms") + .longConf() + .createWithDefault(500)); + + public static RssConfigEntry RSS_TEST_FLAG = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.test") + .stringConf() + .createWithDefault("")); + + public static RssConfigEntry RSS_REMOTE_STORAGE_PATH = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.remote.storage.path") + .stringConf() + .createWithDefault("")); + + public static RssConfigEntry RSS_INDEX_READ_LIMIT = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.index.read.limit") + .intConf() + .createWithDefault(500)); + + public static RssConfigEntry RSS_CLIENT_TYPE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.type") + .stringConf() + .createWithDefault("GRPC")); + + public static RssConfigEntry RSS_STORAGE_TYPE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.storage.type") + .stringConf() + .createWithDefault("")); + + public static RssConfigEntry RSS_CLIENT_RETRY_MAX = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.retry.max") + .intConf() + .createWithDefault(100)); + + public static RssConfigEntry RSS_CLIENT_RETRY_INTERVAL_MAX = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.retry.interval.max") + .longConf() + .createWithDefault(10000)); + + public static RssConfigEntry RSS_CLIENT_HEARTBEAT_THREAD_NUM = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.heartBeat.threadNum") + .intConf() + .createWithDefault(4)); + + public static RssConfigEntry RSS_CLIENT_SEND_SIZE_LIMIT = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.send.size.limit") + .stringConf() + .createWithDefault("16m")); + + public static RssConfigEntry RSS_CLIENT_READ_BUFFER_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.read.buffer.size") + .stringConf() + .createWithDefault("14m")); + + public static RssConfigEntry RSS_HEARTBEAT_INTERVAL = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.heartbeat.interval") + .longConf() + .createWithDefault(10 * 1000L)); + + public static RssConfigEntry RSS_HEARTBEAT_TIMEOUT = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.heartbeat.timeout") + .longConf() + .createWithDefault(5 * 1000L)); + + public static RssConfigEntry RSS_CLIENT_SEND_THREAD_POOL_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.send.threadPool.size") + .intConf() + .createWithDefault(10)); + + public static RssConfigEntry RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.send.threadPool.keepalive") + .intConf() + .createWithDefault(60)); + + public static RssConfigEntry RSS_DATA_REPLICA = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.data.replica") + .intConf() + .createWithDefault(1)); + + public static RssConfigEntry RSS_DATA_REPLICA_WRITE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.data.replica.write") + .intConf() + .createWithDefault(1)); + + public static RssConfigEntry RSS_DATA_REPLICA_READ = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.data.replica.read") + .intConf() + .createWithDefault(1)); + + public static RssConfigEntry RSS_DATA_REPLICA_SKIP_ENABLED = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.data.replica.skip.enabled") + .booleanConf() + .createWithDefault(true)); + + public static RssConfigEntry RSS_DATA_TRANSFER_POOL_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.data.transfer.pool.size") + .intConf() + .createWithDefault(Runtime.getRuntime().availableProcessors())); + + public static RssConfigEntry RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.ozone.dfs.namenode.odfs.enable") + .booleanConf() + .createWithDefault(false)); + + public static RssConfigEntry RSS_OZONE_FS_HDFS_IMPL = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.ozone.fs.hdfs.impl") + .stringConf() + .createWithDefault("org.apache.hadoop.odfs.HdfsOdfsFilesystem")); + + public static RssConfigEntry RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.ozone.fs.AbstractFileSystem.hdfs.impl") + .stringConf() + .createWithDefault("org.apache.hadoop.odfs.HdfsOdfs")); + + public static RssConfigEntry RSS_CLIENT_BITMAP_SPLIT_NUM = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.bitmap.splitNum") + .intConf() + .createWithDefault(1)); + + public static RssConfigEntry RSS_ACCESS_ID = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.access.id") + .stringConf() + .createWithDefault("")); + + public static RssConfigEntry RSS_ACCESS_TIMEOUT_MS = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.access.timeout.ms") + .intConf() + .createWithDefault(10000)); + + public static RssConfigEntry RSS_ENABLED = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.enabled") + .booleanConf() + .createWithDefault(false)); + + public static RssConfigEntry RSS_DYNAMIC_CLIENT_CONF_ENABLED = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.dynamicClientConf.enabled") + .booleanConf() + .createWithDefault(true)); + + public static RssConfigEntry RSS_CLIENT_ASSIGNMENT_TAGS = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.assignment.tags") + .stringConf() + .createWithDefault("")); + + public static RssConfigEntry RSS_COORDINATOR_QUORUM = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.coordinator.quorum") + .stringConf() + .createWithDefault("")); + + public static final Set RSS_MANDATORY_CLUSTER_CONF = + Sets.newHashSet(RSS_STORAGE_TYPE.key, RSS_REMOTE_STORAGE_PATH.key); + public static final String SPARK_RSS_CONFIG_PREFIX = "spark."; - public static final String RSS_PARTITION_NUM_PER_RANGE = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_PARTITION_NUM_PER_RANGE; - public static final int RSS_PARTITION_NUM_PER_RANGE_DEFAULT_VALUE = - RssClientConfig.RSS_PARTITION_NUM_PER_RANGE_DEFAULT_VALUE; - public static final String RSS_WRITER_BUFFER_SIZE = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_WRITER_BUFFER_SIZE; - public static final String RSS_WRITER_BUFFER_SIZE_DEFAULT_VALUE = "3m"; - public static final String RSS_WRITER_SERIALIZER_BUFFER_SIZE = - SPARK_RSS_CONFIG_PREFIX + "rss.writer.serializer.buffer.size"; - public static final String RSS_WRITER_SERIALIZER_BUFFER_SIZE_DEFAULT_VALUE = "3k"; - public static final String RSS_WRITER_BUFFER_SEGMENT_SIZE = - SPARK_RSS_CONFIG_PREFIX + "rss.writer.buffer.segment.size"; - public static final String RSS_WRITER_BUFFER_SEGMENT_SIZE_DEFAULT_VALUE = "3k"; - public static final String RSS_WRITER_BUFFER_SPILL_SIZE = - SPARK_RSS_CONFIG_PREFIX + "rss.writer.buffer.spill.size"; - public static final String RSS_WRITER_BUFFER_SPILL_SIZE_DEFAULT_VALUE = "128m"; - public static final String RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE = - SPARK_RSS_CONFIG_PREFIX + "rss.writer.pre.allocated.buffer.size"; - public static final String RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE_DEFAULT_VALUE = "16m"; - public static final String RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX = - SPARK_RSS_CONFIG_PREFIX + "rss.writer.require.memory.retryMax"; - public static final int RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX_DEFAULT_VALUE = 1200; - public static final String RSS_WRITER_REQUIRE_MEMORY_INTERVAL = - SPARK_RSS_CONFIG_PREFIX + "rss.writer.require.memory.interval"; - public static final long RSS_WRITER_REQUIRE_MEMORY_INTERVAL_DEFAULT_VALUE = 1000; // 1s - public static final String RSS_COORDINATOR_QUORUM = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_COORDINATOR_QUORUM; - public static final String RSS_CLIENT_SEND_CHECK_TIMEOUT_MS = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS; - public static final long RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE = - RssClientConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE; // 10 min - public static final String RSS_CLIENT_SEND_CHECK_INTERVAL_MS = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS; - public static final long RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE = - RssClientConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE; - public static final String RSS_TEST_FLAG = SPARK_RSS_CONFIG_PREFIX + "rss.test"; - public static final String RSS_REMOTE_STORAGE_PATH = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_REMOTE_STORAGE_PATH; - public static final String RSS_INDEX_READ_LIMIT = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_INDEX_READ_LIMIT; - public static final int RSS_INDEX_READ_LIMIT_DEFAULT_VALUE = 500; - public static final String RSS_CLIENT_TYPE = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_TYPE; - public static final String RSS_CLIENT_TYPE_DEFAULT_VALUE = RssClientConfig.RSS_CLIENT_TYPE_DEFAULT_VALUE; - public static final String RSS_STORAGE_TYPE = SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_STORAGE_TYPE; - public static final String RSS_CLIENT_RETRY_MAX = SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_RETRY_MAX; - public static final int RSS_CLIENT_RETRY_MAX_DEFAULT_VALUE = RssClientConfig.RSS_CLIENT_RETRY_MAX_DEFAULT_VALUE; - public static final String RSS_CLIENT_RETRY_INTERVAL_MAX = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_RETRY_INTERVAL_MAX; - public static final long RSS_CLIENT_RETRY_INTERVAL_MAX_DEFAULT_VALUE = - RssClientConfig.RSS_CLIENT_RETRY_INTERVAL_MAX_DEFAULT_VALUE; - public static final String RSS_CLIENT_HEARTBEAT_THREAD_NUM = - SPARK_RSS_CONFIG_PREFIX + "rss.client.heartBeat.threadNum"; - public static final int RSS_CLIENT_HEARTBEAT_THREAD_NUM_DEFAULT_VALUE = 4; - public static final String RSS_CLIENT_SEND_SIZE_LIMIT = SPARK_RSS_CONFIG_PREFIX + "rss.client.send.size.limit"; - public static final String RSS_CLIENT_SEND_SIZE_LIMIT_DEFAULT_VALUE = "16m"; - public static final String RSS_CLIENT_READ_BUFFER_SIZE = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_READ_BUFFER_SIZE; - // When the size of read buffer reaches the half of JVM region (i.e., 32m), - // it will incur humongous allocation, so we set it to 14m. - public static final String RSS_CLIENT_READ_BUFFER_SIZE_DEFAULT_VALUE = - RssClientConfig.RSS_CLIENT_READ_BUFFER_SIZE_DEFAULT_VALUE; - public static final String RSS_HEARTBEAT_INTERVAL = SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_HEARTBEAT_INTERVAL; - public static final long RSS_HEARTBEAT_INTERVAL_DEFAULT_VALUE = RssClientConfig.RSS_HEARTBEAT_INTERVAL_DEFAULT_VALUE; - public static final String RSS_HEARTBEAT_TIMEOUT = SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_HEARTBEAT_TIMEOUT; - public static final String RSS_CLIENT_SEND_THREAD_POOL_SIZE = - SPARK_RSS_CONFIG_PREFIX + "rss.client.send.threadPool.size"; - public static final int RSS_CLIENT_SEND_THREAD_POOL_SIZE_DEFAULT_VALUE = 10; - public static final String RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE = - SPARK_RSS_CONFIG_PREFIX + "rss.client.send.threadPool.keepalive"; - public static final int RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE_DEFAULT_VALUE = 60; - public static final String RSS_DATA_REPLICA = SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA; - public static final int RSS_DATA_REPLICA_DEFAULT_VALUE = RssClientConfig.RSS_DATA_REPLICA_DEFAULT_VALUE; - public static final String RSS_DATA_REPLICA_WRITE = SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA_WRITE; - public static final int RSS_DATA_REPLICA_WRITE_DEFAULT_VALUE = RssClientConfig.RSS_DATA_REPLICA_WRITE_DEFAULT_VALUE; - public static final String RSS_DATA_REPLICA_READ = SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA_READ; - public static final int RSS_DATA_REPLICA_READ_DEFAULT_VALUE = RssClientConfig.RSS_DATA_REPLICA_READ_DEFAULT_VALUE; - public static final String RSS_DATA_REPLICA_SKIP_ENABLED = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA_SKIP_ENABLED; - public static final String RSS_DATA_TRANSFER_POOL_SIZE = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_TRANSFER_POOL_SIZE; - public static final int RSS_DATA_TRANSFER_POOL_SIZE_DEFAULT_VALUE = - RssClientConfig.RSS_DATA_TRANFER_POOL_SIZE_DEFAULT_VALUE; - public static final String RSS_DATA_COMMIT_POOL_SIZE = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_COMMIT_POOL_SIZE; - public static final int RSS_DATA_COMMIT_POOL_SIZE_DEFAULT_VALUE = - RssClientConfig.RSS_DATA_COMMIT_POOL_SIZE_DEFAULT_VALUE; - - public static final boolean RSS_DATA_REPLICA_SKIP_ENABLED_DEFAULT_VALUE = - RssClientConfig.RSS_DATA_REPLICA_SKIP_ENABLED_DEFAULT_VALUE; - public static final String RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE = - SPARK_RSS_CONFIG_PREFIX + "rss.ozone.dfs.namenode.odfs.enable"; + public static final boolean RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE_DEFAULT_VALUE = false; - public static final String RSS_OZONE_FS_HDFS_IMPL = SPARK_RSS_CONFIG_PREFIX + "rss.ozone.fs.hdfs.impl"; - public static final String RSS_OZONE_FS_HDFS_IMPL_DEFAULT_VALUE = "org.apache.hadoop.odfs.HdfsOdfsFilesystem"; - public static final String RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL = - SPARK_RSS_CONFIG_PREFIX + "rss.ozone.fs.AbstractFileSystem.hdfs.impl"; - public static final String RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL_DEFAULT_VALUE = - "org.apache.hadoop.odfs.HdfsOdfs"; - // todo: remove unnecessary configuration - public static final String RSS_CLIENT_BITMAP_SPLIT_NUM = - SPARK_RSS_CONFIG_PREFIX + "rss.client.bitmap.splitNum"; - public static final int RSS_CLIENT_BITMAP_SPLIT_NUM_DEFAULT_VALUE = 1; - public static final String RSS_ACCESS_ID = SPARK_RSS_CONFIG_PREFIX + "rss.access.id"; - public static final String RSS_ACCESS_TIMEOUT_MS = SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_ACCESS_TIMEOUT_MS; - public static final int RSS_ACCESS_TIMEOUT_MS_DEFAULT_VALUE = RssClientConfig.RSS_ACCESS_TIMEOUT_MS_DEFAULT_VALUE; - public static final String RSS_ENABLED = SPARK_RSS_CONFIG_PREFIX + "rss.enabled"; - public static final boolean RSS_USE_RSS_SHUFFLE_MANAGER_DEFAULT_VALUE = false; - public static final String RSS_DYNAMIC_CLIENT_CONF_ENABLED = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED; - public static final boolean RSS_DYNAMIC_CLIENT_CONF_ENABLED_DEFAULT_VALUE = - RssClientConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED_DEFAULT_VALUE; - public static final String RSS_CLIENT_ASSIGNMENT_TAGS = - SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_ASSIGNMENT_TAGS; - public static final Set RSS_MANDATORY_CLUSTER_CONF = - Sets.newHashSet(RSS_STORAGE_TYPE, RSS_REMOTE_STORAGE_PATH); } diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java index a4874f5129..9e5f99f038 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java @@ -44,19 +44,20 @@ 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.getBoolean(RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE.key, + RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE.getValue()); 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.key, + RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.getValue())); conf.set( - RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.substring(OZONE_PREFIX_LEN), + 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, - RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL_DEFAULT_VALUE)); + RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.key, + RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.getValue())); } return conf; @@ -77,9 +78,9 @@ public static ShuffleManager loadShuffleManager(String name, SparkConf conf, boo } public static List createCoordinatorClients(SparkConf sparkConf) throws RuntimeException { - String clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE, - RssSparkConfig.RSS_CLIENT_TYPE_DEFAULT_VALUE); - String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM); + String clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key, + RssSparkConfig.RSS_CLIENT_TYPE.getValue()); + String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM.key); CoordinatorClientFactory coordinatorClientFactory = new CoordinatorClientFactory(clientType); return coordinatorClientFactory.createCoordinatorClient(coordinators); } @@ -110,7 +111,7 @@ public static void applyDynamicClientConf(SparkConf sparkConf, Map getAssignmentTags(SparkConf sparkConf) { Set 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(","))); diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java index f817c00414..e027c1a1e9 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java @@ -35,38 +35,38 @@ 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.getValue()); + serializerBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key, + RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.getValue()); + bufferSegmentSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key, + RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.getValue()); + bufferSpillThreshold = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key, + RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.getValue()); + preAllocatedBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE.key, + RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE.getValue()); + requireMemoryInterval = sparkConf.getLong(RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_INTERVAL.key, + RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_INTERVAL.getValue()); + requireMemoryRetryMax = sparkConf.getInt(RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX.key, + RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX.getValue()); + 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 + "]"); } } diff --git a/client-spark/common/src/test/java/org/apache/spark/shuffle/RssSparkShuffleUtilsTest.java b/client-spark/common/src/test/java/org/apache/spark/shuffle/RssSparkShuffleUtilsTest.java index f15c4cfddc..8a5f6591b1 100644 --- a/client-spark/common/src/test/java/org/apache/spark/shuffle/RssSparkShuffleUtilsTest.java +++ b/client-spark/common/src/test/java/org/apache/spark/shuffle/RssSparkShuffleUtilsTest.java @@ -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 iterator = tags.iterator(); @@ -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")); @@ -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)); } } diff --git a/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/WriteBufferManagerTest.java b/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/WriteBufferManagerTest.java index 3e62f94926..0fc55f52bf 100644 --- a/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/WriteBufferManagerTest.java +++ b/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/WriteBufferManagerTest.java @@ -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; } diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java index 01c4cf6e22..45d2934820 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java @@ -47,8 +47,8 @@ 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); + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key, + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getValue()); if (isDriver) { coordinatorClients = RssSparkShuffleUtils.createCoordinatorClients(sparkConf); delegate = createShuffleManagerInDriver(); @@ -69,7 +69,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; @@ -80,7 +80,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) { @@ -92,7 +92,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; @@ -128,7 +128,7 @@ private ShuffleManager createShuffleManagerInExecutor() throws RssException { ShuffleManager shuffleManager; // get useRSS from spark conf boolean useRSS = sparkConf.getBoolean( - RssSparkConfig.RSS_ENABLED, + RssSparkConfig.RSS_ENABLED.key, RssSparkConfig.RSS_USE_RSS_SHUFFLE_MANAGER_DEFAULT_VALUE); if (useRSS) { // Executor will not do any fallback diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java index 19012584aa..4e51f3c606 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java @@ -105,20 +105,20 @@ public RssShuffleWriter( this.partitioner = shuffleDependency.partitioner(); this.shuffleManager = shuffleManager; this.shouldPartition = partitioner.numPartitions() > 1; - this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS, - RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE); - this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS, - RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE); - this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT, - RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT_DEFAULT_VALUE); - this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM, - RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM_DEFAULT_VALUE); + this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, + RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.getValue()); + this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, + RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.getValue()); + this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key, + RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.getValue()); + this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.key, + RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.getValue()); this.partitionToBlockIds = Maps.newConcurrentMap(); this.shuffleWriteClient = shuffleWriteClient; this.shuffleServersForData = rssHandle.getShuffleServersForData(); this.partitionToServers = rssHandle.getPartitionToServers(); this.isMemoryShuffleEnabled = isMemoryShuffleEnabled( - sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE)); + sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key)); } private boolean isMemoryShuffleEnabled(String storageType) { diff --git a/client-spark/spark2/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java b/client-spark/spark2/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java index 5936c9a4b7..c357bcbfa4 100644 --- a/client-spark/spark2/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java +++ b/client-spark/spark2/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java @@ -65,7 +65,7 @@ public void testCreateInDriverDenied() throws Exception { mockedStaticRssShuffleUtils.when(() -> RssSparkShuffleUtils.createCoordinatorClients(any())).thenReturn(coordinatorClients); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED, "false"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); assertCreateSortShuffleManager(conf); } @@ -81,15 +81,15 @@ public void testCreateInDriver() throws Exception { SparkConf conf = new SparkConf(); assertCreateSortShuffleManager(conf); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED, "false"); - conf.set(RssSparkConfig.RSS_ACCESS_ID, "mockId"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); + conf.set(RssSparkConfig.RSS_ACCESS_ID.key, "mockId"); conf.set("spark.rss.storage.type", StorageType.LOCALFILE.name()); assertCreateSortShuffleManager(conf); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); assertCreateRssShuffleManager(conf); conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); when(mockCoordinatorClient.accessCluster(any())).thenReturn( new RssAccessClusterResponse(SUCCESS, "")); assertCreateSortShuffleManager(conf); @@ -99,7 +99,7 @@ public void testCreateInDriver() throws Exception { public void testCreateInExecutor() throws Exception { DelegationRssShuffleManager delegationRssShuffleManager; SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); delegationRssShuffleManager = new DelegationRssShuffleManager(conf, false); assertFalse(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); assertTrue(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); @@ -115,15 +115,15 @@ public void testCreateFallback() throws Exception { RssSparkShuffleUtils.createCoordinatorClients(any())).thenReturn(coordinatorClients); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED, "false"); - conf.set(RssSparkConfig.RSS_ACCESS_ID, "mockId"); - conf.set(RssSparkConfig.RSS_ENABLED, "true"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); + conf.set(RssSparkConfig.RSS_ACCESS_ID.key, "mockId"); + conf.set(RssSparkConfig.RSS_ENABLED.key, "true"); // fall back to SortShuffleManager in driver assertCreateSortShuffleManager(conf); // No fall back in executor - conf.set(RssSparkConfig.RSS_ENABLED, "true"); + conf.set(RssSparkConfig.RSS_ENABLED.key, "true"); boolean hasException = false; try { new DelegationRssShuffleManager(conf, false); @@ -138,7 +138,7 @@ private DelegationRssShuffleManager assertCreateSortShuffleManager(SparkConf con DelegationRssShuffleManager delegationRssShuffleManager = new DelegationRssShuffleManager(conf, true); assertTrue(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); assertFalse(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); - assertFalse(conf.getBoolean(RssSparkConfig.RSS_ENABLED, false)); + assertFalse(conf.getBoolean(RssSparkConfig.RSS_ENABLED.key, false)); assertEquals("sort", conf.get("spark.shuffle.manager")); return delegationRssShuffleManager; } @@ -147,7 +147,7 @@ private DelegationRssShuffleManager assertCreateRssShuffleManager(SparkConf conf DelegationRssShuffleManager delegationRssShuffleManager = new DelegationRssShuffleManager(conf, true); assertFalse(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); assertTrue(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); - assertTrue(Boolean.parseBoolean(conf.get(RssSparkConfig.RSS_ENABLED))); + assertTrue(Boolean.parseBoolean(conf.get(RssSparkConfig.RSS_ENABLED.key))); assertEquals(RssShuffleManager.class.getCanonicalName(), conf.get("spark.shuffle.manager")); return delegationRssShuffleManager; } diff --git a/client-spark/spark2/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java b/client-spark/spark2/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java index 1ef7d2b99e..f67171850a 100644 --- a/client-spark/spark2/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java +++ b/client-spark/spark2/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java @@ -68,11 +68,11 @@ public void checkBlockSendResultTest() { String taskId = "checkBlockSendResultTest_taskId"; conf.setAppName("testApp") .setMaster("local[2]") - .set(RssSparkConfig.RSS_TEST_FLAG, "true") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS, "10000") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS, "1000") - .set(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.LOCALFILE.name()) - .set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "127.0.0.1:12345,127.0.0.1:12346"); + .set(RssSparkConfig.RSS_TEST_FLAG.key, "true") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, "10000") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, "1000") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()) + .set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "127.0.0.1:12345,127.0.0.1:12346"); // init SparkContext SparkContext sc = SparkContext.getOrCreate(conf); RssShuffleManager manager = new RssShuffleManager(conf, false); @@ -131,15 +131,15 @@ public void checkBlockSendResultTest() { public void writeTest() throws Exception { SparkConf conf = new SparkConf(); conf.setAppName("testApp").setMaster("local[2]") - .set(RssSparkConfig.RSS_TEST_FLAG, "true") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE, "32") - .set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE, "32") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE, "64") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE, "128") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS, "10000") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS, "1000") - .set(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.LOCALFILE.name()) - .set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "127.0.0.1:12345,127.0.0.1:12346"); + .set(RssSparkConfig.RSS_TEST_FLAG.key, "true") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key, "32") + .set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key, "32") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key, "64") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key, "128") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, "10000") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, "1000") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()) + .set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "127.0.0.1:12345,127.0.0.1:12346"); // init SparkContext SparkContext sc = SparkContext.getOrCreate(conf); RssShuffleManager manager = new RssShuffleManager(conf, false); @@ -273,8 +273,8 @@ public void onError(Throwable e) { when(mockHandle.getDependency()).thenReturn(mockDependency); ShuffleWriteClient mockWriteClient = mock(ShuffleWriteClient.class); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT, "64") - .set(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.LOCALFILE.name()); + conf.set(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key, "64") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()); RssShuffleWriter writer = new RssShuffleWriter("appId", 0, "taskId", 1L, mockBufferManager, mockMetrics, mockShuffleManager, conf, mockWriteClient, mockHandle); diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java index 6ec8901013..27d76a6b66 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java @@ -47,8 +47,8 @@ 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); + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key, + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getValue()); if (isDriver) { coordinatorClients = RssSparkShuffleUtils.createCoordinatorClients(sparkConf); delegate = createShuffleManagerInDriver(); @@ -69,7 +69,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; @@ -80,7 +80,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) { @@ -92,7 +92,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; @@ -128,8 +128,8 @@ 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); + RssSparkConfig.RSS_ENABLED.key, + RssSparkConfig.RSS_ENABLED.getValue()); if (useRSS) { // Executor will not do any fallback shuffleManager = new RssShuffleManager(sparkConf, false); diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java index bed94034ea..7d3b21f551 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java @@ -106,14 +106,14 @@ public RssShuffleWriter( this.shuffleDependency = rssHandle.getDependency(); this.partitioner = shuffleDependency.partitioner(); this.shouldPartition = partitioner.numPartitions() > 1; - this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS, - RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE); - this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS, - RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE); - this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT, - RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT_DEFAULT_VALUE); - this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM, - RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM_DEFAULT_VALUE); + this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, + RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.getValue()); + this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, + RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.getValue()); + this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key, + RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.getValue()); + this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.key, + RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.getValue()); this.partitionToBlockIds = Maps.newConcurrentMap(); this.shuffleWriteClient = shuffleWriteClient; this.shuffleServersForData = rssHandle.getShuffleServersForData(); @@ -121,7 +121,7 @@ public RssShuffleWriter( Arrays.fill(partitionLengths, 0); partitionToServers = rssHandle.getPartitionToServers(); this.isMemoryShuffleEnabled = isMemoryShuffleEnabled( - sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE)); + sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key)); } private boolean isMemoryShuffleEnabled(String storageType) { diff --git a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java index a96b52ac58..88999f4d7e 100644 --- a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java +++ b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java @@ -66,7 +66,7 @@ public void testCreateInDriverDenied() throws Exception { mockedStaticRssShuffleUtils.when(() -> RssSparkShuffleUtils.createCoordinatorClients(any())).thenReturn(coordinatorClients); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED, "false"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); assertCreateSortShuffleManager(conf); } @@ -82,15 +82,15 @@ public void testCreateInDriver() throws Exception { SparkConf conf = new SparkConf(); assertCreateSortShuffleManager(conf); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED, "false"); - conf.set(RssSparkConfig.RSS_ACCESS_ID, "mockId"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); + conf.set(RssSparkConfig.RSS_ACCESS_ID.key, "mockId"); assertCreateSortShuffleManager(conf); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); conf.set("spark.rss.storage.type", StorageType.LOCALFILE.name()); assertCreateRssShuffleManager(conf); conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); when(mockCoordinatorClient.accessCluster(any())).thenReturn( new RssAccessClusterResponse(SUCCESS, "")); assertCreateSortShuffleManager(conf); @@ -100,7 +100,7 @@ public void testCreateInDriver() throws Exception { public void testCreateInExecutor() throws Exception { DelegationRssShuffleManager delegationRssShuffleManager; SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); delegationRssShuffleManager = new DelegationRssShuffleManager(conf, false); assertFalse(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); assertTrue(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); @@ -116,15 +116,15 @@ public void testCreateFallback() throws Exception { RssSparkShuffleUtils.createCoordinatorClients(any())).thenReturn(coordinatorClients); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED, "false"); - conf.set(RssSparkConfig.RSS_ACCESS_ID, "mockId"); - conf.set(RssSparkConfig.RSS_ENABLED, "true"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); + conf.set(RssSparkConfig.RSS_ACCESS_ID.key, "mockId"); + conf.set(RssSparkConfig.RSS_ENABLED.key, "true"); // fall back to SortShuffleManager in driver assertCreateSortShuffleManager(conf); // No fall back in executor - conf.set(RssSparkConfig.RSS_ENABLED, "true"); + conf.set(RssSparkConfig.RSS_ENABLED.key, "true"); boolean hasException = false; try { new DelegationRssShuffleManager(conf, false); @@ -139,7 +139,7 @@ private DelegationRssShuffleManager assertCreateSortShuffleManager(SparkConf con DelegationRssShuffleManager delegationRssShuffleManager = new DelegationRssShuffleManager(conf, true); assertTrue(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); assertFalse(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); - assertFalse(conf.getBoolean(RssSparkConfig.RSS_ENABLED, false)); + assertFalse(conf.getBoolean(RssSparkConfig.RSS_ENABLED.key, false)); assertEquals("sort", conf.get("spark.shuffle.manager")); return delegationRssShuffleManager; } @@ -148,7 +148,7 @@ private DelegationRssShuffleManager assertCreateRssShuffleManager(SparkConf conf DelegationRssShuffleManager delegationRssShuffleManager = new DelegationRssShuffleManager(conf, true); assertFalse(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); assertTrue(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); - assertTrue(Boolean.parseBoolean(conf.get(RssSparkConfig.RSS_ENABLED))); + assertTrue(Boolean.parseBoolean(conf.get(RssSparkConfig.RSS_ENABLED.key))); assertEquals(RssShuffleManager.class.getCanonicalName(), conf.get("spark.shuffle.manager")); return delegationRssShuffleManager; } diff --git a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java index a943a453b5..8c2c03bd15 100644 --- a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java +++ b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java @@ -68,11 +68,11 @@ public void checkBlockSendResultTest() { SparkConf conf = new SparkConf(); conf.setAppName("testApp") .setMaster("local[2]") - .set(RssSparkConfig.RSS_TEST_FLAG, "true") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS, "10000") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS, "1000") - .set(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.LOCALFILE.name()) - .set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "127.0.0.1:12345,127.0.0.1:12346"); + .set(RssSparkConfig.RSS_TEST_FLAG.key, "true") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, "10000") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, "1000") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()) + .set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "127.0.0.1:12345,127.0.0.1:12346"); // init SparkContext SparkContext sc = SparkContext.getOrCreate(conf); Map> failBlocks = Maps.newConcurrentMap(); @@ -135,15 +135,15 @@ public void checkBlockSendResultTest() { public void writeTest() throws Exception { SparkConf conf = new SparkConf(); conf.setAppName("testApp").setMaster("local[2]") - .set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE, "32") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE, "32") - .set(RssSparkConfig.RSS_TEST_FLAG, "true") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE, "64") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS, "10000") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS, "1000") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE, "128") - .set(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.LOCALFILE.name()) - .set(RssSparkConfig.RSS_COORDINATOR_QUORUM, "127.0.0.1:12345,127.0.0.1:12346"); + .set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key, "32") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key, "32") + .set(RssSparkConfig.RSS_TEST_FLAG.key, "true") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key, "64") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, "10000") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, "1000") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key, "128") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()) + .set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "127.0.0.1:12345,127.0.0.1:12346"); // init SparkContext List shuffleBlockInfos = Lists.newArrayList(); SparkContext sc = SparkContext.getOrCreate(conf); @@ -303,8 +303,8 @@ public void onError(Throwable e) { when(mockHandle.getDependency()).thenReturn(mockDependency); ShuffleWriteClient mockWriteClient = mock(ShuffleWriteClient.class); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT, "64") - .set(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE.name()); + conf.set(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key, "64") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE.name()); List shuffleBlockInfoList = createShuffleBlockList(1, 31); RssShuffleWriter writer = new RssShuffleWriter("appId", 0, "taskId", 1L, mockBufferManager, mockMetrics, mockShuffleManager, conf, mockWriteClient, mockHandle); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/AutoAccessTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/AutoAccessTest.java index 15523ba048..eedcce7e37 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/AutoAccessTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/AutoAccessTest.java @@ -48,9 +48,9 @@ public class AutoAccessTest extends IntegrationTestBase { public void test() throws Exception { SparkConf sparkConf = new SparkConf(); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.DelegationRssShuffleManager"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); sparkConf.set("spark.mock.2", "no-overwrite-conf"); - sparkConf.set(RssSparkConfig.RSS_REMOTE_STORAGE_PATH, "overwrite-path"); + sparkConf.set(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key, "overwrite-path"); sparkConf.set("spark.shuffle.service.enabled", "true"); String cfgFile = HDFS_URI + "/test/client_conf"; @@ -61,7 +61,7 @@ public void test() throws Exception { printWriter.println(" spark.mock.2 overwrite-conf "); printWriter.println(" spark.mock.3 true "); printWriter.println("spark.rss.storage.type " + StorageType.MEMORY_LOCALFILE_HDFS.name()); - printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH + " expectedPath"); + printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key + " expectedPath"); printWriter.flush(); printWriter.close(); @@ -100,7 +100,7 @@ public void test() throws Exception { ShuffleManager shuffleManager = delegationRssShuffleManager.getDelegate(); assertTrue(shuffleManager instanceof SortShuffleManager); assertTrue(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); - assertEquals("overwrite-path", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH)); + assertEquals("overwrite-path", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); assertFalse(sparkConf.contains("spark.rss.storage.type")); // wrong access id @@ -108,7 +108,7 @@ public void test() throws Exception { delegationRssShuffleManager = new DelegationRssShuffleManager(sparkConf, true); shuffleManager = delegationRssShuffleManager.getDelegate(); assertTrue(shuffleManager instanceof SortShuffleManager); - assertEquals("overwrite-path", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH)); + assertEquals("overwrite-path", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); assertTrue(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); assertFalse(sparkConf.contains("spark.rss.storage.type")); @@ -122,7 +122,7 @@ public void test() throws Exception { assertEquals("no-overwrite-conf", sparkConf.get("spark.mock.2")); assertTrue(sparkConf.getBoolean("spark.mock.3", false)); assertEquals(StorageType.MEMORY_LOCALFILE_HDFS.name(), sparkConf.get("spark.rss.storage.type")); - assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH)); + assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); assertFalse(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); // update candidates file @@ -149,7 +149,7 @@ public void test() throws Exception { assertEquals("no-overwrite-conf", sparkConf.get("spark.mock.2")); assertTrue(sparkConf.getBoolean("spark.mock.3", false)); assertEquals(StorageType.MEMORY_LOCALFILE_HDFS.name(), sparkConf.get("spark.rss.storage.type")); - assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH)); + assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); assertFalse(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); // update client conf file @@ -162,7 +162,7 @@ public void test() throws Exception { printWriter.println(" spark.mock.2 overwrite-conf "); printWriter.println(" spark.mock.3 false "); printWriter.println("spark.rss.storage.type " + StorageType.MEMORY_LOCALFILE_HDFS.name()); - printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH + " expectedPathNew"); + printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key + " expectedPathNew"); printWriter.flush(); printWriter.close(); fs.rename(tmpPath, path); @@ -177,7 +177,7 @@ public void test() throws Exception { assertEquals("overwrite-conf", sparkConf.get("spark.mock.2")); assertTrue(sparkConf.getBoolean("spark.mock.3", false)); assertEquals(StorageType.MEMORY_LOCALFILE_HDFS.name(), sparkConf.get("spark.rss.storage.type")); - assertEquals("expectedPathNew", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH)); + assertEquals("expectedPathNew", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); assertFalse(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); } } diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/DynamicFetchClientConfTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/DynamicFetchClientConfTest.java index e4f5375f95..e72f2fa20d 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/DynamicFetchClientConfTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/DynamicFetchClientConfTest.java @@ -42,7 +42,7 @@ public class DynamicFetchClientConfTest extends IntegrationTestBase { public void test() throws Exception { SparkConf sparkConf = new SparkConf(); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.RssShuffleManager"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); sparkConf.set("spark.mock.2", "no-overwrite-conf"); sparkConf.set("spark.shuffle.service.enabled", "true"); @@ -54,7 +54,7 @@ public void test() throws Exception { printWriter.println(" spark.mock.2 overwrite-conf "); printWriter.println(" spark.mock.3 true "); printWriter.println("spark.rss.storage.type " + StorageType.MEMORY_LOCALFILE_HDFS.name()); - printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH + " expectedPath"); + printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key + " expectedPath"); printWriter.flush(); printWriter.close(); for (String k : RssSparkConfig.RSS_MANDATORY_CLUSTER_CONF) { @@ -74,10 +74,10 @@ public void test() throws Exception { assertFalse(sparkConf.contains("spark.mock.1")); assertEquals("no-overwrite-conf", sparkConf.get("spark.mock.2")); assertFalse(sparkConf.contains("spark.mock.3")); - assertEquals("Dummy-" + RssSparkConfig.RSS_STORAGE_TYPE, - sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE)); - assertEquals("Dummy-" + RssSparkConfig.RSS_REMOTE_STORAGE_PATH, - sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH)); + assertEquals("Dummy-" + RssSparkConfig.RSS_STORAGE_TYPE.key, + sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key)); + assertEquals("Dummy-" + RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key, + sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); assertTrue(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); RssShuffleManager rssShuffleManager = new RssShuffleManager(sparkConf, true); @@ -86,14 +86,14 @@ public void test() throws Exception { assertEquals(1234, sparkConf1.getInt("spark.mock.1", 0)); assertEquals("no-overwrite-conf", sparkConf1.get("spark.mock.2")); assertEquals(StorageType.MEMORY_LOCALFILE_HDFS.name(), sparkConf.get("spark.rss.storage.type")); - assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH)); + assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); assertFalse(sparkConf1.getBoolean("spark.shuffle.service.enabled", true)); fs.delete(path, true); shutdownServers(); sparkConf = new SparkConf(); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.RssShuffleManager"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); sparkConf.set("spark.mock.2", "no-overwrite-conf"); sparkConf.set("spark.shuffle.service.enabled", "true"); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithHdfsMultiStorageRssTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithHdfsMultiStorageRssTest.java index 10fcd51218..b0ce89e536 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithHdfsMultiStorageRssTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithHdfsMultiStorageRssTest.java @@ -36,7 +36,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); dynamicConf.put(CoordinatorConf.COORDINATOR_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.LOCALFILE_HDFS.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE_HDFS.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithLocalFileRssTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithLocalFileRssTest.java index 06a529ee14..6f446a962d 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithLocalFileRssTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithLocalFileRssTest.java @@ -36,7 +36,7 @@ public class RepartitionWithLocalFileRssTest extends RepartitionTest { public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.LOCALFILE.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryMultiStorageRssTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryMultiStorageRssTest.java index 2c7933f0c8..b07aa1f368 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryMultiStorageRssTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryMultiStorageRssTest.java @@ -36,7 +36,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); dynamicConf.put(CoordinatorConf.COORDINATOR_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE_HDFS.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE_HDFS.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryRssTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryRssTest.java index 63a477a695..3397c61e4e 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryRssTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryRssTest.java @@ -38,7 +38,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); coordinatorConf.set(CoordinatorConf.COORDINATOR_APP_EXPIRED, 5000L); Map dynamicConf = Maps.newHashMap(); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SimpleTestBase.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SimpleTestBase.java index f08d286810..09a204b873 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SimpleTestBase.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SimpleTestBase.java @@ -34,7 +34,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); dynamicConf.put(CoordinatorConf.COORDINATOR_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE_HDFS.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE_HDFS.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkIntegrationTestBase.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkIntegrationTestBase.java index 1f94319112..da164004dc 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkIntegrationTestBase.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkIntegrationTestBase.java @@ -85,18 +85,18 @@ protected SparkConf createSparkConf() { public void updateSparkConfWithRss(SparkConf sparkConf) { sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.RssShuffleManager"); sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE, "4m"); - sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE, "32m"); - sparkConf.set(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE, "2m"); - sparkConf.set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE, "128k"); - sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE, "256k"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, COORDINATOR_QUORUM); - sparkConf.set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS, "30000"); - sparkConf.set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS, "1000"); - sparkConf.set(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX, "1000"); - sparkConf.set(RssSparkConfig.RSS_INDEX_READ_LIMIT, "100"); - sparkConf.set(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE, "1m"); - sparkConf.set(RssSparkConfig.RSS_HEARTBEAT_INTERVAL, "2000"); + sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key, "4m"); + sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key, "32m"); + sparkConf.set(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key, "2m"); + sparkConf.set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key, "128k"); + sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key, "256k"); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, "30000"); + sparkConf.set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, "1000"); + sparkConf.set(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key, "1000"); + sparkConf.set(RssSparkConfig.RSS_INDEX_READ_LIMIT.key, "100"); + sparkConf.set(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key, "1m"); + sparkConf.set(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key, "2000"); } private void verifyTestResult(Map expected, Map actual) { diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManager.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManager.java index 517bb20532..5da718f5cc 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManager.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManager.java @@ -47,7 +47,7 @@ public static void setupServers() throws Exception { coordinatorConf.set(CoordinatorConf.COORDINATOR_APP_EXPIRED, 5000L); coordinatorConf.set(CoordinatorConf.COORDINATOR_ACCESS_LOADCHECKER_SERVER_NUM_THRESHOLD, 1); Map dynamicConf = Maps.newHashMap(); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); @@ -67,7 +67,7 @@ public static void setupServers() throws Exception { @Override public void updateRssStorage(SparkConf sparkConf) { - sparkConf.set(RssSparkConfig.RSS_ACCESS_ID, "test_access_id"); + sparkConf.set(RssSparkConfig.RSS_ACCESS_ID.key, "test_access_id"); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.DelegationRssShuffleManager"); } diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManagerFallback.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManagerFallback.java index 6b60933ec1..8b47767444 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManagerFallback.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManagerFallback.java @@ -47,7 +47,7 @@ public static void setupServers() throws Exception { coordinatorConf.set(CoordinatorConf.COORDINATOR_APP_EXPIRED, 5000L); coordinatorConf.set(CoordinatorConf.COORDINATOR_ACCESS_LOADCHECKER_SERVER_NUM_THRESHOLD, 1); Map dynamicConf = Maps.newHashMap(); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); @@ -68,7 +68,7 @@ public static void setupServers() throws Exception { @Override public void updateRssStorage(SparkConf sparkConf) { - sparkConf.set(RssSparkConfig.RSS_ACCESS_ID, "wrong_id"); + sparkConf.set(RssSparkConfig.RSS_ACCESS_ID.key, "wrong_id"); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.DelegationRssShuffleManager"); } diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithMemoryLocalTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithMemoryLocalTest.java index 649b20a9e8..f4d0a0bf55 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithMemoryLocalTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithMemoryLocalTest.java @@ -41,7 +41,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); coordinatorConf.setLong("rss.coordinator.app.expired", 5000); Map dynamicConf = Maps.newHashMap(); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark2/src/test/java/org/apache/uniffle/test/GetReaderTest.java b/integration-test/spark2/src/test/java/org/apache/uniffle/test/GetReaderTest.java index 24edd03545..d15e601a06 100644 --- a/integration-test/spark2/src/test/java/org/apache/uniffle/test/GetReaderTest.java +++ b/integration-test/spark2/src/test/java/org/apache/uniffle/test/GetReaderTest.java @@ -55,7 +55,7 @@ public class GetReaderTest extends IntegrationTestBase { public void test() throws Exception { SparkConf sparkConf = new SparkConf(); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.RssShuffleManager"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); sparkConf.setMaster("local[4]"); final String remoteStorage1 = "hdfs://h1/p1"; final String remoteStorage2 = "hdfs://h2/p2"; diff --git a/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQERepartitionTest.java b/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQERepartitionTest.java index 58860b60d9..6a43606a86 100644 --- a/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQERepartitionTest.java +++ b/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQERepartitionTest.java @@ -47,7 +47,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); dynamicConf.put(CoordinatorConf.COORDINATOR_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE_HDFS.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE_HDFS.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQESkewedJoinTest.java b/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQESkewedJoinTest.java index dea7527681..ce215a12da 100644 --- a/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQESkewedJoinTest.java +++ b/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQESkewedJoinTest.java @@ -49,7 +49,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); dynamicConf.put(CoordinatorConf.COORDINATOR_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE, StorageType.MEMORY_LOCALFILE_HDFS.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE_HDFS.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); @@ -69,8 +69,8 @@ public void updateCommonSparkConf(SparkConf sparkConf) { @Override public void updateSparkConfCustomer(SparkConf sparkConf) { - sparkConf.set(RssSparkConfig.RSS_STORAGE_TYPE, "HDFS"); - sparkConf.set(RssSparkConfig.RSS_REMOTE_STORAGE_PATH, HDFS_URI + "rss/test"); + sparkConf.set(RssSparkConfig.RSS_STORAGE_TYPE.key, "HDFS"); + sparkConf.set(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key, HDFS_URI + "rss/test"); } @Test diff --git a/integration-test/spark3/src/test/java/org/apache/uniffle/test/GetReaderTest.java b/integration-test/spark3/src/test/java/org/apache/uniffle/test/GetReaderTest.java index 286245cc12..5739598386 100644 --- a/integration-test/spark3/src/test/java/org/apache/uniffle/test/GetReaderTest.java +++ b/integration-test/spark3/src/test/java/org/apache/uniffle/test/GetReaderTest.java @@ -68,7 +68,7 @@ public class GetReaderTest extends IntegrationTestBase { public void test() throws Exception { SparkConf sparkConf = new SparkConf(); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.RssShuffleManager"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); sparkConf.setMaster("local[4]"); final String remoteStorage1 = "hdfs://h1/p1"; final String remoteStorage2 = "hdfs://h2/p2"; From ec502130990e8510d8eb24f4c5d6f31e1fef40df Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Fri, 29 Jul 2022 19:34:21 +0800 Subject: [PATCH 03/22] fix RssShuffleManager --- .../apache/spark/shuffle/RssSparkConfig.java | 2 +- .../spark/shuffle/RssShuffleManager.java | 87 ++++++------ .../spark/shuffle/RssShuffleManager.java | 128 +++++++++--------- 3 files changed, 107 insertions(+), 110 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index 459ee9078b..875fa2d8f7 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -44,7 +44,7 @@ public RssConfigEntry(ConfigEntry entry) { } public T getValue() { - return (T) entry.defaultValue(). get(); + return (T) entry.defaultValue().get(); } } diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java index 4198460073..591aff3b2b 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java @@ -141,36 +141,35 @@ public RssShuffleManager(SparkConf sparkConf, boolean isDriver) { this.sparkConf = sparkConf; // set & check replica config - this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA, - RssSparkConfig.RSS_DATA_REPLICA_DEFAULT_VALUE); - this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE, - RssSparkConfig.RSS_DATA_REPLICA_WRITE_DEFAULT_VALUE); - this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ, - RssSparkConfig.RSS_DATA_REPLICA_READ_DEFAULT_VALUE); - this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE, - RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE_DEFAULT_VALUE); - this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED, - RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED_DEFAULT_VALUE); + this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA.key, + RssSparkConfig.RSS_DATA_REPLICA.getValue()); + this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key, + RssSparkConfig.RSS_DATA_REPLICA_WRITE.getValue()); + this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ.key, + RssSparkConfig.RSS_DATA_REPLICA_READ.getValue()); + this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.key, + RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.getValue()); + this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.key, LOG.info("Check quorum config [" + dataReplica + ":" + dataReplicaWrite + ":" + dataReplicaRead + ":" + dataReplicaSkipEnabled + "]"); RssUtils.checkQuorumSetting(dataReplica, dataReplicaWrite, dataReplicaRead); - this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE, - RssSparkConfig.RSS_CLIENT_TYPE_DEFAULT_VALUE); - this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL, - RssSparkConfig.RSS_HEARTBEAT_INTERVAL_DEFAULT_VALUE); - this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT, heartbeatInterval / 2); + this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key, + RssSparkConfig.RSS_CLIENT_TYPE.getValue()); + this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key, + RssSparkConfig.RSS_HEARTBEAT_INTERVAL.getValue()); + this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT.key, heartbeatInterval / 2); this.dynamicConfEnabled = sparkConf.getBoolean( - RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED, - RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED_DEFAULT_VALUE); - int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX, - RssSparkConfig.RSS_CLIENT_RETRY_MAX_DEFAULT_VALUE); - long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX, - RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX_DEFAULT_VALUE); - int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM, - RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM_DEFAULT_VALUE); - this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE, - RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE_DEFAULT_VALUE); + RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, + RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.getValue()); + int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key, + RssSparkConfig.RSS_CLIENT_RETRY_MAX.getValue()); + long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key, + RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.getValue()); + int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.key, + RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.getValue()); + this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.key, + RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.getValue()); this.shuffleWriteClient = ShuffleClientFactory .getInstance() .createShuffleWriteClient(clientType, retryMax, retryIntervalMax, heartBeatThreadNum, @@ -180,22 +179,22 @@ public RssShuffleManager(SparkConf sparkConf, boolean isDriver) { // fetch client conf and apply them if necessary and disable ESS if (isDriver && dynamicConfEnabled) { Map clusterClientConf = shuffleWriteClient.fetchClientConf( - sparkConf.getInt(RssSparkConfig.RSS_ACCESS_TIMEOUT_MS, - RssSparkConfig.RSS_ACCESS_TIMEOUT_MS_DEFAULT_VALUE)); + sparkConf.getInt(RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key, + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getValue())); RssSparkShuffleUtils.applyDynamicClientConf(sparkConf, clusterClientConf); } RssSparkShuffleUtils.validateRssClientConf(sparkConf); // External shuffle service is not supported when using remote shuffle service sparkConf.set("spark.shuffle.service.enabled", "false"); LOG.info("Disable external shuffle service in RssShuffleManager."); - if (!sparkConf.getBoolean(RssSparkConfig.RSS_TEST_FLAG, false)) { + if (!sparkConf.getBoolean(RssSparkConfig.RSS_TEST_FLAG.key, false)) { // for non-driver executor, start a thread for sending shuffle data to shuffle server LOG.info("RSS data send thread is starting"); eventLoop.start(); - int poolSize = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE, - RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE_DEFAULT_VALUE); - int keepAliveTime = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE, - RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE_DEFAULT_VALUE); + int poolSize = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.key, + RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.getValue()); + int keepAliveTime = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.key, + RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.getValue()); threadPoolExecutor = new ThreadPoolExecutor(poolSize, poolSize * 2, keepAliveTime, TimeUnit.SECONDS, Queues.newLinkedBlockingQueue(Integer.MAX_VALUE), ThreadUtils.getThreadFactory("SendData")); @@ -224,13 +223,13 @@ public ShuffleHandle registerShuffle(int shuffleId, int numMaps, Shuff LOG.info("Generate application id used in rss: " + appId); } - String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE); - remoteStorage = new RemoteStorageInfo(sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH, "")); + String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key); + remoteStorage = new RemoteStorageInfo(sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key, "")); remoteStorage = ClientUtils.fetchRemoteStorage( appId, remoteStorage, dynamicConfEnabled, storageType, shuffleWriteClient); - int partitionNumPerRange = sparkConf.getInt(RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE, - RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE_DEFAULT_VALUE); + int partitionNumPerRange = sparkConf.getInt(RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.key, + RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.getValue()); // get all register info according to coordinator's response Set assignmentTags = RssSparkShuffleUtils.getAssignmentTags(sparkConf); @@ -286,7 +285,7 @@ protected void registerShuffleServers( @VisibleForTesting protected void registerCoordinator() { - String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM); + String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM.key); LOG.info("Registering coordinators {}", coordinators); shuffleWriteClient.registerCoordinators(coordinators); } @@ -323,15 +322,15 @@ public ShuffleWriter getWriter(ShuffleHandle handle, int mapId, public ShuffleReader getReader(ShuffleHandle handle, int startPartition, int endPartition, TaskContext context) { if (handle instanceof RssShuffleHandle) { - final String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE); - final int indexReadLimit = sparkConf.getInt(RssSparkConfig.RSS_INDEX_READ_LIMIT, - RssSparkConfig.RSS_INDEX_READ_LIMIT_DEFAULT_VALUE); + final String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key); + final int indexReadLimit = sparkConf.getInt(RssSparkConfig.RSS_INDEX_READ_LIMIT.key, + RssSparkConfig.RSS_INDEX_READ_LIMIT.getValue()); RssShuffleHandle rssShuffleHandle = (RssShuffleHandle) handle; - final int partitionNumPerRange = sparkConf.getInt(RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE, - RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE_DEFAULT_VALUE); + final int partitionNumPerRange = sparkConf.getInt(RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.key, + RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.getValue()); final int partitionNum = rssShuffleHandle.getDependency().partitioner().numPartitions(); - long readBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE, - RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE_DEFAULT_VALUE); + long readBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key, + RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.getValue()); if (readBufferSize > Integer.MAX_VALUE) { LOG.warn(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE + " can support 2g as max"); readBufferSize = Integer.MAX_VALUE; diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java index 124ae6134b..14063c8a40 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java @@ -140,38 +140,36 @@ public RssShuffleManager(SparkConf conf, boolean isDriver) { this.sparkConf = conf; // set & check replica config - this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA, - RssSparkConfig.RSS_DATA_REPLICA_DEFAULT_VALUE); - this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE, - RssSparkConfig.RSS_DATA_REPLICA_WRITE_DEFAULT_VALUE); - this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ, - RssSparkConfig.RSS_DATA_REPLICA_READ_DEFAULT_VALUE); - this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED, - RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED_DEFAULT_VALUE); + this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA.key, + RssSparkConfig.RSS_DATA_REPLICA.getValue()); + this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key, + RssSparkConfig.RSS_DATA_REPLICA_WRITE.getValue()); + this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ.key, + RssSparkConfig.RSS_DATA_REPLICA_READ.getValue()); + this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.key, + RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.getValue()); LOG.info("Check quorum config [" + dataReplica + ":" + dataReplicaWrite + ":" + dataReplicaRead + ":" + dataReplicaSkipEnabled + "]"); RssUtils.checkQuorumSetting(dataReplica, dataReplicaWrite, dataReplicaRead); - this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL, - RssSparkConfig.RSS_HEARTBEAT_INTERVAL_DEFAULT_VALUE); - this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT, heartbeatInterval / 2); - final int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX, - RssSparkConfig.RSS_CLIENT_RETRY_MAX_DEFAULT_VALUE); - this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE, - RssSparkConfig.RSS_CLIENT_TYPE_DEFAULT_VALUE); + this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key, + RssSparkConfig.RSS_HEARTBEAT_INTERVAL.getValue()); + this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT.key, heartbeatInterval / 2); + final int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key, + RssSparkConfig.RSS_CLIENT_RETRY_MAX.getValue()); + this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key, + RssSparkConfig.RSS_CLIENT_TYPE.getValue()); this.dynamicConfEnabled = sparkConf.getBoolean( - RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED, - RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED_DEFAULT_VALUE); - - long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX, - RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX_DEFAULT_VALUE); - int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM, - RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM_DEFAULT_VALUE); - - this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE, - RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE_DEFAULT_VALUE); - this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE, - RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE_DEFAULT_VALUE); + RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, + RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.getValue()); + long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key, + RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.getValue()); + int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.key, + RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.getValue()); + this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.key, + RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.getValue()); + this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.key, + RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.getValue()); shuffleWriteClient = ShuffleClientFactory .getInstance() @@ -182,8 +180,8 @@ public RssShuffleManager(SparkConf conf, boolean isDriver) { // fetch client conf and apply them if necessary and disable ESS if (isDriver && dynamicConfEnabled) { Map clusterClientConf = shuffleWriteClient.fetchClientConf( - sparkConf.getInt(RssSparkConfig.RSS_ACCESS_TIMEOUT_MS, - RssSparkConfig.RSS_ACCESS_TIMEOUT_MS_DEFAULT_VALUE)); + sparkConf.getInt(RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key, + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getValue())); RssSparkShuffleUtils.applyDynamicClientConf(sparkConf, clusterClientConf); } RssSparkShuffleUtils.validateRssClientConf(sparkConf); @@ -196,10 +194,10 @@ public RssShuffleManager(SparkConf conf, boolean isDriver) { LOG.info("RSS data send thread is starting"); eventLoop = defaultEventLoop; eventLoop.start(); - int poolSize = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE, - RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE_DEFAULT_VALUE); - int keepAliveTime = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE, - RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE_DEFAULT_VALUE); + int poolSize = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.key, + RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.getValue()); + int keepAliveTime = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.key, + RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.getValue()); threadPoolExecutor = new ThreadPoolExecutor(poolSize, poolSize * 2, keepAliveTime, TimeUnit.SECONDS, Queues.newLinkedBlockingQueue(Integer.MAX_VALUE)); if (isDriver) { @@ -217,33 +215,33 @@ public RssShuffleManager(SparkConf conf, boolean isDriver) { Map> taskToSuccessBlockIds, Map> taskToFailedBlockIds) { this.sparkConf = conf; - this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE, - RssSparkConfig.RSS_CLIENT_TYPE_DEFAULT_VALUE); - this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL, - RssSparkConfig.RSS_HEARTBEAT_INTERVAL_DEFAULT_VALUE); - this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT, heartbeatInterval / 2); - this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA, - RssSparkConfig.RSS_DATA_REPLICA_DEFAULT_VALUE); - this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE, - RssSparkConfig.RSS_DATA_REPLICA_WRITE_DEFAULT_VALUE); - this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ, - RssSparkConfig.RSS_DATA_REPLICA_READ_DEFAULT_VALUE); - this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED, - RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED_DEFAULT_VALUE); + this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key, + RssSparkConfig.RSS_CLIENT_TYPE.getValue()); + this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key, + RssSparkConfig.RSS_HEARTBEAT_INTERVAL.getValue()); + this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT.key, heartbeatInterval / 2); + this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA.key, + RssSparkConfig.RSS_DATA_REPLICA.getValue()); + this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key, + RssSparkConfig.RSS_DATA_REPLICA_WRITE.getValue()); + this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ.key, + RssSparkConfig.RSS_DATA_REPLICA_READ.getValue()); + this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.key, + RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.getValue()); LOG.info("Check quorum config [" + dataReplica + ":" + dataReplicaWrite + ":" + dataReplicaRead + ":" + dataReplicaSkipEnabled + "]"); RssUtils.checkQuorumSetting(dataReplica, dataReplicaWrite, dataReplicaRead); - int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX, - RssSparkConfig.RSS_CLIENT_RETRY_MAX_DEFAULT_VALUE); - long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX, - RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX_DEFAULT_VALUE); - int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM, - RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM_DEFAULT_VALUE); - this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE, - RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE_DEFAULT_VALUE); - this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE, - RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE_DEFAULT_VALUE); + int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key, + RssSparkConfig.RSS_CLIENT_RETRY_MAX.getValue()); + long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key, + RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.getValue()); + int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.key, + RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.getValue()); + this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.key, + RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.getValue()); + this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.key, + RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.getValue()); shuffleWriteClient = ShuffleClientFactory .getInstance() @@ -276,9 +274,9 @@ public ShuffleHandle registerShuffle(int shuffleId, ShuffleDependency< } LOG.info("Generate application id used in rss: " + id.get()); - String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE); + String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key); remoteStorage = new RemoteStorageInfo( - sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH, "")); + sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key, "")); remoteStorage = ClientUtils.fetchRemoteStorage( id.get(), remoteStorage, dynamicConfEnabled, storageType, shuffleWriteClient); @@ -407,15 +405,15 @@ public ShuffleReader getReaderImpl( if (!(handle instanceof RssShuffleHandle)) { throw new RuntimeException("Unexpected ShuffleHandle:" + handle.getClass().getName()); } - final String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE); - final int indexReadLimit = sparkConf.getInt(RssSparkConfig.RSS_INDEX_READ_LIMIT, - RssSparkConfig.RSS_INDEX_READ_LIMIT_DEFAULT_VALUE); + final String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key); + final int indexReadLimit = sparkConf.getInt(RssSparkConfig.RSS_INDEX_READ_LIMIT.key, + RssSparkConfig.RSS_INDEX_READ_LIMIT.getValue()); RssShuffleHandle rssShuffleHandle = (RssShuffleHandle) handle; final int partitionNum = rssShuffleHandle.getDependency().partitioner().numPartitions(); - long readBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE, - RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE_DEFAULT_VALUE); + long readBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key, + RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.getValue()); if (readBufferSize > Integer.MAX_VALUE) { - LOG.warn(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE + " can support 2g as max"); + LOG.warn(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key + " can support 2g as max"); readBufferSize = Integer.MAX_VALUE; } int shuffleId = rssShuffleHandle.getShuffleId(); @@ -600,7 +598,7 @@ protected void registerShuffleServers(String appId, int shuffleId, @VisibleForTesting protected void registerCoordinator() { - String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM); + String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM.key); LOG.info("Start Registering coordinators {}", coordinators); shuffleWriteClient.registerCoordinators(coordinators); } From a5c418de45150b414318a64148c3aa8f258324de Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Fri, 29 Jul 2022 19:41:05 +0800 Subject: [PATCH 04/22] fix wrong order --- .../main/java/org/apache/spark/shuffle/RssSparkConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index 894a54238b..0ad3631918 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -17,12 +17,12 @@ package org.apache.spark.shuffle; +import java.util.Set; + import com.google.common.collect.Sets; import org.apache.spark.internal.config.ConfigBuilder; import org.apache.spark.internal.config.ConfigEntry; -import java.util.Set; - public class RssSparkConfig { public static final class SparkConfigBuilder { From ffbf49e07df933e951319c0342af0286dfcbd772 Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Fri, 29 Jul 2022 19:44:40 +0800 Subject: [PATCH 05/22] add final to modify params --- .../apache/spark/shuffle/RssSparkConfig.java | 78 +++++++++---------- 1 file changed, 39 insertions(+), 39 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index 0ad3631918..9f98c53981 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -48,236 +48,236 @@ public T getValue() { } } - public static RssConfigEntry RSS_PARTITION_NUM_PER_RANGE = new RssConfigEntry( + public static final RssConfigEntry RSS_PARTITION_NUM_PER_RANGE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.partitionNum.per.range") .intConf() .createWithDefault(1)); - public static RssConfigEntry RSS_WRITER_BUFFER_SIZE = new RssConfigEntry( + public static final RssConfigEntry RSS_WRITER_BUFFER_SIZE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.writer.buffer.size") .doc("controls the buffer flushing size during shuffle write") .stringConf() .createWithDefault("3m")); - public static RssConfigEntry RSS_WRITER_SERIALIZER_BUFFER_SIZE = new RssConfigEntry( + public static final RssConfigEntry RSS_WRITER_SERIALIZER_BUFFER_SIZE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.writer.serializer.buffer.size") .stringConf() .createWithDefault("3k")); - public static RssConfigEntry RSS_WRITER_BUFFER_SEGMENT_SIZE = new RssConfigEntry( + public static final RssConfigEntry RSS_WRITER_BUFFER_SEGMENT_SIZE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.writer.buffer.segment.size") .stringConf() .createWithDefault("3k")); - public static RssConfigEntry RSS_WRITER_BUFFER_SPILL_SIZE = new RssConfigEntry( + public static final RssConfigEntry RSS_WRITER_BUFFER_SPILL_SIZE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.writer.buffer.spill.size") .stringConf() .createWithDefault("128m")); - public static RssConfigEntry RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE = new RssConfigEntry( + public static final RssConfigEntry RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.writer.pre.allocated.buffer.size") .stringConf() .createWithDefault("16m")); - public static RssConfigEntry RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX = new RssConfigEntry( + public static final RssConfigEntry RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.writer.require.memory.retryMax") .intConf() .createWithDefault(1200)); - public static RssConfigEntry RSS_WRITER_REQUIRE_MEMORY_INTERVAL = new RssConfigEntry( + public static final RssConfigEntry RSS_WRITER_REQUIRE_MEMORY_INTERVAL = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.writer.require.memory.interval") .longConf() .createWithDefault(1000)); - public static RssConfigEntry RSS_CLIENT_SEND_CHECK_TIMEOUT_MS = new RssConfigEntry( + public static final RssConfigEntry RSS_CLIENT_SEND_CHECK_TIMEOUT_MS = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.send.check.timeout.ms") .longConf() .createWithDefault(60 * 1000 * 10)); - public static RssConfigEntry RSS_CLIENT_SEND_CHECK_INTERVAL_MS = new RssConfigEntry( + public static final RssConfigEntry RSS_CLIENT_SEND_CHECK_INTERVAL_MS = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.send.check.interval.ms") .longConf() .createWithDefault(500)); - public static RssConfigEntry RSS_TEST_FLAG = new RssConfigEntry( + public static final RssConfigEntry RSS_TEST_FLAG = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.test") .stringConf() .createWithDefault("")); - public static RssConfigEntry RSS_REMOTE_STORAGE_PATH = new RssConfigEntry( + public static final RssConfigEntry RSS_REMOTE_STORAGE_PATH = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.remote.storage.path") .stringConf() .createWithDefault("")); - public static RssConfigEntry RSS_INDEX_READ_LIMIT = new RssConfigEntry( + public static final RssConfigEntry RSS_INDEX_READ_LIMIT = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.index.read.limit") .intConf() .createWithDefault(500)); - public static RssConfigEntry RSS_CLIENT_TYPE = new RssConfigEntry( + public static final RssConfigEntry RSS_CLIENT_TYPE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.type") .stringConf() .createWithDefault("GRPC")); - public static RssConfigEntry RSS_STORAGE_TYPE = new RssConfigEntry( + public static final RssConfigEntry RSS_STORAGE_TYPE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.storage.type") .stringConf() .createWithDefault("")); - public static RssConfigEntry RSS_CLIENT_RETRY_MAX = new RssConfigEntry( + public static final RssConfigEntry RSS_CLIENT_RETRY_MAX = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.retry.max") .intConf() .createWithDefault(100)); - public static RssConfigEntry RSS_CLIENT_RETRY_INTERVAL_MAX = new RssConfigEntry( + public static final RssConfigEntry RSS_CLIENT_RETRY_INTERVAL_MAX = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.retry.interval.max") .longConf() .createWithDefault(10000)); - public static RssConfigEntry RSS_CLIENT_HEARTBEAT_THREAD_NUM = new RssConfigEntry( + public static final RssConfigEntry RSS_CLIENT_HEARTBEAT_THREAD_NUM = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.heartBeat.threadNum") .intConf() .createWithDefault(4)); - public static RssConfigEntry RSS_CLIENT_SEND_SIZE_LIMIT = new RssConfigEntry( + public static final RssConfigEntry RSS_CLIENT_SEND_SIZE_LIMIT = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.send.size.limit") .stringConf() .createWithDefault("16m")); - public static RssConfigEntry RSS_CLIENT_READ_BUFFER_SIZE = new RssConfigEntry( + public static final RssConfigEntry RSS_CLIENT_READ_BUFFER_SIZE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.read.buffer.size") .stringConf() .createWithDefault("14m")); - public static RssConfigEntry RSS_HEARTBEAT_INTERVAL = new RssConfigEntry( + public static final RssConfigEntry RSS_HEARTBEAT_INTERVAL = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.heartbeat.interval") .longConf() .createWithDefault(10 * 1000L)); - public static RssConfigEntry RSS_HEARTBEAT_TIMEOUT = new RssConfigEntry( + public static final RssConfigEntry RSS_HEARTBEAT_TIMEOUT = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.heartbeat.timeout") .longConf() .createWithDefault(5 * 1000L)); - public static RssConfigEntry RSS_CLIENT_SEND_THREAD_POOL_SIZE = new RssConfigEntry( + public static final RssConfigEntry RSS_CLIENT_SEND_THREAD_POOL_SIZE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.send.threadPool.size") .intConf() .createWithDefault(10)); - public static RssConfigEntry RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE = new RssConfigEntry( + public static final RssConfigEntry RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.send.threadPool.keepalive") .intConf() .createWithDefault(60)); - public static RssConfigEntry RSS_DATA_REPLICA = new RssConfigEntry( + public static final RssConfigEntry RSS_DATA_REPLICA = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.data.replica") .intConf() .createWithDefault(1)); - public static RssConfigEntry RSS_DATA_REPLICA_WRITE = new RssConfigEntry( + public static final RssConfigEntry RSS_DATA_REPLICA_WRITE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.data.replica.write") .intConf() .createWithDefault(1)); - public static RssConfigEntry RSS_DATA_REPLICA_READ = new RssConfigEntry( + public static final RssConfigEntry RSS_DATA_REPLICA_READ = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.data.replica.read") .intConf() .createWithDefault(1)); - public static RssConfigEntry RSS_DATA_REPLICA_SKIP_ENABLED = new RssConfigEntry( + public static final RssConfigEntry RSS_DATA_REPLICA_SKIP_ENABLED = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.data.replica.skip.enabled") .booleanConf() .createWithDefault(true)); - public static RssConfigEntry RSS_DATA_TRANSFER_POOL_SIZE = new RssConfigEntry( + public static final RssConfigEntry RSS_DATA_TRANSFER_POOL_SIZE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.data.transfer.pool.size") .intConf() .createWithDefault(Runtime.getRuntime().availableProcessors())); - public static RssConfigEntry RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE = new RssConfigEntry( + public static final RssConfigEntry RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.ozone.dfs.namenode.odfs.enable") .booleanConf() .createWithDefault(false)); - public static RssConfigEntry RSS_OZONE_FS_HDFS_IMPL = new RssConfigEntry( + public static final RssConfigEntry RSS_OZONE_FS_HDFS_IMPL = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.ozone.fs.hdfs.impl") .stringConf() .createWithDefault("org.apache.hadoop.odfs.HdfsOdfsFilesystem")); - public static RssConfigEntry RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL = new RssConfigEntry( + public static final RssConfigEntry RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.ozone.fs.AbstractFileSystem.hdfs.impl") .stringConf() .createWithDefault("org.apache.hadoop.odfs.HdfsOdfs")); - public static RssConfigEntry RSS_CLIENT_BITMAP_SPLIT_NUM = new RssConfigEntry( + public static final RssConfigEntry RSS_CLIENT_BITMAP_SPLIT_NUM = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.bitmap.splitNum") .intConf() .createWithDefault(1)); - public static RssConfigEntry RSS_ACCESS_ID = new RssConfigEntry( + public static final RssConfigEntry RSS_ACCESS_ID = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.access.id") .stringConf() .createWithDefault("")); - public static RssConfigEntry RSS_ACCESS_TIMEOUT_MS = new RssConfigEntry( + public static final RssConfigEntry RSS_ACCESS_TIMEOUT_MS = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.access.timeout.ms") .intConf() .createWithDefault(10000)); - public static RssConfigEntry RSS_ENABLED = new RssConfigEntry( + public static final RssConfigEntry RSS_ENABLED = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.enabled") .booleanConf() .createWithDefault(false)); - public static RssConfigEntry RSS_DYNAMIC_CLIENT_CONF_ENABLED = new RssConfigEntry( + public static final RssConfigEntry RSS_DYNAMIC_CLIENT_CONF_ENABLED = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.dynamicClientConf.enabled") .booleanConf() .createWithDefault(true)); - public static RssConfigEntry RSS_CLIENT_ASSIGNMENT_TAGS = new RssConfigEntry( + public static final RssConfigEntry RSS_CLIENT_ASSIGNMENT_TAGS = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.assignment.tags") .stringConf() .createWithDefault("")); - public static RssConfigEntry RSS_COORDINATOR_QUORUM = new RssConfigEntry( + public static final RssConfigEntry RSS_COORDINATOR_QUORUM = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.coordinator.quorum") .stringConf() From 5a33c8de021e3641331ceaa9765c3bd14ef25bd2 Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Fri, 29 Jul 2022 20:08:36 +0800 Subject: [PATCH 06/22] add RSS_DATA_COMMIT_POOL_SIZE --- .../main/java/org/apache/spark/shuffle/RssSparkConfig.java | 6 ++++++ .../java/org/apache/spark/shuffle/RssShuffleManager.java | 1 + 2 files changed, 7 insertions(+) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index 9f98c53981..65b8c1a59c 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -283,6 +283,12 @@ public T getValue() { .stringConf() .createWithDefault("")); + public static final RssConfigEntry RSS_DATA_COMMIT_POOL_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.data.commit.pool.size") + .intConf() + .createWithDefault(-1)); + public static final Set RSS_MANDATORY_CLUSTER_CONF = Sets.newHashSet(RSS_STORAGE_TYPE.key, RSS_REMOTE_STORAGE_PATH.key); diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java index 591aff3b2b..6e89ca71c7 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java @@ -150,6 +150,7 @@ public RssShuffleManager(SparkConf sparkConf, boolean isDriver) { this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.key, RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.getValue()); this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.key, + RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.getValue()); LOG.info("Check quorum config [" + dataReplica + ":" + dataReplicaWrite + ":" + dataReplicaRead + ":" + dataReplicaSkipEnabled + "]"); RssUtils.checkQuorumSetting(dataReplica, dataReplicaWrite, dataReplicaRead); From 37b468de20428c276c80e1ba40f1b4acfa992e01 Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Fri, 29 Jul 2022 20:23:30 +0800 Subject: [PATCH 07/22] change String to Integer --- .../src/main/java/org/apache/spark/shuffle/RssSparkConfig.java | 2 +- .../main/java/org/apache/spark/shuffle/RssShuffleManager.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index 65b8c1a59c..beb64cee85 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -283,7 +283,7 @@ public T getValue() { .stringConf() .createWithDefault("")); - public static final RssConfigEntry RSS_DATA_COMMIT_POOL_SIZE = new RssConfigEntry( + public static final RssConfigEntry RSS_DATA_COMMIT_POOL_SIZE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.data.commit.pool.size") .intConf() diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java index 6e89ca71c7..50c7d8e53b 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java @@ -248,7 +248,7 @@ public ShuffleHandle registerShuffle(int shuffleId, int numMaps, Shuff } private void startHeartbeat() { - if (!sparkConf.getBoolean(RssSparkConfig.RSS_TEST_FLAG, false) && !heartbeatStarted) { + if (!sparkConf.getBoolean(RssSparkConfig.RSS_TEST_FLAG.key, false) && !heartbeatStarted) { heartBeatScheduledExecutorService.scheduleAtFixedRate( () -> { try { From 771ef2bc2c60c1c444a68e7adf87b0d0d9641833 Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Fri, 29 Jul 2022 20:49:00 +0800 Subject: [PATCH 08/22] Modifying default parameters --- .../apache/spark/shuffle/RssSparkConfig.java | 43 ++++++++++--------- 1 file changed, 22 insertions(+), 21 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index beb64cee85..cfd4ddd520 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -22,6 +22,7 @@ import com.google.common.collect.Sets; import org.apache.spark.internal.config.ConfigBuilder; import org.apache.spark.internal.config.ConfigEntry; +import org.apache.uniffle.client.util.RssClientConfig; public class RssSparkConfig { @@ -52,7 +53,7 @@ public T getValue() { SparkConfigBuilder .key("spark.rss.partitionNum.per.range") .intConf() - .createWithDefault(1)); + .createWithDefault(RssClientConfig.RSS_PARTITION_NUM_PER_RANGE_DEFAULT_VALUE)); public static final RssConfigEntry RSS_WRITER_BUFFER_SIZE = new RssConfigEntry( SparkConfigBuilder @@ -101,13 +102,13 @@ public T getValue() { SparkConfigBuilder .key("spark.rss.client.send.check.timeout.ms") .longConf() - .createWithDefault(60 * 1000 * 10)); + .createWithDefault(RssClientConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE)); public static final RssConfigEntry RSS_CLIENT_SEND_CHECK_INTERVAL_MS = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.send.check.interval.ms") .longConf() - .createWithDefault(500)); + .createWithDefault(RssClientConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE)); public static final RssConfigEntry RSS_TEST_FLAG = new RssConfigEntry( SparkConfigBuilder @@ -131,7 +132,7 @@ public T getValue() { SparkConfigBuilder .key("spark.rss.client.type") .stringConf() - .createWithDefault("GRPC")); + .createWithDefault(RssClientConfig.RSS_CLIENT_TYPE_DEFAULT_VALUE)); public static final RssConfigEntry RSS_STORAGE_TYPE = new RssConfigEntry( SparkConfigBuilder @@ -143,13 +144,13 @@ public T getValue() { SparkConfigBuilder .key("spark.rss.client.retry.max") .intConf() - .createWithDefault(100)); + .createWithDefault(RssClientConfig.RSS_CLIENT_RETRY_MAX_DEFAULT_VALUE)); public static final RssConfigEntry RSS_CLIENT_RETRY_INTERVAL_MAX = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.retry.interval.max") .longConf() - .createWithDefault(10000)); + .createWithDefault(RssClientConfig.RSS_CLIENT_RETRY_INTERVAL_MAX_DEFAULT_VALUE)); public static final RssConfigEntry RSS_CLIENT_HEARTBEAT_THREAD_NUM = new RssConfigEntry( SparkConfigBuilder @@ -167,13 +168,13 @@ public T getValue() { SparkConfigBuilder .key("spark.rss.client.read.buffer.size") .stringConf() - .createWithDefault("14m")); + .createWithDefault(RssClientConfig.RSS_CLIENT_READ_BUFFER_SIZE_DEFAULT_VALUE)); public static final RssConfigEntry RSS_HEARTBEAT_INTERVAL = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.heartbeat.interval") .longConf() - .createWithDefault(10 * 1000L)); + .createWithDefault(RssClientConfig.RSS_HEARTBEAT_INTERVAL_DEFAULT_VALUE)); public static final RssConfigEntry RSS_HEARTBEAT_TIMEOUT = new RssConfigEntry( SparkConfigBuilder @@ -197,31 +198,37 @@ public T getValue() { SparkConfigBuilder .key("spark.rss.data.replica") .intConf() - .createWithDefault(1)); + .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_DEFAULT_VALUE)); public static final RssConfigEntry RSS_DATA_REPLICA_WRITE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.data.replica.write") .intConf() - .createWithDefault(1)); + .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_WRITE_DEFAULT_VALUE)); public static final RssConfigEntry RSS_DATA_REPLICA_READ = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.data.replica.read") .intConf() - .createWithDefault(1)); + .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_READ_DEFAULT_VALUE)); public static final RssConfigEntry RSS_DATA_REPLICA_SKIP_ENABLED = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.data.replica.skip.enabled") .booleanConf() - .createWithDefault(true)); + .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_SKIP_ENABLED_DEFAULT_VALUE)); public static final RssConfigEntry RSS_DATA_TRANSFER_POOL_SIZE = new RssConfigEntry( SparkConfigBuilder .key("spark.rss.client.data.transfer.pool.size") .intConf() - .createWithDefault(Runtime.getRuntime().availableProcessors())); + .createWithDefault(RssClientConfig.RSS_DATA_TRANFER_POOL_SIZE_DEFAULT_VALUE)); + + public static final RssConfigEntry RSS_DATA_COMMIT_POOL_SIZE = new RssConfigEntry( + SparkConfigBuilder + .key("spark.rss.client.data.commit.pool.size") + .intConf() + .createWithDefault(RssClientConfig.RSS_DATA_COMMIT_POOL_SIZE_DEFAULT_VALUE)); public static final RssConfigEntry RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE = new RssConfigEntry( SparkConfigBuilder @@ -257,7 +264,7 @@ public T getValue() { SparkConfigBuilder .key("spark.rss.access.timeout.ms") .intConf() - .createWithDefault(10000)); + .createWithDefault(RssClientConfig.RSS_ACCESS_TIMEOUT_MS_DEFAULT_VALUE)); public static final RssConfigEntry RSS_ENABLED = new RssConfigEntry( SparkConfigBuilder @@ -269,7 +276,7 @@ public T getValue() { SparkConfigBuilder .key("spark.rss.dynamicClientConf.enabled") .booleanConf() - .createWithDefault(true)); + .createWithDefault(RssClientConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED_DEFAULT_VALUE)); public static final RssConfigEntry RSS_CLIENT_ASSIGNMENT_TAGS = new RssConfigEntry( SparkConfigBuilder @@ -283,12 +290,6 @@ public T getValue() { .stringConf() .createWithDefault("")); - public static final RssConfigEntry RSS_DATA_COMMIT_POOL_SIZE = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.client.data.commit.pool.size") - .intConf() - .createWithDefault(-1)); - public static final Set RSS_MANDATORY_CLUSTER_CONF = Sets.newHashSet(RSS_STORAGE_TYPE.key, RSS_REMOTE_STORAGE_PATH.key); From 1d1d7105ba208d0f328ff3adf4d57746d5d81e57 Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Fri, 29 Jul 2022 21:06:35 +0800 Subject: [PATCH 09/22] change method name --- .../apache/spark/shuffle/RssSparkConfig.java | 2 +- .../spark/shuffle/RssSparkShuffleUtils.java | 8 +-- .../shuffle/writer/BufferManagerOptions.java | 14 ++--- .../shuffle/DelegationRssShuffleManager.java | 2 +- .../spark/shuffle/RssShuffleManager.java | 38 ++++++------- .../shuffle/writer/RssShuffleWriter.java | 8 +-- .../shuffle/DelegationRssShuffleManager.java | 4 +- .../spark/shuffle/RssShuffleManager.java | 56 +++++++++---------- 8 files changed, 66 insertions(+), 66 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index cfd4ddd520..9cfe0453a2 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -44,7 +44,7 @@ public RssConfigEntry(ConfigEntry entry) { this.key = entry.key(); } - public T getValue() { + public T getDefaultValue() { return (T) entry.defaultValue().get(); } } diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java index 9e5f99f038..6e1e94e34c 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java @@ -45,19 +45,19 @@ public static Configuration newHadoopConfiguration(SparkConf sparkConf) { Configuration conf = util.newConfiguration(sparkConf); boolean useOdfs = sparkConf.getBoolean(RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE.key, - RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE.getValue()); + RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE.getDefaultValue()); if (useOdfs) { final int OZONE_PREFIX_LEN = "spark.rss.ozone.".length(); conf.setBoolean(RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE.key.substring(OZONE_PREFIX_LEN), useOdfs); conf.set( RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.key.substring(OZONE_PREFIX_LEN), sparkConf.get(RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.key, - RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.getValue())); + RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.getDefaultValue())); conf.set( 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.key, - RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.getValue())); + RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.getDefaultValue())); } return conf; @@ -79,7 +79,7 @@ public static ShuffleManager loadShuffleManager(String name, SparkConf conf, boo public static List createCoordinatorClients(SparkConf sparkConf) throws RuntimeException { String clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key, - RssSparkConfig.RSS_CLIENT_TYPE.getValue()); + RssSparkConfig.RSS_CLIENT_TYPE.getDefaultValue()); String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM.key); CoordinatorClientFactory coordinatorClientFactory = new CoordinatorClientFactory(clientType); return coordinatorClientFactory.createCoordinatorClient(coordinators); diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java index e027c1a1e9..23c319e4b2 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java @@ -36,19 +36,19 @@ public class BufferManagerOptions { public BufferManagerOptions(SparkConf sparkConf) { bufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key, - RssSparkConfig.RSS_WRITER_BUFFER_SIZE.getValue()); + RssSparkConfig.RSS_WRITER_BUFFER_SIZE.getDefaultValue()); serializerBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key, - RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.getValue()); + RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.getDefaultValue()); bufferSegmentSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key, - RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.getValue()); + RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.getDefaultValue()); bufferSpillThreshold = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key, - RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.getValue()); + RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.getDefaultValue()); preAllocatedBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE.key, - RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE.getValue()); + RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE.getDefaultValue()); requireMemoryInterval = sparkConf.getLong(RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_INTERVAL.key, - RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_INTERVAL.getValue()); + RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_INTERVAL.getDefaultValue()); requireMemoryRetryMax = sparkConf.getInt(RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX.key, - RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX.getValue()); + RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX.getDefaultValue()); 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); diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java index 45d2934820..0b373c4d14 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java @@ -48,7 +48,7 @@ public DelegationRssShuffleManager(SparkConf sparkConf, boolean isDriver) throws this.sparkConf = sparkConf; accessTimeoutMs = sparkConf.getInt( RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key, - RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getValue()); + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getDefaultValue()); if (isDriver) { coordinatorClients = RssSparkShuffleUtils.createCoordinatorClients(sparkConf); delegate = createShuffleManagerInDriver(); diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java index 50c7d8e53b..15dbe808cb 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java @@ -142,35 +142,35 @@ public RssShuffleManager(SparkConf sparkConf, boolean isDriver) { // set & check replica config this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA.key, - RssSparkConfig.RSS_DATA_REPLICA.getValue()); + RssSparkConfig.RSS_DATA_REPLICA.getDefaultValue()); this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key, - RssSparkConfig.RSS_DATA_REPLICA_WRITE.getValue()); + RssSparkConfig.RSS_DATA_REPLICA_WRITE.getDefaultValue()); this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ.key, - RssSparkConfig.RSS_DATA_REPLICA_READ.getValue()); + RssSparkConfig.RSS_DATA_REPLICA_READ.getDefaultValue()); this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.key, - RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.getValue()); + RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.getDefaultValue()); this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.key, - RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.getValue()); + RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.getDefaultValue()); LOG.info("Check quorum config [" + dataReplica + ":" + dataReplicaWrite + ":" + dataReplicaRead + ":" + dataReplicaSkipEnabled + "]"); RssUtils.checkQuorumSetting(dataReplica, dataReplicaWrite, dataReplicaRead); this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key, - RssSparkConfig.RSS_CLIENT_TYPE.getValue()); + RssSparkConfig.RSS_CLIENT_TYPE.getDefaultValue()); this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key, - RssSparkConfig.RSS_HEARTBEAT_INTERVAL.getValue()); + RssSparkConfig.RSS_HEARTBEAT_INTERVAL.getDefaultValue()); this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT.key, heartbeatInterval / 2); this.dynamicConfEnabled = sparkConf.getBoolean( RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, - RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.getValue()); + RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.getDefaultValue()); int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key, - RssSparkConfig.RSS_CLIENT_RETRY_MAX.getValue()); + RssSparkConfig.RSS_CLIENT_RETRY_MAX.getDefaultValue()); long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key, - RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.getValue()); + RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.getDefaultValue()); int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.key, - RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.getValue()); + RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.getDefaultValue()); this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.key, - RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.getValue()); + RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.getDefaultValue()); this.shuffleWriteClient = ShuffleClientFactory .getInstance() .createShuffleWriteClient(clientType, retryMax, retryIntervalMax, heartBeatThreadNum, @@ -181,7 +181,7 @@ public RssShuffleManager(SparkConf sparkConf, boolean isDriver) { if (isDriver && dynamicConfEnabled) { Map clusterClientConf = shuffleWriteClient.fetchClientConf( sparkConf.getInt(RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key, - RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getValue())); + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getDefaultValue())); RssSparkShuffleUtils.applyDynamicClientConf(sparkConf, clusterClientConf); } RssSparkShuffleUtils.validateRssClientConf(sparkConf); @@ -193,9 +193,9 @@ public RssShuffleManager(SparkConf sparkConf, boolean isDriver) { LOG.info("RSS data send thread is starting"); eventLoop.start(); int poolSize = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.key, - RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.getValue()); + RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.getDefaultValue()); int keepAliveTime = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.key, - RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.getValue()); + RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.getDefaultValue()); threadPoolExecutor = new ThreadPoolExecutor(poolSize, poolSize * 2, keepAliveTime, TimeUnit.SECONDS, Queues.newLinkedBlockingQueue(Integer.MAX_VALUE), ThreadUtils.getThreadFactory("SendData")); @@ -230,7 +230,7 @@ public ShuffleHandle registerShuffle(int shuffleId, int numMaps, Shuff appId, remoteStorage, dynamicConfEnabled, storageType, shuffleWriteClient); int partitionNumPerRange = sparkConf.getInt(RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.key, - RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.getValue()); + RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.getDefaultValue()); // get all register info according to coordinator's response Set assignmentTags = RssSparkShuffleUtils.getAssignmentTags(sparkConf); @@ -325,13 +325,13 @@ public ShuffleReader getReader(ShuffleHandle handle, if (handle instanceof RssShuffleHandle) { final String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key); final int indexReadLimit = sparkConf.getInt(RssSparkConfig.RSS_INDEX_READ_LIMIT.key, - RssSparkConfig.RSS_INDEX_READ_LIMIT.getValue()); + RssSparkConfig.RSS_INDEX_READ_LIMIT.getDefaultValue()); RssShuffleHandle rssShuffleHandle = (RssShuffleHandle) handle; final int partitionNumPerRange = sparkConf.getInt(RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.key, - RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.getValue()); + RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.getDefaultValue()); final int partitionNum = rssShuffleHandle.getDependency().partitioner().numPartitions(); long readBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key, - RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.getValue()); + RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.getDefaultValue()); if (readBufferSize > Integer.MAX_VALUE) { LOG.warn(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE + " can support 2g as max"); readBufferSize = Integer.MAX_VALUE; diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java index 4e51f3c606..3a216d7428 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java @@ -106,13 +106,13 @@ public RssShuffleWriter( this.shuffleManager = shuffleManager; this.shouldPartition = partitioner.numPartitions() > 1; this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, - RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.getValue()); + RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.getDefaultValue()); this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, - RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.getValue()); + RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.getDefaultValue()); this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key, - RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.getValue()); + RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.getDefaultValue()); this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.key, - RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.getValue()); + RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.getDefaultValue()); this.partitionToBlockIds = Maps.newConcurrentMap(); this.shuffleWriteClient = shuffleWriteClient; this.shuffleServersForData = rssHandle.getShuffleServersForData(); diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java index 27d76a6b66..6afad97f9f 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java @@ -48,7 +48,7 @@ public DelegationRssShuffleManager(SparkConf sparkConf, boolean isDriver) throws this.sparkConf = sparkConf; accessTimeoutMs = sparkConf.getInt( RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key, - RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getValue()); + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getDefaultValue()); if (isDriver) { coordinatorClients = RssSparkShuffleUtils.createCoordinatorClients(sparkConf); delegate = createShuffleManagerInDriver(); @@ -129,7 +129,7 @@ private ShuffleManager createShuffleManagerInExecutor() throws RssException { // get useRSS from spark conf boolean useRSS = sparkConf.getBoolean( RssSparkConfig.RSS_ENABLED.key, - RssSparkConfig.RSS_ENABLED.getValue()); + RssSparkConfig.RSS_ENABLED.getDefaultValue()); if (useRSS) { // Executor will not do any fallback shuffleManager = new RssShuffleManager(sparkConf, false); diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java index 14063c8a40..9d76886b94 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java @@ -141,35 +141,35 @@ public RssShuffleManager(SparkConf conf, boolean isDriver) { // set & check replica config this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA.key, - RssSparkConfig.RSS_DATA_REPLICA.getValue()); + RssSparkConfig.RSS_DATA_REPLICA.getDefaultValue()); this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key, - RssSparkConfig.RSS_DATA_REPLICA_WRITE.getValue()); + RssSparkConfig.RSS_DATA_REPLICA_WRITE.getDefaultValue()); this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ.key, - RssSparkConfig.RSS_DATA_REPLICA_READ.getValue()); + RssSparkConfig.RSS_DATA_REPLICA_READ.getDefaultValue()); this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.key, - RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.getValue()); + RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.getDefaultValue()); LOG.info("Check quorum config [" + dataReplica + ":" + dataReplicaWrite + ":" + dataReplicaRead + ":" + dataReplicaSkipEnabled + "]"); RssUtils.checkQuorumSetting(dataReplica, dataReplicaWrite, dataReplicaRead); this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key, - RssSparkConfig.RSS_HEARTBEAT_INTERVAL.getValue()); + RssSparkConfig.RSS_HEARTBEAT_INTERVAL.getDefaultValue()); this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT.key, heartbeatInterval / 2); final int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key, - RssSparkConfig.RSS_CLIENT_RETRY_MAX.getValue()); + RssSparkConfig.RSS_CLIENT_RETRY_MAX.getDefaultValue()); this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key, - RssSparkConfig.RSS_CLIENT_TYPE.getValue()); + RssSparkConfig.RSS_CLIENT_TYPE.getDefaultValue()); this.dynamicConfEnabled = sparkConf.getBoolean( RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, - RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.getValue()); + RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.getDefaultValue()); long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key, - RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.getValue()); + RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.getDefaultValue()); int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.key, - RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.getValue()); + RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.getDefaultValue()); this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.key, - RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.getValue()); + RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.getDefaultValue()); this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.key, - RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.getValue()); + RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.getDefaultValue()); shuffleWriteClient = ShuffleClientFactory .getInstance() @@ -181,7 +181,7 @@ public RssShuffleManager(SparkConf conf, boolean isDriver) { if (isDriver && dynamicConfEnabled) { Map clusterClientConf = shuffleWriteClient.fetchClientConf( sparkConf.getInt(RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key, - RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getValue())); + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getDefaultValue())); RssSparkShuffleUtils.applyDynamicClientConf(sparkConf, clusterClientConf); } RssSparkShuffleUtils.validateRssClientConf(sparkConf); @@ -195,9 +195,9 @@ public RssShuffleManager(SparkConf conf, boolean isDriver) { eventLoop = defaultEventLoop; eventLoop.start(); int poolSize = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.key, - RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.getValue()); + RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.getDefaultValue()); int keepAliveTime = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.key, - RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.getValue()); + RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.getDefaultValue()); threadPoolExecutor = new ThreadPoolExecutor(poolSize, poolSize * 2, keepAliveTime, TimeUnit.SECONDS, Queues.newLinkedBlockingQueue(Integer.MAX_VALUE)); if (isDriver) { @@ -216,32 +216,32 @@ public RssShuffleManager(SparkConf conf, boolean isDriver) { Map> taskToFailedBlockIds) { this.sparkConf = conf; this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key, - RssSparkConfig.RSS_CLIENT_TYPE.getValue()); + RssSparkConfig.RSS_CLIENT_TYPE.getDefaultValue()); this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key, - RssSparkConfig.RSS_HEARTBEAT_INTERVAL.getValue()); + RssSparkConfig.RSS_HEARTBEAT_INTERVAL.getDefaultValue()); this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT.key, heartbeatInterval / 2); this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA.key, - RssSparkConfig.RSS_DATA_REPLICA.getValue()); + RssSparkConfig.RSS_DATA_REPLICA.getDefaultValue()); this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key, - RssSparkConfig.RSS_DATA_REPLICA_WRITE.getValue()); + RssSparkConfig.RSS_DATA_REPLICA_WRITE.getDefaultValue()); this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ.key, - RssSparkConfig.RSS_DATA_REPLICA_READ.getValue()); + RssSparkConfig.RSS_DATA_REPLICA_READ.getDefaultValue()); this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.key, - RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.getValue()); + RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.getDefaultValue()); LOG.info("Check quorum config [" + dataReplica + ":" + dataReplicaWrite + ":" + dataReplicaRead + ":" + dataReplicaSkipEnabled + "]"); RssUtils.checkQuorumSetting(dataReplica, dataReplicaWrite, dataReplicaRead); int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key, - RssSparkConfig.RSS_CLIENT_RETRY_MAX.getValue()); + RssSparkConfig.RSS_CLIENT_RETRY_MAX.getDefaultValue()); long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key, - RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.getValue()); + RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.getDefaultValue()); int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.key, - RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.getValue()); + RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.getDefaultValue()); this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.key, - RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.getValue()); + RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.getDefaultValue()); this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.key, - RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.getValue()); + RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.getDefaultValue()); shuffleWriteClient = ShuffleClientFactory .getInstance() @@ -407,11 +407,11 @@ public ShuffleReader getReaderImpl( } final String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key); final int indexReadLimit = sparkConf.getInt(RssSparkConfig.RSS_INDEX_READ_LIMIT.key, - RssSparkConfig.RSS_INDEX_READ_LIMIT.getValue()); + RssSparkConfig.RSS_INDEX_READ_LIMIT.getDefaultValue()); RssShuffleHandle rssShuffleHandle = (RssShuffleHandle) handle; final int partitionNum = rssShuffleHandle.getDependency().partitioner().numPartitions(); long readBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key, - RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.getValue()); + RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.getDefaultValue()); if (readBufferSize > Integer.MAX_VALUE) { LOG.warn(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key + " can support 2g as max"); readBufferSize = Integer.MAX_VALUE; From b5562d367eb2b95a30e8c74a2191f493e07816c0 Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Fri, 29 Jul 2022 21:44:30 +0800 Subject: [PATCH 10/22] change getDefaultValue to getValue --- .../java/org/apache/spark/shuffle/RssSparkConfig.java | 1 + .../org/apache/spark/shuffle/writer/RssShuffleWriter.java | 8 ++++---- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index 9cfe0453a2..d0ed3e3b47 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -22,6 +22,7 @@ import com.google.common.collect.Sets; import org.apache.spark.internal.config.ConfigBuilder; import org.apache.spark.internal.config.ConfigEntry; + import org.apache.uniffle.client.util.RssClientConfig; public class RssSparkConfig { diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java index 7d3b21f551..432daa444d 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java @@ -107,13 +107,13 @@ public RssShuffleWriter( this.partitioner = shuffleDependency.partitioner(); this.shouldPartition = partitioner.numPartitions() > 1; this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, - RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.getValue()); + RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.getDefaultValue()); this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, - RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.getValue()); + RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.getDefaultValue()); this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key, - RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.getValue()); + RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.getDefaultValue()); this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.key, - RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.getValue()); + RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.getDefaultValue()); this.partitionToBlockIds = Maps.newConcurrentMap(); this.shuffleWriteClient = shuffleWriteClient; this.shuffleServersForData = rssHandle.getShuffleServersForData(); From 407baf3c2c9b0ecb0af174f83df28891ee118c2f Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Sun, 31 Jul 2022 01:38:35 +0800 Subject: [PATCH 11/22] use ConfigBuilder --- .../apache/spark/shuffle/RssSparkConfig.java | 509 +++++++++--------- .../spark/shuffle/RssSparkShuffleUtils.java | 28 +- .../shuffle/writer/BufferManagerOptions.java | 42 +- .../shuffle/RssSparkShuffleUtilsTest.java | 44 +- .../writer/WriteBufferManagerTest.java | 10 +- .../shuffle/DelegationRssShuffleManager.java | 12 +- .../spark/shuffle/RssShuffleManager.java | 90 ++-- .../shuffle/writer/RssShuffleWriter.java | 18 +- .../DelegationRssShuffleManagerTest.java | 24 +- .../shuffle/writer/RssShuffleWriterTest.java | 32 +- .../shuffle/DelegationRssShuffleManager.java | 14 +- .../spark/shuffle/RssShuffleManager.java | 126 ++--- .../shuffle/writer/RssShuffleWriter.java | 18 +- .../DelegationRssShuffleManagerTest.java | 24 +- .../shuffle/writer/RssShuffleWriterTest.java | 32 +- .../uniffle/client/util/RssClientConfig.java | 6 +- .../apache/uniffle/test/AutoAccessTest.java | 8 +- .../test/DynamicFetchClientConfTest.java | 16 +- ...epartitionWithHdfsMultiStorageRssTest.java | 2 +- .../test/RepartitionWithLocalFileRssTest.java | 2 +- ...artitionWithMemoryMultiStorageRssTest.java | 2 +- .../test/RepartitionWithMemoryRssTest.java | 2 +- .../apache/uniffle/test/SimpleTestBase.java | 2 +- .../test/SparkIntegrationTestBase.java | 24 +- .../SparkSQLWithDelegationShuffleManager.java | 4 +- ...LWithDelegationShuffleManagerFallback.java | 4 +- .../test/SparkSQLWithMemoryLocalTest.java | 2 +- .../apache/uniffle/test/GetReaderTest.java | 2 +- .../uniffle/test/AQERepartitionTest.java | 2 +- .../uniffle/test/AQESkewedJoinTest.java | 6 +- .../apache/uniffle/test/GetReaderTest.java | 2 +- 31 files changed, 548 insertions(+), 561 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index d0ed3e3b47..6990463402 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -22,280 +22,267 @@ import com.google.common.collect.Sets; import org.apache.spark.internal.config.ConfigBuilder; import org.apache.spark.internal.config.ConfigEntry; +import org.apache.spark.internal.config.TypedConfigBuilder; +import scala.Serializable; +import scala.runtime.AbstractFunction1; + +import org.apache.uniffle.common.config.ConfigUtils; import org.apache.uniffle.client.util.RssClientConfig; public class RssSparkConfig { - public static final class SparkConfigBuilder { - public static ConfigBuilder configBuilder; + public static final ConfigEntry RSS_PARTITION_NUM_PER_RANGE = createIntegerBuilder( + new ConfigBuilder("spark.rss.partitionNum.per.range") + .doc("xxxxxx")) + .createWithDefault(10); + + public static final ConfigEntry RSS_WRITER_BUFFER_SIZE = createStringBuilder( + new ConfigBuilder("spark.rss.writer.buffer.size") + .doc("controls the buffer flushing size during shuffle write")) + .createWithDefault("3m"); + + public static final ConfigEntry RSS_WRITER_SERIALIZER_BUFFER_SIZE = createStringBuilder( + new ConfigBuilder("spark.rss.writer.serializer.buffer.size") + .doc("")) + .createWithDefault("3k"); + + public static final ConfigEntry RSS_WRITER_BUFFER_SEGMENT_SIZE = createStringBuilder( + new ConfigBuilder("spark.rss.writer.buffer.segment.size") + .doc("")) + .createWithDefault("3k"); + + public static final ConfigEntry RSS_WRITER_BUFFER_SPILL_SIZE = createStringBuilder( + new ConfigBuilder("spark.rss.writer.buffer.spill.size") + .doc("")) + .createWithDefault("128m"); + + public static final ConfigEntry RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE = createStringBuilder( + new ConfigBuilder("spark.rss.writer.pre.allocated.buffer.size") + .doc("")) + .createWithDefault("16m"); + + public static final ConfigEntry RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX = createIntegerBuilder( + new ConfigBuilder("spark.rss.writer.require.memory.retryMax") + .doc("")) + .createWithDefault(1200); + + public static final ConfigEntry RSS_WRITER_REQUIRE_MEMORY_INTERVAL = createLongBuilder( + new ConfigBuilder("spark.rss.writer.require.memory.interval") + .doc("")) + .createWithDefault(1000L); + + public static final ConfigEntry RSS_CLIENT_SEND_CHECK_TIMEOUT_MS = createLongBuilder( + new ConfigBuilder("spark.rss.client.send.check.timeout.ms") + .doc("")) + .createWithDefault(RssClientConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE); + + public static final ConfigEntry RSS_CLIENT_SEND_CHECK_INTERVAL_MS = createLongBuilder( + new ConfigBuilder("spark.rss.client.send.check.interval.ms") + .doc("")) + .createWithDefault(RssClientConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE); + + public static final ConfigEntry RSS_TEST_FLAG = createStringBuilder( + new ConfigBuilder("spark.rss.test") + .doc("")) + .createWithDefault(""); + + public static final ConfigEntry RSS_REMOTE_STORAGE_PATH = createStringBuilder( + new ConfigBuilder("spark.rss.remote.storage.path") + .doc("")) + .createWithDefault(""); + + public static final ConfigEntry RSS_INDEX_READ_LIMIT = createIntegerBuilder( + new ConfigBuilder("spark.rss.index.read.limit") + .doc("")) + .createWithDefault(500); + + public static final ConfigEntry RSS_CLIENT_TYPE = createStringBuilder( + new ConfigBuilder("spark.rss.client.type") + .doc("")) + .createWithDefault(RssClientConfig.RSS_CLIENT_TYPE_DEFAULT_VALUE); + + public static final ConfigEntry RSS_STORAGE_TYPE = createStringBuilder( + new ConfigBuilder("spark.rss.storage.type") + .doc("")) + .createWithDefault(""); + + public static final ConfigEntry RSS_CLIENT_RETRY_MAX = createIntegerBuilder( + new ConfigBuilder("spark.rss.client.retry.max") + .doc("")) + .createWithDefault(RssClientConfig.RSS_CLIENT_RETRY_MAX_DEFAULT_VALUE); + + public static final ConfigEntry RSS_CLIENT_RETRY_INTERVAL_MAX = createLongBuilder( + new ConfigBuilder("spark.rss.client.retry.interval.max") + .doc("")) + .createWithDefault(RssClientConfig.RSS_CLIENT_RETRY_INTERVAL_MAX_DEFAULT_VALUE); + + public static final ConfigEntry RSS_CLIENT_HEARTBEAT_THREAD_NUM = createIntegerBuilder( + new ConfigBuilder("spark.rss.client.heartBeat.threadNum") + .doc("")) + .createWithDefault(4); + + public static final ConfigEntry RSS_CLIENT_SEND_SIZE_LIMIT = createStringBuilder( + new ConfigBuilder("spark.rss.client.send.size.limit") + .doc("")) + .createWithDefault("16m"); + + public static final ConfigEntry RSS_CLIENT_READ_BUFFER_SIZE = createStringBuilder( + new ConfigBuilder("spark.rss.client.read.buffer.size") + .doc("")) + .createWithDefault(RssClientConfig.RSS_CLIENT_READ_BUFFER_SIZE_DEFAULT_VALUE); + + public static final ConfigEntry RSS_HEARTBEAT_INTERVAL = createLongBuilder( + new ConfigBuilder("spark.rss.heartbeat.interval") + .doc("")) + .createWithDefault(RssClientConfig.RSS_HEARTBEAT_INTERVAL_DEFAULT_VALUE); + + public static final ConfigEntry RSS_HEARTBEAT_TIMEOUT = createLongBuilder( + new ConfigBuilder("spark.rss.heartbeat.timeout") + .doc("")) + .createWithDefault(5 * 1000L); + + public static final ConfigEntry RSS_CLIENT_SEND_THREAD_POOL_SIZE = createIntegerBuilder( + new ConfigBuilder("spark.rss.client.send.threadPool.size") + .doc("")) + .createWithDefault(10); + + public static final ConfigEntry RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE = createIntegerBuilder( + new ConfigBuilder("spark.rss.client.send.threadPool.keepalive") + .doc("")) + .createWithDefault(60); + + public static final ConfigEntry RSS_DATA_REPLICA = createIntegerBuilder( + new ConfigBuilder("spark.rss.data.replica") + .doc("")) + .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_DEFAULT_VALUE); + + public static final ConfigEntry RSS_DATA_REPLICA_WRITE = createIntegerBuilder( + new ConfigBuilder("spark.rss.data.replica.write") + .doc("")) + .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_WRITE_DEFAULT_VALUE); + + public static final ConfigEntry RSS_DATA_REPLICA_READ = createIntegerBuilder( + new ConfigBuilder("spark.rss.data.replica.read") + .doc("")) + .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_READ_DEFAULT_VALUE); + + public static final ConfigEntry RSS_DATA_REPLICA_SKIP_ENABLED = createBooleanBuilder( + new ConfigBuilder("spark.rss.data.replica.skip.enabled") + .doc("")) + .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_SKIP_ENABLED_DEFAULT_VALUE); + + public static final ConfigEntry RSS_DATA_TRANSFER_POOL_SIZE = createIntegerBuilder( + new ConfigBuilder("spark.rss.client.data.transfer.pool.size") + .doc("")) + .createWithDefault(RssClientConfig.RSS_DATA_TRANFER_POOL_SIZE_DEFAULT_VALUE); + + public static final ConfigEntry RSS_DATA_COMMIT_POOL_SIZE = createIntegerBuilder( + new ConfigBuilder("spark.rss.client.data.commit.pool.size") + .doc("")) + .createWithDefault(RssClientConfig.RSS_DATA_COMMIT_POOL_SIZE_DEFAULT_VALUE); + + public static final ConfigEntry RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE = createBooleanBuilder( + new ConfigBuilder("spark.rss.ozone.dfs.namenode.odfs.enable") + .doc("")) + .createWithDefault(false); + + public static final ConfigEntry RSS_OZONE_FS_HDFS_IMPL = createStringBuilder( + new ConfigBuilder("spark.rss.ozone.fs.hdfs.impl") + .doc("")) + .createWithDefault("org.apache.hadoop.odfs.HdfsOdfsFilesystem"); + + public static final ConfigEntry RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL = createStringBuilder( + new ConfigBuilder("spark.rss.ozone.fs.AbstractFileSystem.hdfs.impl") + .doc("")) + .createWithDefault("org.apache.hadoop.odfs.HdfsOdfs"); + + public static final ConfigEntry RSS_CLIENT_BITMAP_SPLIT_NUM = createIntegerBuilder( + new ConfigBuilder("spark.rss.client.bitmap.splitNum") + .doc("")) + .createWithDefault(1); + + public static final ConfigEntry RSS_ACCESS_ID = createStringBuilder( + new ConfigBuilder("spark.rss.access.id") + .doc("")) + .createWithDefault(""); + + public static final ConfigEntry RSS_ACCESS_TIMEOUT_MS = createIntegerBuilder( + new ConfigBuilder("spark.rss.access.timeout.ms") + .doc("")) + .createWithDefault(RssClientConfig.RSS_ACCESS_TIMEOUT_MS_DEFAULT_VALUE); + + public static final ConfigEntry RSS_ENABLED = createBooleanBuilder( + new ConfigBuilder("spark.rss.enabled") + .doc("")) + .createWithDefault(false); + + public static final ConfigEntry RSS_DYNAMIC_CLIENT_CONF_ENABLED = createBooleanBuilder( + new ConfigBuilder("spark.rss.dynamicClientConf.enabled") + .doc("")) + .createWithDefault(RssClientConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED_DEFAULT_VALUE); + + public static final ConfigEntry RSS_CLIENT_ASSIGNMENT_TAGS = createStringBuilder( + new ConfigBuilder("spark.rss.client.assignment.tags") + .doc("")) + .createWithDefault(""); + + public static final ConfigEntry RSS_COORDINATOR_QUORUM = createStringBuilder( + new ConfigBuilder("spark.rss.coordinator.quorum") + .doc("")) + .createWithDefault(""); - public static ConfigBuilder key(String key) { - configBuilder = new ConfigBuilder(key); - return configBuilder; - } - } + public static final Set RSS_MANDATORY_CLUSTER_CONF = + Sets.newHashSet(RSS_STORAGE_TYPE.key(), RSS_REMOTE_STORAGE_PATH.key()); - public static class RssConfigEntry { - public ConfigEntry entry; - public String key; + public static final String SPARK_RSS_CONFIG_PREFIX = "spark."; - public RssConfigEntry(ConfigEntry entry) { - this.entry = entry; - this.key = entry.key(); - } + public static final boolean RSS_USE_RSS_SHUFFLE_MANAGER_DEFAULT_VALUE = false; - public T getDefaultValue() { - return (T) entry.defaultValue().get(); - } + public static TypedConfigBuilder createIntegerBuilder(ConfigBuilder builder) { + scala.Function1 f = new SerializableFunction1() { + @Override + public Integer apply(String in) { + return ConfigUtils.convertValue(in, Integer.class); + } + }; + return new TypedConfigBuilder<>(builder, f); } - public static final RssConfigEntry RSS_PARTITION_NUM_PER_RANGE = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.partitionNum.per.range") - .intConf() - .createWithDefault(RssClientConfig.RSS_PARTITION_NUM_PER_RANGE_DEFAULT_VALUE)); - - public static final RssConfigEntry RSS_WRITER_BUFFER_SIZE = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.writer.buffer.size") - .doc("controls the buffer flushing size during shuffle write") - .stringConf() - .createWithDefault("3m")); - - public static final RssConfigEntry RSS_WRITER_SERIALIZER_BUFFER_SIZE = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.writer.serializer.buffer.size") - .stringConf() - .createWithDefault("3k")); - - public static final RssConfigEntry RSS_WRITER_BUFFER_SEGMENT_SIZE = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.writer.buffer.segment.size") - .stringConf() - .createWithDefault("3k")); - - public static final RssConfigEntry RSS_WRITER_BUFFER_SPILL_SIZE = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.writer.buffer.spill.size") - .stringConf() - .createWithDefault("128m")); - - public static final RssConfigEntry RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.writer.pre.allocated.buffer.size") - .stringConf() - .createWithDefault("16m")); - - public static final RssConfigEntry RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.writer.require.memory.retryMax") - .intConf() - .createWithDefault(1200)); - - public static final RssConfigEntry RSS_WRITER_REQUIRE_MEMORY_INTERVAL = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.writer.require.memory.interval") - .longConf() - .createWithDefault(1000)); - - public static final RssConfigEntry RSS_CLIENT_SEND_CHECK_TIMEOUT_MS = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.client.send.check.timeout.ms") - .longConf() - .createWithDefault(RssClientConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE)); - - public static final RssConfigEntry RSS_CLIENT_SEND_CHECK_INTERVAL_MS = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.client.send.check.interval.ms") - .longConf() - .createWithDefault(RssClientConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE)); - - public static final RssConfigEntry RSS_TEST_FLAG = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.test") - .stringConf() - .createWithDefault("")); - - public static final RssConfigEntry RSS_REMOTE_STORAGE_PATH = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.remote.storage.path") - .stringConf() - .createWithDefault("")); - - public static final RssConfigEntry RSS_INDEX_READ_LIMIT = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.index.read.limit") - .intConf() - .createWithDefault(500)); - - public static final RssConfigEntry RSS_CLIENT_TYPE = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.client.type") - .stringConf() - .createWithDefault(RssClientConfig.RSS_CLIENT_TYPE_DEFAULT_VALUE)); - - public static final RssConfigEntry RSS_STORAGE_TYPE = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.storage.type") - .stringConf() - .createWithDefault("")); - - public static final RssConfigEntry RSS_CLIENT_RETRY_MAX = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.client.retry.max") - .intConf() - .createWithDefault(RssClientConfig.RSS_CLIENT_RETRY_MAX_DEFAULT_VALUE)); - - public static final RssConfigEntry RSS_CLIENT_RETRY_INTERVAL_MAX = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.client.retry.interval.max") - .longConf() - .createWithDefault(RssClientConfig.RSS_CLIENT_RETRY_INTERVAL_MAX_DEFAULT_VALUE)); - - public static final RssConfigEntry RSS_CLIENT_HEARTBEAT_THREAD_NUM = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.client.heartBeat.threadNum") - .intConf() - .createWithDefault(4)); - - public static final RssConfigEntry RSS_CLIENT_SEND_SIZE_LIMIT = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.client.send.size.limit") - .stringConf() - .createWithDefault("16m")); - - public static final RssConfigEntry RSS_CLIENT_READ_BUFFER_SIZE = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.client.read.buffer.size") - .stringConf() - .createWithDefault(RssClientConfig.RSS_CLIENT_READ_BUFFER_SIZE_DEFAULT_VALUE)); - - public static final RssConfigEntry RSS_HEARTBEAT_INTERVAL = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.heartbeat.interval") - .longConf() - .createWithDefault(RssClientConfig.RSS_HEARTBEAT_INTERVAL_DEFAULT_VALUE)); - - public static final RssConfigEntry RSS_HEARTBEAT_TIMEOUT = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.heartbeat.timeout") - .longConf() - .createWithDefault(5 * 1000L)); - - public static final RssConfigEntry RSS_CLIENT_SEND_THREAD_POOL_SIZE = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.client.send.threadPool.size") - .intConf() - .createWithDefault(10)); - - public static final RssConfigEntry RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.client.send.threadPool.keepalive") - .intConf() - .createWithDefault(60)); - - public static final RssConfigEntry RSS_DATA_REPLICA = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.data.replica") - .intConf() - .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_DEFAULT_VALUE)); - - public static final RssConfigEntry RSS_DATA_REPLICA_WRITE = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.data.replica.write") - .intConf() - .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_WRITE_DEFAULT_VALUE)); - - public static final RssConfigEntry RSS_DATA_REPLICA_READ = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.data.replica.read") - .intConf() - .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_READ_DEFAULT_VALUE)); - - public static final RssConfigEntry RSS_DATA_REPLICA_SKIP_ENABLED = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.data.replica.skip.enabled") - .booleanConf() - .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_SKIP_ENABLED_DEFAULT_VALUE)); - - public static final RssConfigEntry RSS_DATA_TRANSFER_POOL_SIZE = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.client.data.transfer.pool.size") - .intConf() - .createWithDefault(RssClientConfig.RSS_DATA_TRANFER_POOL_SIZE_DEFAULT_VALUE)); - - public static final RssConfigEntry RSS_DATA_COMMIT_POOL_SIZE = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.client.data.commit.pool.size") - .intConf() - .createWithDefault(RssClientConfig.RSS_DATA_COMMIT_POOL_SIZE_DEFAULT_VALUE)); - - public static final RssConfigEntry RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.ozone.dfs.namenode.odfs.enable") - .booleanConf() - .createWithDefault(false)); - - public static final RssConfigEntry RSS_OZONE_FS_HDFS_IMPL = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.ozone.fs.hdfs.impl") - .stringConf() - .createWithDefault("org.apache.hadoop.odfs.HdfsOdfsFilesystem")); - - public static final RssConfigEntry RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.ozone.fs.AbstractFileSystem.hdfs.impl") - .stringConf() - .createWithDefault("org.apache.hadoop.odfs.HdfsOdfs")); - - public static final RssConfigEntry RSS_CLIENT_BITMAP_SPLIT_NUM = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.client.bitmap.splitNum") - .intConf() - .createWithDefault(1)); - - public static final RssConfigEntry RSS_ACCESS_ID = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.access.id") - .stringConf() - .createWithDefault("")); - - public static final RssConfigEntry RSS_ACCESS_TIMEOUT_MS = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.access.timeout.ms") - .intConf() - .createWithDefault(RssClientConfig.RSS_ACCESS_TIMEOUT_MS_DEFAULT_VALUE)); - - public static final RssConfigEntry RSS_ENABLED = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.enabled") - .booleanConf() - .createWithDefault(false)); - - public static final RssConfigEntry RSS_DYNAMIC_CLIENT_CONF_ENABLED = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.dynamicClientConf.enabled") - .booleanConf() - .createWithDefault(RssClientConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED_DEFAULT_VALUE)); - - public static final RssConfigEntry RSS_CLIENT_ASSIGNMENT_TAGS = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.client.assignment.tags") - .stringConf() - .createWithDefault("")); - - public static final RssConfigEntry RSS_COORDINATOR_QUORUM = new RssConfigEntry( - SparkConfigBuilder - .key("spark.rss.coordinator.quorum") - .stringConf() - .createWithDefault("")); + public static TypedConfigBuilder createLongBuilder(ConfigBuilder builder) { + scala.Function1 f = new SerializableFunction1() { + @Override + public Long apply(String in) { + return ConfigUtils.convertValue(in, Long.class); + } + }; + return new TypedConfigBuilder<>(builder, f); + } - public static final Set RSS_MANDATORY_CLUSTER_CONF = - Sets.newHashSet(RSS_STORAGE_TYPE.key, RSS_REMOTE_STORAGE_PATH.key); + public static TypedConfigBuilder createBooleanBuilder(ConfigBuilder builder) { + scala.Function1 f = new SerializableFunction1() { + @Override + public Boolean apply(String in) { + return ConfigUtils.convertValue(in, Boolean.class); + } + }; + return new TypedConfigBuilder<>(builder, f); + } - public static final String SPARK_RSS_CONFIG_PREFIX = "spark."; + public static TypedConfigBuilder createDoubleBuilder(ConfigBuilder builder) { + scala.Function1 f = new SerializableFunction1() { + @Override + public Double apply(String in) { + return ConfigUtils.convertValue(in, Double.class); + } + }; + return new TypedConfigBuilder<>(builder, f); + } - public static final boolean RSS_USE_RSS_SHUFFLE_MANAGER_DEFAULT_VALUE = false; + public static TypedConfigBuilder createStringBuilder(ConfigBuilder builder) { + return builder.stringConf(); + } +} +abstract class SerializableFunction1 extends AbstractFunction1 implements Serializable { } diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java index 6e1e94e34c..2763c66853 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java @@ -44,20 +44,20 @@ 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.key, - RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE.getDefaultValue()); + boolean useOdfs = sparkConf.getBoolean(RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE.key(), + RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE.defaultValue().get()); if (useOdfs) { final int OZONE_PREFIX_LEN = "spark.rss.ozone.".length(); - conf.setBoolean(RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE.key.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.key.substring(OZONE_PREFIX_LEN), - sparkConf.get(RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.key, - RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.getDefaultValue())); + RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.key().substring(OZONE_PREFIX_LEN), + sparkConf.get(RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.key(), + RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.defaultValue().get())); conf.set( - RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.key.substring(OZONE_PREFIX_LEN), + 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.key, - RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.getDefaultValue())); + RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.key(), + RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.defaultValue().get())); } return conf; @@ -78,9 +78,9 @@ public static ShuffleManager loadShuffleManager(String name, SparkConf conf, boo } public static List createCoordinatorClients(SparkConf sparkConf) throws RuntimeException { - String clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key, - RssSparkConfig.RSS_CLIENT_TYPE.getDefaultValue()); - String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM.key); + String clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key(), + RssSparkConfig.RSS_CLIENT_TYPE.defaultValue().get()); + String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM.key()); CoordinatorClientFactory coordinatorClientFactory = new CoordinatorClientFactory(clientType); return coordinatorClientFactory.createCoordinatorClient(coordinators); } @@ -111,7 +111,7 @@ public static void applyDynamicClientConf(SparkConf sparkConf, Map getAssignmentTags(SparkConf sparkConf) { Set assignmentTags = new HashSet<>(); - String rawTags = sparkConf.get(RssSparkConfig.RSS_CLIENT_ASSIGNMENT_TAGS.key, ""); + String rawTags = sparkConf.get(RssSparkConfig.RSS_CLIENT_ASSIGNMENT_TAGS.key(), ""); if (StringUtils.isNotEmpty(rawTags)) { rawTags = rawTags.trim(); assignmentTags.addAll(Arrays.asList(rawTags.split(","))); diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java index 23c319e4b2..feab47a085 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java @@ -35,38 +35,38 @@ public class BufferManagerOptions { private int requireMemoryRetryMax; public BufferManagerOptions(SparkConf sparkConf) { - bufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key, - RssSparkConfig.RSS_WRITER_BUFFER_SIZE.getDefaultValue()); - serializerBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key, - RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.getDefaultValue()); - bufferSegmentSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key, - RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.getDefaultValue()); - bufferSpillThreshold = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key, - RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.getDefaultValue()); - preAllocatedBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE.key, - RssSparkConfig.RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE.getDefaultValue()); - requireMemoryInterval = sparkConf.getLong(RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_INTERVAL.key, - RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_INTERVAL.getDefaultValue()); - requireMemoryRetryMax = sparkConf.getInt(RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX.key, - RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX.getDefaultValue()); - 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); + 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.getLong(RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_INTERVAL.key(), + RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_INTERVAL.defaultValue().get()); + requireMemoryRetryMax = sparkConf.getInt(RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX.key(), + RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX.defaultValue().get()); + 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.key + 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.key + 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.key + "[" + bufferSegmentSize + "] should be less than " - + RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key + "[" + bufferSize + "]"); + LOG.warn(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key() + "[" + bufferSegmentSize + "] should be less than " + + RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key() + "[" + bufferSize + "]"); } } diff --git a/client-spark/common/src/test/java/org/apache/spark/shuffle/RssSparkShuffleUtilsTest.java b/client-spark/common/src/test/java/org/apache/spark/shuffle/RssSparkShuffleUtilsTest.java index 8a5f6591b1..cdd30ff687 100644 --- a/client-spark/common/src/test/java/org/apache/spark/shuffle/RssSparkShuffleUtilsTest.java +++ b/client-spark/common/src/test/java/org/apache/spark/shuffle/RssSparkShuffleUtilsTest.java @@ -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.key, " a,b"); + conf.set(RssSparkConfig.RSS_CLIENT_ASSIGNMENT_TAGS.key(), " a,b"); tags = RssSparkShuffleUtils.getAssignmentTags(conf); assertEquals(3, tags.size()); Iterator iterator = tags.iterator(); @@ -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.key, "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.key, "expect_odfs_impl"); - conf.set(RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.key, "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")); @@ -116,47 +116,47 @@ public void applyDynamicClientConfTest() { clientConf.put(mockKey, mockValue); RssSparkShuffleUtils.applyDynamicClientConf(conf, clientConf); - assertEquals(remoteStoragePath, conf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); + assertEquals(remoteStoragePath, conf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key())); assertEquals(RssClientConfig.RSS_CLIENT_TYPE_DEFAULT_VALUE, - conf.get(RssSparkConfig.RSS_CLIENT_TYPE.key)); + conf.get(RssSparkConfig.RSS_CLIENT_TYPE.key())); assertEquals(Integer.toString(RssClientConfig.RSS_CLIENT_RETRY_MAX_DEFAULT_VALUE), - conf.get(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key)); + 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.key)); + conf.get(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key())); assertEquals(Integer.toString(RssClientConfig.RSS_DATA_REPLICA_DEFAULT_VALUE), - conf.get(RssSparkConfig.RSS_DATA_REPLICA.key)); + conf.get(RssSparkConfig.RSS_DATA_REPLICA.key())); assertEquals(Integer.toString(RssClientConfig.RSS_DATA_REPLICA_WRITE_DEFAULT_VALUE), - conf.get(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key)); + 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.key)); + conf.get(RssSparkConfig.RSS_DATA_REPLICA_READ.key())); assertEquals(Long.toString(RssClientConfig.RSS_HEARTBEAT_INTERVAL_DEFAULT_VALUE), - conf.get(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key)); - assertEquals(StorageType.MEMORY_LOCALFILE_HDFS.name(), conf.get(RssSparkConfig.RSS_STORAGE_TYPE.key)); + 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.key)); + 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.key)); + 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.key)); + 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.key)); + 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.key)); + 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.key, 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.key)); - assertEquals(StorageType.MEMORY_HDFS.name(), conf.get(RssSparkConfig.RSS_STORAGE_TYPE.key)); + 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.key)); + conf.get(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key())); } } diff --git a/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/WriteBufferManagerTest.java b/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/WriteBufferManagerTest.java index 0fc55f52bf..eeb88b937a 100644 --- a/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/WriteBufferManagerTest.java +++ b/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/WriteBufferManagerTest.java @@ -53,11 +53,11 @@ private WriteBufferManager createManager(SparkConf conf) { private SparkConf getConf() { SparkConf conf = new SparkConf(false); - 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"); + 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; } diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java index 0b373c4d14..0b58fb086a 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java @@ -47,8 +47,8 @@ public class DelegationRssShuffleManager implements ShuffleManager { public DelegationRssShuffleManager(SparkConf sparkConf, boolean isDriver) throws Exception { this.sparkConf = sparkConf; accessTimeoutMs = sparkConf.getInt( - RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key, - RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getDefaultValue()); + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key(), + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.defaultValue().get()); if (isDriver) { coordinatorClients = RssSparkShuffleUtils.createCoordinatorClients(sparkConf); delegate = createShuffleManagerInDriver(); @@ -69,7 +69,7 @@ private ShuffleManager createShuffleManagerInDriver() throws RssException { if (canAccess) { try { shuffleManager = new RssShuffleManager(sparkConf, true); - sparkConf.set(RssSparkConfig.RSS_ENABLED.key, "true"); + sparkConf.set(RssSparkConfig.RSS_ENABLED.key(), "true"); sparkConf.set("spark.shuffle.manager", RssShuffleManager.class.getCanonicalName()); LOG.info("Use RssShuffleManager"); return shuffleManager; @@ -80,7 +80,7 @@ private ShuffleManager createShuffleManagerInDriver() throws RssException { try { shuffleManager = RssSparkShuffleUtils.loadShuffleManager(Constants.SORT_SHUFFLE_MANAGER_NAME, sparkConf, true); - sparkConf.set(RssSparkConfig.RSS_ENABLED.key, "false"); + sparkConf.set(RssSparkConfig.RSS_ENABLED.key(), "false"); sparkConf.set("spark.shuffle.manager", "sort"); LOG.info("Use SortShuffleManager"); } catch (Exception e) { @@ -92,7 +92,7 @@ private ShuffleManager createShuffleManagerInDriver() throws RssException { private boolean tryAccessCluster() { String accessId = sparkConf.get( - RssSparkConfig.RSS_ACCESS_ID.key, "").trim(); + RssSparkConfig.RSS_ACCESS_ID.key(), "").trim(); if (StringUtils.isEmpty(accessId)) { LOG.warn("Access id key is empty"); return false; @@ -128,7 +128,7 @@ private ShuffleManager createShuffleManagerInExecutor() throws RssException { ShuffleManager shuffleManager; // get useRSS from spark conf boolean useRSS = sparkConf.getBoolean( - RssSparkConfig.RSS_ENABLED.key, + RssSparkConfig.RSS_ENABLED.key(), RssSparkConfig.RSS_USE_RSS_SHUFFLE_MANAGER_DEFAULT_VALUE); if (useRSS) { // Executor will not do any fallback diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java index 15dbe808cb..1a80b3064a 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java @@ -141,36 +141,36 @@ public RssShuffleManager(SparkConf sparkConf, boolean isDriver) { this.sparkConf = sparkConf; // set & check replica config - this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA.key, - RssSparkConfig.RSS_DATA_REPLICA.getDefaultValue()); - this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key, - RssSparkConfig.RSS_DATA_REPLICA_WRITE.getDefaultValue()); - this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ.key, - RssSparkConfig.RSS_DATA_REPLICA_READ.getDefaultValue()); - this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.key, - RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.getDefaultValue()); - this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.key, - RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.getDefaultValue()); + this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA.key(), + RssSparkConfig.RSS_DATA_REPLICA.defaultValue().get()); + this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key(), + RssSparkConfig.RSS_DATA_REPLICA_WRITE.defaultValue().get()); + this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ.key(), + RssSparkConfig.RSS_DATA_REPLICA_READ.defaultValue().get()); + this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.key(), + RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.defaultValue().get()); + this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.key(), + RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.defaultValue().get()); LOG.info("Check quorum config [" + dataReplica + ":" + dataReplicaWrite + ":" + dataReplicaRead + ":" + dataReplicaSkipEnabled + "]"); RssUtils.checkQuorumSetting(dataReplica, dataReplicaWrite, dataReplicaRead); - this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key, - RssSparkConfig.RSS_CLIENT_TYPE.getDefaultValue()); - this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key, - RssSparkConfig.RSS_HEARTBEAT_INTERVAL.getDefaultValue()); - this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT.key, heartbeatInterval / 2); + this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key(), + RssSparkConfig.RSS_CLIENT_TYPE.defaultValue().get()); + this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key(), + RssSparkConfig.RSS_HEARTBEAT_INTERVAL.defaultValue().get()); + this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT.key(), heartbeatInterval / 2); this.dynamicConfEnabled = sparkConf.getBoolean( - RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, - RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.getDefaultValue()); - int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key, - RssSparkConfig.RSS_CLIENT_RETRY_MAX.getDefaultValue()); - long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key, - RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.getDefaultValue()); - int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.key, - RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.getDefaultValue()); - this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.key, - RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.getDefaultValue()); + RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key(), + RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.defaultValue().get()); + int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key(), + RssSparkConfig.RSS_CLIENT_RETRY_MAX.defaultValue().get()); + long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key(), + RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.defaultValue().get()); + int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.key(), + RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.defaultValue().get()); + this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.key(), + RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.defaultValue().get()); this.shuffleWriteClient = ShuffleClientFactory .getInstance() .createShuffleWriteClient(clientType, retryMax, retryIntervalMax, heartBeatThreadNum, @@ -180,22 +180,22 @@ public RssShuffleManager(SparkConf sparkConf, boolean isDriver) { // fetch client conf and apply them if necessary and disable ESS if (isDriver && dynamicConfEnabled) { Map clusterClientConf = shuffleWriteClient.fetchClientConf( - sparkConf.getInt(RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key, - RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getDefaultValue())); + sparkConf.getInt(RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key(), + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.defaultValue().get())); RssSparkShuffleUtils.applyDynamicClientConf(sparkConf, clusterClientConf); } RssSparkShuffleUtils.validateRssClientConf(sparkConf); // External shuffle service is not supported when using remote shuffle service sparkConf.set("spark.shuffle.service.enabled", "false"); LOG.info("Disable external shuffle service in RssShuffleManager."); - if (!sparkConf.getBoolean(RssSparkConfig.RSS_TEST_FLAG.key, false)) { + if (!sparkConf.getBoolean(RssSparkConfig.RSS_TEST_FLAG.key(), false)) { // for non-driver executor, start a thread for sending shuffle data to shuffle server LOG.info("RSS data send thread is starting"); eventLoop.start(); - int poolSize = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.key, - RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.getDefaultValue()); - int keepAliveTime = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.key, - RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.getDefaultValue()); + int poolSize = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.key(), + RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.defaultValue().get()); + int keepAliveTime = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.key(), + RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.defaultValue().get()); threadPoolExecutor = new ThreadPoolExecutor(poolSize, poolSize * 2, keepAliveTime, TimeUnit.SECONDS, Queues.newLinkedBlockingQueue(Integer.MAX_VALUE), ThreadUtils.getThreadFactory("SendData")); @@ -224,13 +224,13 @@ public ShuffleHandle registerShuffle(int shuffleId, int numMaps, Shuff LOG.info("Generate application id used in rss: " + appId); } - String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key); - remoteStorage = new RemoteStorageInfo(sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key, "")); + String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key()); + remoteStorage = new RemoteStorageInfo(sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key(), "")); remoteStorage = ClientUtils.fetchRemoteStorage( appId, remoteStorage, dynamicConfEnabled, storageType, shuffleWriteClient); - int partitionNumPerRange = sparkConf.getInt(RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.key, - RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.getDefaultValue()); + int partitionNumPerRange = sparkConf.getInt(RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.key(), + RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.defaultValue().get()); // get all register info according to coordinator's response Set assignmentTags = RssSparkShuffleUtils.getAssignmentTags(sparkConf); @@ -248,7 +248,7 @@ public ShuffleHandle registerShuffle(int shuffleId, int numMaps, Shuff } private void startHeartbeat() { - if (!sparkConf.getBoolean(RssSparkConfig.RSS_TEST_FLAG.key, false) && !heartbeatStarted) { + if (!sparkConf.getBoolean(RssSparkConfig.RSS_TEST_FLAG.key(), false) && !heartbeatStarted) { heartBeatScheduledExecutorService.scheduleAtFixedRate( () -> { try { @@ -286,7 +286,7 @@ protected void registerShuffleServers( @VisibleForTesting protected void registerCoordinator() { - String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM.key); + String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM.key()); LOG.info("Registering coordinators {}", coordinators); shuffleWriteClient.registerCoordinators(coordinators); } @@ -323,15 +323,15 @@ public ShuffleWriter getWriter(ShuffleHandle handle, int mapId, public ShuffleReader getReader(ShuffleHandle handle, int startPartition, int endPartition, TaskContext context) { if (handle instanceof RssShuffleHandle) { - final String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key); - final int indexReadLimit = sparkConf.getInt(RssSparkConfig.RSS_INDEX_READ_LIMIT.key, - RssSparkConfig.RSS_INDEX_READ_LIMIT.getDefaultValue()); + final String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key()); + final int indexReadLimit = sparkConf.getInt(RssSparkConfig.RSS_INDEX_READ_LIMIT.key(), + RssSparkConfig.RSS_INDEX_READ_LIMIT.defaultValue().get()); RssShuffleHandle rssShuffleHandle = (RssShuffleHandle) handle; - final int partitionNumPerRange = sparkConf.getInt(RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.key, - RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.getDefaultValue()); + final int partitionNumPerRange = sparkConf.getInt(RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.key(), + RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.defaultValue().get()); final int partitionNum = rssShuffleHandle.getDependency().partitioner().numPartitions(); - long readBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key, - RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.getDefaultValue()); + long readBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key(), + RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.defaultValue().get()); if (readBufferSize > Integer.MAX_VALUE) { LOG.warn(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE + " can support 2g as max"); readBufferSize = Integer.MAX_VALUE; diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java index 3a216d7428..51a25452e6 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java @@ -105,20 +105,20 @@ public RssShuffleWriter( this.partitioner = shuffleDependency.partitioner(); this.shuffleManager = shuffleManager; this.shouldPartition = partitioner.numPartitions() > 1; - this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, - RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.getDefaultValue()); - this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, - RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.getDefaultValue()); - this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key, - RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.getDefaultValue()); - this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.key, - RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.getDefaultValue()); + this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key(), + RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.defaultValue().get()); + this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key(), + RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.defaultValue().get()); + this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key(), + RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.defaultValue().get()); + this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.key(), + RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.defaultValue().get()); this.partitionToBlockIds = Maps.newConcurrentMap(); this.shuffleWriteClient = shuffleWriteClient; this.shuffleServersForData = rssHandle.getShuffleServersForData(); this.partitionToServers = rssHandle.getPartitionToServers(); this.isMemoryShuffleEnabled = isMemoryShuffleEnabled( - sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key)); + sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key())); } private boolean isMemoryShuffleEnabled(String storageType) { diff --git a/client-spark/spark2/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java b/client-spark/spark2/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java index c357bcbfa4..6f3d35b700 100644 --- a/client-spark/spark2/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java +++ b/client-spark/spark2/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java @@ -65,7 +65,7 @@ public void testCreateInDriverDenied() throws Exception { mockedStaticRssShuffleUtils.when(() -> RssSparkShuffleUtils.createCoordinatorClients(any())).thenReturn(coordinatorClients); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key(), "false"); assertCreateSortShuffleManager(conf); } @@ -81,15 +81,15 @@ public void testCreateInDriver() throws Exception { SparkConf conf = new SparkConf(); assertCreateSortShuffleManager(conf); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); - conf.set(RssSparkConfig.RSS_ACCESS_ID.key, "mockId"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key(), "false"); + conf.set(RssSparkConfig.RSS_ACCESS_ID.key(), "mockId"); conf.set("spark.rss.storage.type", StorageType.LOCALFILE.name()); assertCreateSortShuffleManager(conf); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key(), "m1:8001,m2:8002"); assertCreateRssShuffleManager(conf); conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key(), "m1:8001,m2:8002"); when(mockCoordinatorClient.accessCluster(any())).thenReturn( new RssAccessClusterResponse(SUCCESS, "")); assertCreateSortShuffleManager(conf); @@ -99,7 +99,7 @@ public void testCreateInDriver() throws Exception { public void testCreateInExecutor() throws Exception { DelegationRssShuffleManager delegationRssShuffleManager; SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key(), "m1:8001,m2:8002"); delegationRssShuffleManager = new DelegationRssShuffleManager(conf, false); assertFalse(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); assertTrue(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); @@ -115,15 +115,15 @@ public void testCreateFallback() throws Exception { RssSparkShuffleUtils.createCoordinatorClients(any())).thenReturn(coordinatorClients); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); - conf.set(RssSparkConfig.RSS_ACCESS_ID.key, "mockId"); - conf.set(RssSparkConfig.RSS_ENABLED.key, "true"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key(), "false"); + conf.set(RssSparkConfig.RSS_ACCESS_ID.key(), "mockId"); + conf.set(RssSparkConfig.RSS_ENABLED.key(), "true"); // fall back to SortShuffleManager in driver assertCreateSortShuffleManager(conf); // No fall back in executor - conf.set(RssSparkConfig.RSS_ENABLED.key, "true"); + conf.set(RssSparkConfig.RSS_ENABLED.key(), "true"); boolean hasException = false; try { new DelegationRssShuffleManager(conf, false); @@ -138,7 +138,7 @@ private DelegationRssShuffleManager assertCreateSortShuffleManager(SparkConf con DelegationRssShuffleManager delegationRssShuffleManager = new DelegationRssShuffleManager(conf, true); assertTrue(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); assertFalse(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); - assertFalse(conf.getBoolean(RssSparkConfig.RSS_ENABLED.key, false)); + assertFalse(conf.getBoolean(RssSparkConfig.RSS_ENABLED.key(), false)); assertEquals("sort", conf.get("spark.shuffle.manager")); return delegationRssShuffleManager; } @@ -147,7 +147,7 @@ private DelegationRssShuffleManager assertCreateRssShuffleManager(SparkConf conf DelegationRssShuffleManager delegationRssShuffleManager = new DelegationRssShuffleManager(conf, true); assertFalse(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); assertTrue(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); - assertTrue(Boolean.parseBoolean(conf.get(RssSparkConfig.RSS_ENABLED.key))); + assertTrue(Boolean.parseBoolean(conf.get(RssSparkConfig.RSS_ENABLED.key()))); assertEquals(RssShuffleManager.class.getCanonicalName(), conf.get("spark.shuffle.manager")); return delegationRssShuffleManager; } diff --git a/client-spark/spark2/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java b/client-spark/spark2/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java index f67171850a..995f14e22c 100644 --- a/client-spark/spark2/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java +++ b/client-spark/spark2/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java @@ -68,11 +68,11 @@ public void checkBlockSendResultTest() { String taskId = "checkBlockSendResultTest_taskId"; conf.setAppName("testApp") .setMaster("local[2]") - .set(RssSparkConfig.RSS_TEST_FLAG.key, "true") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, "10000") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, "1000") - .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()) - .set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "127.0.0.1:12345,127.0.0.1:12346"); + .set(RssSparkConfig.RSS_TEST_FLAG.key(), "true") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key(), "10000") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key(), "1000") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.LOCALFILE.name()) + .set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key(), "127.0.0.1:12345,127.0.0.1:12346"); // init SparkContext SparkContext sc = SparkContext.getOrCreate(conf); RssShuffleManager manager = new RssShuffleManager(conf, false); @@ -131,15 +131,15 @@ public void checkBlockSendResultTest() { public void writeTest() throws Exception { SparkConf conf = new SparkConf(); conf.setAppName("testApp").setMaster("local[2]") - .set(RssSparkConfig.RSS_TEST_FLAG.key, "true") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key, "32") - .set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key, "32") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key, "64") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key, "128") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, "10000") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, "1000") - .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()) - .set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "127.0.0.1:12345,127.0.0.1:12346"); + .set(RssSparkConfig.RSS_TEST_FLAG.key(), "true") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key(), "32") + .set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key(), "32") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key(), "64") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key(), "128") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key(), "10000") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key(), "1000") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.LOCALFILE.name()) + .set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key(), "127.0.0.1:12345,127.0.0.1:12346"); // init SparkContext SparkContext sc = SparkContext.getOrCreate(conf); RssShuffleManager manager = new RssShuffleManager(conf, false); @@ -273,8 +273,8 @@ public void onError(Throwable e) { when(mockHandle.getDependency()).thenReturn(mockDependency); ShuffleWriteClient mockWriteClient = mock(ShuffleWriteClient.class); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key, "64") - .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()); + conf.set(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key(), "64") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.LOCALFILE.name()); RssShuffleWriter writer = new RssShuffleWriter("appId", 0, "taskId", 1L, mockBufferManager, mockMetrics, mockShuffleManager, conf, mockWriteClient, mockHandle); diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java index 6afad97f9f..cf3833ca86 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java @@ -47,8 +47,8 @@ public class DelegationRssShuffleManager implements ShuffleManager { public DelegationRssShuffleManager(SparkConf sparkConf, boolean isDriver) throws Exception { this.sparkConf = sparkConf; accessTimeoutMs = sparkConf.getInt( - RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key, - RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getDefaultValue()); + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key(), + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.defaultValue().get()); if (isDriver) { coordinatorClients = RssSparkShuffleUtils.createCoordinatorClients(sparkConf); delegate = createShuffleManagerInDriver(); @@ -69,7 +69,7 @@ private ShuffleManager createShuffleManagerInDriver() throws RssException { if (canAccess) { try { shuffleManager = new RssShuffleManager(sparkConf, true); - sparkConf.set(RssSparkConfig.RSS_ENABLED.key, "true"); + sparkConf.set(RssSparkConfig.RSS_ENABLED.key(), "true"); sparkConf.set("spark.shuffle.manager", RssShuffleManager.class.getCanonicalName()); LOG.info("Use RssShuffleManager"); return shuffleManager; @@ -80,7 +80,7 @@ private ShuffleManager createShuffleManagerInDriver() throws RssException { try { shuffleManager = RssSparkShuffleUtils.loadShuffleManager(Constants.SORT_SHUFFLE_MANAGER_NAME, sparkConf, true); - sparkConf.set(RssSparkConfig.RSS_ENABLED.key, "false"); + sparkConf.set(RssSparkConfig.RSS_ENABLED.key(), "false"); sparkConf.set("spark.shuffle.manager", "sort"); LOG.info("Use SortShuffleManager"); } catch (Exception e) { @@ -92,7 +92,7 @@ private ShuffleManager createShuffleManagerInDriver() throws RssException { private boolean tryAccessCluster() { String accessId = sparkConf.get( - RssSparkConfig.RSS_ACCESS_ID.key, "").trim(); + RssSparkConfig.RSS_ACCESS_ID.key(), "").trim(); if (StringUtils.isEmpty(accessId)) { LOG.warn("Access id key is empty"); return false; @@ -128,8 +128,8 @@ private ShuffleManager createShuffleManagerInExecutor() throws RssException { ShuffleManager shuffleManager; // get useRSS from spark conf boolean useRSS = sparkConf.getBoolean( - RssSparkConfig.RSS_ENABLED.key, - RssSparkConfig.RSS_ENABLED.getDefaultValue()); + RssSparkConfig.RSS_ENABLED.key(), + RssSparkConfig.RSS_ENABLED.defaultValue().get()); if (useRSS) { // Executor will not do any fallback shuffleManager = new RssShuffleManager(sparkConf, false); diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java index 9d76886b94..ad76559595 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java @@ -140,36 +140,36 @@ public RssShuffleManager(SparkConf conf, boolean isDriver) { this.sparkConf = conf; // set & check replica config - this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA.key, - RssSparkConfig.RSS_DATA_REPLICA.getDefaultValue()); - this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key, - RssSparkConfig.RSS_DATA_REPLICA_WRITE.getDefaultValue()); - this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ.key, - RssSparkConfig.RSS_DATA_REPLICA_READ.getDefaultValue()); - this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.key, - RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.getDefaultValue()); + this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA.key(), + RssSparkConfig.RSS_DATA_REPLICA.defaultValue().get()); + this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key(), + RssSparkConfig.RSS_DATA_REPLICA_WRITE.defaultValue().get()); + this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ.key(), + RssSparkConfig.RSS_DATA_REPLICA_READ.defaultValue().get()); + this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.key(), + RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.defaultValue().get()); LOG.info("Check quorum config [" + dataReplica + ":" + dataReplicaWrite + ":" + dataReplicaRead + ":" + dataReplicaSkipEnabled + "]"); RssUtils.checkQuorumSetting(dataReplica, dataReplicaWrite, dataReplicaRead); - this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key, - RssSparkConfig.RSS_HEARTBEAT_INTERVAL.getDefaultValue()); - this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT.key, heartbeatInterval / 2); - final int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key, - RssSparkConfig.RSS_CLIENT_RETRY_MAX.getDefaultValue()); - this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key, - RssSparkConfig.RSS_CLIENT_TYPE.getDefaultValue()); + this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key(), + RssSparkConfig.RSS_HEARTBEAT_INTERVAL.defaultValue().get()); + this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT.key(), heartbeatInterval / 2); + final int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key(), + RssSparkConfig.RSS_CLIENT_RETRY_MAX.defaultValue().get()); + this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key(), + RssSparkConfig.RSS_CLIENT_TYPE.defaultValue().get()); this.dynamicConfEnabled = sparkConf.getBoolean( - RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, - RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.getDefaultValue()); - long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key, - RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.getDefaultValue()); - int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.key, - RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.getDefaultValue()); - this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.key, - RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.getDefaultValue()); - this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.key, - RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.getDefaultValue()); + RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key(), + RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.defaultValue().get()); + long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key(), + RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.defaultValue().get()); + int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.key(), + RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.defaultValue().get()); + this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.key(), + RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.defaultValue().get()); + this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.key(), + RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.defaultValue().get()); shuffleWriteClient = ShuffleClientFactory .getInstance() @@ -180,8 +180,8 @@ public RssShuffleManager(SparkConf conf, boolean isDriver) { // fetch client conf and apply them if necessary and disable ESS if (isDriver && dynamicConfEnabled) { Map clusterClientConf = shuffleWriteClient.fetchClientConf( - sparkConf.getInt(RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key, - RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.getDefaultValue())); + sparkConf.getInt(RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key(), + RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.defaultValue().get())); RssSparkShuffleUtils.applyDynamicClientConf(sparkConf, clusterClientConf); } RssSparkShuffleUtils.validateRssClientConf(sparkConf); @@ -194,10 +194,10 @@ public RssShuffleManager(SparkConf conf, boolean isDriver) { LOG.info("RSS data send thread is starting"); eventLoop = defaultEventLoop; eventLoop.start(); - int poolSize = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.key, - RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.getDefaultValue()); - int keepAliveTime = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.key, - RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.getDefaultValue()); + int poolSize = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.key(), + RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.defaultValue().get()); + int keepAliveTime = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.key(), + RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.defaultValue().get()); threadPoolExecutor = new ThreadPoolExecutor(poolSize, poolSize * 2, keepAliveTime, TimeUnit.SECONDS, Queues.newLinkedBlockingQueue(Integer.MAX_VALUE)); if (isDriver) { @@ -215,33 +215,33 @@ public RssShuffleManager(SparkConf conf, boolean isDriver) { Map> taskToSuccessBlockIds, Map> taskToFailedBlockIds) { this.sparkConf = conf; - this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key, - RssSparkConfig.RSS_CLIENT_TYPE.getDefaultValue()); - this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key, - RssSparkConfig.RSS_HEARTBEAT_INTERVAL.getDefaultValue()); - this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT.key, heartbeatInterval / 2); - this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA.key, - RssSparkConfig.RSS_DATA_REPLICA.getDefaultValue()); - this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key, - RssSparkConfig.RSS_DATA_REPLICA_WRITE.getDefaultValue()); - this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ.key, - RssSparkConfig.RSS_DATA_REPLICA_READ.getDefaultValue()); - this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.key, - RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.getDefaultValue()); + this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key(), + RssSparkConfig.RSS_CLIENT_TYPE.defaultValue().get()); + this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key(), + RssSparkConfig.RSS_HEARTBEAT_INTERVAL.defaultValue().get()); + this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT.key(), heartbeatInterval / 2); + this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA.key(), + RssSparkConfig.RSS_DATA_REPLICA.defaultValue().get()); + this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key(), + RssSparkConfig.RSS_DATA_REPLICA_WRITE.defaultValue().get()); + this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ.key(), + RssSparkConfig.RSS_DATA_REPLICA_READ.defaultValue().get()); + this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.key(), + RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.defaultValue().get()); LOG.info("Check quorum config [" + dataReplica + ":" + dataReplicaWrite + ":" + dataReplicaRead + ":" + dataReplicaSkipEnabled + "]"); RssUtils.checkQuorumSetting(dataReplica, dataReplicaWrite, dataReplicaRead); - int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key, - RssSparkConfig.RSS_CLIENT_RETRY_MAX.getDefaultValue()); - long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key, - RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.getDefaultValue()); - int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.key, - RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.getDefaultValue()); - this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.key, - RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.getDefaultValue()); - this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.key, - RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.getDefaultValue()); + int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key(), + RssSparkConfig.RSS_CLIENT_RETRY_MAX.defaultValue().get()); + long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key(), + RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.defaultValue().get()); + int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.key(), + RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.defaultValue().get()); + this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.key(), + RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.defaultValue().get()); + this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.key(), + RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.defaultValue().get()); shuffleWriteClient = ShuffleClientFactory .getInstance() @@ -274,9 +274,9 @@ public ShuffleHandle registerShuffle(int shuffleId, ShuffleDependency< } LOG.info("Generate application id used in rss: " + id.get()); - String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key); + String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key()); remoteStorage = new RemoteStorageInfo( - sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key, "")); + sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key(), "")); remoteStorage = ClientUtils.fetchRemoteStorage( id.get(), remoteStorage, dynamicConfEnabled, storageType, shuffleWriteClient); @@ -405,15 +405,15 @@ public ShuffleReader getReaderImpl( if (!(handle instanceof RssShuffleHandle)) { throw new RuntimeException("Unexpected ShuffleHandle:" + handle.getClass().getName()); } - final String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key); - final int indexReadLimit = sparkConf.getInt(RssSparkConfig.RSS_INDEX_READ_LIMIT.key, - RssSparkConfig.RSS_INDEX_READ_LIMIT.getDefaultValue()); + final String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key()); + final int indexReadLimit = sparkConf.getInt(RssSparkConfig.RSS_INDEX_READ_LIMIT.key(), + RssSparkConfig.RSS_INDEX_READ_LIMIT.defaultValue().get()); RssShuffleHandle rssShuffleHandle = (RssShuffleHandle) handle; final int partitionNum = rssShuffleHandle.getDependency().partitioner().numPartitions(); - long readBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key, - RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.getDefaultValue()); + long readBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key(), + RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.defaultValue().get()); if (readBufferSize > Integer.MAX_VALUE) { - LOG.warn(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key + " can support 2g as max"); + LOG.warn(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key()+ " can support 2g as max"); readBufferSize = Integer.MAX_VALUE; } int shuffleId = rssShuffleHandle.getShuffleId(); @@ -598,7 +598,7 @@ protected void registerShuffleServers(String appId, int shuffleId, @VisibleForTesting protected void registerCoordinator() { - String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM.key); + String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM.key()); LOG.info("Start Registering coordinators {}", coordinators); shuffleWriteClient.registerCoordinators(coordinators); } diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java index 432daa444d..7dc35e6545 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java @@ -106,14 +106,14 @@ public RssShuffleWriter( this.shuffleDependency = rssHandle.getDependency(); this.partitioner = shuffleDependency.partitioner(); this.shouldPartition = partitioner.numPartitions() > 1; - this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, - RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.getDefaultValue()); - this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, - RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.getDefaultValue()); - this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key, - RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.getDefaultValue()); - this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.key, - RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.getDefaultValue()); + this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key(), + RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.defaultValue().get()); + this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key(), + RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.defaultValue().get()); + this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key(), + RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.defaultValue().get()); + this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.key(), + RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.defaultValue().get()); this.partitionToBlockIds = Maps.newConcurrentMap(); this.shuffleWriteClient = shuffleWriteClient; this.shuffleServersForData = rssHandle.getShuffleServersForData(); @@ -121,7 +121,7 @@ public RssShuffleWriter( Arrays.fill(partitionLengths, 0); partitionToServers = rssHandle.getPartitionToServers(); this.isMemoryShuffleEnabled = isMemoryShuffleEnabled( - sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key)); + sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key())); } private boolean isMemoryShuffleEnabled(String storageType) { diff --git a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java index 88999f4d7e..6382e0ae33 100644 --- a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java +++ b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/DelegationRssShuffleManagerTest.java @@ -66,7 +66,7 @@ public void testCreateInDriverDenied() throws Exception { mockedStaticRssShuffleUtils.when(() -> RssSparkShuffleUtils.createCoordinatorClients(any())).thenReturn(coordinatorClients); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key(), "false"); assertCreateSortShuffleManager(conf); } @@ -82,15 +82,15 @@ public void testCreateInDriver() throws Exception { SparkConf conf = new SparkConf(); assertCreateSortShuffleManager(conf); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); - conf.set(RssSparkConfig.RSS_ACCESS_ID.key, "mockId"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key(), "false"); + conf.set(RssSparkConfig.RSS_ACCESS_ID.key(), "mockId"); assertCreateSortShuffleManager(conf); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key(), "m1:8001,m2:8002"); conf.set("spark.rss.storage.type", StorageType.LOCALFILE.name()); assertCreateRssShuffleManager(conf); conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key(), "m1:8001,m2:8002"); when(mockCoordinatorClient.accessCluster(any())).thenReturn( new RssAccessClusterResponse(SUCCESS, "")); assertCreateSortShuffleManager(conf); @@ -100,7 +100,7 @@ public void testCreateInDriver() throws Exception { public void testCreateInExecutor() throws Exception { DelegationRssShuffleManager delegationRssShuffleManager; SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "m1:8001,m2:8002"); + conf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key(), "m1:8001,m2:8002"); delegationRssShuffleManager = new DelegationRssShuffleManager(conf, false); assertFalse(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); assertTrue(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); @@ -116,15 +116,15 @@ public void testCreateFallback() throws Exception { RssSparkShuffleUtils.createCoordinatorClients(any())).thenReturn(coordinatorClients); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key, "false"); - conf.set(RssSparkConfig.RSS_ACCESS_ID.key, "mockId"); - conf.set(RssSparkConfig.RSS_ENABLED.key, "true"); + conf.set(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key(), "false"); + conf.set(RssSparkConfig.RSS_ACCESS_ID.key(), "mockId"); + conf.set(RssSparkConfig.RSS_ENABLED.key(), "true"); // fall back to SortShuffleManager in driver assertCreateSortShuffleManager(conf); // No fall back in executor - conf.set(RssSparkConfig.RSS_ENABLED.key, "true"); + conf.set(RssSparkConfig.RSS_ENABLED.key(), "true"); boolean hasException = false; try { new DelegationRssShuffleManager(conf, false); @@ -139,7 +139,7 @@ private DelegationRssShuffleManager assertCreateSortShuffleManager(SparkConf con DelegationRssShuffleManager delegationRssShuffleManager = new DelegationRssShuffleManager(conf, true); assertTrue(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); assertFalse(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); - assertFalse(conf.getBoolean(RssSparkConfig.RSS_ENABLED.key, false)); + assertFalse(conf.getBoolean(RssSparkConfig.RSS_ENABLED.key(), false)); assertEquals("sort", conf.get("spark.shuffle.manager")); return delegationRssShuffleManager; } @@ -148,7 +148,7 @@ private DelegationRssShuffleManager assertCreateRssShuffleManager(SparkConf conf DelegationRssShuffleManager delegationRssShuffleManager = new DelegationRssShuffleManager(conf, true); assertFalse(delegationRssShuffleManager.getDelegate() instanceof SortShuffleManager); assertTrue(delegationRssShuffleManager.getDelegate() instanceof RssShuffleManager); - assertTrue(Boolean.parseBoolean(conf.get(RssSparkConfig.RSS_ENABLED.key))); + assertTrue(Boolean.parseBoolean(conf.get(RssSparkConfig.RSS_ENABLED.key()))); assertEquals(RssShuffleManager.class.getCanonicalName(), conf.get("spark.shuffle.manager")); return delegationRssShuffleManager; } diff --git a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java index 8c2c03bd15..093f3c0f31 100644 --- a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java +++ b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java @@ -68,11 +68,11 @@ public void checkBlockSendResultTest() { SparkConf conf = new SparkConf(); conf.setAppName("testApp") .setMaster("local[2]") - .set(RssSparkConfig.RSS_TEST_FLAG.key, "true") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, "10000") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, "1000") - .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()) - .set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "127.0.0.1:12345,127.0.0.1:12346"); + .set(RssSparkConfig.RSS_TEST_FLAG.key(), "true") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key(), "10000") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key(), "1000") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.LOCALFILE.name()) + .set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key(), "127.0.0.1:12345,127.0.0.1:12346"); // init SparkContext SparkContext sc = SparkContext.getOrCreate(conf); Map> failBlocks = Maps.newConcurrentMap(); @@ -135,15 +135,15 @@ public void checkBlockSendResultTest() { public void writeTest() throws Exception { SparkConf conf = new SparkConf(); conf.setAppName("testApp").setMaster("local[2]") - .set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key, "32") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key, "32") - .set(RssSparkConfig.RSS_TEST_FLAG.key, "true") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key, "64") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, "10000") - .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, "1000") - .set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key, "128") - .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()) - .set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, "127.0.0.1:12345,127.0.0.1:12346"); + .set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key(), "32") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key(), "32") + .set(RssSparkConfig.RSS_TEST_FLAG.key(), "true") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key(), "64") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key(), "10000") + .set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key(), "1000") + .set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key(), "128") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.LOCALFILE.name()) + .set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key(), "127.0.0.1:12345,127.0.0.1:12346"); // init SparkContext List shuffleBlockInfos = Lists.newArrayList(); SparkContext sc = SparkContext.getOrCreate(conf); @@ -303,8 +303,8 @@ public void onError(Throwable e) { when(mockHandle.getDependency()).thenReturn(mockDependency); ShuffleWriteClient mockWriteClient = mock(ShuffleWriteClient.class); SparkConf conf = new SparkConf(); - conf.set(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key, "64") - .set(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE.name()); + conf.set(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key(), "64") + .set(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.MEMORY_LOCALFILE.name()); List shuffleBlockInfoList = createShuffleBlockList(1, 31); RssShuffleWriter writer = new RssShuffleWriter("appId", 0, "taskId", 1L, mockBufferManager, mockMetrics, mockShuffleManager, conf, mockWriteClient, mockHandle); diff --git a/client/src/main/java/org/apache/uniffle/client/util/RssClientConfig.java b/client/src/main/java/org/apache/uniffle/client/util/RssClientConfig.java index 43204e2b5d..0b42d49a20 100644 --- a/client/src/main/java/org/apache/uniffle/client/util/RssClientConfig.java +++ b/client/src/main/java/org/apache/uniffle/client/util/RssClientConfig.java @@ -24,7 +24,7 @@ public class RssClientConfig { public static final String RSS_CLIENT_RETRY_MAX = "rss.client.retry.max"; public static final int RSS_CLIENT_RETRY_MAX_DEFAULT_VALUE = 100; public static final String RSS_CLIENT_RETRY_INTERVAL_MAX = "rss.client.retry.interval.max"; - public static final long RSS_CLIENT_RETRY_INTERVAL_MAX_DEFAULT_VALUE = 10000; + public static final long RSS_CLIENT_RETRY_INTERVAL_MAX_DEFAULT_VALUE = 10000L; public static final String RSS_COORDINATOR_QUORUM = "rss.coordinator.quorum"; public static final String RSS_DATA_REPLICA = "rss.data.replica"; public static final int RSS_DATA_REPLICA_DEFAULT_VALUE = 1; @@ -43,9 +43,9 @@ public class RssClientConfig { public static final String RSS_HEARTBEAT_TIMEOUT = "rss.heartbeat.timeout"; public static final String RSS_STORAGE_TYPE = "rss.storage.type"; public static final String RSS_CLIENT_SEND_CHECK_INTERVAL_MS = "rss.client.send.check.interval.ms"; - public static final long RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE = 500; + public static final long RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE = 500L; public static final String RSS_CLIENT_SEND_CHECK_TIMEOUT_MS = "rss.client.send.check.timeout.ms"; - public static final long RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE = 60 * 1000 * 10; + public static final long RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE = 60 * 1000 * 10L; public static final String RSS_WRITER_BUFFER_SIZE = "rss.writer.buffer.size"; public static final String RSS_PARTITION_NUM_PER_RANGE = "rss.partitionNum.per.range"; public static final int RSS_PARTITION_NUM_PER_RANGE_DEFAULT_VALUE = 1; diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/AutoAccessTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/AutoAccessTest.java index eedcce7e37..6c269ba642 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/AutoAccessTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/AutoAccessTest.java @@ -48,9 +48,9 @@ public class AutoAccessTest extends IntegrationTestBase { public void test() throws Exception { SparkConf sparkConf = new SparkConf(); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.DelegationRssShuffleManager"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key(), COORDINATOR_QUORUM); sparkConf.set("spark.mock.2", "no-overwrite-conf"); - sparkConf.set(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key, "overwrite-path"); + sparkConf.set(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key(), "overwrite-path"); sparkConf.set("spark.shuffle.service.enabled", "true"); String cfgFile = HDFS_URI + "/test/client_conf"; @@ -61,7 +61,7 @@ public void test() throws Exception { printWriter.println(" spark.mock.2 overwrite-conf "); printWriter.println(" spark.mock.3 true "); printWriter.println("spark.rss.storage.type " + StorageType.MEMORY_LOCALFILE_HDFS.name()); - printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key + " expectedPath"); + printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key()+ " expectedPath"); printWriter.flush(); printWriter.close(); @@ -162,7 +162,7 @@ public void test() throws Exception { printWriter.println(" spark.mock.2 overwrite-conf "); printWriter.println(" spark.mock.3 false "); printWriter.println("spark.rss.storage.type " + StorageType.MEMORY_LOCALFILE_HDFS.name()); - printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key + " expectedPathNew"); + printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key()+ " expectedPathNew"); printWriter.flush(); printWriter.close(); fs.rename(tmpPath, path); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/DynamicFetchClientConfTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/DynamicFetchClientConfTest.java index e72f2fa20d..ee5cde5e6d 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/DynamicFetchClientConfTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/DynamicFetchClientConfTest.java @@ -42,7 +42,7 @@ public class DynamicFetchClientConfTest extends IntegrationTestBase { public void test() throws Exception { SparkConf sparkConf = new SparkConf(); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.RssShuffleManager"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key(), COORDINATOR_QUORUM); sparkConf.set("spark.mock.2", "no-overwrite-conf"); sparkConf.set("spark.shuffle.service.enabled", "true"); @@ -54,7 +54,7 @@ public void test() throws Exception { printWriter.println(" spark.mock.2 overwrite-conf "); printWriter.println(" spark.mock.3 true "); printWriter.println("spark.rss.storage.type " + StorageType.MEMORY_LOCALFILE_HDFS.name()); - printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key + " expectedPath"); + printWriter.println(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key() + " expectedPath"); printWriter.flush(); printWriter.close(); for (String k : RssSparkConfig.RSS_MANDATORY_CLUSTER_CONF) { @@ -74,10 +74,10 @@ public void test() throws Exception { assertFalse(sparkConf.contains("spark.mock.1")); assertEquals("no-overwrite-conf", sparkConf.get("spark.mock.2")); assertFalse(sparkConf.contains("spark.mock.3")); - assertEquals("Dummy-" + RssSparkConfig.RSS_STORAGE_TYPE.key, - sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key)); - assertEquals("Dummy-" + RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key, - sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); + assertEquals("Dummy-" + RssSparkConfig.RSS_STORAGE_TYPE.key(), + sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key())); + assertEquals("Dummy-" + RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key(), + sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key())); assertTrue(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); RssShuffleManager rssShuffleManager = new RssShuffleManager(sparkConf, true); @@ -86,14 +86,14 @@ public void test() throws Exception { assertEquals(1234, sparkConf1.getInt("spark.mock.1", 0)); assertEquals("no-overwrite-conf", sparkConf1.get("spark.mock.2")); assertEquals(StorageType.MEMORY_LOCALFILE_HDFS.name(), sparkConf.get("spark.rss.storage.type")); - assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); + assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key())); assertFalse(sparkConf1.getBoolean("spark.shuffle.service.enabled", true)); fs.delete(path, true); shutdownServers(); sparkConf = new SparkConf(); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.RssShuffleManager"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key(), COORDINATOR_QUORUM); sparkConf.set("spark.mock.2", "no-overwrite-conf"); sparkConf.set("spark.shuffle.service.enabled", "true"); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithHdfsMultiStorageRssTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithHdfsMultiStorageRssTest.java index b0ce89e536..4552a2d36b 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithHdfsMultiStorageRssTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithHdfsMultiStorageRssTest.java @@ -36,7 +36,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); dynamicConf.put(CoordinatorConf.COORDINATOR_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE_HDFS.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.LOCALFILE_HDFS.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithLocalFileRssTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithLocalFileRssTest.java index 6f446a962d..82586d9113 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithLocalFileRssTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithLocalFileRssTest.java @@ -36,7 +36,7 @@ public class RepartitionWithLocalFileRssTest extends RepartitionTest { public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.LOCALFILE.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.LOCALFILE.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryMultiStorageRssTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryMultiStorageRssTest.java index b07aa1f368..2cbe5acbbd 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryMultiStorageRssTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryMultiStorageRssTest.java @@ -36,7 +36,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); dynamicConf.put(CoordinatorConf.COORDINATOR_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE_HDFS.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.MEMORY_LOCALFILE_HDFS.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryRssTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryRssTest.java index 3397c61e4e..f15e142df9 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryRssTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RepartitionWithMemoryRssTest.java @@ -38,7 +38,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); coordinatorConf.set(CoordinatorConf.COORDINATOR_APP_EXPIRED, 5000L); Map dynamicConf = Maps.newHashMap(); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.MEMORY_LOCALFILE.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SimpleTestBase.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SimpleTestBase.java index 09a204b873..5b7e4d5319 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SimpleTestBase.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SimpleTestBase.java @@ -34,7 +34,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); dynamicConf.put(CoordinatorConf.COORDINATOR_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE_HDFS.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.MEMORY_LOCALFILE_HDFS.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkIntegrationTestBase.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkIntegrationTestBase.java index da164004dc..3a38efb309 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkIntegrationTestBase.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkIntegrationTestBase.java @@ -85,18 +85,18 @@ protected SparkConf createSparkConf() { public void updateSparkConfWithRss(SparkConf sparkConf) { sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.RssShuffleManager"); sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key, "4m"); - sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key, "32m"); - sparkConf.set(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key, "2m"); - sparkConf.set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key, "128k"); - sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key, "256k"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); - sparkConf.set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key, "30000"); - sparkConf.set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key, "1000"); - sparkConf.set(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key, "1000"); - sparkConf.set(RssSparkConfig.RSS_INDEX_READ_LIMIT.key, "100"); - sparkConf.set(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key, "1m"); - sparkConf.set(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key, "2000"); + sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key(), "4m"); + sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SPILL_SIZE.key(), "32m"); + sparkConf.set(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key(), "2m"); + sparkConf.set(RssSparkConfig.RSS_WRITER_SERIALIZER_BUFFER_SIZE.key(), "128k"); + sparkConf.set(RssSparkConfig.RSS_WRITER_BUFFER_SEGMENT_SIZE.key(), "256k"); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key(), COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key(), "30000"); + sparkConf.set(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key(), "1000"); + sparkConf.set(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key(), "1000"); + sparkConf.set(RssSparkConfig.RSS_INDEX_READ_LIMIT.key(), "100"); + sparkConf.set(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key(), "1m"); + sparkConf.set(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key(), "2000"); } private void verifyTestResult(Map expected, Map actual) { diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManager.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManager.java index 5da718f5cc..4c6d2aa46f 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManager.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManager.java @@ -47,7 +47,7 @@ public static void setupServers() throws Exception { coordinatorConf.set(CoordinatorConf.COORDINATOR_APP_EXPIRED, 5000L); coordinatorConf.set(CoordinatorConf.COORDINATOR_ACCESS_LOADCHECKER_SERVER_NUM_THRESHOLD, 1); Map dynamicConf = Maps.newHashMap(); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.MEMORY_LOCALFILE.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); @@ -67,7 +67,7 @@ public static void setupServers() throws Exception { @Override public void updateRssStorage(SparkConf sparkConf) { - sparkConf.set(RssSparkConfig.RSS_ACCESS_ID.key, "test_access_id"); + sparkConf.set(RssSparkConfig.RSS_ACCESS_ID.key(), "test_access_id"); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.DelegationRssShuffleManager"); } diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManagerFallback.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManagerFallback.java index 8b47767444..3e2cee461f 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManagerFallback.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithDelegationShuffleManagerFallback.java @@ -47,7 +47,7 @@ public static void setupServers() throws Exception { coordinatorConf.set(CoordinatorConf.COORDINATOR_APP_EXPIRED, 5000L); coordinatorConf.set(CoordinatorConf.COORDINATOR_ACCESS_LOADCHECKER_SERVER_NUM_THRESHOLD, 1); Map dynamicConf = Maps.newHashMap(); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.MEMORY_LOCALFILE.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); @@ -68,7 +68,7 @@ public static void setupServers() throws Exception { @Override public void updateRssStorage(SparkConf sparkConf) { - sparkConf.set(RssSparkConfig.RSS_ACCESS_ID.key, "wrong_id"); + sparkConf.set(RssSparkConfig.RSS_ACCESS_ID.key(), "wrong_id"); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.DelegationRssShuffleManager"); } diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithMemoryLocalTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithMemoryLocalTest.java index f4d0a0bf55..98432720ae 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithMemoryLocalTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SparkSQLWithMemoryLocalTest.java @@ -41,7 +41,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); coordinatorConf.setLong("rss.coordinator.app.expired", 5000); Map dynamicConf = Maps.newHashMap(); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.MEMORY_LOCALFILE.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark2/src/test/java/org/apache/uniffle/test/GetReaderTest.java b/integration-test/spark2/src/test/java/org/apache/uniffle/test/GetReaderTest.java index d15e601a06..0545885999 100644 --- a/integration-test/spark2/src/test/java/org/apache/uniffle/test/GetReaderTest.java +++ b/integration-test/spark2/src/test/java/org/apache/uniffle/test/GetReaderTest.java @@ -55,7 +55,7 @@ public class GetReaderTest extends IntegrationTestBase { public void test() throws Exception { SparkConf sparkConf = new SparkConf(); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.RssShuffleManager"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key(), COORDINATOR_QUORUM); sparkConf.setMaster("local[4]"); final String remoteStorage1 = "hdfs://h1/p1"; final String remoteStorage2 = "hdfs://h2/p2"; diff --git a/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQERepartitionTest.java b/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQERepartitionTest.java index 6a43606a86..316040a92f 100644 --- a/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQERepartitionTest.java +++ b/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQERepartitionTest.java @@ -47,7 +47,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); dynamicConf.put(CoordinatorConf.COORDINATOR_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE_HDFS.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.MEMORY_LOCALFILE_HDFS.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); diff --git a/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQESkewedJoinTest.java b/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQESkewedJoinTest.java index ce215a12da..60867faf1d 100644 --- a/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQESkewedJoinTest.java +++ b/integration-test/spark3/src/test/java/org/apache/uniffle/test/AQESkewedJoinTest.java @@ -49,7 +49,7 @@ public static void setupServers() throws Exception { CoordinatorConf coordinatorConf = getCoordinatorConf(); Map dynamicConf = Maps.newHashMap(); dynamicConf.put(CoordinatorConf.COORDINATOR_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key, StorageType.MEMORY_LOCALFILE_HDFS.name()); + dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.MEMORY_LOCALFILE_HDFS.name()); addDynamicConf(coordinatorConf, dynamicConf); createCoordinatorServer(coordinatorConf); ShuffleServerConf shuffleServerConf = getShuffleServerConf(); @@ -69,8 +69,8 @@ public void updateCommonSparkConf(SparkConf sparkConf) { @Override public void updateSparkConfCustomer(SparkConf sparkConf) { - sparkConf.set(RssSparkConfig.RSS_STORAGE_TYPE.key, "HDFS"); - sparkConf.set(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key, HDFS_URI + "rss/test"); + sparkConf.set(RssSparkConfig.RSS_STORAGE_TYPE.key(), "HDFS"); + sparkConf.set(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); } @Test diff --git a/integration-test/spark3/src/test/java/org/apache/uniffle/test/GetReaderTest.java b/integration-test/spark3/src/test/java/org/apache/uniffle/test/GetReaderTest.java index 5739598386..95e2b39e0f 100644 --- a/integration-test/spark3/src/test/java/org/apache/uniffle/test/GetReaderTest.java +++ b/integration-test/spark3/src/test/java/org/apache/uniffle/test/GetReaderTest.java @@ -68,7 +68,7 @@ public class GetReaderTest extends IntegrationTestBase { public void test() throws Exception { SparkConf sparkConf = new SparkConf(); sparkConf.set("spark.shuffle.manager", "org.apache.spark.shuffle.RssShuffleManager"); - sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key, COORDINATOR_QUORUM); + sparkConf.set(RssSparkConfig.RSS_COORDINATOR_QUORUM.key(), COORDINATOR_QUORUM); sparkConf.setMaster("local[4]"); final String remoteStorage1 = "hdfs://h1/p1"; final String remoteStorage2 = "hdfs://h2/p2"; From 541b88041ed1c63458b7595208415edbbc809425 Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Sun, 31 Jul 2022 01:50:01 +0800 Subject: [PATCH 12/22] pass checkStyle --- .../org/apache/spark/shuffle/RssSparkConfig.java | 14 +++++--------- .../org/apache/uniffle/test/AutoAccessTest.java | 10 +++++----- 2 files changed, 10 insertions(+), 14 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index 6990463402..286eef0058 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -24,11 +24,10 @@ import org.apache.spark.internal.config.ConfigEntry; import org.apache.spark.internal.config.TypedConfigBuilder; -import scala.Serializable; import scala.runtime.AbstractFunction1; -import org.apache.uniffle.common.config.ConfigUtils; import org.apache.uniffle.client.util.RssClientConfig; +import org.apache.uniffle.common.config.ConfigUtils; public class RssSparkConfig { @@ -240,7 +239,7 @@ public class RssSparkConfig { public static final boolean RSS_USE_RSS_SHUFFLE_MANAGER_DEFAULT_VALUE = false; public static TypedConfigBuilder createIntegerBuilder(ConfigBuilder builder) { - scala.Function1 f = new SerializableFunction1() { + scala.Function1 f = new AbstractFunction1() { @Override public Integer apply(String in) { return ConfigUtils.convertValue(in, Integer.class); @@ -250,7 +249,7 @@ public Integer apply(String in) { } public static TypedConfigBuilder createLongBuilder(ConfigBuilder builder) { - scala.Function1 f = new SerializableFunction1() { + scala.Function1 f = new AbstractFunction1() { @Override public Long apply(String in) { return ConfigUtils.convertValue(in, Long.class); @@ -260,7 +259,7 @@ public Long apply(String in) { } public static TypedConfigBuilder createBooleanBuilder(ConfigBuilder builder) { - scala.Function1 f = new SerializableFunction1() { + scala.Function1 f = new AbstractFunction1() { @Override public Boolean apply(String in) { return ConfigUtils.convertValue(in, Boolean.class); @@ -270,7 +269,7 @@ public Boolean apply(String in) { } public static TypedConfigBuilder createDoubleBuilder(ConfigBuilder builder) { - scala.Function1 f = new SerializableFunction1() { + scala.Function1 f = new AbstractFunction1() { @Override public Double apply(String in) { return ConfigUtils.convertValue(in, Double.class); @@ -283,6 +282,3 @@ public static TypedConfigBuilder createStringBuilder(ConfigBuilder build return builder.stringConf(); } } - -abstract class SerializableFunction1 extends AbstractFunction1 implements Serializable { -} diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/AutoAccessTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/AutoAccessTest.java index 6c269ba642..a9d6a34d16 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/AutoAccessTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/AutoAccessTest.java @@ -100,7 +100,7 @@ public void test() throws Exception { ShuffleManager shuffleManager = delegationRssShuffleManager.getDelegate(); assertTrue(shuffleManager instanceof SortShuffleManager); assertTrue(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); - assertEquals("overwrite-path", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); + assertEquals("overwrite-path", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key())); assertFalse(sparkConf.contains("spark.rss.storage.type")); // wrong access id @@ -108,7 +108,7 @@ public void test() throws Exception { delegationRssShuffleManager = new DelegationRssShuffleManager(sparkConf, true); shuffleManager = delegationRssShuffleManager.getDelegate(); assertTrue(shuffleManager instanceof SortShuffleManager); - assertEquals("overwrite-path", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); + assertEquals("overwrite-path", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key())); assertTrue(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); assertFalse(sparkConf.contains("spark.rss.storage.type")); @@ -122,7 +122,7 @@ public void test() throws Exception { assertEquals("no-overwrite-conf", sparkConf.get("spark.mock.2")); assertTrue(sparkConf.getBoolean("spark.mock.3", false)); assertEquals(StorageType.MEMORY_LOCALFILE_HDFS.name(), sparkConf.get("spark.rss.storage.type")); - assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); + assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key())); assertFalse(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); // update candidates file @@ -149,7 +149,7 @@ public void test() throws Exception { assertEquals("no-overwrite-conf", sparkConf.get("spark.mock.2")); assertTrue(sparkConf.getBoolean("spark.mock.3", false)); assertEquals(StorageType.MEMORY_LOCALFILE_HDFS.name(), sparkConf.get("spark.rss.storage.type")); - assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); + assertEquals("expectedPath", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key())); assertFalse(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); // update client conf file @@ -177,7 +177,7 @@ public void test() throws Exception { assertEquals("overwrite-conf", sparkConf.get("spark.mock.2")); assertTrue(sparkConf.getBoolean("spark.mock.3", false)); assertEquals(StorageType.MEMORY_LOCALFILE_HDFS.name(), sparkConf.get("spark.rss.storage.type")); - assertEquals("expectedPathNew", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key)); + assertEquals("expectedPathNew", sparkConf.get(RssSparkConfig.RSS_REMOTE_STORAGE_PATH.key())); assertFalse(sparkConf.getBoolean("spark.shuffle.service.enabled", true)); } } From 2452fbee2450bceee8120040f29ba9bc512afdbd Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Sun, 31 Jul 2022 02:14:30 +0800 Subject: [PATCH 13/22] change value --- .../src/main/java/org/apache/spark/shuffle/RssSparkConfig.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index 286eef0058..2cefea205e 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -24,6 +24,7 @@ import org.apache.spark.internal.config.ConfigEntry; import org.apache.spark.internal.config.TypedConfigBuilder; + import scala.runtime.AbstractFunction1; import org.apache.uniffle.client.util.RssClientConfig; @@ -34,7 +35,7 @@ public class RssSparkConfig { public static final ConfigEntry RSS_PARTITION_NUM_PER_RANGE = createIntegerBuilder( new ConfigBuilder("spark.rss.partitionNum.per.range") .doc("xxxxxx")) - .createWithDefault(10); + .createWithDefault(1); public static final ConfigEntry RSS_WRITER_BUFFER_SIZE = createStringBuilder( new ConfigBuilder("spark.rss.writer.buffer.size") From 1fc9e6f64e07ea8c785fe7057238faa6c0b7aca5 Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Sun, 31 Jul 2022 02:37:29 +0800 Subject: [PATCH 14/22] fix style --- .../src/main/java/org/apache/spark/shuffle/RssSparkConfig.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index 2cefea205e..d6a4e63e0c 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -23,8 +23,6 @@ import org.apache.spark.internal.config.ConfigBuilder; import org.apache.spark.internal.config.ConfigEntry; import org.apache.spark.internal.config.TypedConfigBuilder; - - import scala.runtime.AbstractFunction1; import org.apache.uniffle.client.util.RssClientConfig; From 650eadbb3f832ba9583df73247cf4dd39673f57c Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Sun, 31 Jul 2022 02:39:31 +0800 Subject: [PATCH 15/22] add whitespace --- .../main/java/org/apache/spark/shuffle/RssShuffleManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java index ad76559595..d759953912 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java @@ -413,7 +413,7 @@ public ShuffleReader getReaderImpl( long readBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key(), RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.defaultValue().get()); if (readBufferSize > Integer.MAX_VALUE) { - LOG.warn(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key()+ " can support 2g as max"); + LOG.warn(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key() + " can support 2g as max"); readBufferSize = Integer.MAX_VALUE; } int shuffleId = rssShuffleHandle.getShuffleId(); From 3a366b38958be28a236ddc5768ba2680c7f9abd9 Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Sun, 31 Jul 2022 16:05:38 +0800 Subject: [PATCH 16/22] change ConfigBuilder key name --- .../apache/spark/shuffle/RssSparkConfig.java | 57 ++++++++++--------- 1 file changed, 29 insertions(+), 28 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index d6a4e63e0c..840765dc70 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -31,12 +31,12 @@ public class RssSparkConfig { public static final ConfigEntry RSS_PARTITION_NUM_PER_RANGE = createIntegerBuilder( - new ConfigBuilder("spark.rss.partitionNum.per.range") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_PARTITION_NUM_PER_RANGE) .doc("xxxxxx")) - .createWithDefault(1); + .createWithDefault(RssClientConfig.RSS_PARTITION_NUM_PER_RANGE_DEFAULT_VALUE); public static final ConfigEntry RSS_WRITER_BUFFER_SIZE = createStringBuilder( - new ConfigBuilder("spark.rss.writer.buffer.size") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_WRITER_BUFFER_SIZE) .doc("controls the buffer flushing size during shuffle write")) .createWithDefault("3m"); @@ -71,52 +71,52 @@ public class RssSparkConfig { .createWithDefault(1000L); public static final ConfigEntry RSS_CLIENT_SEND_CHECK_TIMEOUT_MS = createLongBuilder( - new ConfigBuilder("spark.rss.client.send.check.timeout.ms") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS) .doc("")) .createWithDefault(RssClientConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE); public static final ConfigEntry RSS_CLIENT_SEND_CHECK_INTERVAL_MS = createLongBuilder( - new ConfigBuilder("spark.rss.client.send.check.interval.ms") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS) .doc("")) .createWithDefault(RssClientConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE); - public static final ConfigEntry RSS_TEST_FLAG = createStringBuilder( + public static final ConfigEntry RSS_TEST_FLAG = createStringBuilder( new ConfigBuilder("spark.rss.test") .doc("")) - .createWithDefault(""); + .createWithDefault(false); public static final ConfigEntry RSS_REMOTE_STORAGE_PATH = createStringBuilder( - new ConfigBuilder("spark.rss.remote.storage.path") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_REMOTE_STORAGE_PATH) .doc("")) .createWithDefault(""); public static final ConfigEntry RSS_INDEX_READ_LIMIT = createIntegerBuilder( - new ConfigBuilder("spark.rss.index.read.limit") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_INDEX_READ_LIMIT) .doc("")) - .createWithDefault(500); + .createWithDefault(RssClientConfig.RSS_INDEX_READ_LIMIT_DEFAULT_VALUE); public static final ConfigEntry RSS_CLIENT_TYPE = createStringBuilder( - new ConfigBuilder("spark.rss.client.type") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_TYPE) .doc("")) .createWithDefault(RssClientConfig.RSS_CLIENT_TYPE_DEFAULT_VALUE); public static final ConfigEntry RSS_STORAGE_TYPE = createStringBuilder( - new ConfigBuilder("spark.rss.storage.type") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_STORAGE_TYPE) .doc("")) .createWithDefault(""); public static final ConfigEntry RSS_CLIENT_RETRY_MAX = createIntegerBuilder( - new ConfigBuilder("spark.rss.client.retry.max") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_RETRY_MAX) .doc("")) .createWithDefault(RssClientConfig.RSS_CLIENT_RETRY_MAX_DEFAULT_VALUE); public static final ConfigEntry RSS_CLIENT_RETRY_INTERVAL_MAX = createLongBuilder( - new ConfigBuilder("spark.rss.client.retry.interval.max") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_RETRY_INTERVAL_MAX) .doc("")) .createWithDefault(RssClientConfig.RSS_CLIENT_RETRY_INTERVAL_MAX_DEFAULT_VALUE); public static final ConfigEntry RSS_CLIENT_HEARTBEAT_THREAD_NUM = createIntegerBuilder( - new ConfigBuilder("spark.rss.client.heartBeat.threadNum") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM) .doc("")) .createWithDefault(4); @@ -125,18 +125,19 @@ public class RssSparkConfig { .doc("")) .createWithDefault("16m"); + public static final ConfigEntry RSS_CLIENT_READ_BUFFER_SIZE = createStringBuilder( - new ConfigBuilder("spark.rss.client.read.buffer.size") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_READ_BUFFER_SIZE) .doc("")) .createWithDefault(RssClientConfig.RSS_CLIENT_READ_BUFFER_SIZE_DEFAULT_VALUE); public static final ConfigEntry RSS_HEARTBEAT_INTERVAL = createLongBuilder( - new ConfigBuilder("spark.rss.heartbeat.interval") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_HEARTBEAT_INTERVAL) .doc("")) .createWithDefault(RssClientConfig.RSS_HEARTBEAT_INTERVAL_DEFAULT_VALUE); public static final ConfigEntry RSS_HEARTBEAT_TIMEOUT = createLongBuilder( - new ConfigBuilder("spark.rss.heartbeat.timeout") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_HEARTBEAT_TIMEOUT) .doc("")) .createWithDefault(5 * 1000L); @@ -151,32 +152,32 @@ public class RssSparkConfig { .createWithDefault(60); public static final ConfigEntry RSS_DATA_REPLICA = createIntegerBuilder( - new ConfigBuilder("spark.rss.data.replica") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA) .doc("")) .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_DEFAULT_VALUE); public static final ConfigEntry RSS_DATA_REPLICA_WRITE = createIntegerBuilder( - new ConfigBuilder("spark.rss.data.replica.write") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA_WRITE) .doc("")) .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_WRITE_DEFAULT_VALUE); public static final ConfigEntry RSS_DATA_REPLICA_READ = createIntegerBuilder( - new ConfigBuilder("spark.rss.data.replica.read") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA_READ) .doc("")) .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_READ_DEFAULT_VALUE); public static final ConfigEntry RSS_DATA_REPLICA_SKIP_ENABLED = createBooleanBuilder( - new ConfigBuilder("spark.rss.data.replica.skip.enabled") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA_SKIP_ENABLED) .doc("")) .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_SKIP_ENABLED_DEFAULT_VALUE); public static final ConfigEntry RSS_DATA_TRANSFER_POOL_SIZE = createIntegerBuilder( - new ConfigBuilder("spark.rss.client.data.transfer.pool.size") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_TRANSFER_POOL_SIZE) .doc("")) .createWithDefault(RssClientConfig.RSS_DATA_TRANFER_POOL_SIZE_DEFAULT_VALUE); public static final ConfigEntry RSS_DATA_COMMIT_POOL_SIZE = createIntegerBuilder( - new ConfigBuilder("spark.rss.client.data.commit.pool.size") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_COMMIT_POOL_SIZE) .doc("")) .createWithDefault(RssClientConfig.RSS_DATA_COMMIT_POOL_SIZE_DEFAULT_VALUE); @@ -206,7 +207,7 @@ public class RssSparkConfig { .createWithDefault(""); public static final ConfigEntry RSS_ACCESS_TIMEOUT_MS = createIntegerBuilder( - new ConfigBuilder("spark.rss.access.timeout.ms") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_ACCESS_TIMEOUT_MS) .doc("")) .createWithDefault(RssClientConfig.RSS_ACCESS_TIMEOUT_MS_DEFAULT_VALUE); @@ -216,17 +217,17 @@ public class RssSparkConfig { .createWithDefault(false); public static final ConfigEntry RSS_DYNAMIC_CLIENT_CONF_ENABLED = createBooleanBuilder( - new ConfigBuilder("spark.rss.dynamicClientConf.enabled") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED) .doc("")) .createWithDefault(RssClientConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED_DEFAULT_VALUE); public static final ConfigEntry RSS_CLIENT_ASSIGNMENT_TAGS = createStringBuilder( - new ConfigBuilder("spark.rss.client.assignment.tags") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_ASSIGNMENT_TAGS) .doc("")) .createWithDefault(""); public static final ConfigEntry RSS_COORDINATOR_QUORUM = createStringBuilder( - new ConfigBuilder("spark.rss.coordinator.quorum") + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX +RssClientConfig.RSS_COORDINATOR_QUORUM) .doc("")) .createWithDefault(""); From 71c50d9088017e74f73cc8c0f1dd6c1b6c7d8079 Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Sun, 31 Jul 2022 16:32:12 +0800 Subject: [PATCH 17/22] change order --- .../java/org/apache/spark/shuffle/RssSparkConfig.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index 840765dc70..8454cc7aa0 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -30,6 +30,8 @@ public class RssSparkConfig { + public static final String SPARK_RSS_CONFIG_PREFIX = "spark."; + public static final ConfigEntry RSS_PARTITION_NUM_PER_RANGE = createIntegerBuilder( new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_PARTITION_NUM_PER_RANGE) .doc("xxxxxx")) @@ -80,7 +82,7 @@ public class RssSparkConfig { .doc("")) .createWithDefault(RssClientConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE); - public static final ConfigEntry RSS_TEST_FLAG = createStringBuilder( + public static final ConfigEntry RSS_TEST_FLAG = createBooleanBuilder( new ConfigBuilder("spark.rss.test") .doc("")) .createWithDefault(false); @@ -116,7 +118,7 @@ public class RssSparkConfig { .createWithDefault(RssClientConfig.RSS_CLIENT_RETRY_INTERVAL_MAX_DEFAULT_VALUE); public static final ConfigEntry RSS_CLIENT_HEARTBEAT_THREAD_NUM = createIntegerBuilder( - new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM) + new ConfigBuilder("spark.rss.client.heartBeat.threadNum") .doc("")) .createWithDefault(4); @@ -227,15 +229,13 @@ public class RssSparkConfig { .createWithDefault(""); public static final ConfigEntry RSS_COORDINATOR_QUORUM = createStringBuilder( - new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX +RssClientConfig.RSS_COORDINATOR_QUORUM) + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_COORDINATOR_QUORUM) .doc("")) .createWithDefault(""); public static final Set RSS_MANDATORY_CLUSTER_CONF = Sets.newHashSet(RSS_STORAGE_TYPE.key(), RSS_REMOTE_STORAGE_PATH.key()); - public static final String SPARK_RSS_CONFIG_PREFIX = "spark."; - public static final boolean RSS_USE_RSS_SHUFFLE_MANAGER_DEFAULT_VALUE = false; public static TypedConfigBuilder createIntegerBuilder(ConfigBuilder builder) { From 809063460c0ae3e53b7945e183936e8d6dacf060 Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Sun, 31 Jul 2022 18:48:50 +0800 Subject: [PATCH 18/22] use configEntry --- .../spark/shuffle/RssSparkShuffleUtils.java | 15 ++-- .../shuffle/writer/BufferManagerOptions.java | 6 +- .../shuffle/DelegationRssShuffleManager.java | 8 +- .../spark/shuffle/RssShuffleManager.java | 55 +++++-------- .../shuffle/writer/RssShuffleWriter.java | 9 +-- .../shuffle/DelegationRssShuffleManager.java | 8 +- .../spark/shuffle/RssShuffleManager.java | 79 ++++++------------- .../shuffle/writer/RssShuffleWriter.java | 9 +-- 8 files changed, 61 insertions(+), 128 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java index 2763c66853..13f7305442 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java @@ -44,20 +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.key(), - RssSparkConfig.RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE.defaultValue().get()); + 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.key().substring(OZONE_PREFIX_LEN), useOdfs); conf.set( RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.key().substring(OZONE_PREFIX_LEN), - sparkConf.get(RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.key(), - RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL.defaultValue().get())); + sparkConf.get(RssSparkConfig.RSS_OZONE_FS_HDFS_IMPL)); conf.set( 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.key(), - RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL.defaultValue().get())); + sparkConf.get(RssSparkConfig.RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL)); } return conf; @@ -78,9 +74,8 @@ public static ShuffleManager loadShuffleManager(String name, SparkConf conf, boo } public static List createCoordinatorClients(SparkConf sparkConf) throws RuntimeException { - String clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key(), - RssSparkConfig.RSS_CLIENT_TYPE.defaultValue().get()); - String coordinators = sparkConf.get(RssSparkConfig.RSS_COORDINATOR_QUORUM.key()); + 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); } diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java index feab47a085..3ae878ccd3 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BufferManagerOptions.java @@ -45,10 +45,8 @@ public BufferManagerOptions(SparkConf sparkConf) { 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.getLong(RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_INTERVAL.key(), - RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_INTERVAL.defaultValue().get()); - requireMemoryRetryMax = sparkConf.getInt(RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX.key(), - RssSparkConfig.RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX.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); diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java index 0b58fb086a..7ff9467835 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java @@ -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.key(), - RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.defaultValue().get()); + accessTimeoutMs = sparkConf.get(RssSparkConfig.RSS_ACCESS_TIMEOUT_MS); if (isDriver) { coordinatorClients = RssSparkShuffleUtils.createCoordinatorClients(sparkConf); delegate = createShuffleManagerInDriver(); @@ -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.key(), - 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); diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java index 1a80b3064a..c313747a70 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java @@ -141,36 +141,23 @@ public RssShuffleManager(SparkConf sparkConf, boolean isDriver) { this.sparkConf = sparkConf; // set & check replica config - this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA.key(), - RssSparkConfig.RSS_DATA_REPLICA.defaultValue().get()); - this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key(), - RssSparkConfig.RSS_DATA_REPLICA_WRITE.defaultValue().get()); - this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ.key(), - RssSparkConfig.RSS_DATA_REPLICA_READ.defaultValue().get()); - this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.key(), - RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.defaultValue().get()); - this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.key(), - RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.defaultValue().get()); + this.dataReplica = sparkConf.get(RssSparkConfig.RSS_DATA_REPLICA); + this.dataReplicaWrite = sparkConf.get(RssSparkConfig.RSS_DATA_REPLICA_WRITE); + this.dataReplicaRead = sparkConf.get(RssSparkConfig.RSS_DATA_REPLICA_READ); + this.dataTransferPoolSize = sparkConf.get(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE); + this.dataReplicaSkipEnabled = sparkConf.get(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED); LOG.info("Check quorum config [" + dataReplica + ":" + dataReplicaWrite + ":" + dataReplicaRead + ":" + dataReplicaSkipEnabled + "]"); RssUtils.checkQuorumSetting(dataReplica, dataReplicaWrite, dataReplicaRead); - this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key(), - RssSparkConfig.RSS_CLIENT_TYPE.defaultValue().get()); - this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key(), - RssSparkConfig.RSS_HEARTBEAT_INTERVAL.defaultValue().get()); + this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE); + this.heartbeatInterval = sparkConf.get(RssSparkConfig.RSS_HEARTBEAT_INTERVAL); this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT.key(), heartbeatInterval / 2); - this.dynamicConfEnabled = sparkConf.getBoolean( - RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key(), - RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.defaultValue().get()); - int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key(), - RssSparkConfig.RSS_CLIENT_RETRY_MAX.defaultValue().get()); - long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key(), - RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.defaultValue().get()); - int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.key(), - RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.defaultValue().get()); - this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.key(), - RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.defaultValue().get()); + this.dynamicConfEnabled = sparkConf.get(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED); + int retryMax = sparkConf.get(RssSparkConfig.RSS_CLIENT_RETRY_MAX); + long retryIntervalMax = sparkConf.get(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX); + int heartBeatThreadNum = sparkConf.get(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM); + this.dataCommitPoolSize = sparkConf.get(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE); this.shuffleWriteClient = ShuffleClientFactory .getInstance() .createShuffleWriteClient(clientType, retryMax, retryIntervalMax, heartBeatThreadNum, @@ -180,8 +167,7 @@ public RssShuffleManager(SparkConf sparkConf, boolean isDriver) { // fetch client conf and apply them if necessary and disable ESS if (isDriver && dynamicConfEnabled) { Map clusterClientConf = shuffleWriteClient.fetchClientConf( - sparkConf.getInt(RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.key(), - RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.defaultValue().get())); + sparkConf.get(RssSparkConfig.RSS_ACCESS_TIMEOUT_MS)); RssSparkShuffleUtils.applyDynamicClientConf(sparkConf, clusterClientConf); } RssSparkShuffleUtils.validateRssClientConf(sparkConf); @@ -192,10 +178,8 @@ public RssShuffleManager(SparkConf sparkConf, boolean isDriver) { // for non-driver executor, start a thread for sending shuffle data to shuffle server LOG.info("RSS data send thread is starting"); eventLoop.start(); - int poolSize = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.key(), - RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.defaultValue().get()); - int keepAliveTime = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.key(), - RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.defaultValue().get()); + int poolSize = sparkConf.get(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE); + int keepAliveTime = sparkConf.get(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE); threadPoolExecutor = new ThreadPoolExecutor(poolSize, poolSize * 2, keepAliveTime, TimeUnit.SECONDS, Queues.newLinkedBlockingQueue(Integer.MAX_VALUE), ThreadUtils.getThreadFactory("SendData")); @@ -229,8 +213,7 @@ public ShuffleHandle registerShuffle(int shuffleId, int numMaps, Shuff remoteStorage = ClientUtils.fetchRemoteStorage( appId, remoteStorage, dynamicConfEnabled, storageType, shuffleWriteClient); - int partitionNumPerRange = sparkConf.getInt(RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.key(), - RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.defaultValue().get()); + int partitionNumPerRange = sparkConf.get(RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE); // get all register info according to coordinator's response Set assignmentTags = RssSparkShuffleUtils.getAssignmentTags(sparkConf); @@ -324,11 +307,9 @@ public ShuffleReader getReader(ShuffleHandle handle, int startPartition, int endPartition, TaskContext context) { if (handle instanceof RssShuffleHandle) { final String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key()); - final int indexReadLimit = sparkConf.getInt(RssSparkConfig.RSS_INDEX_READ_LIMIT.key(), - RssSparkConfig.RSS_INDEX_READ_LIMIT.defaultValue().get()); + final int indexReadLimit = sparkConf.get(RssSparkConfig.RSS_INDEX_READ_LIMIT); RssShuffleHandle rssShuffleHandle = (RssShuffleHandle) handle; - final int partitionNumPerRange = sparkConf.getInt(RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.key(), - RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE.defaultValue().get()); + final int partitionNumPerRange = sparkConf.get(RssSparkConfig.RSS_PARTITION_NUM_PER_RANGE); final int partitionNum = rssShuffleHandle.getDependency().partitioner().numPartitions(); long readBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key(), RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.defaultValue().get()); diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java index 51a25452e6..982b4eeab3 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java @@ -105,14 +105,11 @@ public RssShuffleWriter( this.partitioner = shuffleDependency.partitioner(); this.shuffleManager = shuffleManager; this.shouldPartition = partitioner.numPartitions() > 1; - this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key(), - RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.defaultValue().get()); - this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key(), - RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.defaultValue().get()); + this.sendCheckTimeout = sparkConf.get(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS); + this.sendCheckInterval = sparkConf.get(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS); this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key(), RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.defaultValue().get()); - this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.key(), - RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.defaultValue().get()); + this.bitmapSplitNum = sparkConf.get(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM); this.partitionToBlockIds = Maps.newConcurrentMap(); this.shuffleWriteClient = shuffleWriteClient; this.shuffleServersForData = rssHandle.getShuffleServersForData(); diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java index cf3833ca86..33d883368f 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/DelegationRssShuffleManager.java @@ -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.key(), - RssSparkConfig.RSS_ACCESS_TIMEOUT_MS.defaultValue().get()); + accessTimeoutMs = sparkConf.get(RssSparkConfig.RSS_ACCESS_TIMEOUT_MS); if (isDriver) { coordinatorClients = RssSparkShuffleUtils.createCoordinatorClients(sparkConf); delegate = createShuffleManagerInDriver(); @@ -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.key(), - RssSparkConfig.RSS_ENABLED.defaultValue().get()); + boolean useRSS = sparkConf.get(RssSparkConfig.RSS_ENABLED); if (useRSS) { // Executor will not do any fallback shuffleManager = new RssShuffleManager(sparkConf, false); diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java index d759953912..32239b31ac 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java @@ -140,36 +140,23 @@ public RssShuffleManager(SparkConf conf, boolean isDriver) { this.sparkConf = conf; // set & check replica config - this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA.key(), - RssSparkConfig.RSS_DATA_REPLICA.defaultValue().get()); - this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key(), - RssSparkConfig.RSS_DATA_REPLICA_WRITE.defaultValue().get()); - this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ.key(), - RssSparkConfig.RSS_DATA_REPLICA_READ.defaultValue().get()); - this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.key(), - RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.defaultValue().get()); + this.dataReplica = sparkConf.get(RssSparkConfig.RSS_DATA_REPLICA); + this.dataReplicaWrite = sparkConf.get(RssSparkConfig.RSS_DATA_REPLICA_WRITE); + this.dataReplicaRead = sparkConf.get(RssSparkConfig.RSS_DATA_REPLICA_READ); + this.dataReplicaSkipEnabled = sparkConf.get(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED); LOG.info("Check quorum config [" + dataReplica + ":" + dataReplicaWrite + ":" + dataReplicaRead + ":" + dataReplicaSkipEnabled + "]"); RssUtils.checkQuorumSetting(dataReplica, dataReplicaWrite, dataReplicaRead); - this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key(), - RssSparkConfig.RSS_HEARTBEAT_INTERVAL.defaultValue().get()); + this.heartbeatInterval = sparkConf.get(RssSparkConfig.RSS_HEARTBEAT_INTERVAL); this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT.key(), heartbeatInterval / 2); - final int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key(), - RssSparkConfig.RSS_CLIENT_RETRY_MAX.defaultValue().get()); - this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key(), - RssSparkConfig.RSS_CLIENT_TYPE.defaultValue().get()); - this.dynamicConfEnabled = sparkConf.getBoolean( - RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.key(), - RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED.defaultValue().get()); - long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key(), - RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.defaultValue().get()); - int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.key(), - RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.defaultValue().get()); - this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.key(), - RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.defaultValue().get()); - this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.key(), - RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.defaultValue().get()); + final int retryMax = sparkConf.get(RssSparkConfig.RSS_CLIENT_RETRY_MAX); + this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE); + this.dynamicConfEnabled = sparkConf.get(RssSparkConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED); + long retryIntervalMax = sparkConf.get(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX); + int heartBeatThreadNum = sparkConf.get(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM); + this.dataTransferPoolSize = sparkConf.get(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE); + this.dataCommitPoolSize = sparkConf.get(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE); shuffleWriteClient = ShuffleClientFactory .getInstance() @@ -194,10 +181,8 @@ public RssShuffleManager(SparkConf conf, boolean isDriver) { LOG.info("RSS data send thread is starting"); eventLoop = defaultEventLoop; eventLoop.start(); - int poolSize = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.key(), - RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE.defaultValue().get()); - int keepAliveTime = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.key(), - RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE.defaultValue().get()); + int poolSize = sparkConf.get(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_SIZE); + int keepAliveTime = sparkConf.get(RssSparkConfig.RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE); threadPoolExecutor = new ThreadPoolExecutor(poolSize, poolSize * 2, keepAliveTime, TimeUnit.SECONDS, Queues.newLinkedBlockingQueue(Integer.MAX_VALUE)); if (isDriver) { @@ -215,33 +200,22 @@ public RssShuffleManager(SparkConf conf, boolean isDriver) { Map> taskToSuccessBlockIds, Map> taskToFailedBlockIds) { this.sparkConf = conf; - this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE.key(), - RssSparkConfig.RSS_CLIENT_TYPE.defaultValue().get()); - this.heartbeatInterval = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_INTERVAL.key(), - RssSparkConfig.RSS_HEARTBEAT_INTERVAL.defaultValue().get()); + this.clientType = sparkConf.get(RssSparkConfig.RSS_CLIENT_TYPE); + this.heartbeatInterval = sparkConf.get(RssSparkConfig.RSS_HEARTBEAT_INTERVAL); this.heartbeatTimeout = sparkConf.getLong(RssSparkConfig.RSS_HEARTBEAT_TIMEOUT.key(), heartbeatInterval / 2); - this.dataReplica = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA.key(), - RssSparkConfig.RSS_DATA_REPLICA.defaultValue().get()); - this.dataReplicaWrite = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_WRITE.key(), - RssSparkConfig.RSS_DATA_REPLICA_WRITE.defaultValue().get()); - this.dataReplicaRead = sparkConf.getInt(RssSparkConfig.RSS_DATA_REPLICA_READ.key(), - RssSparkConfig.RSS_DATA_REPLICA_READ.defaultValue().get()); - this.dataReplicaSkipEnabled = sparkConf.getBoolean(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.key(), - RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED.defaultValue().get()); + this.dataReplica = sparkConf.get(RssSparkConfig.RSS_DATA_REPLICA); + this.dataReplicaWrite = sparkConf.get(RssSparkConfig.RSS_DATA_REPLICA_WRITE); + this.dataReplicaRead = sparkConf.get(RssSparkConfig.RSS_DATA_REPLICA_READ); + this.dataReplicaSkipEnabled = sparkConf.get(RssSparkConfig.RSS_DATA_REPLICA_SKIP_ENABLED); LOG.info("Check quorum config [" + dataReplica + ":" + dataReplicaWrite + ":" + dataReplicaRead + ":" + dataReplicaSkipEnabled + "]"); RssUtils.checkQuorumSetting(dataReplica, dataReplicaWrite, dataReplicaRead); - int retryMax = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_RETRY_MAX.key(), - RssSparkConfig.RSS_CLIENT_RETRY_MAX.defaultValue().get()); - long retryIntervalMax = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.key(), - RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX.defaultValue().get()); - int heartBeatThreadNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.key(), - RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM.defaultValue().get()); - this.dataTransferPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.key(), - RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE.defaultValue().get()); - this.dataCommitPoolSize = sparkConf.getInt(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.key(), - RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE.defaultValue().get()); + int retryMax = sparkConf.get(RssSparkConfig.RSS_CLIENT_RETRY_MAX); + long retryIntervalMax = sparkConf.get(RssSparkConfig.RSS_CLIENT_RETRY_INTERVAL_MAX); + int heartBeatThreadNum = sparkConf.get(RssSparkConfig.RSS_CLIENT_HEARTBEAT_THREAD_NUM); + this.dataTransferPoolSize = sparkConf.get(RssSparkConfig.RSS_DATA_TRANSFER_POOL_SIZE); + this.dataCommitPoolSize = sparkConf.get(RssSparkConfig.RSS_DATA_COMMIT_POOL_SIZE); shuffleWriteClient = ShuffleClientFactory .getInstance() @@ -406,8 +380,7 @@ public ShuffleReader getReaderImpl( throw new RuntimeException("Unexpected ShuffleHandle:" + handle.getClass().getName()); } final String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key()); - final int indexReadLimit = sparkConf.getInt(RssSparkConfig.RSS_INDEX_READ_LIMIT.key(), - RssSparkConfig.RSS_INDEX_READ_LIMIT.defaultValue().get()); + final int indexReadLimit = sparkConf.get(RssSparkConfig.RSS_INDEX_READ_LIMIT); RssShuffleHandle rssShuffleHandle = (RssShuffleHandle) handle; final int partitionNum = rssShuffleHandle.getDependency().partitioner().numPartitions(); long readBufferSize = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_READ_BUFFER_SIZE.key(), diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java index 7dc35e6545..dbebc338f6 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java @@ -106,14 +106,11 @@ public RssShuffleWriter( this.shuffleDependency = rssHandle.getDependency(); this.partitioner = shuffleDependency.partitioner(); this.shouldPartition = partitioner.numPartitions() > 1; - this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.key(), - RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS.defaultValue().get()); - this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.key(), - RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS.defaultValue().get()); + this.sendCheckTimeout = sparkConf.get(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS); + this.sendCheckInterval = sparkConf.get(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS); this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.key(), RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT.defaultValue().get()); - this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.key(), - RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM.defaultValue().get()); + this.bitmapSplitNum = sparkConf.get(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM); this.partitionToBlockIds = Maps.newConcurrentMap(); this.shuffleWriteClient = shuffleWriteClient; this.shuffleServersForData = rssHandle.getShuffleServersForData(); From 63e28d28b71a7756a5a139885e19e89fa90d8e8e Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Sun, 31 Jul 2022 19:09:43 +0800 Subject: [PATCH 19/22] add notes and supplement doc --- .../apache/spark/shuffle/RssSparkConfig.java | 29 ++++++++++--------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index 8454cc7aa0..a392a1335b 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -39,7 +39,7 @@ public class RssSparkConfig { public static final ConfigEntry RSS_WRITER_BUFFER_SIZE = createStringBuilder( new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_WRITER_BUFFER_SIZE) - .doc("controls the buffer flushing size during shuffle write")) + .doc("Buffer size for single partition data")) .createWithDefault("3m"); public static final ConfigEntry RSS_WRITER_SERIALIZER_BUFFER_SIZE = createStringBuilder( @@ -54,7 +54,7 @@ public class RssSparkConfig { public static final ConfigEntry RSS_WRITER_BUFFER_SPILL_SIZE = createStringBuilder( new ConfigBuilder("spark.rss.writer.buffer.spill.size") - .doc("")) + .doc("Buffer size for total partition data")) .createWithDefault("128m"); public static final ConfigEntry RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE = createStringBuilder( @@ -104,7 +104,7 @@ public class RssSparkConfig { public static final ConfigEntry RSS_STORAGE_TYPE = createStringBuilder( new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_STORAGE_TYPE) - .doc("")) + .doc("Supports MEMORY_LOCALFILE, MEMORY_HDFS, MEMORY_LOCALFILE_HDFS")) .createWithDefault(""); public static final ConfigEntry RSS_CLIENT_RETRY_MAX = createIntegerBuilder( @@ -124,13 +124,14 @@ public class RssSparkConfig { public static final ConfigEntry RSS_CLIENT_SEND_SIZE_LIMIT = createStringBuilder( new ConfigBuilder("spark.rss.client.send.size.limit") - .doc("")) + .doc("The max data size sent to shuffle server")) .createWithDefault("16m"); - + // When the size of read buffer reaches the half of JVM region (i.e., 32m), + // it will incur humongous allocation, so we set it to 14m. public static final ConfigEntry RSS_CLIENT_READ_BUFFER_SIZE = createStringBuilder( new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_READ_BUFFER_SIZE) - .doc("")) + .doc("The max data size read from storage")) .createWithDefault(RssClientConfig.RSS_CLIENT_READ_BUFFER_SIZE_DEFAULT_VALUE); public static final ConfigEntry RSS_HEARTBEAT_INTERVAL = createLongBuilder( @@ -145,7 +146,7 @@ public class RssSparkConfig { public static final ConfigEntry RSS_CLIENT_SEND_THREAD_POOL_SIZE = createIntegerBuilder( new ConfigBuilder("spark.rss.client.send.threadPool.size") - .doc("")) + .doc("The thread size for send shuffle data to shuffle server")) .createWithDefault(10); public static final ConfigEntry RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE = createIntegerBuilder( @@ -155,17 +156,17 @@ public class RssSparkConfig { public static final ConfigEntry RSS_DATA_REPLICA = createIntegerBuilder( new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA) - .doc("")) + .doc("The max server number that each block can be send by client in quorum protocol")) .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_DEFAULT_VALUE); public static final ConfigEntry RSS_DATA_REPLICA_WRITE = createIntegerBuilder( new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA_WRITE) - .doc("")) + .doc("The min server number that each block should be send by client successfully")) .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_WRITE_DEFAULT_VALUE); public static final ConfigEntry RSS_DATA_REPLICA_READ = createIntegerBuilder( new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA_READ) - .doc("")) + .doc("The min server number that metadata should be fetched by client successfully")) .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_READ_DEFAULT_VALUE); public static final ConfigEntry RSS_DATA_REPLICA_SKIP_ENABLED = createBooleanBuilder( @@ -180,7 +181,7 @@ public class RssSparkConfig { public static final ConfigEntry RSS_DATA_COMMIT_POOL_SIZE = createIntegerBuilder( new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_COMMIT_POOL_SIZE) - .doc("")) + .doc("The thread size for sending commit to shuffle servers")) .createWithDefault(RssClientConfig.RSS_DATA_COMMIT_POOL_SIZE_DEFAULT_VALUE); public static final ConfigEntry RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE = createBooleanBuilder( @@ -225,12 +226,14 @@ public class RssSparkConfig { public static final ConfigEntry RSS_CLIENT_ASSIGNMENT_TAGS = createStringBuilder( new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_ASSIGNMENT_TAGS) - .doc("")) + .doc("The comma-separated list of tags for deciding assignment shuffle servers. " + + "Notice that the SHUFFLE_SERVER_VERSION will always as the assignment tag " + + "whether this conf is set or not")) .createWithDefault(""); public static final ConfigEntry RSS_COORDINATOR_QUORUM = createStringBuilder( new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_COORDINATOR_QUORUM) - .doc("")) + .doc("Coordinator quorum")) .createWithDefault(""); public static final Set RSS_MANDATORY_CLUSTER_CONF = From bc7d6b319d2b4ba72c7b5c24a09ad935d5d30747 Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Sun, 31 Jul 2022 19:10:20 +0800 Subject: [PATCH 20/22] delete sth --- .../src/main/java/org/apache/spark/shuffle/RssSparkConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index a392a1335b..632a000756 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -34,7 +34,7 @@ public class RssSparkConfig { public static final ConfigEntry RSS_PARTITION_NUM_PER_RANGE = createIntegerBuilder( new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_PARTITION_NUM_PER_RANGE) - .doc("xxxxxx")) + .doc("")) .createWithDefault(RssClientConfig.RSS_PARTITION_NUM_PER_RANGE_DEFAULT_VALUE); public static final ConfigEntry RSS_WRITER_BUFFER_SIZE = createStringBuilder( From 697d220afa5eb59f1977bb367bad1fbeae9e780f Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Sun, 31 Jul 2022 19:15:09 +0800 Subject: [PATCH 21/22] fix style --- .../main/java/org/apache/spark/shuffle/RssSparkConfig.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index 632a000756..d84eac8287 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -226,9 +226,9 @@ public class RssSparkConfig { public static final ConfigEntry RSS_CLIENT_ASSIGNMENT_TAGS = createStringBuilder( new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_ASSIGNMENT_TAGS) - .doc("The comma-separated list of tags for deciding assignment shuffle servers. " + - "Notice that the SHUFFLE_SERVER_VERSION will always as the assignment tag " + - "whether this conf is set or not")) + .doc("The comma-separated list of tags for deciding assignment shuffle servers. " + + "Notice that the SHUFFLE_SERVER_VERSION will always as the assignment tag " + + "whether this conf is set or not")) .createWithDefault(""); public static final ConfigEntry RSS_COORDINATOR_QUORUM = createStringBuilder( From c0937b5948f1fab900669bada3e623eee21f5a4c Mon Sep 17 00:00:00 2001 From: smallzhongfeng Date: Mon, 1 Aug 2022 12:15:21 +0800 Subject: [PATCH 22/22] remove doc --- .../apache/spark/shuffle/RssSparkConfig.java | 84 +++++++------------ 1 file changed, 28 insertions(+), 56 deletions(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index d84eac8287..875c9a5fbd 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -33,8 +33,7 @@ public class RssSparkConfig { public static final String SPARK_RSS_CONFIG_PREFIX = "spark."; public static final ConfigEntry RSS_PARTITION_NUM_PER_RANGE = createIntegerBuilder( - new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_PARTITION_NUM_PER_RANGE) - .doc("")) + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_PARTITION_NUM_PER_RANGE)) .createWithDefault(RssClientConfig.RSS_PARTITION_NUM_PER_RANGE_DEFAULT_VALUE); public static final ConfigEntry RSS_WRITER_BUFFER_SIZE = createStringBuilder( @@ -43,13 +42,11 @@ public class RssSparkConfig { .createWithDefault("3m"); public static final ConfigEntry RSS_WRITER_SERIALIZER_BUFFER_SIZE = createStringBuilder( - new ConfigBuilder("spark.rss.writer.serializer.buffer.size") - .doc("")) + new ConfigBuilder("spark.rss.writer.serializer.buffer.size")) .createWithDefault("3k"); public static final ConfigEntry RSS_WRITER_BUFFER_SEGMENT_SIZE = createStringBuilder( - new ConfigBuilder("spark.rss.writer.buffer.segment.size") - .doc("")) + new ConfigBuilder("spark.rss.writer.buffer.segment.size")) .createWithDefault("3k"); public static final ConfigEntry RSS_WRITER_BUFFER_SPILL_SIZE = createStringBuilder( @@ -58,48 +55,39 @@ public class RssSparkConfig { .createWithDefault("128m"); public static final ConfigEntry RSS_WRITER_PRE_ALLOCATED_BUFFER_SIZE = createStringBuilder( - new ConfigBuilder("spark.rss.writer.pre.allocated.buffer.size") - .doc("")) + new ConfigBuilder("spark.rss.writer.pre.allocated.buffer.size")) .createWithDefault("16m"); public static final ConfigEntry RSS_WRITER_REQUIRE_MEMORY_RETRY_MAX = createIntegerBuilder( - new ConfigBuilder("spark.rss.writer.require.memory.retryMax") - .doc("")) + new ConfigBuilder("spark.rss.writer.require.memory.retryMax")) .createWithDefault(1200); public static final ConfigEntry RSS_WRITER_REQUIRE_MEMORY_INTERVAL = createLongBuilder( - new ConfigBuilder("spark.rss.writer.require.memory.interval") - .doc("")) + new ConfigBuilder("spark.rss.writer.require.memory.interval")) .createWithDefault(1000L); public static final ConfigEntry RSS_CLIENT_SEND_CHECK_TIMEOUT_MS = createLongBuilder( - new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS) - .doc("")) + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS)) .createWithDefault(RssClientConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE); public static final ConfigEntry RSS_CLIENT_SEND_CHECK_INTERVAL_MS = createLongBuilder( - new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS) - .doc("")) + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS)) .createWithDefault(RssClientConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE); public static final ConfigEntry RSS_TEST_FLAG = createBooleanBuilder( - new ConfigBuilder("spark.rss.test") - .doc("")) + new ConfigBuilder("spark.rss.test")) .createWithDefault(false); public static final ConfigEntry RSS_REMOTE_STORAGE_PATH = createStringBuilder( - new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_REMOTE_STORAGE_PATH) - .doc("")) + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_REMOTE_STORAGE_PATH)) .createWithDefault(""); public static final ConfigEntry RSS_INDEX_READ_LIMIT = createIntegerBuilder( - new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_INDEX_READ_LIMIT) - .doc("")) + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_INDEX_READ_LIMIT)) .createWithDefault(RssClientConfig.RSS_INDEX_READ_LIMIT_DEFAULT_VALUE); public static final ConfigEntry RSS_CLIENT_TYPE = createStringBuilder( - new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_TYPE) - .doc("")) + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_TYPE)) .createWithDefault(RssClientConfig.RSS_CLIENT_TYPE_DEFAULT_VALUE); public static final ConfigEntry RSS_STORAGE_TYPE = createStringBuilder( @@ -108,18 +96,15 @@ public class RssSparkConfig { .createWithDefault(""); public static final ConfigEntry RSS_CLIENT_RETRY_MAX = createIntegerBuilder( - new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_RETRY_MAX) - .doc("")) + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_RETRY_MAX)) .createWithDefault(RssClientConfig.RSS_CLIENT_RETRY_MAX_DEFAULT_VALUE); public static final ConfigEntry RSS_CLIENT_RETRY_INTERVAL_MAX = createLongBuilder( - new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_RETRY_INTERVAL_MAX) - .doc("")) + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_RETRY_INTERVAL_MAX)) .createWithDefault(RssClientConfig.RSS_CLIENT_RETRY_INTERVAL_MAX_DEFAULT_VALUE); public static final ConfigEntry RSS_CLIENT_HEARTBEAT_THREAD_NUM = createIntegerBuilder( - new ConfigBuilder("spark.rss.client.heartBeat.threadNum") - .doc("")) + new ConfigBuilder("spark.rss.client.heartBeat.threadNum")) .createWithDefault(4); public static final ConfigEntry RSS_CLIENT_SEND_SIZE_LIMIT = createStringBuilder( @@ -135,13 +120,11 @@ public class RssSparkConfig { .createWithDefault(RssClientConfig.RSS_CLIENT_READ_BUFFER_SIZE_DEFAULT_VALUE); public static final ConfigEntry RSS_HEARTBEAT_INTERVAL = createLongBuilder( - new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_HEARTBEAT_INTERVAL) - .doc("")) + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_HEARTBEAT_INTERVAL)) .createWithDefault(RssClientConfig.RSS_HEARTBEAT_INTERVAL_DEFAULT_VALUE); public static final ConfigEntry RSS_HEARTBEAT_TIMEOUT = createLongBuilder( - new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_HEARTBEAT_TIMEOUT) - .doc("")) + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_HEARTBEAT_TIMEOUT)) .createWithDefault(5 * 1000L); public static final ConfigEntry RSS_CLIENT_SEND_THREAD_POOL_SIZE = createIntegerBuilder( @@ -150,8 +133,7 @@ public class RssSparkConfig { .createWithDefault(10); public static final ConfigEntry RSS_CLIENT_SEND_THREAD_POOL_KEEPALIVE = createIntegerBuilder( - new ConfigBuilder("spark.rss.client.send.threadPool.keepalive") - .doc("")) + new ConfigBuilder("spark.rss.client.send.threadPool.keepalive")) .createWithDefault(60); public static final ConfigEntry RSS_DATA_REPLICA = createIntegerBuilder( @@ -170,13 +152,11 @@ public class RssSparkConfig { .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_READ_DEFAULT_VALUE); public static final ConfigEntry RSS_DATA_REPLICA_SKIP_ENABLED = createBooleanBuilder( - new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA_SKIP_ENABLED) - .doc("")) + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_REPLICA_SKIP_ENABLED)) .createWithDefault(RssClientConfig.RSS_DATA_REPLICA_SKIP_ENABLED_DEFAULT_VALUE); public static final ConfigEntry RSS_DATA_TRANSFER_POOL_SIZE = createIntegerBuilder( - new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_TRANSFER_POOL_SIZE) - .doc("")) + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DATA_TRANSFER_POOL_SIZE)) .createWithDefault(RssClientConfig.RSS_DATA_TRANFER_POOL_SIZE_DEFAULT_VALUE); public static final ConfigEntry RSS_DATA_COMMIT_POOL_SIZE = createIntegerBuilder( @@ -185,43 +165,35 @@ public class RssSparkConfig { .createWithDefault(RssClientConfig.RSS_DATA_COMMIT_POOL_SIZE_DEFAULT_VALUE); public static final ConfigEntry RSS_OZONE_DFS_NAMENODE_ODFS_ENABLE = createBooleanBuilder( - new ConfigBuilder("spark.rss.ozone.dfs.namenode.odfs.enable") - .doc("")) + new ConfigBuilder("spark.rss.ozone.dfs.namenode.odfs.enable")) .createWithDefault(false); public static final ConfigEntry RSS_OZONE_FS_HDFS_IMPL = createStringBuilder( - new ConfigBuilder("spark.rss.ozone.fs.hdfs.impl") - .doc("")) + new ConfigBuilder("spark.rss.ozone.fs.hdfs.impl")) .createWithDefault("org.apache.hadoop.odfs.HdfsOdfsFilesystem"); public static final ConfigEntry RSS_OZONE_FS_ABSTRACT_FILE_SYSTEM_HDFS_IMPL = createStringBuilder( - new ConfigBuilder("spark.rss.ozone.fs.AbstractFileSystem.hdfs.impl") - .doc("")) + new ConfigBuilder("spark.rss.ozone.fs.AbstractFileSystem.hdfs.impl")) .createWithDefault("org.apache.hadoop.odfs.HdfsOdfs"); public static final ConfigEntry RSS_CLIENT_BITMAP_SPLIT_NUM = createIntegerBuilder( - new ConfigBuilder("spark.rss.client.bitmap.splitNum") - .doc("")) + new ConfigBuilder("spark.rss.client.bitmap.splitNum")) .createWithDefault(1); public static final ConfigEntry RSS_ACCESS_ID = createStringBuilder( - new ConfigBuilder("spark.rss.access.id") - .doc("")) + new ConfigBuilder("spark.rss.access.id")) .createWithDefault(""); public static final ConfigEntry RSS_ACCESS_TIMEOUT_MS = createIntegerBuilder( - new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_ACCESS_TIMEOUT_MS) - .doc("")) + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_ACCESS_TIMEOUT_MS)) .createWithDefault(RssClientConfig.RSS_ACCESS_TIMEOUT_MS_DEFAULT_VALUE); public static final ConfigEntry RSS_ENABLED = createBooleanBuilder( - new ConfigBuilder("spark.rss.enabled") - .doc("")) + new ConfigBuilder("spark.rss.enabled")) .createWithDefault(false); public static final ConfigEntry RSS_DYNAMIC_CLIENT_CONF_ENABLED = createBooleanBuilder( - new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED) - .doc("")) + new ConfigBuilder(SPARK_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED)) .createWithDefault(RssClientConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED_DEFAULT_VALUE); public static final ConfigEntry RSS_CLIENT_ASSIGNMENT_TAGS = createStringBuilder(