diff --git a/atlasdb-config/readme.md b/atlasdb-config/readme.md index ad32c1fff60..1421fffe001 100644 --- a/atlasdb-config/readme.md +++ b/atlasdb-config/readme.md @@ -13,13 +13,10 @@ Creating a `TransactionManager`: ```java AtlasDbConfig atlasConfig = ... Schema atlasSchema = ... -Optional sslSocketFactory = ... - -SerializableTransactionManager tm = TransactionManagers.create( - atlasConfig, - sslSocketFactory, - atlasSchema, - (resource) -> {}); +SerializableTransactionManager tm = TransactionManagers.builder() + .config(atlasConfig) + .schemas(ImmutableSet.of(atlasSchema)) + .buildSerializable(); ``` The last item is a consumer of resources meant to be exposed to as web @@ -48,12 +45,10 @@ And initialization code to your run method: ```java public void run(AtlasDbServerConfiguration config, Environment env) throws Exception { - TransactionManager transactionManager = - TransactionManagers.create( - config.getAtlas(), - Optional.absent(), - ImmutableSet.of(), - env.jersey()::register); + TransactionManager transactionManager = TransactionManagers.builder() + .config(config.getAtlas()) + .registrar(env.jersey()::register) + .buildSerializable(); ... ``` diff --git a/atlasdb-config/src/main/java/com/palantir/atlasdb/factory/Leaders.java b/atlasdb-config/src/main/java/com/palantir/atlasdb/factory/Leaders.java index dc9fd512b13..2986ff25490 100644 --- a/atlasdb-config/src/main/java/com/palantir/atlasdb/factory/Leaders.java +++ b/atlasdb-config/src/main/java/com/palantir/atlasdb/factory/Leaders.java @@ -23,6 +23,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.Executors; +import java.util.function.Consumer; import javax.net.ssl.SSLSocketFactory; @@ -36,7 +37,6 @@ import com.google.common.net.HostAndPort; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.palantir.atlasdb.config.LeaderConfig; -import com.palantir.atlasdb.factory.TransactionManagers.Environment; import com.palantir.atlasdb.http.AtlasDbHttpClients; import com.palantir.atlasdb.http.NotCurrentLeaderExceptionMapper; import com.palantir.atlasdb.http.UserAgents; @@ -62,22 +62,22 @@ private Leaders() { * Creates a LeaderElectionService using the supplied configuration and * registers appropriate endpoints for that service. */ - public static LeaderElectionService create(Environment env, LeaderConfig config) { + public static LeaderElectionService create(Consumer env, LeaderConfig config) { return create(env, config, UserAgents.DEFAULT_USER_AGENT); } - public static LeaderElectionService create(Environment env, LeaderConfig config, String userAgent) { + public static LeaderElectionService create(Consumer env, LeaderConfig config, String userAgent) { return createAndRegisterLocalServices(env, config, userAgent).leaderElectionService(); } public static LocalPaxosServices createAndRegisterLocalServices( - Environment env, LeaderConfig config, String userAgent) { + Consumer env, LeaderConfig config, String userAgent) { LocalPaxosServices localPaxosServices = createInstrumentedLocalServices(config, userAgent); - env.register(localPaxosServices.ourAcceptor()); - env.register(localPaxosServices.ourLearner()); - env.register(localPaxosServices.pingableLeader()); - env.register(new NotCurrentLeaderExceptionMapper()); + env.accept(localPaxosServices.ourAcceptor()); + env.accept(localPaxosServices.ourLearner()); + env.accept(localPaxosServices.pingableLeader()); + env.accept(new NotCurrentLeaderExceptionMapper()); return localPaxosServices; } diff --git a/atlasdb-config/src/main/java/com/palantir/atlasdb/factory/TransactionManagers.java b/atlasdb-config/src/main/java/com/palantir/atlasdb/factory/TransactionManagers.java index c2be544fcca..3b0d41b4d58 100644 --- a/atlasdb-config/src/main/java/com/palantir/atlasdb/factory/TransactionManagers.java +++ b/atlasdb-config/src/main/java/com/palantir/atlasdb/factory/TransactionManagers.java @@ -15,10 +15,12 @@ */ package com.palantir.atlasdb.factory; +import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; +import java.util.function.Supplier; import javax.ws.rs.ClientErrorException; @@ -27,9 +29,9 @@ import org.slf4j.LoggerFactory; import com.codahale.metrics.MetricRegistry; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -106,12 +108,57 @@ import com.palantir.timestamp.TimestampStoreInvalidator; import com.palantir.util.OptionalResolver; -public final class TransactionManagers { - +@Value.Immutable +public abstract class TransactionManagers { private static final int LOGGING_INTERVAL = 60; private static final Logger log = LoggerFactory.getLogger(TransactionManagers.class); + public static final LockClient LOCK_CLIENT = LockClient.of("atlas instance"); + abstract AtlasDbConfig config(); + + @Value.Default + Supplier> runtimeConfigSupplier() { + return Optional::empty; + } + + abstract Set schemas(); + + @Value.Default + Consumer registrar() { + return resource -> { }; + } + + @Value.Default + LockServerOptions lockServerOptions() { + return LockServerOptions.DEFAULT; + } + + @Value.Default + boolean allowHiddenTableAccess() { + return false; + } + + abstract Optional> callingClass(); + + abstract Optional userAgent(); + + // directly specified -> inferred from caller -> default + @Value.Derived + String derivedUserAgent() { + return userAgent().orElse(callingClass().map(UserAgents::fromClass).orElse(UserAgents.DEFAULT_USER_AGENT)); + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder extends ImmutableTransactionManagers.Builder { + public SerializableTransactionManager buildSerializable() { + return build().serializable(); + } + } + @VisibleForTesting static Consumer runAsync = task -> { Thread thread = new Thread(task); @@ -119,10 +166,6 @@ public final class TransactionManagers { thread.start(); }; - private TransactionManagers() { - // Utility class - } - /** * Accepts a single {@link Schema}. * @see TransactionManagers#createInMemory(Set) @@ -137,23 +180,20 @@ public static SerializableTransactionManager createInMemory(Schema schema) { * purposes only. */ public static SerializableTransactionManager createInMemory(Set schemas) { - AtlasDbConfig config = ImmutableAtlasDbConfig.builder() - .keyValueService(new InMemoryAtlasDbConfig()) - .build(); - return create(config, - java.util.Optional::empty, - schemas, - x -> { }, - false); + AtlasDbConfig config = ImmutableAtlasDbConfig.builder().keyValueService(new InMemoryAtlasDbConfig()).build(); + return builder().config(config).schemas(schemas).buildSerializable(); } + // Begin deprecated creation methods + /** - * Create a {@link SerializableTransactionManager} with provided configurations, {@link Schema}, - * and an environment in which to register HTTP server endpoints. + * @deprecated Use {@link #builder()} to create a {@link Builder}, and use {@link Builder#buildSerializable()} to + * generate a {@link SerializableTransactionManager} from it. */ + @Deprecated public static SerializableTransactionManager create( AtlasDbConfig config, - java.util.function.Supplier> runtimeConfigSupplier, + Supplier> runtimeConfigSupplier, Schema schema, Environment env, boolean allowHiddenTableAccess) { @@ -161,12 +201,13 @@ public static SerializableTransactionManager create( } /** - * Create a {@link SerializableTransactionManager} with provided configurations, a set of - * {@link Schema}s, and an environment in which to register HTTP server endpoints. + * @deprecated Use {@link #builder()} to create a {@link Builder}, and use {@link Builder#buildSerializable()} to + * generate a {@link SerializableTransactionManager} from it. */ + @Deprecated public static SerializableTransactionManager create( AtlasDbConfig config, - java.util.function.Supplier> runtimeConfigSupplier, + Supplier> runtimeConfigSupplier, Set schemas, Environment env, boolean allowHiddenTableAccess) { @@ -176,45 +217,95 @@ public static SerializableTransactionManager create( } /** - * Create a {@link SerializableTransactionManager} with provided configurations, a set of - * {@link Schema}s, {@link LockServerOptions}, and an environment in which to register HTTP server endpoints. + * @deprecated Use {@link #builder()} to create a {@link Builder}, and use {@link Builder#buildSerializable()} to + * generate a {@link SerializableTransactionManager} from it. */ + @Deprecated public static SerializableTransactionManager create( AtlasDbConfig config, - java.util.function.Supplier> runtimeConfigSupplier, + Supplier> runtimeConfigSupplier, Set schemas, Environment env, LockServerOptions lockServerOptions, boolean allowHiddenTableAccess) { - return create(config, runtimeConfigSupplier, schemas, env, lockServerOptions, allowHiddenTableAccess, - UserAgents.DEFAULT_USER_AGENT); + return builder() + .config(config) + .runtimeConfigSupplier(runtimeConfigSupplier) + .schemas(schemas) + .registrar(env::register) + .lockServerOptions(lockServerOptions) + .allowHiddenTableAccess(allowHiddenTableAccess) + .buildSerializable(); } + /** + * @deprecated Use {@link #builder()} to create a {@link Builder}, and use {@link Builder#buildSerializable()} to + * generate a {@link SerializableTransactionManager} from it. + */ + @Deprecated public static SerializableTransactionManager create( AtlasDbConfig config, - java.util.function.Supplier> runtimeConfigSupplier, + Supplier> runtimeConfigSupplier, Set schemas, Environment env, LockServerOptions lockServerOptions, boolean allowHiddenTableAccess, Class callingClass) { - return create(config, runtimeConfigSupplier, schemas, env, lockServerOptions, allowHiddenTableAccess, - UserAgents.fromClass(callingClass)); + return builder() + .config(config) + .runtimeConfigSupplier(runtimeConfigSupplier) + .schemas(schemas) + .registrar(env::register) + .lockServerOptions(lockServerOptions) + .allowHiddenTableAccess(allowHiddenTableAccess) + .callingClass(callingClass) + .buildSerializable(); } + /** + * @deprecated Use {@link #builder()} to create a {@link Builder}, and use {@link Builder#buildSerializable()} to + * generate a {@link SerializableTransactionManager} from it. + */ + @Deprecated public static SerializableTransactionManager create( AtlasDbConfig config, - java.util.function.Supplier> optionalRuntimeConfigSupplier, + Supplier> optionalRuntimeConfigSupplier, Set schemas, Environment env, LockServerOptions lockServerOptions, boolean allowHiddenTableAccess, String userAgent) { + return builder() + .config(config) + .runtimeConfigSupplier(optionalRuntimeConfigSupplier) + .schemas(schemas) + .registrar(env::register) + .lockServerOptions(lockServerOptions) + .allowHiddenTableAccess(allowHiddenTableAccess) + .userAgent(userAgent) + .buildSerializable(); + } + + /** + * @deprecated This interface is deprecated and is not meant for use publicly. When creating a + * {@link SerializableTransactionManager} via the {@link Builder}, specify a {@link Consumer}. + */ + @Deprecated + public interface Environment { + void register(Object resource); + } + + // End deprecated creation methods + + @JsonIgnore + @Value.Derived + SerializableTransactionManager serializable() { + final AtlasDbConfig config = config(); checkInstallConfig(config); AtlasDbRuntimeConfig defaultRuntime = AtlasDbRuntimeConfig.defaultRuntimeConfig(); java.util.function.Supplier runtimeConfigSupplier = - () -> optionalRuntimeConfigSupplier.get().orElse(defaultRuntime); + () -> runtimeConfigSupplier().get().orElse(defaultRuntime); ServiceDiscoveringAtlasSupplier atlasFactory = new ServiceDiscoveringAtlasSupplier(config.keyValueService(), config.leader(), config.namespace(), @@ -226,31 +317,34 @@ public static SerializableTransactionManager create( LockAndTimestampServices lockAndTimestampServices = createLockAndTimestampServices( config, () -> runtimeConfigSupplier.get().timestampClient(), - env, - () -> LockServiceImpl.create(lockServerOptions), + registrar(), + () -> LockServiceImpl.create(lockServerOptions()), atlasFactory::getTimestampService, atlasFactory.getTimestampStoreInvalidator(), - userAgent); + derivedUserAgent()); KvsProfilingLogger.setSlowLogThresholdMillis(config.getKvsSlowLogThresholdMillis()); KeyValueService kvs = ProfilingKeyValueService.create(rawKvs); - kvs = SweepStatsKeyValueService.create(kvs, new TimelockTimestampServiceAdapter(lockAndTimestampServices.timelock())); kvs = TracingKeyValueService.create(kvs); kvs = AtlasDbMetrics.instrument(KeyValueService.class, kvs, MetricRegistry.name(KeyValueService.class)); kvs = ValidatingQueryRewritingKeyValueService.create(kvs); - TransactionManagersInitializer initializer = TransactionManagersInitializer.createInitialTables(kvs, schemas, + TransactionManagersInitializer initializer = TransactionManagersInitializer.createInitialTables( + kvs, + schemas(), config.initializeAsync()); - PersistentLockService persistentLockService = createAndRegisterPersistentLockService(kvs, env, + PersistentLockService persistentLockService = createAndRegisterPersistentLockService( + kvs, + registrar(), config.initializeAsync()); TransactionService transactionService = TransactionServices.createTransactionService(kvs); ConflictDetectionManager conflictManager = ConflictDetectionManagers.create(kvs); SweepStrategyManager sweepStrategyManager = SweepStrategyManagers.createDefault(kvs); - CleanupFollower follower = CleanupFollower.create(schemas); + CleanupFollower follower = CleanupFollower.create(schemas()); Cleaner cleaner = new DefaultCleanerBuilder( kvs, @@ -276,7 +370,7 @@ public static SerializableTransactionManager create( sweepStrategyManager, cleaner, initializer, - allowHiddenTableAccess, + allowHiddenTableAccess(), () -> runtimeConfigSupplier.get().transaction().getLockAcquireTimeoutMillis(), config.keyValueService().concurrentGetRangesThreadPoolSize(), config.initializeAsync()); @@ -285,7 +379,7 @@ public static SerializableTransactionManager create( persistentLockService, config.getSweepPersistentLockWaitMillis()); initializeSweepEndpointAndBackgroundProcess(runtimeConfigSupplier, - env, + registrar(), kvs, transactionService, sweepStrategyManager, @@ -308,8 +402,8 @@ private static void checkInstallConfig(AtlasDbConfig config) { } private static void initializeSweepEndpointAndBackgroundProcess( - java.util.function.Supplier runtimeConfigSupplier, - Environment env, + Supplier runtimeConfigSupplier, + Consumer env, KeyValueService kvs, TransactionService transactionService, SweepStrategyManager sweepStrategyManager, @@ -329,7 +423,8 @@ private static void initializeSweepEndpointAndBackgroundProcess( sweepStrategyManager, cellsSweeper); BackgroundSweeperPerformanceLogger sweepPerfLogger = new NoOpBackgroundSweeperPerformanceLogger(); - Supplier sweepBatchConfig = () -> getSweepBatchConfig(runtimeConfigSupplier.get().sweep()); + com.google.common.base.Supplier sweepBatchConfig = () -> + getSweepBatchConfig(runtimeConfigSupplier.get().sweep()); SweepMetrics sweepMetrics = new SweepMetrics(); SpecificTableSweeper specificTableSweeper = initializeSweepEndpoint( @@ -352,12 +447,12 @@ private static void initializeSweepEndpointAndBackgroundProcess( } private static SpecificTableSweeper initializeSweepEndpoint( - Environment env, + Consumer env, KeyValueService kvs, SerializableTransactionManager transactionManager, SweepTaskRunner sweepRunner, BackgroundSweeperPerformanceLogger sweepPerfLogger, - Supplier sweepBatchConfig, + com.google.common.base.Supplier sweepBatchConfig, SweepMetrics sweepMetrics) { SpecificTableSweeper specificTableSweeper = SpecificTableSweeper.create( transactionManager, @@ -367,7 +462,7 @@ private static SpecificTableSweeper initializeSweepEndpoint( SweepTableFactory.of(), sweepPerfLogger, sweepMetrics); - env.register(new SweeperServiceImpl(specificTableSweeper)); + env.accept(new SweeperServiceImpl(specificTableSweeper)); return specificTableSweeper; } @@ -379,15 +474,17 @@ private static SweepBatchConfig getSweepBatchConfig(SweepConfig sweepConfig) { .build(); } - private static PersistentLockService createAndRegisterPersistentLockService(KeyValueService kvs, Environment env, + private static PersistentLockService createAndRegisterPersistentLockService( + KeyValueService kvs, + Consumer env, boolean initializeAsync) { if (!kvs.supportsCheckAndSet()) { return new NoOpPersistentLockService(); } PersistentLockService pls = KvsBackedPersistentLockService.create(kvs, initializeAsync); - env.register(pls); - env.register(new CheckAndSetExceptionMapper()); + env.accept(pls); + env.accept(new CheckAndSetExceptionMapper()); return pls; } @@ -400,9 +497,9 @@ private static PersistentLockService createAndRegisterPersistentLockService(KeyV @Deprecated public static LockAndTimestampServices createLockAndTimestampServices( AtlasDbConfig config, - Environment env, - Supplier lock, - Supplier time) { + Consumer env, + com.google.common.base.Supplier lock, + com.google.common.base.Supplier time) { LockAndTimestampServices lockAndTimestampServices = createRawInstrumentedServices(config, env, @@ -420,9 +517,9 @@ public static LockAndTimestampServices createLockAndTimestampServices( static LockAndTimestampServices createLockAndTimestampServices( AtlasDbConfig config, java.util.function.Supplier runtimeConfigSupplier, - Environment env, - Supplier lock, - Supplier time, + Consumer env, + com.google.common.base.Supplier lock, + com.google.common.base.Supplier time, TimestampStoreInvalidator invalidator, String userAgent) { LockAndTimestampServices lockAndTimestampServices = @@ -459,9 +556,9 @@ private static LockAndTimestampServices withRequestBatchingTimestampService( @VisibleForTesting static LockAndTimestampServices createRawInstrumentedServices( AtlasDbConfig config, - Environment env, - Supplier lock, - Supplier time, + Consumer env, + com.google.common.base.Supplier lock, + com.google.common.base.Supplier time, TimestampStoreInvalidator invalidator, String userAgent) { if (config.leader().isPresent()) { @@ -513,11 +610,11 @@ private static LockAndTimestampServices getLockAndTimestampServices( private static LockAndTimestampServices createRawLeaderServices( LeaderConfig leaderConfig, - Environment env, - Supplier lock, - Supplier time, + Consumer env, + com.google.common.base.Supplier lock, + com.google.common.base.Supplier time, String userAgent) { - // Create local services, that may or may not end up being registered in an environment. + // Create local services, that may or may not end up being registered in an Consumer. LocalPaxosServices localPaxosServices = Leaders.createAndRegisterLocalServices(env, leaderConfig, userAgent); LeaderElectionService leader = localPaxosServices.leaderElectionService(); LockService localLock = ServiceCreator.createInstrumentedService( @@ -526,8 +623,8 @@ private static LockAndTimestampServices createRawLeaderServices( TimestampService localTime = ServiceCreator.createInstrumentedService( AwaitingLeadershipProxy.newProxyInstance(TimestampService.class, time, leader), TimestampService.class); - env.register(localLock); - env.register(localTime); + env.accept(localLock); + env.accept(localTime); // Create remote services, that may end up calling our own local services. ImmutableServerListConfig serverListConfig = ImmutableServerListConfig.builder() @@ -611,14 +708,14 @@ private static LockAndTimestampServices createRawRemoteServices(AtlasDbConfig co } private static LockAndTimestampServices createRawEmbeddedServices( - Environment env, - Supplier lock, - Supplier time) { + Consumer env, + com.google.common.base.Supplier lock, + com.google.common.base.Supplier time) { LockService lockService = ServiceCreator.createInstrumentedService(lock.get(), LockService.class); TimestampService timeService = ServiceCreator.createInstrumentedService(time.get(), TimestampService.class); - env.register(lockService); - env.register(timeService); + env.accept(lockService); + env.accept(timeService); return ImmutableLockAndTimestampServices.builder() .lock(lockService) @@ -633,8 +730,4 @@ public interface LockAndTimestampServices { TimestampService timestamp(); TimelockService timelock(); } - - public interface Environment { - void register(Object resource); - } } diff --git a/atlasdb-config/src/test/java/com/palantir/atlasdb/factory/TransactionManagersTest.java b/atlasdb-config/src/test/java/com/palantir/atlasdb/factory/TransactionManagersTest.java index a9f58c69304..7d02281947e 100644 --- a/atlasdb-config/src/test/java/com/palantir/atlasdb/factory/TransactionManagersTest.java +++ b/atlasdb-config/src/test/java/com/palantir/atlasdb/factory/TransactionManagersTest.java @@ -56,7 +56,6 @@ import com.github.tomakehurst.wiremock.core.WireMockConfiguration; import com.github.tomakehurst.wiremock.junit.WireMockRule; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSortedMap; import com.jayway.awaitility.Awaitility; import com.palantir.atlasdb.config.AtlasDbConfig; @@ -129,7 +128,7 @@ public class TransactionManagersTest { private AtlasDbConfig config; private AtlasDbRuntimeConfig runtimeConfig; - private TransactionManagers.Environment environment; + private Consumer environment; private TimestampStoreInvalidator invalidator; private Consumer originalAsyncMethod; @@ -177,7 +176,7 @@ public void setup() throws JsonProcessingException { runtimeConfig = mock(AtlasDbRuntimeConfig.class); when(runtimeConfig.timestampClient()).thenReturn(ImmutableTimestampClientConfig.of(false)); - environment = mock(TransactionManagers.Environment.class); + environment = mock(Consumer.class); invalidator = mock(TimestampStoreInvalidator.class); when(invalidator.backupAndInvalidate()).thenReturn(EMBEDDED_BOUND); @@ -260,8 +259,10 @@ public void setsGlobalDefaultLockTimeout() { .keyValueService(new InMemoryAtlasDbConfig()) .defaultLockTimeoutSeconds((int) expectedTimeout.getTime()) .build(); - TransactionManagers.create(realConfig, Optional::empty, - ImmutableSet.of(), environment, false); + TransactionManagers.builder() + .config(realConfig) + .registrar(environment) + .buildSerializable(); assertEquals(expectedTimeout, LockRequest.getDefaultLockTimeout()); @@ -300,8 +301,10 @@ public void runsClosingCallbackOnShutdown() throws Exception { Runnable callback = mock(Runnable.class); - SerializableTransactionManager manager = TransactionManagers.create( - realConfig, Optional::empty, ImmutableSet.of(), environment, false); + SerializableTransactionManager manager = TransactionManagers.builder() + .config(realConfig) + .registrar(environment) + .buildSerializable(); manager.registerClosingCallback(callback); manager.close(); verify(callback, times(1)).run(); @@ -313,7 +316,10 @@ public void keyValueServiceMetricsDoNotContainUserAgent() { .keyValueService(new InMemoryAtlasDbConfig()) .build(); - TransactionManagers.create(realConfig, Optional::empty, ImmutableSet.of(), environment, false); + TransactionManagers.builder() + .config(realConfig) + .registrar(environment) + .buildSerializable(); assertThat(metricsRule.metrics().getNames().stream() .anyMatch(metricName -> metricName.contains(USER_AGENT_NAME)), is(false)); } @@ -373,7 +379,7 @@ private void setUpForLocalServices() throws IOException { .withStatus(200) .withBody(("\"" + localPingableLeader.getUUID().toString() + "\"").getBytes()))); return null; - }).when(environment).register(isA(PingableLeader.class)); + }).when(environment).accept(isA(PingableLeader.class)); setupLeaderBlockInConfig(); } diff --git a/atlasdb-console/src/main/groovy/com/palantir/atlasdb/console/module/AtlasCoreModule.groovy b/atlasdb-console/src/main/groovy/com/palantir/atlasdb/console/module/AtlasCoreModule.groovy index 83da1d2f12e..27cdb9bb8c3 100644 --- a/atlasdb-console/src/main/groovy/com/palantir/atlasdb/console/module/AtlasCoreModule.groovy +++ b/atlasdb-console/src/main/groovy/com/palantir/atlasdb/console/module/AtlasCoreModule.groovy @@ -21,7 +21,6 @@ import com.palantir.atlasdb.api.AtlasDbService import com.palantir.atlasdb.api.TransactionToken import com.palantir.atlasdb.config.AtlasDbConfig import com.palantir.atlasdb.config.AtlasDbConfigs -import com.palantir.atlasdb.config.AtlasDbRuntimeConfig import com.palantir.atlasdb.console.AtlasConsoleModule import com.palantir.atlasdb.console.AtlasConsoleService import com.palantir.atlasdb.console.AtlasConsoleServiceImpl @@ -31,12 +30,10 @@ import com.palantir.atlasdb.factory.TransactionManagers import com.palantir.atlasdb.impl.AtlasDbServiceImpl import com.palantir.atlasdb.impl.TableMetadataCache import com.palantir.atlasdb.jackson.AtlasJacksonModule -import com.palantir.atlasdb.table.description.Schema import com.palantir.atlasdb.transaction.impl.SerializableTransactionManager import groovy.json.JsonBuilder import groovy.json.JsonOutput import groovy.transform.CompileStatic -import java.util.function.Supplier /** * Public methods that clients can call within AtlasConsole. @@ -217,20 +214,10 @@ class AtlasCoreModule implements AtlasConsoleModule { } private setupConnection(AtlasDbConfig config) { - SerializableTransactionManager tm = TransactionManagers.create( - config, - new Supplier>() { - @Override - Optional get() { - return Optional.empty() - } - }, - ImmutableSet.of(), - new com.palantir.atlasdb.factory.TransactionManagers.Environment() { - @Override - public void register(Object resource) { - } - }, true) + SerializableTransactionManager tm = TransactionManagers.builder() + .config(config) + .allowHiddenTableAccess(true) + .buildSerializable(); TableMetadataCache cache = new TableMetadataCache(tm.getKeyValueService()) AtlasDbService service = new AtlasDbServiceImpl(tm.getKeyValueService(), tm, cache) ObjectMapper serviceMapper = new ObjectMapper() diff --git a/atlasdb-ete-tests/src/main/java/com/palantir/atlasdb/AtlasDbEteServer.java b/atlasdb-ete-tests/src/main/java/com/palantir/atlasdb/AtlasDbEteServer.java index c260b5ea4ff..e46d4fda10d 100644 --- a/atlasdb-ete-tests/src/main/java/com/palantir/atlasdb/AtlasDbEteServer.java +++ b/atlasdb-ete-tests/src/main/java/com/palantir/atlasdb/AtlasDbEteServer.java @@ -15,7 +15,6 @@ */ package com.palantir.atlasdb; -import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -50,8 +49,6 @@ public class AtlasDbEteServer extends Application { private static final Logger log = LoggerFactory.getLogger(AtlasDbEteServer.class); private static final long CREATE_TRANSACTION_MANAGER_MAX_WAIT_TIME_SECS = 60; private static final long CREATE_TRANSACTION_MANAGER_POLL_INTERVAL_SECS = 5; - - private static final boolean DONT_SHOW_HIDDEN_TABLES = false; private static final Set ETE_SCHEMAS = ImmutableSet.of( CheckAndSetSchema.getSchema(), TodoSchema.getSchema()); @@ -101,12 +98,11 @@ private TransactionManager createTransactionManagerWithRetry(AtlasDbConfig confi } private TransactionManager createTransactionManager(AtlasDbConfig config, Environment environment) { - return TransactionManagers.create( - config, - Optional::empty, - ETE_SCHEMAS, - environment.jersey()::register, - DONT_SHOW_HIDDEN_TABLES); + return TransactionManagers.builder() + .config(config) + .schemas(ETE_SCHEMAS) + .registrar(environment.jersey()::register) + .buildSerializable(); } private void enableEnvironmentVariablesInConfig(Bootstrap bootstrap) { diff --git a/atlasdb-service-server/src/main/java/com/palantir/atlasdb/server/AtlasDbServiceServer.java b/atlasdb-service-server/src/main/java/com/palantir/atlasdb/server/AtlasDbServiceServer.java index cd1fcdf01ae..b91364cfd7b 100644 --- a/atlasdb-service-server/src/main/java/com/palantir/atlasdb/server/AtlasDbServiceServer.java +++ b/atlasdb-service-server/src/main/java/com/palantir/atlasdb/server/AtlasDbServiceServer.java @@ -15,9 +15,6 @@ */ package com.palantir.atlasdb.server; -import java.util.Optional; - -import com.google.common.collect.ImmutableSet; import com.palantir.atlasdb.factory.TransactionManagers; import com.palantir.atlasdb.impl.AtlasDbServiceImpl; import com.palantir.atlasdb.impl.TableMetadataCache; @@ -46,12 +43,10 @@ public void initialize(Bootstrap bootstrap) { public void run(AtlasDbServiceServerConfiguration config, final Environment environment) throws Exception { AtlasDbMetrics.setMetricRegistry(environment.metrics()); - SerializableTransactionManager tm = TransactionManagers.create( - config.getConfig(), - Optional::empty, - ImmutableSet.of(), - environment.jersey()::register, - false); + SerializableTransactionManager tm = TransactionManagers.builder() + .config(config.getConfig()) + .registrar(environment.jersey()::register) + .buildSerializable(); TableMetadataCache cache = new TableMetadataCache(tm.getKeyValueService()); diff --git a/docs/source/release_notes/release-notes.rst b/docs/source/release_notes/release-notes.rst index b6f1ee93545..62624dc9adf 100644 --- a/docs/source/release_notes/release-notes.rst +++ b/docs/source/release_notes/release-notes.rst @@ -50,6 +50,12 @@ develop Now, we only request the latest version of the specific column requested, if only one column is requested. Requesting multiple columns still results in the previous behavior, however this will also be optimized in a future release. (`Pull Request `__) + * - |deprecated| |improved| + - ``SerializableTransactionManager`` is now created via an immutable builder instead of a long list of individual arguments. Use ``TransactionManagers.builder()`` + to get the builder and once completely configured, build the transaction manager via the builder's ``.buildSerializable()`` method. + The existing ``create`` methods are deprecated and will be removed by November 15th, 2017. + (`Pull Request `__) + .. <<<<------------------------------------------------------------------------------------------------------------->>>> ======= diff --git a/timelock-server/src/integTest/java/com/palantir/atlasdb/timelock/AsyncTimelockServiceTransactionIntegrationTest.java b/timelock-server/src/integTest/java/com/palantir/atlasdb/timelock/AsyncTimelockServiceTransactionIntegrationTest.java index 79faecdc4a9..fa9dbdd4616 100644 --- a/timelock-server/src/integTest/java/com/palantir/atlasdb/timelock/AsyncTimelockServiceTransactionIntegrationTest.java +++ b/timelock-server/src/integTest/java/com/palantir/atlasdb/timelock/AsyncTimelockServiceTransactionIntegrationTest.java @@ -22,7 +22,6 @@ import java.nio.file.Paths; import java.util.List; -import java.util.Optional; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -87,8 +86,7 @@ public AsyncTimelockServiceTransactionIntegrationTest(TestableTimelockCluster cl .build()) .build()) .build(); - txnManager = TransactionManagers.create(config, () -> Optional.empty(), ImmutableSet.of(), - ignored -> { }, false); + txnManager = TransactionManagers.builder().config(config).buildSerializable(); txnManager.getKeyValueService().createTable(TABLE, AtlasDbConstants.GENERIC_TABLE_METADATA); }