diff --git a/core/trino-main/src/main/java/io/trino/FeaturesConfig.java b/core/trino-main/src/main/java/io/trino/FeaturesConfig.java index 1acbaabdd4e7..d2c5363961dc 100644 --- a/core/trino-main/src/main/java/io/trino/FeaturesConfig.java +++ b/core/trino-main/src/main/java/io/trino/FeaturesConfig.java @@ -14,7 +14,6 @@ package io.trino; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; @@ -276,10 +275,11 @@ public List getSpillerSpillPaths() } @Config("spiller-spill-path") - public FeaturesConfig setSpillerSpillPaths(String spillPaths) + public FeaturesConfig setSpillerSpillPaths(List spillPaths) { - List spillPathsSplit = ImmutableList.copyOf(Splitter.on(",").trimResults().omitEmptyStrings().split(spillPaths)); - this.spillerSpillPaths = spillPathsSplit.stream().map(Paths::get).collect(toImmutableList()); + this.spillerSpillPaths = spillPaths.stream() + .map(Paths::get) + .collect(toImmutableList()); return this; } diff --git a/core/trino-main/src/main/java/io/trino/connector/FileCatalogStoreConfig.java b/core/trino-main/src/main/java/io/trino/connector/FileCatalogStoreConfig.java index 79bfc2cb0ba0..e52ebfb0bb52 100644 --- a/core/trino-main/src/main/java/io/trino/connector/FileCatalogStoreConfig.java +++ b/core/trino-main/src/main/java/io/trino/connector/FileCatalogStoreConfig.java @@ -13,7 +13,6 @@ */ package io.trino.connector; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import io.airlift.configuration.LegacyConfig; @@ -24,10 +23,8 @@ public class FileCatalogStoreConfig { - private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); - private File catalogConfigurationDir = new File("etc/catalog/"); - private List disabledCatalogs; + private List disabledCatalogs = ImmutableList.of(); private boolean readOnly; @NotNull @@ -50,15 +47,9 @@ public List getDisabledCatalogs() } @Config("catalog.disabled-catalogs") - public FileCatalogStoreConfig setDisabledCatalogs(String catalogs) - { - this.disabledCatalogs = (catalogs == null) ? null : SPLITTER.splitToList(catalogs); - return this; - } - public FileCatalogStoreConfig setDisabledCatalogs(List catalogs) { - this.disabledCatalogs = (catalogs == null) ? null : ImmutableList.copyOf(catalogs); + this.disabledCatalogs = ImmutableList.copyOf(catalogs); return this; } diff --git a/core/trino-main/src/main/java/io/trino/connector/StaticCatalogManagerConfig.java b/core/trino-main/src/main/java/io/trino/connector/StaticCatalogManagerConfig.java index 0484685aee15..b2f6b5076f8b 100644 --- a/core/trino-main/src/main/java/io/trino/connector/StaticCatalogManagerConfig.java +++ b/core/trino-main/src/main/java/io/trino/connector/StaticCatalogManagerConfig.java @@ -13,7 +13,6 @@ */ package io.trino.connector; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import io.airlift.configuration.LegacyConfig; @@ -24,10 +23,8 @@ public class StaticCatalogManagerConfig { - private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); - private File catalogConfigurationDir = new File("etc/catalog/"); - private List disabledCatalogs; + private List disabledCatalogs = ImmutableList.of(); @NotNull public File getCatalogConfigurationDir() @@ -49,15 +46,9 @@ public List getDisabledCatalogs() } @Config("catalog.disabled-catalogs") - public StaticCatalogManagerConfig setDisabledCatalogs(String catalogs) - { - this.disabledCatalogs = (catalogs == null) ? null : SPLITTER.splitToList(catalogs); - return this; - } - public StaticCatalogManagerConfig setDisabledCatalogs(List catalogs) { - this.disabledCatalogs = (catalogs == null) ? null : ImmutableList.copyOf(catalogs); + this.disabledCatalogs = ImmutableList.copyOf(catalogs); return this; } } diff --git a/core/trino-main/src/main/java/io/trino/eventlistener/EventListenerConfig.java b/core/trino-main/src/main/java/io/trino/eventlistener/EventListenerConfig.java index 8cd69fe1ce0f..5f9ef30ef294 100644 --- a/core/trino-main/src/main/java/io/trino/eventlistener/EventListenerConfig.java +++ b/core/trino-main/src/main/java/io/trino/eventlistener/EventListenerConfig.java @@ -13,7 +13,6 @@ */ package io.trino.eventlistener; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import io.airlift.configuration.validation.FileExists; @@ -26,7 +25,6 @@ public class EventListenerConfig { - private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); private List eventListenerFiles = ImmutableList.of(); @NotNull @@ -36,17 +34,11 @@ public class EventListenerConfig } @Config("event-listener.config-files") - public EventListenerConfig setEventListenerFiles(String eventListenerFiles) + public EventListenerConfig setEventListenerFiles(List eventListenerFiles) { - this.eventListenerFiles = SPLITTER.splitToList(eventListenerFiles).stream() + this.eventListenerFiles = eventListenerFiles.stream() .map(File::new) .collect(toImmutableList()); return this; } - - public EventListenerConfig setEventListenerFiles(List eventListenerFiles) - { - this.eventListenerFiles = ImmutableList.copyOf(eventListenerFiles); - return this; - } } diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/SubnetTopologyConfig.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/SubnetTopologyConfig.java index 31510ff8198e..83451d9b023d 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/SubnetTopologyConfig.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/SubnetTopologyConfig.java @@ -13,13 +13,11 @@ */ package io.trino.execution.scheduler; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import java.util.List; -import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.execution.scheduler.SubnetBasedTopology.AddressProtocol.IPv4; public class SubnetTopologyConfig @@ -33,11 +31,9 @@ public List getCidrPrefixLengths() } @Config("node-scheduler.network-topology.subnet.cidr-prefix-lengths") - public SubnetTopologyConfig setCidrPrefixLengths(String commaSeparatedLengths) + public SubnetTopologyConfig setCidrPrefixLengths(List cidrPrefixLengths) { - this.cidrPrefixLengths = Splitter.on(',').omitEmptyStrings().splitToList(commaSeparatedLengths).stream() - .map(Integer::parseInt) - .collect(toImmutableList()); + this.cidrPrefixLengths = ImmutableList.copyOf(cidrPrefixLengths); return this; } diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/TopologyAwareNodeSelectorConfig.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/TopologyAwareNodeSelectorConfig.java index b2a68a7321ae..ef397b9d4b1d 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/TopologyAwareNodeSelectorConfig.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/TopologyAwareNodeSelectorConfig.java @@ -13,15 +13,12 @@ */ package io.trino.execution.scheduler; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import jakarta.validation.constraints.NotNull; import java.util.List; -import static java.util.Objects.requireNonNull; - public class TopologyAwareNodeSelectorConfig { public enum TopologyType @@ -51,19 +48,10 @@ public List getLocationSegmentNames() return locationSegmentNames; } - public TopologyAwareNodeSelectorConfig setLocationSegmentNames(List locationSegmentNames) - { - this.locationSegmentNames = requireNonNull(locationSegmentNames, "locationSegmentNames is null"); - return this; - } - @Config("node-scheduler.network-topology.segments") - public TopologyAwareNodeSelectorConfig setLocationSegmentNames(String locationSegmentNames) + public TopologyAwareNodeSelectorConfig setLocationSegmentNames(List locationSegmentNames) { - this.locationSegmentNames = Splitter.on(",") - .trimResults() - .omitEmptyStrings() - .splitToList(locationSegmentNames); + this.locationSegmentNames = ImmutableList.copyOf(locationSegmentNames); return this; } } diff --git a/core/trino-main/src/main/java/io/trino/security/AccessControlConfig.java b/core/trino-main/src/main/java/io/trino/security/AccessControlConfig.java index 6879ee330db3..70b8d56a1d34 100644 --- a/core/trino-main/src/main/java/io/trino/security/AccessControlConfig.java +++ b/core/trino-main/src/main/java/io/trino/security/AccessControlConfig.java @@ -13,7 +13,6 @@ */ package io.trino.security; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import io.airlift.configuration.validation.FileExists; @@ -26,7 +25,6 @@ public class AccessControlConfig { - private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); private List accessControlFiles = ImmutableList.of(); @NotNull @@ -36,17 +34,11 @@ public class AccessControlConfig } @Config("access-control.config-files") - public AccessControlConfig setAccessControlFiles(String accessControlFiles) + public AccessControlConfig setAccessControlFiles(List accessControlFiles) { - this.accessControlFiles = SPLITTER.splitToList(accessControlFiles).stream() + this.accessControlFiles = accessControlFiles.stream() .map(File::new) .collect(toImmutableList()); return this; } - - public AccessControlConfig setAccessControlFiles(List accessControlFiles) - { - this.accessControlFiles = ImmutableList.copyOf(accessControlFiles); - return this; - } } diff --git a/core/trino-main/src/main/java/io/trino/server/security/PasswordAuthenticatorConfig.java b/core/trino-main/src/main/java/io/trino/server/security/PasswordAuthenticatorConfig.java index 08632a23bc4f..a3caac610b4e 100644 --- a/core/trino-main/src/main/java/io/trino/server/security/PasswordAuthenticatorConfig.java +++ b/core/trino-main/src/main/java/io/trino/server/security/PasswordAuthenticatorConfig.java @@ -13,7 +13,6 @@ */ package io.trino.server.security; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; @@ -29,7 +28,6 @@ public class PasswordAuthenticatorConfig { - private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); private Optional userMappingPattern = Optional.empty(); private Optional userMappingFile = Optional.empty(); private List passwordAuthenticatorFiles = ImmutableList.of(new File("etc/password-authenticator.properties")); @@ -67,9 +65,9 @@ public PasswordAuthenticatorConfig setUserMappingFile(File userMappingFile) @Config("password-authenticator.config-files") @ConfigDescription("Ordered list of password authenticator config files") - public PasswordAuthenticatorConfig setPasswordAuthenticatorFiles(String passwordAuthenticatorFiles) + public PasswordAuthenticatorConfig setPasswordAuthenticatorFiles(List passwordAuthenticatorFiles) { - this.passwordAuthenticatorFiles = SPLITTER.splitToList(passwordAuthenticatorFiles).stream() + this.passwordAuthenticatorFiles = passwordAuthenticatorFiles.stream() .map(File::new) .collect(toImmutableList()); return this; diff --git a/core/trino-main/src/main/java/io/trino/server/security/SecurityConfig.java b/core/trino-main/src/main/java/io/trino/server/security/SecurityConfig.java index c547ebb78029..17bfeefc12f7 100644 --- a/core/trino-main/src/main/java/io/trino/server/security/SecurityConfig.java +++ b/core/trino-main/src/main/java/io/trino/server/security/SecurityConfig.java @@ -13,7 +13,6 @@ */ package io.trino.server.security; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; @@ -30,8 +29,6 @@ "dispatcher.forwarded-header"}) public class SecurityConfig { - private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); - private boolean insecureAuthenticationOverHttpAllowed = true; private List authenticationTypes = ImmutableList.of("insecure"); private Optional fixedManagementUser = Optional.empty(); @@ -57,17 +54,11 @@ public List getAuthenticationTypes() return authenticationTypes; } - public SecurityConfig setAuthenticationTypes(List authenticationTypes) - { - this.authenticationTypes = ImmutableList.copyOf(authenticationTypes); - return this; - } - @Config("http-server.authentication.type") @ConfigDescription("Ordered list of authentication types") - public SecurityConfig setAuthenticationTypes(String types) + public SecurityConfig setAuthenticationTypes(List types) { - authenticationTypes = Optional.ofNullable(types).map(SPLITTER::splitToList).orElse(null); + authenticationTypes = ImmutableList.copyOf(types); return this; } diff --git a/core/trino-main/src/main/java/io/trino/server/security/oauth2/OAuth2Config.java b/core/trino-main/src/main/java/io/trino/server/security/oauth2/OAuth2Config.java index f0c575290076..0fa00d306290 100644 --- a/core/trino-main/src/main/java/io/trino/server/security/oauth2/OAuth2Config.java +++ b/core/trino-main/src/main/java/io/trino/server/security/oauth2/OAuth2Config.java @@ -13,7 +13,6 @@ */ package io.trino.server.security.oauth2; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import io.airlift.configuration.Config; @@ -32,7 +31,6 @@ import java.util.Set; import java.util.concurrent.TimeUnit; -import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.trino.server.security.oauth2.OAuth2Service.OPENID_SCOPE; public class OAuth2Config @@ -132,9 +130,9 @@ public Set getScopes() @Config("http-server.authentication.oauth2.scopes") @ConfigDescription("Scopes requested by the server during OAuth2 authorization challenge") - public OAuth2Config setScopes(String scopes) + public OAuth2Config setScopes(Set scopes) { - this.scopes = Splitter.on(',').trimResults().omitEmptyStrings().splitToStream(scopes).collect(toImmutableSet()); + this.scopes = ImmutableSet.copyOf(scopes); return this; } diff --git a/core/trino-main/src/test/java/io/trino/connector/TestFileCatalogStoreConfig.java b/core/trino-main/src/test/java/io/trino/connector/TestFileCatalogStoreConfig.java index 92a123ca26cf..60c22e5b56ba 100644 --- a/core/trino-main/src/test/java/io/trino/connector/TestFileCatalogStoreConfig.java +++ b/core/trino-main/src/test/java/io/trino/connector/TestFileCatalogStoreConfig.java @@ -31,7 +31,7 @@ public void testDefaults() { assertRecordedDefaults(recordDefaults(FileCatalogStoreConfig.class) .setCatalogConfigurationDir(new File("etc/catalog")) - .setDisabledCatalogs((String) null) + .setDisabledCatalogs(ImmutableList.of()) .setReadOnly(false)); } diff --git a/core/trino-main/src/test/java/io/trino/connector/TestStaticCatalogManagerConfig.java b/core/trino-main/src/test/java/io/trino/connector/TestStaticCatalogManagerConfig.java index 66b83e2ac0b5..a8a32b1b797f 100644 --- a/core/trino-main/src/test/java/io/trino/connector/TestStaticCatalogManagerConfig.java +++ b/core/trino-main/src/test/java/io/trino/connector/TestStaticCatalogManagerConfig.java @@ -31,7 +31,7 @@ public void testDefaults() { assertRecordedDefaults(recordDefaults(StaticCatalogManagerConfig.class) .setCatalogConfigurationDir(new File("etc/catalog")) - .setDisabledCatalogs((String) null)); + .setDisabledCatalogs(ImmutableList.of())); } @Test diff --git a/core/trino-main/src/test/java/io/trino/eventlistener/TestEventListenerConfig.java b/core/trino-main/src/test/java/io/trino/eventlistener/TestEventListenerConfig.java index eb4e56a9aadd..20ccadc9df1a 100644 --- a/core/trino-main/src/test/java/io/trino/eventlistener/TestEventListenerConfig.java +++ b/core/trino-main/src/test/java/io/trino/eventlistener/TestEventListenerConfig.java @@ -32,7 +32,7 @@ public class TestEventListenerConfig public void testDefaults() { assertRecordedDefaults(ConfigAssertions.recordDefaults(EventListenerConfig.class) - .setEventListenerFiles("")); + .setEventListenerFiles(ImmutableList.of())); } @Test @@ -45,7 +45,7 @@ public void testExplicitPropertyMappings() Map properties = ImmutableMap.of("event-listener.config-files", config1.toString() + "," + config2.toString()); EventListenerConfig expected = new EventListenerConfig() - .setEventListenerFiles(ImmutableList.of(config1.toFile(), config2.toFile())); + .setEventListenerFiles(ImmutableList.of(config1.toFile().getPath(), config2.toFile().getPath())); assertFullMapping(properties, expected); } diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSubnetTopologyConfig.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSubnetTopologyConfig.java index 681f7caf5089..b4fa94c93faf 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSubnetTopologyConfig.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSubnetTopologyConfig.java @@ -13,6 +13,7 @@ */ package io.trino.execution.scheduler; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.configuration.testing.ConfigAssertions; import org.junit.jupiter.api.Test; @@ -29,7 +30,7 @@ public class TestSubnetTopologyConfig public void testDefaults() { ConfigAssertions.assertRecordedDefaults(ConfigAssertions.recordDefaults(SubnetTopologyConfig.class) - .setCidrPrefixLengths("") + .setCidrPrefixLengths(ImmutableList.of()) .setAddressProtocol(IPv4)); } @@ -43,7 +44,7 @@ public void testExplicitPropertyMappings() .buildOrThrow(); SubnetTopologyConfig expected = new SubnetTopologyConfig() - .setCidrPrefixLengths("24,26") + .setCidrPrefixLengths(ImmutableList.of(24, 26)) .setAddressProtocol(IPv6); ConfigAssertions.assertFullMapping(properties, expected); diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestTopologyAwareNodeSelectorConfig.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestTopologyAwareNodeSelectorConfig.java index ba50998a1d93..2d63fe00978c 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestTopologyAwareNodeSelectorConfig.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestTopologyAwareNodeSelectorConfig.java @@ -28,7 +28,7 @@ public void testDefaults() { ConfigAssertions.assertRecordedDefaults(ConfigAssertions.recordDefaults(TopologyAwareNodeSelectorConfig.class) .setType(TopologyType.FLAT) - .setLocationSegmentNames("machine")); + .setLocationSegmentNames(ImmutableList.of("machine"))); } @Test diff --git a/core/trino-main/src/test/java/io/trino/security/TestAccessControlConfig.java b/core/trino-main/src/test/java/io/trino/security/TestAccessControlConfig.java index c97d1e8c50df..471e1a89d268 100644 --- a/core/trino-main/src/test/java/io/trino/security/TestAccessControlConfig.java +++ b/core/trino-main/src/test/java/io/trino/security/TestAccessControlConfig.java @@ -29,7 +29,7 @@ public class TestAccessControlConfig public void testDefaults() { ConfigAssertions.assertRecordedDefaults(ConfigAssertions.recordDefaults(AccessControlConfig.class) - .setAccessControlFiles("")); + .setAccessControlFiles(ImmutableList.of())); } @Test @@ -42,7 +42,7 @@ public void testExplicitPropertyMappings() Map properties = ImmutableMap.of("access-control.config-files", config1.toString() + "," + config2.toString()); AccessControlConfig expected = new AccessControlConfig() - .setAccessControlFiles(ImmutableList.of(config1.toFile(), config2.toFile())); + .setAccessControlFiles(ImmutableList.of(config1.toFile().getAbsolutePath(), config2.toFile().getAbsolutePath())); ConfigAssertions.assertFullMapping(properties, expected); } diff --git a/core/trino-main/src/test/java/io/trino/security/TestAccessControlManager.java b/core/trino-main/src/test/java/io/trino/security/TestAccessControlManager.java index 8c442a2eef35..5af5ab760fc7 100644 --- a/core/trino-main/src/test/java/io/trino/security/TestAccessControlManager.java +++ b/core/trino-main/src/test/java/io/trino/security/TestAccessControlManager.java @@ -392,7 +392,7 @@ private static AccessControlManager createAccessControlManager(TestingEventListe return createAccessControlManager( eventListenerManager, - new AccessControlConfig().setAccessControlFiles(accessControlConfigPath)); + new AccessControlConfig().setAccessControlFiles(ImmutableList.of(accessControlConfigPath))); } private static AccessControlManager createAccessControlManager(TransactionManager testTransactionManager) diff --git a/core/trino-main/src/test/java/io/trino/server/security/TestPasswordAuthenticatorConfig.java b/core/trino-main/src/test/java/io/trino/server/security/TestPasswordAuthenticatorConfig.java index 4531b245be70..e01c1b498181 100644 --- a/core/trino-main/src/test/java/io/trino/server/security/TestPasswordAuthenticatorConfig.java +++ b/core/trino-main/src/test/java/io/trino/server/security/TestPasswordAuthenticatorConfig.java @@ -13,6 +13,7 @@ */ package io.trino.server.security; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; @@ -33,7 +34,7 @@ public void testDefaults() assertRecordedDefaults(recordDefaults(PasswordAuthenticatorConfig.class) .setUserMappingPattern(null) .setUserMappingFile(null) - .setPasswordAuthenticatorFiles("etc/password-authenticator.properties")); + .setPasswordAuthenticatorFiles(ImmutableList.of("etc/password-authenticator.properties"))); } @Test @@ -53,7 +54,7 @@ public void testExplicitPropertyMappings() PasswordAuthenticatorConfig expected = new PasswordAuthenticatorConfig() .setUserMappingPattern("(.*)@something") .setUserMappingFile(userMappingFile.toFile()) - .setPasswordAuthenticatorFiles(config1 + "," + config2); + .setPasswordAuthenticatorFiles(ImmutableList.of(config1.toAbsolutePath().toString(), config2.toAbsolutePath().toString())); assertFullMapping(properties, expected); } diff --git a/core/trino-main/src/test/java/io/trino/server/security/TestPasswordAuthenticatorManager.java b/core/trino-main/src/test/java/io/trino/server/security/TestPasswordAuthenticatorManager.java index 0b981989fcf6..09ff0acaca71 100644 --- a/core/trino-main/src/test/java/io/trino/server/security/TestPasswordAuthenticatorManager.java +++ b/core/trino-main/src/test/java/io/trino/server/security/TestPasswordAuthenticatorManager.java @@ -41,7 +41,7 @@ public void testMultipleConfigFiles() Files.write(config2, ImmutableList.of("password-authenticator.name=type2")); PasswordAuthenticatorManager manager = new PasswordAuthenticatorManager(new PasswordAuthenticatorConfig() - .setPasswordAuthenticatorFiles(config1.toAbsolutePath() + "," + config2.toAbsolutePath())); + .setPasswordAuthenticatorFiles(ImmutableList.of(config1.toAbsolutePath().toString(), config2.toAbsolutePath().toString()))); manager.setRequired(); manager.addPasswordAuthenticatorFactory(new TestingPasswordAuthenticatorFactory("type1", "password1")); manager.addPasswordAuthenticatorFactory(new TestingPasswordAuthenticatorFactory("type2", "password2")); diff --git a/core/trino-main/src/test/java/io/trino/server/security/TestSecurityConfig.java b/core/trino-main/src/test/java/io/trino/server/security/TestSecurityConfig.java index 975d92a6554a..4d783b4996a1 100644 --- a/core/trino-main/src/test/java/io/trino/server/security/TestSecurityConfig.java +++ b/core/trino-main/src/test/java/io/trino/server/security/TestSecurityConfig.java @@ -29,7 +29,7 @@ public class TestSecurityConfig public void testDefaults() { assertRecordedDefaults(recordDefaults(SecurityConfig.class) - .setAuthenticationTypes("insecure") + .setAuthenticationTypes(ImmutableList.of("insecure")) .setInsecureAuthenticationOverHttpAllowed(true) .setFixedManagementUser(null) .setFixedManagementUserForHttps(false)); diff --git a/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestOAuth2Config.java b/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestOAuth2Config.java index 2c0eb66e3709..1a7da2496360 100644 --- a/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestOAuth2Config.java +++ b/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestOAuth2Config.java @@ -14,6 +14,7 @@ package io.trino.server.security.oauth2; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import io.airlift.units.Duration; import org.junit.jupiter.api.Test; @@ -40,7 +41,7 @@ public void testDefaults() .setIssuer(null) .setClientId(null) .setClientSecret(null) - .setScopes("openid") + .setScopes(ImmutableSet.of("openid")) .setChallengeTimeout(new Duration(15, MINUTES)) .setPrincipalField("sub") .setGroupsField(null) @@ -81,7 +82,7 @@ public void testExplicitPropertyMappings() .setIssuer("http://127.0.0.1:9000/oauth2") .setClientId("another-consumer") .setClientSecret("consumer-secret") - .setScopes("email, offline") + .setScopes(ImmutableSet.of("email", "offline")) .setPrincipalField("some-field") .setGroupsField("groups") .setAdditionalAudiences(List.of("test-aud1", "test-aud2")) diff --git a/core/trino-main/src/test/java/io/trino/spiller/TestBinaryFileSpiller.java b/core/trino-main/src/test/java/io/trino/spiller/TestBinaryFileSpiller.java index 924d821836bd..1aadf0aa64ae 100644 --- a/core/trino-main/src/test/java/io/trino/spiller/TestBinaryFileSpiller.java +++ b/core/trino-main/src/test/java/io/trino/spiller/TestBinaryFileSpiller.java @@ -77,7 +77,7 @@ public void setUp() { spillerStats = new SpillerStats(); FeaturesConfig featuresConfig = new FeaturesConfig(); - featuresConfig.setSpillerSpillPaths(spillPath.getAbsolutePath()); + featuresConfig.setSpillerSpillPaths(ImmutableList.of(spillPath.getAbsolutePath())); featuresConfig.setSpillMaxUsedSpaceThreshold(1.0); NodeSpillConfig nodeSpillConfig = new NodeSpillConfig(); BlockEncodingSerde blockEncodingSerde = new TestingBlockEncodingSerde(); diff --git a/core/trino-main/src/test/java/io/trino/spiller/TestGenericPartitioningSpiller.java b/core/trino-main/src/test/java/io/trino/spiller/TestGenericPartitioningSpiller.java index a297a1f7b4fb..17214e24145f 100644 --- a/core/trino-main/src/test/java/io/trino/spiller/TestGenericPartitioningSpiller.java +++ b/core/trino-main/src/test/java/io/trino/spiller/TestGenericPartitioningSpiller.java @@ -78,7 +78,7 @@ public void setUp() { tempDirectory = createTempDirectory(getClass().getSimpleName()); FeaturesConfig featuresConfig = new FeaturesConfig(); - featuresConfig.setSpillerSpillPaths(tempDirectory.toString()); + featuresConfig.setSpillerSpillPaths(ImmutableList.of(tempDirectory.toString())); featuresConfig.setSpillerThreads(8); featuresConfig.setSpillMaxUsedSpaceThreshold(1.0); SingleStreamSpillerFactory singleStreamSpillerFactory = new FileSingleStreamSpillerFactory( diff --git a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestFeaturesConfig.java b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestFeaturesConfig.java index cf268923d19d..bd7a6d271d95 100644 --- a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestFeaturesConfig.java +++ b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestFeaturesConfig.java @@ -13,6 +13,7 @@ */ package io.trino.sql.analyzer; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.units.DataSize; import io.trino.FeaturesConfig; @@ -47,7 +48,7 @@ public void testDefaults() .setRe2JDfaRetries(5) .setSpillEnabled(false) .setAggregationOperatorUnspillMemoryLimit(DataSize.valueOf("4MB")) - .setSpillerSpillPaths("") + .setSpillerSpillPaths(ImmutableList.of()) .setSpillerThreads(4) .setSpillMaxUsedSpaceThreshold(0.9) .setMemoryRevokingThreshold(0.9) @@ -112,7 +113,7 @@ public void testExplicitPropertyMappings() .setRe2JDfaRetries(42) .setSpillEnabled(true) .setAggregationOperatorUnspillMemoryLimit(DataSize.valueOf("100MB")) - .setSpillerSpillPaths("/tmp/custom/spill/path1,/tmp/custom/spill/path2") + .setSpillerSpillPaths(ImmutableList.of("/tmp/custom/spill/path1", "/tmp/custom/spill/path2")) .setSpillerThreads(42) .setSpillMaxUsedSpaceThreshold(0.8) .setMemoryRevokingThreshold(0.2) diff --git a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCacheConfig.java b/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCacheConfig.java index ae776a205283..a45a03ace579 100644 --- a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCacheConfig.java +++ b/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCacheConfig.java @@ -13,7 +13,6 @@ */ package io.trino.filesystem.alluxio; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; @@ -29,18 +28,13 @@ import java.util.List; import java.util.Optional; -import static com.google.common.base.MoreObjects.firstNonNull; -import static com.google.common.collect.ImmutableList.toImmutableList; - public class AlluxioFileSystemCacheConfig { - private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); - static final String CACHE_DIRECTORIES = "fs.cache.directories"; static final String CACHE_MAX_SIZES = "fs.cache.max-sizes"; static final String CACHE_MAX_PERCENTAGES = "fs.cache.max-disk-usage-percentages"; - private List cacheDirectories; + private List cacheDirectories = ImmutableList.of(); private List maxCacheSizes = ImmutableList.of(); private Optional cacheTTL = Optional.of(Duration.valueOf("7d")); private List maxCacheDiskUsagePercentages = ImmutableList.of(); @@ -54,9 +48,9 @@ public List getCacheDirectories() @Config(CACHE_DIRECTORIES) @ConfigDescription("Base directory to cache data. Use a comma-separated list to cache data in multiple directories.") - public AlluxioFileSystemCacheConfig setCacheDirectories(String cacheDirectories) + public AlluxioFileSystemCacheConfig setCacheDirectories(List cacheDirectories) { - this.cacheDirectories = cacheDirectories == null ? null : SPLITTER.splitToList(cacheDirectories); + this.cacheDirectories = ImmutableList.copyOf(cacheDirectories); return this; } @@ -67,9 +61,9 @@ public List getMaxCacheSizes() @Config(CACHE_MAX_SIZES) @ConfigDescription("The maximum cache size for a cache directory. Use a comma-separated list of sizes to specify allowed maximum values for each directory.") - public AlluxioFileSystemCacheConfig setMaxCacheSizes(String maxCacheSizes) + public AlluxioFileSystemCacheConfig setMaxCacheSizes(List maxCacheSizes) { - this.maxCacheSizes = SPLITTER.splitToStream(firstNonNull(maxCacheSizes, "")).map(DataSize::valueOf).collect(toImmutableList()); + this.maxCacheSizes = ImmutableList.copyOf(maxCacheSizes); return this; } @@ -100,11 +94,9 @@ public AlluxioFileSystemCacheConfig disableTTL() @Config(CACHE_MAX_PERCENTAGES) @ConfigDescription("The maximum percentage (0-100) of total disk size the cache can use. Use a comma-separated list of percentage values if supplying several cache directories.") - public AlluxioFileSystemCacheConfig setMaxCacheDiskUsagePercentages(String maxCacheDiskUsagePercentages) + public AlluxioFileSystemCacheConfig setMaxCacheDiskUsagePercentages(List maxCacheDiskUsagePercentages) { - this.maxCacheDiskUsagePercentages = SPLITTER.splitToStream(firstNonNull(maxCacheDiskUsagePercentages, "")) - .map(Integer::valueOf) - .collect(toImmutableList()); + this.maxCacheDiskUsagePercentages = ImmutableList.copyOf(maxCacheDiskUsagePercentages); return this; } diff --git a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioCacheFileSystem.java b/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioCacheFileSystem.java index 7811a4036a61..b831b0015afb 100644 --- a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioCacheFileSystem.java +++ b/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioCacheFileSystem.java @@ -13,6 +13,7 @@ */ package io.trino.filesystem.alluxio; +import com.google.common.collect.ImmutableList; import io.airlift.units.DataSize; import io.trino.filesystem.AbstractTestTrinoFileSystem; import io.trino.filesystem.Location; @@ -49,10 +50,10 @@ void beforeAll() Path cacheDirectory = tempDirectory.resolve("cache"); Files.createDirectory(cacheDirectory); AlluxioFileSystemCacheConfig configuration = new AlluxioFileSystemCacheConfig() - .setCacheDirectories(cacheDirectory.toAbsolutePath().toString()) + .setCacheDirectories(ImmutableList.of(cacheDirectory.toAbsolutePath().toString())) .setCachePageSize(DataSize.valueOf("32003B")) .disableTTL() - .setMaxCacheSizes("100MB"); + .setMaxCacheSizes(ImmutableList.of(DataSize.valueOf("100MB"))); memoryFileSystem = new IncompleteStreamMemoryFileSystem(); cache = new AlluxioFileSystemCache(noopTracer(), configuration, new AlluxioCacheStats()); fileSystem = new CacheFileSystem(memoryFileSystem, cache, new DefaultCacheKeyProvider()); diff --git a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioCacheFileSystemAccessOperations.java b/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioCacheFileSystemAccessOperations.java index f2a0839267b3..37bce4b3ff9f 100644 --- a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioCacheFileSystemAccessOperations.java +++ b/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioCacheFileSystemAccessOperations.java @@ -17,6 +17,7 @@ import alluxio.client.file.cache.PageStore; import alluxio.client.file.cache.store.PageStoreOptions; import com.google.common.collect.HashMultiset; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMultiset; import com.google.common.collect.Iterables; import com.google.common.collect.Multiset; @@ -83,10 +84,10 @@ public void setUp() Path cacheDirectory = Files.createDirectory(tempDirectory.resolve("cache")); AlluxioFileSystemCacheConfig configuration = new AlluxioFileSystemCacheConfig() - .setCacheDirectories(cacheDirectory.toAbsolutePath().toString()) + .setCacheDirectories(ImmutableList.of(cacheDirectory.toAbsolutePath().toString())) .disableTTL() .setCachePageSize(DataSize.ofBytes(PAGE_SIZE)) - .setMaxCacheSizes(DataSize.ofBytes(CACHE_SIZE).toBytesValueString()); + .setMaxCacheSizes(ImmutableList.of(DataSize.ofBytes(CACHE_SIZE))); tracingFileSystemFactory = new TracingFileSystemFactory(testingTelemetry.getTracer(), new MemoryFileSystemFactory()); alluxioCache = new AlluxioFileSystemCache(testingTelemetry.getTracer(), configuration, new AlluxioCacheStats()); diff --git a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioFileSystemCacheConfig.java b/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioFileSystemCacheConfig.java index 4958e7691399..957332086ab6 100644 --- a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioFileSystemCacheConfig.java +++ b/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioFileSystemCacheConfig.java @@ -13,6 +13,7 @@ */ package io.trino.filesystem.alluxio; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.units.DataSize; import io.airlift.units.Duration; @@ -38,23 +39,23 @@ public void testInvalidConfiguration() assertThatThrownBy(() -> AlluxioConfigurationFactory.create( new AlluxioFileSystemCacheConfig() - .setCacheDirectories("/cache1,/cache2") - .setMaxCacheDiskUsagePercentages("0") - .setMaxCacheSizes("1B"))) + .setCacheDirectories(ImmutableList.of("/cache1", "/cache2")) + .setMaxCacheDiskUsagePercentages(ImmutableList.of(0)) + .setMaxCacheSizes(ImmutableList.of(DataSize.valueOf("1B"))))) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Either fs.cache.max-sizes or fs.cache.max-disk-usage-percentages must be specified"); assertThatThrownBy(() -> AlluxioConfigurationFactory.create( new AlluxioFileSystemCacheConfig() - .setCacheDirectories("/cache1,/cache2") - .setMaxCacheSizes("1B"))) + .setCacheDirectories(ImmutableList.of("/cache1", "/cache2")) + .setMaxCacheSizes(ImmutableList.of(DataSize.valueOf("1B"))))) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("fs.cache.directories and fs.cache.max-sizes must have the same size"); assertThatThrownBy(() -> AlluxioConfigurationFactory.create( new AlluxioFileSystemCacheConfig() - .setCacheDirectories("/cache1,/cache2") - .setMaxCacheDiskUsagePercentages("0"))) + .setCacheDirectories(ImmutableList.of("/cache1", "/cache2")) + .setMaxCacheDiskUsagePercentages(ImmutableList.of(0)))) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("fs.cache.directories and fs.cache.max-disk-usage-percentages must have the same size"); } @@ -63,10 +64,10 @@ public void testInvalidConfiguration() void testDefaults() { assertRecordedDefaults(recordDefaults(AlluxioFileSystemCacheConfig.class) - .setCacheDirectories(null) + .setCacheDirectories(ImmutableList.of()) .setCachePageSize(DataSize.valueOf("1MB")) - .setMaxCacheSizes(null) - .setMaxCacheDiskUsagePercentages(null) + .setMaxCacheSizes(ImmutableList.of()) + .setMaxCacheDiskUsagePercentages(ImmutableList.of()) .setCacheTTL(Duration.valueOf("7d"))); } @@ -85,10 +86,10 @@ public void testExplicitPropertyMappings() .buildOrThrow(); AlluxioFileSystemCacheConfig expected = new AlluxioFileSystemCacheConfig() - .setCacheDirectories(cacheDirectory.toString()) + .setCacheDirectories(ImmutableList.of(cacheDirectory.toString())) .setCachePageSize(DataSize.valueOf("7MB")) - .setMaxCacheSizes("1GB") - .setMaxCacheDiskUsagePercentages("50") + .setMaxCacheSizes(ImmutableList.of(DataSize.valueOf("1GB"))) + .setMaxCacheDiskUsagePercentages(ImmutableList.of(50)) .setCacheTTL(Duration.valueOf("1d")); assertFullMapping(properties, expected); diff --git a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestFuzzAlluxioCacheFileSystem.java b/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestFuzzAlluxioCacheFileSystem.java index 192b039a6103..fd64558ec0ae 100644 --- a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestFuzzAlluxioCacheFileSystem.java +++ b/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestFuzzAlluxioCacheFileSystem.java @@ -13,6 +13,7 @@ */ package io.trino.filesystem.alluxio; +import com.google.common.collect.ImmutableList; import io.airlift.slice.Slice; import io.airlift.slice.Slices; import io.airlift.tracing.Tracing; @@ -34,13 +35,14 @@ import java.util.Random; import static java.lang.Math.min; +import static java.lang.Math.toIntExact; import static org.assertj.core.api.Assertions.assertThat; @TestInstance(Lifecycle.PER_METHOD) public class TestFuzzAlluxioCacheFileSystem { - private static final int CACHE_SIZE = 8 * 1024; - private static final int PAGE_SIZE = 128; + private static final DataSize CACHE_SIZE = DataSize.ofBytes(8 * 1024); + private static final DataSize PAGE_SIZE = DataSize.ofBytes(128); @Test public void testFuzzTrinoInputReadFully() @@ -91,7 +93,7 @@ private void fuzzTrinoInputOperation(CreateTrinoInput createInput, TrinoI Location expectedLocation = expectedFileSystemState.tempLocation(); Location testLocation = actualFileSystemState.tempLocation(); - int fileSize = random.nextInt(0, CACHE_SIZE / 2); + int fileSize = random.nextInt(0, toIntExact(CACHE_SIZE.toBytes() / 2)); createTestFile(expectedFileSystem, expectedLocation, fileSize); createTestFile(testFileSystem, testLocation, fileSize); @@ -179,10 +181,10 @@ public TrinoFileSystem create() Path cacheDirectory = Files.createDirectory(tempDirectory.resolve("cache")); AlluxioFileSystemCacheConfig configuration = new AlluxioFileSystemCacheConfig() - .setCacheDirectories(cacheDirectory.toAbsolutePath().toString()) - .setCachePageSize(DataSize.ofBytes(PAGE_SIZE)) + .setCacheDirectories(ImmutableList.of(cacheDirectory.toAbsolutePath().toString())) + .setCachePageSize(PAGE_SIZE) .disableTTL() - .setMaxCacheSizes(CACHE_SIZE + "B"); + .setMaxCacheSizes(ImmutableList.of(CACHE_SIZE)); AlluxioFileSystemCache alluxioCache = new AlluxioFileSystemCache(Tracing.noopTracer(), configuration, new AlluxioCacheStats()); return new CacheFileSystem(new IncompleteStreamMemoryFileSystem(), alluxioCache, new TestingCacheKeyProvider()); } diff --git a/lib/trino-hdfs/src/main/java/io/trino/hdfs/HdfsConfig.java b/lib/trino-hdfs/src/main/java/io/trino/hdfs/HdfsConfig.java index c0e9518be0d1..5292ca3ed3b8 100644 --- a/lib/trino-hdfs/src/main/java/io/trino/hdfs/HdfsConfig.java +++ b/lib/trino-hdfs/src/main/java/io/trino/hdfs/HdfsConfig.java @@ -13,7 +13,6 @@ */ package io.trino.hdfs; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import com.google.common.net.HostAndPort; import com.google.common.primitives.Shorts; @@ -39,8 +38,6 @@ public class HdfsConfig { public static final String SKIP_DIR_PERMISSIONS = "skip"; - private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); - private List resourceConfigFiles = ImmutableList.of(); private String newDirectoryPermissions = "0777"; private boolean newFileInheritOwnership; @@ -63,20 +60,14 @@ public class HdfsConfig } @Config("hive.config.resources") - public HdfsConfig setResourceConfigFiles(String files) + public HdfsConfig setResourceConfigFiles(List files) { - this.resourceConfigFiles = SPLITTER.splitToList(files).stream() + this.resourceConfigFiles = files.stream() .map(File::new) .collect(toImmutableList()); return this; } - public HdfsConfig setResourceConfigFiles(List files) - { - this.resourceConfigFiles = ImmutableList.copyOf(files); - return this; - } - public Optional getNewDirectoryFsPermissions() { if (newDirectoryPermissions.equalsIgnoreCase(HdfsConfig.SKIP_DIR_PERMISSIONS)) { diff --git a/lib/trino-hdfs/src/test/java/io/trino/hdfs/TestHdfsConfig.java b/lib/trino-hdfs/src/test/java/io/trino/hdfs/TestHdfsConfig.java index 03bf0b6219ff..5ef3a41aa9eb 100644 --- a/lib/trino-hdfs/src/test/java/io/trino/hdfs/TestHdfsConfig.java +++ b/lib/trino-hdfs/src/test/java/io/trino/hdfs/TestHdfsConfig.java @@ -37,7 +37,7 @@ public class TestHdfsConfig public void testDefaults() { assertRecordedDefaults(recordDefaults(HdfsConfig.class) - .setResourceConfigFiles("") + .setResourceConfigFiles(ImmutableList.of()) .setNewDirectoryPermissions("0777") .setNewFileInheritOwnership(false) .setVerifyChecksum(true) @@ -78,7 +78,7 @@ public void testExplicitPropertyMappings() .buildOrThrow(); HdfsConfig expected = new HdfsConfig() - .setResourceConfigFiles(ImmutableList.of(resource1.toFile(), resource2.toFile())) + .setResourceConfigFiles(ImmutableList.of(resource1.toFile().getAbsolutePath(), resource2.toFile().getAbsolutePath())) .setNewDirectoryPermissions("0700") .setNewFileInheritOwnership(true) .setVerifyChecksum(false) diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcConfig.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcConfig.java index 44efaa48390d..1ff349e65a0a 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcConfig.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcConfig.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.jdbc; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableSet; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; @@ -27,7 +26,6 @@ import java.util.Optional; import java.util.Set; -import static com.google.common.base.Strings.nullToEmpty; import static jakarta.validation.constraints.Pattern.Flag.CASE_INSENSITIVE; import static java.util.concurrent.TimeUnit.SECONDS; @@ -70,9 +68,9 @@ public Set getJdbcTypesMappedToVarchar() } @Config("jdbc-types-mapped-to-varchar") - public BaseJdbcConfig setJdbcTypesMappedToVarchar(String jdbcTypesMappedToVarchar) + public BaseJdbcConfig setJdbcTypesMappedToVarchar(Set jdbcTypesMappedToVarchar) { - this.jdbcTypesMappedToVarchar = ImmutableSet.copyOf(Splitter.on(",").omitEmptyStrings().trimResults().split(nullToEmpty(jdbcTypesMappedToVarchar))); + this.jdbcTypesMappedToVarchar = ImmutableSet.copyOf(jdbcTypesMappedToVarchar); return this; } diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestBaseJdbcConfig.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestBaseJdbcConfig.java index a8dfaba46f99..45bb27892d51 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestBaseJdbcConfig.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestBaseJdbcConfig.java @@ -14,6 +14,7 @@ package io.trino.plugin.jdbc; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import io.airlift.configuration.ConfigurationFactory; import io.airlift.units.Duration; import jakarta.validation.constraints.AssertTrue; @@ -38,7 +39,7 @@ public void testDefaults() { assertRecordedDefaults(recordDefaults(BaseJdbcConfig.class) .setConnectionUrl(null) - .setJdbcTypesMappedToVarchar("") + .setJdbcTypesMappedToVarchar(ImmutableSet.of()) .setMetadataCacheTtl(ZERO) .setSchemaNamesCacheTtl(null) .setTableNamesCacheTtl(null) @@ -63,7 +64,7 @@ public void testExplicitPropertyMappings() BaseJdbcConfig expected = new BaseJdbcConfig() .setConnectionUrl("jdbc:h2:mem:config") - .setJdbcTypesMappedToVarchar("mytype, struct_type1") + .setJdbcTypesMappedToVarchar(ImmutableSet.of("mytype", "struct_type1")) .setMetadataCacheTtl(new Duration(1, SECONDS)) .setSchemaNamesCacheTtl(new Duration(2, SECONDS)) .setTableNamesCacheTtl(new Duration(3, SECONDS)) diff --git a/plugin/trino-cassandra/src/main/java/io/trino/plugin/cassandra/CassandraClientConfig.java b/plugin/trino-cassandra/src/main/java/io/trino/plugin/cassandra/CassandraClientConfig.java index a8a3de53ad61..c476eda31514 100644 --- a/plugin/trino-cassandra/src/main/java/io/trino/plugin/cassandra/CassandraClientConfig.java +++ b/plugin/trino-cassandra/src/main/java/io/trino/plugin/cassandra/CassandraClientConfig.java @@ -17,7 +17,6 @@ import com.datastax.oss.driver.api.core.DefaultConsistencyLevel; import com.datastax.oss.driver.api.core.DefaultProtocolVersion; import com.datastax.oss.driver.api.core.ProtocolVersion; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; @@ -33,7 +32,6 @@ import jakarta.validation.constraints.Size; import java.io.File; -import java.util.Arrays; import java.util.List; import java.util.Optional; @@ -57,8 +55,6 @@ }) public class CassandraClientConfig { - private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); - private ConsistencyLevel consistencyLevel = ConsistencyLevel.ONE; private int fetchSize = 5_000; private List contactPoints = ImmutableList.of(); @@ -96,15 +92,9 @@ public List getContactPoints() } @Config("cassandra.contact-points") - public CassandraClientConfig setContactPoints(String commaSeparatedList) - { - this.contactPoints = SPLITTER.splitToList(commaSeparatedList); - return this; - } - - public CassandraClientConfig setContactPoints(String... contactPoints) + public CassandraClientConfig setContactPoints(List contactPoints) { - this.contactPoints = Arrays.asList(contactPoints); + this.contactPoints = ImmutableList.copyOf(contactPoints); return this; } diff --git a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraClientConfig.java b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraClientConfig.java index ada4ca86577c..ac6619239d49 100644 --- a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraClientConfig.java +++ b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraClientConfig.java @@ -15,6 +15,7 @@ import com.datastax.oss.driver.api.core.DefaultConsistencyLevel; import com.datastax.oss.driver.api.core.DefaultProtocolVersion; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.units.Duration; import org.junit.jupiter.api.Test; @@ -39,7 +40,7 @@ public void testDefaults() assertRecordedDefaults(recordDefaults(CassandraClientConfig.class) .setFetchSize(5_000) .setConsistencyLevel(DefaultConsistencyLevel.ONE) - .setContactPoints("") + .setContactPoints(ImmutableList.of()) .setNativeProtocolPort(9042) .setPartitionSizeForBatchSelect(100) .setSplitSize(1_024) @@ -106,7 +107,7 @@ public void testExplicitPropertyMappings() .buildOrThrow(); CassandraClientConfig expected = new CassandraClientConfig() - .setContactPoints("host1", "host2") + .setContactPoints(ImmutableList.of("host1", "host2")) .setNativeProtocolPort(9999) .setFetchSize(10_000) .setConsistencyLevel(DefaultConsistencyLevel.TWO) diff --git a/plugin/trino-exchange-hdfs/src/main/java/io/trino/plugin/exchange/hdfs/ExchangeHdfsConfig.java b/plugin/trino-exchange-hdfs/src/main/java/io/trino/plugin/exchange/hdfs/ExchangeHdfsConfig.java index ab3e59ee6c87..092ba2688056 100644 --- a/plugin/trino-exchange-hdfs/src/main/java/io/trino/plugin/exchange/hdfs/ExchangeHdfsConfig.java +++ b/plugin/trino-exchange-hdfs/src/main/java/io/trino/plugin/exchange/hdfs/ExchangeHdfsConfig.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.exchange.hdfs; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; @@ -57,9 +56,9 @@ public ExchangeHdfsConfig setHdfsStorageBlockSize(DataSize hdfsStorageBlockSize) } @Config("hdfs.config.resources") - public ExchangeHdfsConfig setResourceConfigFiles(String files) + public ExchangeHdfsConfig setResourceConfigFiles(List files) { - this.resourceConfigFiles = Splitter.on(',').trimResults().omitEmptyStrings().splitToList(files).stream() + this.resourceConfigFiles = files.stream() .map(File::new) .collect(toImmutableList()); return this; diff --git a/plugin/trino-exchange-hdfs/src/test/java/io/trino/plugin/exchange/hdfs/TestExchangeHdfsConfig.java b/plugin/trino-exchange-hdfs/src/test/java/io/trino/plugin/exchange/hdfs/TestExchangeHdfsConfig.java index c35ef7a9644b..190258e93c81 100644 --- a/plugin/trino-exchange-hdfs/src/test/java/io/trino/plugin/exchange/hdfs/TestExchangeHdfsConfig.java +++ b/plugin/trino-exchange-hdfs/src/test/java/io/trino/plugin/exchange/hdfs/TestExchangeHdfsConfig.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.exchange.hdfs; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.units.DataSize; import org.junit.jupiter.api.Test; @@ -33,7 +34,7 @@ public class TestExchangeHdfsConfig public void testDefaults() { assertRecordedDefaults(recordDefaults(ExchangeHdfsConfig.class) - .setResourceConfigFiles("") + .setResourceConfigFiles(ImmutableList.of()) .setHdfsStorageBlockSize(DataSize.of(4, MEGABYTE))); } @@ -50,7 +51,7 @@ public void testExplicitPropertyMappings() .buildOrThrow(); ExchangeHdfsConfig expected = new ExchangeHdfsConfig() - .setResourceConfigFiles(resource1 + "," + resource2) + .setResourceConfigFiles(ImmutableList.of(resource1.toString(), resource2.toString())) .setHdfsStorageBlockSize(DataSize.of(8, MEGABYTE)); assertFullMapping(properties, expected); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java index 5995e37e34a6..5bb13734d3d4 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.hive; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import io.airlift.configuration.Config; @@ -39,6 +38,7 @@ import java.util.Set; import java.util.TimeZone; +import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.airlift.units.DataSize.Unit.GIGABYTE; import static io.airlift.units.DataSize.Unit.KILOBYTE; import static io.airlift.units.DataSize.Unit.MEGABYTE; @@ -68,8 +68,6 @@ public class HiveConfig { public static final String CONFIGURATION_HIVE_PARTITION_PROJECTION_ENABLED = "hive.partition-projection-enabled"; - private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); - private boolean singleStatementWritesOnly; private DataSize maxSplitSize = DataSize.of(64, MEGABYTE); @@ -767,9 +765,9 @@ public List getFileStatusCacheTables() } @Config("hive.file-status-cache-tables") - public HiveConfig setFileStatusCacheTables(String fileStatusCacheTables) + public HiveConfig setFileStatusCacheTables(List fileStatusCacheTables) { - this.fileStatusCacheTables = SPLITTER.splitToList(fileStatusCacheTables); + this.fileStatusCacheTables = ImmutableList.copyOf(fileStatusCacheTables); return this; } @@ -1117,9 +1115,11 @@ public Set getQueryPartitionFilterRequiredSchemas() @Config("hive.query-partition-filter-required-schemas") @ConfigDescription("List of schemas for which filter on partition column is enforced") - public HiveConfig setQueryPartitionFilterRequiredSchemas(String queryPartitionFilterRequiredSchemas) + public HiveConfig setQueryPartitionFilterRequiredSchemas(List queryPartitionFilterRequiredSchemas) { - this.queryPartitionFilterRequiredSchemas = ImmutableSet.copyOf(SPLITTER.splitToList(queryPartitionFilterRequiredSchemas.toLowerCase(ENGLISH))); + this.queryPartitionFilterRequiredSchemas = queryPartitionFilterRequiredSchemas.stream() + .map(value -> value.toLowerCase(ENGLISH)) + .collect(toImmutableSet()); return this; } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/StaticMetastoreConfig.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/StaticMetastoreConfig.java index 99bbe53acf79..b4cbab73c2c6 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/StaticMetastoreConfig.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/StaticMetastoreConfig.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.hive.metastore.thrift; -import com.google.common.base.Splitter; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; import jakarta.validation.constraints.AssertFalse; @@ -23,14 +22,10 @@ import java.util.List; import static com.google.common.collect.ImmutableList.toImmutableList; -import static com.google.common.collect.Streams.stream; public class StaticMetastoreConfig { public static final String HIVE_METASTORE_USERNAME = "hive.metastore.username"; - - private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); - private List metastoreUris; private String metastoreUsername; @@ -42,14 +37,14 @@ public List getMetastoreUris() @Config("hive.metastore.uri") @ConfigDescription("Hive metastore URIs (comma separated)") - public StaticMetastoreConfig setMetastoreUris(String uris) + public StaticMetastoreConfig setMetastoreUris(List uris) { if (uris == null) { this.metastoreUris = null; return this; } - this.metastoreUris = stream(SPLITTER.split(uris)) + this.metastoreUris = uris.stream() .map(URI::create) .collect(toImmutableList()); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java index ab0aa4cd3447..670d3ac0e259 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.hive; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.units.DataSize; import io.airlift.units.DataSize.Unit; @@ -97,7 +98,7 @@ public void testDefaults() .setDelegateTransactionalManagedTableLocationToMetastore(false) .setFileStatusCacheExpireAfterWrite(new Duration(1, TimeUnit.MINUTES)) .setFileStatusCacheMaxRetainedSize(DataSize.of(1, GIGABYTE)) - .setFileStatusCacheTables("") + .setFileStatusCacheTables(ImmutableList.of()) .setPerTransactionFileStatusCacheMaxRetainedSize(DataSize.of(100, MEGABYTE)) .setTranslateHiveViews(false) .setLegacyHiveViewTranslation(false) @@ -106,7 +107,7 @@ public void testDefaults() .setHiveTransactionHeartbeatThreads(5) .setAllowRegisterPartition(false) .setQueryPartitionFilterRequired(false) - .setQueryPartitionFilterRequiredSchemas("") + .setQueryPartitionFilterRequiredSchemas(ImmutableList.of()) .setProjectionPushdownEnabled(true) .setDynamicFilteringWaitTimeout(new Duration(0, TimeUnit.MINUTES)) .setTimestampPrecision(HiveTimestampPrecision.DEFAULT_PRECISION) @@ -262,7 +263,7 @@ public void testExplicitPropertyMappings() .setTemporaryStagingDirectoryEnabled(false) .setTemporaryStagingDirectoryPath("updated") .setDelegateTransactionalManagedTableLocationToMetastore(true) - .setFileStatusCacheTables("foo.bar1,foo.bar2") + .setFileStatusCacheTables(ImmutableList.of("foo.bar1", "foo.bar2")) .setFileStatusCacheMaxRetainedSize(DataSize.ofBytes(1000)) .setFileStatusCacheExpireAfterWrite(new Duration(30, TimeUnit.MINUTES)) .setPerTransactionFileStatusCacheMaxRetainedSize(DataSize.ofBytes(42)) @@ -273,7 +274,7 @@ public void testExplicitPropertyMappings() .setHiveTransactionHeartbeatThreads(10) .setAllowRegisterPartition(true) .setQueryPartitionFilterRequired(true) - .setQueryPartitionFilterRequiredSchemas("foo, bar") + .setQueryPartitionFilterRequiredSchemas(ImmutableList.of("foo", "bar")) .setProjectionPushdownEnabled(false) .setDynamicFilteringWaitTimeout(new Duration(10, TimeUnit.SECONDS)) .setTimestampPrecision(HiveTimestampPrecision.NANOSECONDS) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestStaticMetastoreConfig.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestStaticMetastoreConfig.java index 9cb092b7ae71..265186f19daf 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestStaticMetastoreConfig.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestStaticMetastoreConfig.java @@ -44,7 +44,7 @@ public void testExplicitPropertyMappingsSingleMetastore() .buildOrThrow(); StaticMetastoreConfig expected = new StaticMetastoreConfig() - .setMetastoreUris("thrift://localhost:9083") + .setMetastoreUris(ImmutableList.of("thrift://localhost:9083")) .setMetastoreUsername("trino"); assertFullMapping(properties, expected); @@ -61,7 +61,7 @@ public void testExplicitPropertyMappingsMultipleMetastores() .buildOrThrow(); StaticMetastoreConfig expected = new StaticMetastoreConfig() - .setMetastoreUris("thrift://localhost:9083,thrift://192.0.2.3:8932") + .setMetastoreUris(ImmutableList.of("thrift://localhost:9083", "thrift://192.0.2.3:8932")) .setMetastoreUsername("trino"); assertFullMapping(properties, expected); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestStaticTokenAwareMetastoreClientFactory.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestStaticTokenAwareMetastoreClientFactory.java index 4eed9b49a139..8cf08a7aa005 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestStaticTokenAwareMetastoreClientFactory.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestStaticTokenAwareMetastoreClientFactory.java @@ -13,8 +13,8 @@ */ package io.trino.plugin.hive.metastore.thrift; -import com.google.common.base.Joiner; import com.google.common.base.Ticker; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.testing.TestingTicker; import io.trino.hive.thrift.metastore.Table; @@ -39,17 +39,17 @@ public class TestStaticTokenAwareMetastoreClientFactory private static final String FALLBACK2_URI = "thrift://fallback2:8090"; private static final StaticMetastoreConfig CONFIG_WITH_FALLBACK = new StaticMetastoreConfig() - .setMetastoreUris(Joiner.on(',').join(DEFAULT_URI, FALLBACK_URI, FALLBACK2_URI)); + .setMetastoreUris(ImmutableList.of(DEFAULT_URI, FALLBACK_URI, FALLBACK2_URI)); private static final StaticMetastoreConfig CONFIG_WITHOUT_FALLBACK = new StaticMetastoreConfig() - .setMetastoreUris(DEFAULT_URI); + .setMetastoreUris(ImmutableList.of(DEFAULT_URI)); private static final StaticMetastoreConfig CONFIG_WITH_FALLBACK_WITH_USER = new StaticMetastoreConfig() - .setMetastoreUris(Joiner.on(',').join(DEFAULT_URI, FALLBACK_URI, FALLBACK2_URI)) + .setMetastoreUris(ImmutableList.of(DEFAULT_URI, FALLBACK_URI, FALLBACK2_URI)) .setMetastoreUsername("presto"); private static final StaticMetastoreConfig CONFIG_WITHOUT_FALLBACK_WITH_USER = new StaticMetastoreConfig() - .setMetastoreUris(DEFAULT_URI) + .setMetastoreUris(ImmutableList.of(DEFAULT_URI)) .setMetastoreUsername("presto"); private static final Map> CLIENTS = ImmutableMap.of(DEFAULT_URI, Optional.of(DEFAULT_CLIENT), FALLBACK_URI, Optional.of(FALLBACK_CLIENT)); diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java index 306287f9ff9f..4479e72c49ea 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.hudi; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; @@ -26,7 +25,6 @@ import java.util.List; -import static com.google.common.base.Strings.nullToEmpty; import static com.google.common.collect.ImmutableList.toImmutableList; import static io.airlift.units.DataSize.Unit.MEGABYTE; import static java.util.Locale.ENGLISH; @@ -38,8 +36,6 @@ }) public class HudiConfig { - private static final Splitter COMMA_SPLITTER = Splitter.on(",").omitEmptyStrings().trimResults(); - private List columnsToHide = ImmutableList.of(); private boolean shouldUseParquetColumnNames = true; private boolean sizeBasedSplitWeightsEnabled = true; @@ -60,9 +56,9 @@ public List getColumnsToHide() @Config("hudi.columns-to-hide") @ConfigDescription("List of column names that will be hidden from the query output. " + "It can be used to hide Hudi meta fields. By default, no fields are hidden.") - public HudiConfig setColumnsToHide(String columnsToHide) + public HudiConfig setColumnsToHide(List columnsToHide) { - this.columnsToHide = COMMA_SPLITTER.splitToStream(nullToEmpty(columnsToHide)) + this.columnsToHide = columnsToHide.stream() .map(s -> s.toLowerCase(ENGLISH)) .collect(toImmutableList()); return this; diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java index 719ef64bce1a..fea569c0d99a 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.hudi; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.units.DataSize; import org.junit.jupiter.api.Test; @@ -30,7 +31,7 @@ public class TestHudiConfig public void testDefaults() { assertRecordedDefaults(recordDefaults(HudiConfig.class) - .setColumnsToHide(null) + .setColumnsToHide(ImmutableList.of()) .setUseParquetColumnNames(true) .setSizeBasedSplitWeightsEnabled(true) .setStandardSplitWeightSize(DataSize.of(128, MEGABYTE)) @@ -61,7 +62,7 @@ public void testExplicitPropertyMappings() .buildOrThrow(); HudiConfig expected = new HudiConfig() - .setColumnsToHide("_hoodie_record_key") + .setColumnsToHide(ImmutableList.of("_hoodie_record_key")) .setUseParquetColumnNames(false) .setSizeBasedSplitWeightsEnabled(false) .setStandardSplitWeightSize(DataSize.of(64, MEGABYTE)) diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSessionProperties.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSessionProperties.java index d36fb8085225..c711a324b06d 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSessionProperties.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSessionProperties.java @@ -28,7 +28,7 @@ public class TestHudiSessionProperties public void testSessionPropertyColumnsToHide() { HudiConfig config = new HudiConfig() - .setColumnsToHide("col1, col2"); + .setColumnsToHide(ImmutableList.of("col1", "col2")); HudiSessionProperties sessionProperties = new HudiSessionProperties(config, new ParquetReaderConfig()); ConnectorSession session = TestingConnectorSession.builder() .setPropertyMetadata(sessionProperties.getSessionProperties()) diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaConfig.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaConfig.java index d75cd380b86c..c34f18c57ddf 100644 --- a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaConfig.java +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaConfig.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.kafka; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import io.airlift.configuration.Config; @@ -35,7 +34,6 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static com.google.common.collect.Streams.stream; @DefunctConfig("kafka.connect-timeout") public class KafkaConfig @@ -60,9 +58,11 @@ public Set getNodes() @Config("kafka.nodes") @ConfigDescription("Seed nodes for Kafka cluster. At least one must exist") - public KafkaConfig setNodes(String nodes) + public KafkaConfig setNodes(Set nodes) { - this.nodes = (nodes == null) ? null : parseNodes(nodes); + this.nodes = nodes.stream() + .map(KafkaConfig::toHostAddress) + .collect(toImmutableSet()); return this; } @@ -120,14 +120,6 @@ public KafkaConfig setHideInternalColumns(boolean hideInternalColumns) return this; } - private static ImmutableSet parseNodes(String nodes) - { - Splitter splitter = Splitter.on(',').omitEmptyStrings().trimResults(); - return stream(splitter.split(nodes)) - .map(KafkaConfig::toHostAddress) - .collect(toImmutableSet()); - } - private static HostAddress toHostAddress(String value) { return HostAddress.fromString(value).withDefaultPort(KAFKA_DEFAULT_PORT); diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/schema/confluent/ConfluentSchemaRegistryConfig.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/schema/confluent/ConfluentSchemaRegistryConfig.java index a484d7fd8c50..1da627d7da7b 100644 --- a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/schema/confluent/ConfluentSchemaRegistryConfig.java +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/schema/confluent/ConfluentSchemaRegistryConfig.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.kafka.schema.confluent; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableSet; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; @@ -29,13 +28,12 @@ import java.util.Set; import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static com.google.common.collect.Streams.stream; import static io.trino.plugin.kafka.schema.confluent.AvroSchemaConverter.EmptyFieldStrategy.IGNORE; import static java.util.concurrent.TimeUnit.SECONDS; public class ConfluentSchemaRegistryConfig { - private Set confluentSchemaRegistryUrls; + private Set confluentSchemaRegistryUrls = ImmutableSet.of(); private int confluentSchemaRegistryClientCacheSize = 1000; private EmptyFieldStrategy emptyFieldStrategy = IGNORE; private Duration confluentSubjectsCacheRefreshInterval = new Duration(1, SECONDS); @@ -48,9 +46,11 @@ public Set getConfluentSchemaRegistryUrls() @Config("kafka.confluent-schema-registry-url") @ConfigDescription("The url of the Confluent Schema Registry") - public ConfluentSchemaRegistryConfig setConfluentSchemaRegistryUrls(String confluentSchemaRegistryUrls) + public ConfluentSchemaRegistryConfig setConfluentSchemaRegistryUrls(Set confluentSchemaRegistryUrls) { - this.confluentSchemaRegistryUrls = (confluentSchemaRegistryUrls == null) ? null : parseNodes(confluentSchemaRegistryUrls); + this.confluentSchemaRegistryUrls = confluentSchemaRegistryUrls.stream() + .map(ConfluentSchemaRegistryConfig::toHostAddress) + .collect(toImmutableSet()); return this; } @@ -97,14 +97,6 @@ public ConfluentSchemaRegistryConfig setConfluentSubjectsCacheRefreshInterval(Du return this; } - private static ImmutableSet parseNodes(String nodes) - { - Splitter splitter = Splitter.on(',').omitEmptyStrings().trimResults(); - return stream(splitter.split(nodes)) - .map(ConfluentSchemaRegistryConfig::toHostAddress) - .collect(toImmutableSet()); - } - private static HostAddress toHostAddress(String value) { return HostAddress.fromString(value); diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/schema/file/FileTableDescriptionSupplierConfig.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/schema/file/FileTableDescriptionSupplierConfig.java index 4f1b68bd23ce..a4052fe3654e 100644 --- a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/schema/file/FileTableDescriptionSupplierConfig.java +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/schema/file/FileTableDescriptionSupplierConfig.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.kafka.schema.file; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableSet; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; @@ -35,9 +34,9 @@ public Set getTableNames() @Config("kafka.table-names") @ConfigDescription("Set of tables known to this connector") - public FileTableDescriptionSupplierConfig setTableNames(String tableNames) + public FileTableDescriptionSupplierConfig setTableNames(Set tableNames) { - this.tableNames = ImmutableSet.copyOf(Splitter.on(',').omitEmptyStrings().trimResults().split(tableNames)); + this.tableNames = ImmutableSet.copyOf(tableNames); return this; } diff --git a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaConfig.java b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaConfig.java index 625fdfec865e..8d21afa75d94 100644 --- a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaConfig.java +++ b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaConfig.java @@ -15,6 +15,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import io.trino.plugin.kafka.schema.file.FileTableDescriptionSupplier; import org.junit.jupiter.api.Test; @@ -34,7 +35,7 @@ public class TestKafkaConfig public void testDefaults() { assertRecordedDefaults(recordDefaults(KafkaConfig.class) - .setNodes("") + .setNodes(ImmutableSet.of()) .setKafkaBufferSize("64kB") .setDefaultSchema("default") .setTableDescriptionSupplier(FileTableDescriptionSupplier.NAME) @@ -67,7 +68,7 @@ public void testExplicitPropertyMappings() KafkaConfig expected = new KafkaConfig() .setDefaultSchema("kafka") .setTableDescriptionSupplier("test") - .setNodes("localhost:12345, localhost:23456") + .setNodes(ImmutableSet.of("localhost:12345", "localhost:23456")) .setKafkaBufferSize("1MB") .setHideInternalColumns(false) .setMessagesPerSplit(1) diff --git a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/schema/confluent/TestConfluentSchemaRegistryConfig.java b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/schema/confluent/TestConfluentSchemaRegistryConfig.java index d807877e915e..d5e23fe40ca1 100644 --- a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/schema/confluent/TestConfluentSchemaRegistryConfig.java +++ b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/schema/confluent/TestConfluentSchemaRegistryConfig.java @@ -14,6 +14,7 @@ package io.trino.plugin.kafka.schema.confluent; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import io.airlift.units.Duration; import org.junit.jupiter.api.Test; @@ -32,7 +33,7 @@ public class TestConfluentSchemaRegistryConfig public void testDefaults() { assertRecordedDefaults(recordDefaults(ConfluentSchemaRegistryConfig.class) - .setConfluentSchemaRegistryUrls(null) + .setConfluentSchemaRegistryUrls(ImmutableSet.of()) .setConfluentSchemaRegistryClientCacheSize(1000) .setEmptyFieldStrategy(IGNORE) .setConfluentSubjectsCacheRefreshInterval(new Duration(1, SECONDS))); @@ -49,7 +50,7 @@ public void testExplicitPropertyMappings() .buildOrThrow(); ConfluentSchemaRegistryConfig expected = new ConfluentSchemaRegistryConfig() - .setConfluentSchemaRegistryUrls("http://schema-registry-a:8081, http://schema-registry-b:8081") + .setConfluentSchemaRegistryUrls(ImmutableSet.of("http://schema-registry-a:8081", "http://schema-registry-b:8081")) .setConfluentSchemaRegistryClientCacheSize(1500) .setEmptyFieldStrategy(MARK) .setConfluentSubjectsCacheRefreshInterval(new Duration(2, SECONDS)); diff --git a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/schema/file/TestFileTableDescriptionSupplierConfig.java b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/schema/file/TestFileTableDescriptionSupplierConfig.java index e22c137d2210..c73e8de3eb9d 100644 --- a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/schema/file/TestFileTableDescriptionSupplierConfig.java +++ b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/schema/file/TestFileTableDescriptionSupplierConfig.java @@ -14,6 +14,7 @@ package io.trino.plugin.kafka.schema.file; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import org.junit.jupiter.api.Test; import java.io.File; @@ -30,7 +31,7 @@ public void testDefaults() { assertRecordedDefaults(recordDefaults(FileTableDescriptionSupplierConfig.class) .setTableDescriptionDir(new File("etc/kafka")) - .setTableNames("")); + .setTableNames(ImmutableSet.of())); } @Test @@ -43,7 +44,7 @@ public void testExplicitPropertyMappings() FileTableDescriptionSupplierConfig expected = new FileTableDescriptionSupplierConfig() .setTableDescriptionDir(new File("/var/lib/kafka")) - .setTableNames("table1, table2, table3"); + .setTableNames(ImmutableSet.of("table1", "table2", "table3")); assertFullMapping(properties, expected); } } diff --git a/plugin/trino-openlineage/src/main/java/io/trino/plugin/openlineage/config/OpenLineageListenerConfig.java b/plugin/trino-openlineage/src/main/java/io/trino/plugin/openlineage/config/OpenLineageListenerConfig.java index 5d1b2bda9b33..9c6298abf7aa 100644 --- a/plugin/trino-openlineage/src/main/java/io/trino/plugin/openlineage/config/OpenLineageListenerConfig.java +++ b/plugin/trino-openlineage/src/main/java/io/trino/plugin/openlineage/config/OpenLineageListenerConfig.java @@ -22,13 +22,9 @@ import jakarta.validation.constraints.NotNull; import java.net.URI; -import java.util.List; import java.util.Optional; import java.util.Set; -import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static java.util.Locale.ENGLISH; - public class OpenLineageListenerConfig { private OpenLineageTransport transport = OpenLineageTransport.CONSOLE; @@ -79,12 +75,9 @@ public Set getIncludeQueryTypes() @Config("openlineage-event-listener.trino.include-query-types") @ConfigDescription("Which query types emitted by Trino should generate OpenLineage events. Other query types will be filtered out.") - public OpenLineageListenerConfig setIncludeQueryTypes(List includeQueryTypes) + public OpenLineageListenerConfig setIncludeQueryTypes(Set includeQueryTypes) { - this.includeQueryTypes = includeQueryTypes.stream() - .map(value -> value.toUpperCase(ENGLISH)) - .map(QueryType::valueOf) - .collect(toImmutableSet()); + this.includeQueryTypes = ImmutableSet.copyOf(includeQueryTypes); return this; } @@ -95,13 +88,10 @@ public Set getDisabledFacets() @Config("openlineage-event-listener.disabled-facets") @ConfigDescription("Which facets should be removed from OpenLineage events.") - public OpenLineageListenerConfig setDisabledFacets(List disabledFacets) + public OpenLineageListenerConfig setDisabledFacets(Set disabledFacets) throws RuntimeException { - this.disabledFacets = disabledFacets.stream() - .map(value -> value.toUpperCase(ENGLISH)) - .map(OpenLineageTrinoFacet::valueOf) - .collect(toImmutableSet()); + this.disabledFacets = ImmutableSet.copyOf(disabledFacets); return this; } diff --git a/plugin/trino-openlineage/src/test/java/io/trino/plugin/openlineage/TestOpenLineageListenerConfig.java b/plugin/trino-openlineage/src/test/java/io/trino/plugin/openlineage/TestOpenLineageListenerConfig.java index 5ccf49c99d15..79340eedfe86 100644 --- a/plugin/trino-openlineage/src/test/java/io/trino/plugin/openlineage/TestOpenLineageListenerConfig.java +++ b/plugin/trino-openlineage/src/test/java/io/trino/plugin/openlineage/TestOpenLineageListenerConfig.java @@ -13,8 +13,8 @@ */ package io.trino.plugin.openlineage; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import io.trino.plugin.openlineage.config.OpenLineageListenerConfig; import org.junit.jupiter.api.Test; @@ -43,14 +43,14 @@ void testDefaults() .setTransport(OpenLineageTransport.CONSOLE) .setTrinoURI(null) .setNamespace(null) - .setDisabledFacets(ImmutableList.of()) - .setIncludeQueryTypes(ImmutableList.of( - ALTER_TABLE_EXECUTE.name(), - DELETE.name(), - INSERT.name(), - MERGE.name(), - UPDATE.name(), - DATA_DEFINITION.name()))); + .setDisabledFacets(ImmutableSet.of()) + .setIncludeQueryTypes(ImmutableSet.of( + ALTER_TABLE_EXECUTE, + DELETE, + INSERT, + MERGE, + UPDATE, + DATA_DEFINITION))); } @Test @@ -68,8 +68,8 @@ void testExplicitPropertyMappings() OpenLineageListenerConfig expected = new OpenLineageListenerConfig() .setTransport(OpenLineageTransport.HTTP) .setTrinoURI(new URI("http://testtrino")) - .setIncludeQueryTypes(ImmutableList.of(SELECT.name(), DELETE.name())) - .setDisabledFacets(ImmutableList.of(TRINO_METADATA.name(), TRINO_QUERY_STATISTICS.name())) + .setIncludeQueryTypes(ImmutableSet.of(SELECT, DELETE)) + .setDisabledFacets(ImmutableSet.of(TRINO_METADATA, TRINO_QUERY_STATISTICS)) .setNamespace("testnamespace"); assertFullMapping(properties, expected); diff --git a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixConfig.java b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixConfig.java index 92b64baf2558..3dbb8b1d3e89 100644 --- a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixConfig.java +++ b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixConfig.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.phoenix5; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; @@ -62,9 +61,9 @@ public PhoenixConfig setConnectionUrl(String connectionUrl) } @Config("phoenix.config.resources") - public PhoenixConfig setResourceConfigFiles(String files) + public PhoenixConfig setResourceConfigFiles(List files) { - this.resourceConfigFiles = Splitter.on(',').trimResults().omitEmptyStrings().splitToList(files); + this.resourceConfigFiles = ImmutableList.copyOf(files); return this; } diff --git a/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixConfig.java b/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixConfig.java index 0a0400dff163..d32fec958cd4 100644 --- a/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixConfig.java +++ b/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixConfig.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.phoenix5; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; @@ -32,7 +33,7 @@ public void testDefaults() { assertRecordedDefaults(recordDefaults(PhoenixConfig.class) .setConnectionUrl(null) - .setResourceConfigFiles("") + .setResourceConfigFiles(ImmutableList.of()) .setMaxScansPerSplit(20) .setReuseConnection(true)); } @@ -52,7 +53,7 @@ public void testExplicitPropertyMappings() PhoenixConfig expected = new PhoenixConfig() .setConnectionUrl("jdbc:phoenix:localhost:2181:/hbase") - .setResourceConfigFiles(configFile.toString()) + .setResourceConfigFiles(ImmutableList.of(configFile.toString())) .setMaxScansPerSplit(1) .setReuseConnection(false); diff --git a/plugin/trino-pinot/src/main/java/io/trino/plugin/pinot/PinotConfig.java b/plugin/trino-pinot/src/main/java/io/trino/plugin/pinot/PinotConfig.java index 723300b1bc90..c681d5f53743 100755 --- a/plugin/trino-pinot/src/main/java/io/trino/plugin/pinot/PinotConfig.java +++ b/plugin/trino-pinot/src/main/java/io/trino/plugin/pinot/PinotConfig.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.pinot; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import com.google.common.net.HostAndPort; import io.airlift.configuration.Config; @@ -49,8 +48,6 @@ }) public class PinotConfig { - private static final Splitter LIST_SPLITTER = Splitter.on(",").trimResults().omitEmptyStrings(); - private List controllerUrls = ImmutableList.of(); private Optional brokerUrl = Optional.empty(); @@ -77,9 +74,9 @@ public List getControllerUrls() } @Config("pinot.controller-urls") - public PinotConfig setControllerUrls(String controllerUrl) + public PinotConfig setControllerUrls(List controllerUrl) { - this.controllerUrls = LIST_SPLITTER.splitToList(controllerUrl).stream() + this.controllerUrls = controllerUrl.stream() .map(PinotConfig::stringToUri) .collect(toImmutableList()); return this; diff --git a/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotClient.java b/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotClient.java index fcdec034438d..c4c0d6c082a2 100755 --- a/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotClient.java +++ b/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotClient.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.pinot; +import com.google.common.collect.ImmutableList; import com.google.common.net.MediaType; import io.airlift.http.client.HttpClient; import io.airlift.http.client.HttpStatus; @@ -78,7 +79,7 @@ public void testBrokersParsed() "}")); PinotConfig pinotConfig = new PinotConfig() .setMetadataCacheExpiry(new Duration(1, TimeUnit.MILLISECONDS)) - .setControllerUrls("localhost:7900"); + .setControllerUrls(ImmutableList.of("localhost:7900")); PinotClient pinotClient = new PinotClient( pinotConfig, new IdentityPinotHostMapper(), diff --git a/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotConfig.java b/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotConfig.java index c5a52a63ac03..21597d7fed52 100755 --- a/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotConfig.java +++ b/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotConfig.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.pinot; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.net.HostAndPort; import io.airlift.configuration.testing.ConfigAssertions; @@ -34,7 +35,7 @@ public void testDefaults() { ConfigAssertions.assertRecordedDefaults( ConfigAssertions.recordDefaults(PinotConfig.class) - .setControllerUrls("") + .setControllerUrls(ImmutableList.of()) .setBrokerUrl(null) .setConnectionTimeout(new Duration(1, TimeUnit.MINUTES)) .setMetadataCacheExpiry(new Duration(2, TimeUnit.MINUTES)) @@ -71,7 +72,7 @@ public void testExplicitPropertyMappings() .buildOrThrow(); PinotConfig expected = new PinotConfig() - .setControllerUrls("https://host1:1111,https://host2:1111") + .setControllerUrls(ImmutableList.of("https://host1:1111", "https://host2:1111")) .setBrokerUrl(HostAndPort.fromString("host1:1111")) .setConnectionTimeout(new Duration(8, TimeUnit.MINUTES)) .setMetadataCacheExpiry(new Duration(1, TimeUnit.MINUTES)) @@ -104,21 +105,21 @@ public void testInvalidCountDistinctPushdown() public void testControllerUrls() { PinotConfig config = new PinotConfig(); - config.setControllerUrls("my-controller-1:8443,my-controller-2:8443"); + config.setControllerUrls(ImmutableList.of("my-controller-1:8443", "my-controller-2:8443")); assertThat(config.allUrlSchemesEqual()).isTrue(); assertThat(config.isTlsEnabled()).isFalse(); - config.setControllerUrls("http://my-controller-1:9000,http://my-controller-2:9000"); + config.setControllerUrls(ImmutableList.of("http://my-controller-1:9000", "http://my-controller-2:9000")); assertThat(config.allUrlSchemesEqual()).isTrue(); assertThat(config.isTlsEnabled()).isFalse(); - config.setControllerUrls("https://my-controller-1:8443,https://my-controller-2:8443"); + config.setControllerUrls(ImmutableList.of("https://my-controller-1:8443", "https://my-controller-2:8443")); assertThat(config.allUrlSchemesEqual()).isTrue(); assertThat(config.isTlsEnabled()).isTrue(); - config.setControllerUrls("my-controller-1:8443,http://my-controller-2:8443"); + config.setControllerUrls(ImmutableList.of("my-controller-1:8443", "http://my-controller-2:8443")); assertThat(config.allUrlSchemesEqual()).isTrue(); assertThat(config.isTlsEnabled()).isFalse(); - config.setControllerUrls("http://my-controller-1:8443,https://my-controller-2:8443"); + config.setControllerUrls(ImmutableList.of("http://my-controller-1:8443", "https://my-controller-2:8443")); assertThat(config.allUrlSchemesEqual()).isFalse(); - config.setControllerUrls("my-controller-1:8443,https://my-controller-2:8443"); + config.setControllerUrls(ImmutableList.of("my-controller-1:8443", "https://my-controller-2:8443")); assertThat(config.allUrlSchemesEqual()).isFalse(); } } diff --git a/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotMetadata.java b/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotMetadata.java index bad57262a4b5..f81bc812b3d9 100755 --- a/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotMetadata.java +++ b/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotMetadata.java @@ -29,7 +29,7 @@ public class TestPinotMetadata { - private final PinotConfig pinotConfig = new PinotConfig().setControllerUrls("localhost:9000"); + private final PinotConfig pinotConfig = new PinotConfig().setControllerUrls(ImmutableList.of("localhost:9000")); private final PinotMetadata metadata = new PinotMetadata(new MockPinotClient(pinotConfig), pinotConfig, Executors.newSingleThreadExecutor(), new PinotTypeConverter(new TestingTypeManager())); @Test diff --git a/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotQueryBase.java b/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotQueryBase.java index bd5060657de2..ea48ea216a45 100755 --- a/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotQueryBase.java +++ b/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotQueryBase.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.pinot; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.trino.plugin.pinot.client.PinotClient; import io.trino.spi.type.TestingTypeManager; @@ -34,7 +35,7 @@ public class TestPinotQueryBase protected static PinotTableHandle realtimeOnlyTable = new PinotTableHandle("schema", "realtimeOnly"); protected static PinotTableHandle hybridTable = new PinotTableHandle("schema", "hybrid"); - protected final PinotConfig pinotConfig = new PinotConfig().setControllerUrls("localhost:9000"); + protected final PinotConfig pinotConfig = new PinotConfig().setControllerUrls(ImmutableList.of("localhost:9000")); protected final PinotClient mockClusterInfoFetcher = new MockPinotClient(pinotConfig, getTestingMetadata()); protected final PinotMetadata pinotMetadata = new PinotMetadata( diff --git a/plugin/trino-redis/src/main/java/io/trino/plugin/redis/RedisConnectorConfig.java b/plugin/trino-redis/src/main/java/io/trino/plugin/redis/RedisConnectorConfig.java index 88a0caa51908..4288ca10622e 100644 --- a/plugin/trino-redis/src/main/java/io/trino/plugin/redis/RedisConnectorConfig.java +++ b/plugin/trino-redis/src/main/java/io/trino/plugin/redis/RedisConnectorConfig.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.redis; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableSet; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; @@ -27,10 +26,10 @@ import jakarta.validation.constraints.Size; import java.io.File; +import java.util.List; import java.util.Set; import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static com.google.common.collect.Streams.stream; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.MINUTES; @@ -90,9 +89,9 @@ public Set getTableNames() @Config("redis.table-names") @ConfigDescription("Set of tables known to this connector. For each table, a description file may be present in the catalog folder which describes columns for the given table") - public RedisConnectorConfig setTableNames(String tableNames) + public RedisConnectorConfig setTableNames(Set tableNames) { - this.tableNames = ImmutableSet.copyOf(Splitter.on(',').omitEmptyStrings().trimResults().split(tableNames)); + this.tableNames = ImmutableSet.copyOf(tableNames); return this; } @@ -118,9 +117,11 @@ public Set getNodes() @Config("redis.nodes") @ConfigDescription("Seed nodes for Redis cluster. At least one must exist") - public RedisConnectorConfig setNodes(String nodes) + public RedisConnectorConfig setNodes(List nodes) { - this.nodes = (nodes == null) ? null : parseNodes(nodes); + this.nodes = nodes.stream() + .map(RedisConnectorConfig::toHostAddress) + .collect(toImmutableSet()); return this; } @@ -245,15 +246,6 @@ public RedisConnectorConfig setKeyPrefixSchemaTable(boolean keyPrefixSchemaTable return this; } - public static ImmutableSet parseNodes(String nodes) - { - Splitter splitter = Splitter.on(',').omitEmptyStrings().trimResults(); - - return stream(splitter.split(nodes)) - .map(RedisConnectorConfig::toHostAddress) - .collect(toImmutableSet()); - } - private static HostAddress toHostAddress(String value) { return HostAddress.fromString(value).withDefaultPort(REDIS_DEFAULT_PORT); diff --git a/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisConnectorConfig.java b/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisConnectorConfig.java index 15b3d22183b2..929884b4709f 100644 --- a/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisConnectorConfig.java +++ b/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisConnectorConfig.java @@ -13,7 +13,9 @@ */ package io.trino.plugin.redis; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import io.airlift.units.Duration; import org.junit.jupiter.api.Test; @@ -32,9 +34,9 @@ public class TestRedisConnectorConfig public void testDefaults() { assertRecordedDefaults(recordDefaults(RedisConnectorConfig.class) - .setNodes("") + .setNodes(ImmutableList.of()) .setDefaultSchema("default") - .setTableNames("") + .setTableNames(ImmutableSet.of()) .setTableDescriptionDir(new File("etc/redis/")) .setTableDescriptionCacheDuration(new Duration(5, MINUTES)) .setKeyPrefixSchemaTable(false) @@ -71,9 +73,9 @@ public void testExplicitPropertyMappings() RedisConnectorConfig expected = new RedisConnectorConfig() .setTableDescriptionDir(new File("/var/lib/redis")) .setTableDescriptionCacheDuration(new Duration(30, SECONDS)) - .setTableNames("table1, table2, table3") + .setTableNames(ImmutableSet.of("table1", "table2", "table3")) .setDefaultSchema("redis") - .setNodes("localhost:12345, localhost:23456") + .setNodes(ImmutableList.of("localhost:12345", "localhost:23456")) .setHideInternalColumns(false) .setRedisScanCount(20) .setRedisMaxKeysPerFetch(10) diff --git a/service/trino-verifier/src/main/java/io/trino/verifier/VerifierConfig.java b/service/trino-verifier/src/main/java/io/trino/verifier/VerifierConfig.java index 22f7b4da045c..2e87580ff3c9 100644 --- a/service/trino-verifier/src/main/java/io/trino/verifier/VerifierConfig.java +++ b/service/trino-verifier/src/main/java/io/trino/verifier/VerifierConfig.java @@ -13,8 +13,6 @@ */ package io.trino.verifier; -import com.google.common.base.Splitter; -import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import io.airlift.configuration.Config; @@ -37,8 +35,6 @@ import static io.trino.verifier.QueryType.CREATE; import static io.trino.verifier.QueryType.MODIFY; import static io.trino.verifier.QueryType.READ; -import static java.util.Locale.ENGLISH; -import static java.util.Objects.requireNonNull; public class VerifierConfig { @@ -46,7 +42,7 @@ public class VerifierConfig private String controlUsernameOverride; private String testPasswordOverride; private String controlPasswordOverride; - private List suites; + private List suites = ImmutableList.of(); private Set controlQueryTypes = ImmutableSet.of(READ, CREATE, MODIFY); private Set testQueryTypes = ImmutableSet.of(READ, CREATE, MODIFY); private String source; @@ -163,7 +159,7 @@ public VerifierConfig setQueryRepetitions(int queryRepetitions) public String getSuite() { - return suites == null ? null : suites.get(0); + return suites.isEmpty() ? null : suites.get(0); } @ConfigDescription("The suites of queries in the query database to run") @@ -184,19 +180,9 @@ public Set getControlQueryTypes() @ConfigDescription("The types of control queries allowed to run [CREATE, READ, MODIFY]") @Config("control.query-types") - public VerifierConfig setControlQueryTypes(String types) + public VerifierConfig setControlQueryTypes(Set controlQueryTypes) { - if (Strings.isNullOrEmpty(types)) { - this.controlQueryTypes = ImmutableSet.of(); - return this; - } - - ImmutableSet.Builder builder = ImmutableSet.builder(); - for (String value : Splitter.on(',').trimResults().omitEmptyStrings().split(types)) { - builder.add(QueryType.valueOf(value.toUpperCase(ENGLISH))); - } - - this.controlQueryTypes = builder.build(); + this.controlQueryTypes = ImmutableSet.copyOf(controlQueryTypes); return this; } @@ -207,19 +193,9 @@ public Set getTestQueryTypes() @ConfigDescription("The types of control queries allowed to run [CREATE, READ, MODIFY]") @Config("test.query-types") - public VerifierConfig setTestQueryTypes(String types) + public VerifierConfig setTestQueryTypes(Set testQueryTypes) { - if (Strings.isNullOrEmpty(types)) { - this.testQueryTypes = ImmutableSet.of(); - return this; - } - - ImmutableSet.Builder builder = ImmutableSet.builder(); - for (String value : Splitter.on(',').trimResults().omitEmptyStrings().split(types)) { - builder.add(QueryType.valueOf(value.toUpperCase(ENGLISH))); - } - - this.testQueryTypes = builder.build(); + this.testQueryTypes = ImmutableSet.copyOf(testQueryTypes); return this; } @@ -231,18 +207,9 @@ public List getSuites() @ConfigDescription("The suites of queries in the query database to run") @Config("suites") - public VerifierConfig setSuites(String suites) + public VerifierConfig setSuites(List suites) { - if (Strings.isNullOrEmpty(suites)) { - return this; - } - - ImmutableList.Builder builder = ImmutableList.builder(); - for (String value : Splitter.on(',').trimResults().omitEmptyStrings().split(suites)) { - builder.add(value); - } - - this.suites = builder.build(); + this.suites = ImmutableList.copyOf(suites); return this; } @@ -283,14 +250,9 @@ public Set getBannedQueries() @ConfigDescription("Names of queries which are banned") @Config("banned-queries") @LegacyConfig("blacklist") - public VerifierConfig setBannedQueries(String bannedQueries) + public VerifierConfig setBannedQueries(Set bannedQueries) { - ImmutableSet.Builder bannedBuilder = ImmutableSet.builder(); - for (String value : Splitter.on(',').trimResults().omitEmptyStrings().split(bannedQueries)) { - bannedBuilder.add(value); - } - - this.bannedQueries = bannedBuilder.build(); + this.bannedQueries = ImmutableSet.copyOf(bannedQueries); return this; } @@ -303,14 +265,9 @@ public Set getAllowedQueries() @ConfigDescription("Names of queries which are allowed. If non-empty, only allowed queries are used.") @Config("allowed-queries") @LegacyConfig("whitelist") - public VerifierConfig setAllowedQueries(String allowedQueries) + public VerifierConfig setAllowedQueries(Set allowedQueries) { - ImmutableSet.Builder allowedBuilder = ImmutableSet.builder(); - for (String value : Splitter.on(',').trimResults().omitEmptyStrings().split(allowedQueries)) { - allowedBuilder.add(value); - } - - this.allowedQueries = allowedBuilder.build(); + this.allowedQueries = ImmutableSet.copyOf(allowedQueries); return this; } @@ -426,15 +383,9 @@ public Set getEventClients() @ConfigDescription("The event client(s) to log the results to") @Config("event-client") - public VerifierConfig setEventClients(String eventClients) + public VerifierConfig setEventClients(Set eventClients) { - requireNonNull(eventClients, "eventClients is null"); - ImmutableSet.Builder builder = ImmutableSet.builder(); - for (String value : Splitter.on(',').trimResults().omitEmptyStrings().split(eventClients)) { - builder.add(value); - } - - this.eventClients = builder.build(); + this.eventClients = ImmutableSet.copyOf(eventClients); return this; } diff --git a/service/trino-verifier/src/test/java/io/trino/verifier/TestVerifierConfig.java b/service/trino-verifier/src/test/java/io/trino/verifier/TestVerifierConfig.java index 89e7afedd7bc..0cba9b4fd6e3 100644 --- a/service/trino-verifier/src/test/java/io/trino/verifier/TestVerifierConfig.java +++ b/service/trino-verifier/src/test/java/io/trino/verifier/TestVerifierConfig.java @@ -13,8 +13,9 @@ */ package io.trino.verifier; -import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import io.airlift.units.Duration; import org.joda.time.DateTime; import org.junit.jupiter.api.Test; @@ -40,20 +41,20 @@ public void testDefaults() .setTestPasswordOverride(null) .setControlPasswordOverride(null) .setSuite(null) - .setSuites(null) - .setControlQueryTypes(Joiner.on(",").join(READ, CREATE, MODIFY)) - .setTestQueryTypes(Joiner.on(",").join(READ, CREATE, MODIFY)) + .setSuites(ImmutableList.of()) + .setControlQueryTypes(ImmutableSet.of(READ, CREATE, MODIFY)) + .setTestQueryTypes(ImmutableSet.of(READ, CREATE, MODIFY)) .setSource(null) .setRunId(new DateTime().toString("yyyy-MM-dd")) - .setEventClients("human-readable") + .setEventClients(ImmutableSet.of("human-readable")) .setThreadCount(10) .setQueryDatabase(null) .setControlGateway(null) .setTestGateway(null) .setControlTimeout(new Duration(10, TimeUnit.MINUTES)) .setTestTimeout(new Duration(1, TimeUnit.HOURS)) - .setBannedQueries("") - .setAllowedQueries("") + .setBannedQueries(ImmutableSet.of()) + .setAllowedQueries(ImmutableSet.of()) .setMaxRowCount(10_000) .setMaxQueries(1_000_000) .setAlwaysReport(false) @@ -94,8 +95,8 @@ public void testExplicitPropertyMappings() Map properties = ImmutableMap.builder() .put("suites", "my_suite") .put("suite", "my_suite") - .put("control.query-types", Joiner.on(",").join(CREATE, MODIFY)) - .put("test.query-types", MODIFY.name()) + .put("control.query-types", "create,modify") + .put("test.query-types", "modify") .put("source", "my_source") .put("run-id", "my_run_id") .put("event-client", "file,human-readable") @@ -146,16 +147,16 @@ public void testExplicitPropertyMappings() .buildOrThrow(); VerifierConfig expected = new VerifierConfig().setTestUsernameOverride("verifier-test") - .setSuites("my_suite") + .setSuites(ImmutableList.of("my_suite")) .setSuite("my_suite") - .setControlQueryTypes(Joiner.on(",").join(CREATE, MODIFY)) - .setTestQueryTypes(MODIFY.name()) + .setControlQueryTypes(ImmutableSet.of(CREATE, MODIFY)) + .setTestQueryTypes(ImmutableSet.of(MODIFY)) .setSource("my_source") .setRunId("my_run_id") - .setEventClients("file,human-readable") + .setEventClients(ImmutableSet.of("file", "human-readable")) .setThreadCount(1) - .setBannedQueries("1,2") - .setAllowedQueries("3,4") + .setBannedQueries(ImmutableSet.of("1", "2")) + .setAllowedQueries(ImmutableSet.of("3", "4")) .setMaxRowCount(1) .setMaxQueries(1) .setAlwaysReport(true) diff --git a/testing/trino-server-dev/src/main/java/io/trino/server/DevelopmentLoaderConfig.java b/testing/trino-server-dev/src/main/java/io/trino/server/DevelopmentLoaderConfig.java index d624a9a7b023..34fd3f650706 100644 --- a/testing/trino-server-dev/src/main/java/io/trino/server/DevelopmentLoaderConfig.java +++ b/testing/trino-server-dev/src/main/java/io/trino/server/DevelopmentLoaderConfig.java @@ -13,7 +13,6 @@ */ package io.trino.server; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import io.airlift.resolver.ArtifactResolver; @@ -23,8 +22,6 @@ public class DevelopmentLoaderConfig { - private static final Splitter SPLITTER = Splitter.on(',').omitEmptyStrings().trimResults(); - private List plugins = ImmutableList.of(); private String mavenLocalRepository = ArtifactResolver.USER_LOCAL_REPO; private List mavenRemoteRepository = ImmutableList.of(ArtifactResolver.MAVEN_CENTRAL_URI); @@ -34,19 +31,13 @@ public List getPlugins() return plugins; } + @Config("plugin.bundles") public DevelopmentLoaderConfig setPlugins(List plugins) { this.plugins = ImmutableList.copyOf(plugins); return this; } - @Config("plugin.bundles") - public DevelopmentLoaderConfig setPlugins(String plugins) - { - this.plugins = SPLITTER.splitToList(plugins); - return this; - } - @NotNull public String getMavenLocalRepository() { @@ -66,16 +57,10 @@ public List getMavenRemoteRepository() return mavenRemoteRepository; } - public DevelopmentLoaderConfig setMavenRemoteRepository(List mavenRemoteRepository) - { - this.mavenRemoteRepository = mavenRemoteRepository; - return this; - } - @Config("maven.repo.remote") - public DevelopmentLoaderConfig setMavenRemoteRepository(String mavenRemoteRepository) + public DevelopmentLoaderConfig setMavenRemoteRepository(List mavenRemoteRepository) { - this.mavenRemoteRepository = ImmutableList.copyOf(Splitter.on(',').omitEmptyStrings().trimResults().split(mavenRemoteRepository)); + this.mavenRemoteRepository = ImmutableList.copyOf(mavenRemoteRepository); return this; } } diff --git a/testing/trino-server-dev/src/test/java/io/trino/server/TestDevelopmentLoaderConfig.java b/testing/trino-server-dev/src/test/java/io/trino/server/TestDevelopmentLoaderConfig.java index 176d095e1ced..1875463d9773 100644 --- a/testing/trino-server-dev/src/test/java/io/trino/server/TestDevelopmentLoaderConfig.java +++ b/testing/trino-server-dev/src/test/java/io/trino/server/TestDevelopmentLoaderConfig.java @@ -30,9 +30,9 @@ public class TestDevelopmentLoaderConfig public void testDefaults() { assertRecordedDefaults(recordDefaults(DevelopmentLoaderConfig.class) - .setPlugins("") + .setPlugins(ImmutableList.of()) .setMavenLocalRepository(ArtifactResolver.USER_LOCAL_REPO) - .setMavenRemoteRepository(ArtifactResolver.MAVEN_CENTRAL_URI)); + .setMavenRemoteRepository(ImmutableList.of(ArtifactResolver.MAVEN_CENTRAL_URI))); } @Test