diff --git a/CHANGELOG.md b/CHANGELOG.md index c04c7c4f42..dda372fdc1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,9 +1,6 @@ -DynoQueueDAO - removed deprecated Constructors -int getLongPollTimeoutInMS() - removed deprecated Worker method in client +DynoQueueDAO - removed deprecated Constructors int getLongPollTimeoutInMS() - removed deprecated Worker method in client -workflow.sqs.event.queue.enabled -workflow.amqp.event.queue.enabled -workflow.nats.event.queue.enabled +workflow.sqs.event.queue.enabled workflow.amqp.event.queue.enabled workflow.nats.event.queue.enabled workflow.nats_stream.event.queue.enabled workflow.executor.service.max.threads=50(default) @@ -12,20 +9,277 @@ workflow.events.default.queue.type=sqs (default)/amqp (Fixed) workflow.listener.queue.prefix -workflow.status.listener.type=stub(default)/archive/queue_status_publisher +workflow.status.listener.type=stub(default)/archive/queue_publisher -conductor.metrics.logger.enabled -conductor.metrics.logger.reportPeriodSeconds +conductor.metrics-logger.enabled HTTP task - removed OAuth support (Create a task for OAuth2 support) Removed deprecated API - /queue/requeue from /tasks +Upgraded protobuf-java to 3.13.0 Upgraded grpc-protobuf to 1.33.+ Renamed DynoProxy to JedisProxy Removed support for +EmbeddedElasticSearch -Upgraded protobuf-java to 3.13.0 -Upgraded grpc-protobuf to 1.33.+ -Renamed DynoProxy to JedisProxy -Removed support for EmbeddedElasticSearch +Ignored a flaky test class - LocalOnlyLockTest. Test Harness module uses TestContainers for MySql,Postgres & +Elasticsearch -Ignored a flaky test class - LocalOnlyLockTest. -Test Harness module uses TestContainers for MySql,Postgres & Elasticsearch +Modified properties in the `azureblob-storage` module: + +| Old | New | +| --- | --- | +| workflow.external.payload.storage.azure_blob.connection_string | conductor.external-payload-storage.azureblob.connectionString | +| workflow.external.payload.storage.azure_blob.container_name | conductor.external-payload-storage.azureblob.containerName | +| workflow.external.payload.storage.azure_blob.endpoint | conductor.external-payload-storage.azureblob.endpoint | +| workflow.external.payload.storage.azure_blob.sas_token | conductor.external-payload-storage.azureblob.sasToken | +| workflow.external.payload.storage.azure_blob.signedurlexpirationseconds | conductor.external-payload-storage.azureblob.signedUrlExpirationSeconds | +| workflow.external.payload.storage.azure_blob.workflow_input_path | conductor.external-payload-storage.azureblob.workflowInputPath | +| workflow.external.payload.storage.azure_blob.workflow_output_path | conductor.external-payload-storage.azureblob.workflowOutputPath | +| workflow.external.payload.storage.azure_blob.task_input_path | conductor.external-payload-storage.azureblob.taskInputPath | +| workflow.external.payload.storage.azure_blob.task_output_path | conductor.external-payload-storage.azureblob.taskOutputPath | + +Modified properties in the `cassandra-persistence` module: + +| Old | New | +| --- | --- | +| workflow.cassandra.host | conductor.cassandra.hostAddress | +| workflow.cassandra.port | conductor.cassandra.port | +| workflow.cassandra.cluster | conductor.cassandra.cluster | +| workflow.cassandra.keyspace | conductor.cassandra.keyspace | +| workflow.cassandra.shard.size | conductor.cassandra.shardSize | +| workflow.cassandra.replication.strategy | conductor.cassandra.replicationStrategy | +| workflow.cassandra.replication.factor.key | conductor.cassandra.replicationFactorKey | +| workflow.cassandra.replication.factor.value | conductor.cassandra.replicationFactorValue | +| workflow.cassandra.read.consistency.level | conductor.cassandra.readConsistencyLevel | +| workflow.cassandra.write.consistency.level | conductor.cassandra.writeConsistencyLevel | +| conductor.taskdef.cache.refresh.time.seconds | conductor.cassandra.taskDefCacheRefreshTimeSecs | +| conductor.eventhandler.cache.refresh.time.seconds | conductor.cassandra.eventHandlerCacheRefreshTimeSecs | +| workflow.event.execution.persistence.ttl.seconds | conductor.cassandra.eventExecutionPersistenceTTLSecs | + +Modified properties in the `contribs` module: + +| Old | New | +| --- | --- | +| workflow.archival.ttl.seconds | conductor.workflow-status-listener.archival.ttlSeconds | +| workflow.archival.delay.queue.worker.thread.count | conductor.workflow-status-listener.archival.delayQueueWorkerThreadCount | +| workflow.archival.delay.seconds | conductor.workflow-status-listener.archival.delaySeconds | +| | | +| workflowstatuslistener.publisher.success.queue | conductor.workflow-status-listener.queue-publisher.successQueue | +| workflowstatuslistener.publisher.failure.queue | conductor.workflow-status-listener.queue-publisher.failureQueue | +| | | +| com.netflix.conductor.contribs.metrics.LoggingMetricsModule.reportPeriodSeconds | conductor.metrics-logger.reportPeriodSeconds | +| | | +| workflow.event.queues.amqp.batchSize | conductor.event-queues.amqp.batchSize | +| workflow.event.queues.amqp.pollTimeInMs | conductor.event-queues.amqp.pollTimeMs | +| workflow.event.queues.amqp.hosts | conductor.event-queues.amqp.hosts | +| workflow.event.queues.amqp.username | conductor.event-queues.amqp.username | +| workflow.event.queues.amqp.password | conductor.event-queues.amqp.password | +| workflow.event.queues.amqp.virtualHost | conductor.event-queues.amqp.virtualHost | +| workflow.event.queues.amqp.port | conductor.event-queues.amqp.port.port | +| workflow.event.queues.amqp.connectionTimeout | conductor.event-queues.amqp.connectionTimeout | +| workflow.event.queues.amqp.useNio | conductor.event-queues.amqp.useNio | +| workflow.event.queues.amqp.durable | conductor.event-queues.amqp.durable | +| workflow.event.queues.amqp.exclusive | conductor.event-queues.amqp.exclusive | +| workflow.event.queues.amqp.autoDelete | conductor.event-queues.amqp.autoDelete | +| workflow.event.queues.amqp.contentType | conductor.event-queues.amqp.contentType | +| workflow.event.queues.amqp.contentEncoding | conductor.event-queues.amqp.contentEncoding | +| workflow.event.queues.amqp.amqp_exchange | conductor.event-queues.amqp.exchangeType | +| workflow.event.queues.amqp.deliveryMode | conductor.event-queues.amqp.deliveryMode | +| workflow.listener.queue.useExchange | conductor.event-queues.amqp.useExchange | +| workflow.listener.queue.prefix | conductor.event-queues.amqp.listenerQueuePrefix | +| | | +| io.nats.streaming.clusterId | conductor.event-queues.nats-stream.clusterId | +| io.nats.streaming.durableName | conductor.event-queues.nats-stream.durableName | +| io.nats.streaming.url | conductor.event-queues.nats-stream.url | +| | | +| workflow.event.queues.sqs.batchSize | conductor.event-queues.sqs.batchSize | +| workflow.event.queues.sqs.pollTimeInMS | conductor.event-queues.sqs.pollTimeMs | +| workflow.event.queues.sqs.visibilityTimeoutInSeconds | conductor.event-queues.sqs.visibilityTimeoutSeconds | +| workflow.listener.queue.prefix | conductor.event-queues.sqs.listenerQueuePrefix | +| workflow.listener.queue.authorizedAccounts | conductor.event-queues.sqs.authorizedAccounts | +| | | +| workflow.external.payload.storage.s3.bucket | conductor.external-payload-storage.s3 | +| workflow.external.payload.storage.s3.signedurlexpirationseconds | conductor.external-payload-storage.s3 | +| workflow.external.payload.storage.s3.region | conductor.external-payload-storage.s3 | +| | | +| http.task.read.timeout | conductor.tasks.http.readTimeout | +| http.task.connect.timeout | conductor.tasks.http.connectTimeout | +| | | +| kafka.publish.request.timeout.ms | conductor.tasks.kafka-publish.requestTimeoutMs | +| kafka.publish.max.block.ms | conductor.tasks.kafka-publish.maxBlockMs | +| kafka.publish.producer.cache.size | conductor.tasks.kafka-publish.cacheSize | +| kafka.publish.producer.cache.time.ms | conductor.tasks.kafka-publish.cacheTimeMs | + +Modified properties in the `core` module: + +| Old | New | +| --- | --- | +| environment | _removed_ | +| STACK | conductor.app.stack | +| APP_ID | conductor.app.appId | +| workflow.executor.service.max.threads | conductor.app.executorServiceMaxThreadCount | +| decider.sweep.frequency.seconds | conductor.app.sweepFrequencySeconds | +| decider.sweep.disable | conductor.app.sweepDisabled | +| workflow.sweeper.thread.count | conductor.app.sweeperThreadCount | +| workflow.event.processor.thread.count | conductor.app.eventProcessorThreadCount | +| workflow.event.message.indexing.enabled | conductor.app.eventMessageIndexingEnabled | +| workflow.event.execution.indexing.enabled | conductor.app.eventExecutionIndexingEnabled | +| workflow.decider.locking.enabled | conductor.app.workflowExecutionLockEnabled | +| workflow.locking.lease.time.ms | conductor.app.lockLeaseTimeMs | +| workflow.locking.time.to.try.ms | conductor.app.lockTimeToTryMs | +| tasks.active.worker.lastpoll | conductor.app.activeWorkerLastPollSecs | +| task.queue.message.postponeSeconds | conductor.app.taskExecutionPostponeSeconds | +| workflow.taskExecLog.indexing.enabled | conductor.app.taskExecLogIndexingEnabled | +| async.indexing.enabled | conductor.app.asyncIndexingEnabled | +| workflow.system.task.worker.thread.count | conductor.app.systemTaskWorkerThreadCount | +| workflow.system.task.worker.callback.seconds | conductor.app.systemTaskWorkerCallbackSeconds | +| workflow.system.task.worker.poll.interval | conductor.app.systemTaskWorkerPollInterval | +| workflow.system.task.worker.executionNameSpace | conductor.app.systemTaskWorkerExecutionNamespace | +| workflow.isolated.system.task.worker.thread.count | conductor.app.isolatedSystemTaskWorkerThreadCount | +| workflow.system.task.queue.pollCount | conductor.app.systemTaskMaxPollCount | +| conductor.disable.async.workers | conductor.app.systemTaskWorkersDisabled | +| async.update.short.workflow.duration.seconds | conductor.app.asyncUpdateShortRunningWorkflowDuration | +| async.update.delay.seconds | conductor.app.asyncUpdateDelay | +| workflow.owner.email.mandatory | conductor.app.ownerEmailMandatory | +| workflow.repairservice.enabled | conductor.app.workflowRepairServiceEnabled | +| workflow.event.queue.scheduler.poll.thread.count | conductor.app.eventSchedulerPollThreadCount | +| workflow.dyno.queues.pollingInterval | conductor.app.eventQueuePollIntervalMs | +| workflow.dyno.queues.pollCount | conductor.app.eventQueuePollCount | +| workflow.dyno.queues.longPollTimeout | conductor.app.eventQueueLongPollTimeout | +| conductor.workflow.input.payload.threshold.kb | conductor.app.workflowInputPayloadSizeThresholdKB | +| conductor.max.workflow.input.payload.threshold.kb | conductor.app.maxWorkflowInputPayloadSizeThresholdKB | +| conductor.workflow.output.payload.threshold.kb | conductor.app.workflowOutputPayloadSizeThresholdKB | +| conductor.max.workflow.output.payload.threshold.kb | conductor.app.maxWorkflowOutputPayloadSizeThresholdKB | +| conductor.task.input.payload.threshold.kb | conductor.app.taskInputPayloadSizeThresholdKB | +| conductor.max.task.input.payload.threshold.kb | conductor.app.maxTaskInputPayloadSizeThresholdKB | +| conductor.task.output.payload.threshold.kb | conductor.app.taskOutputPayloadSizeThresholdKB | +| conductor.max.task.output.payload.threshold.kb | conductor.app.maxTaskOutputPayloadSizeThresholdKB | +| conductor.max.workflow.variables.payload.threshold.kb | conductor.app.maxWorkflowVariablesPayloadSizeThresholdKB | +| | | +| workflow.isolated.system.task.enable | conductor.app.isolatedSystemTaskEnabled | +| workflow.isolated.system.task.poll.time.secs | conductor.app.isolatedSystemTaskQueuePollIntervalSecs | +| | | +| workflow.task.pending.time.threshold.minutes | conductor.app.taskPendingTimeThresholdMins | +| | | +| workflow.monitor.metadata.refresh.counter | conductor.workflow-monitor.metadataRefreshInterval | +| workflow.monitor.stats.freq.seconds | conductor.workflow-monitor.statsFrequencySeconds | + +Modified properties in the `es6-persistence` module: + +| Old | New | +| --- | --- | +| workflow.elasticsearch.version | conductor.elasticsearch.version | +| workflow.elasticsearch.url | conductor.elasticsearch.url | +| workflow.elasticsearch.index.name | conductor.elasticsearch.indexName | +| workflow.elasticsearch.tasklog.index.name | conductor.elasticsearch.taskLogIndexName | +| workflow.elasticsearch.cluster.health.color | conductor.elasticsearch.clusterHealthColor | +| workflow.elasticsearch.archive.search.batchSize | conductor.elasticsearch.archiveSearchBatchSize | +| workflow.elasticsearch.index.batchSize | conductor.elasticsearch.indexBatchSize | +| workflow.elasticsearch.async.dao.worker.queue.size | conductor.elasticsearch.asyncWorkerQueueSize | +| workflow.elasticsearch.async.dao.max.pool.size | conductor.elasticsearch.asyncMaxPoolSize | +| workflow.elasticsearch.async.buffer.flush.timeout.seconds | conductor.elasticsearch.asyncBufferFlushTimeoutSecs | +| workflow.elasticsearch.index.shard.count | conductor.elasticsearch.indexShardCount | +| workflow.elasticsearch.index.replicas.count | conductor.elasticsearch.indexReplicasCount | +| tasklog.elasticsearch.query.size | conductor.elasticsearch.taskLogResultLimit | +| workflow.elasticsearch.rest.client.connectionRequestTimeout.milliseconds | conductor.elasticsearch.restClientConnectionRequestTimeoutMs | +| workflow.elasticsearch.auto.index.management.enabled | conductor.elasticsearch.autoIndexManagementEnabled | +| workflow.elasticsearch.document.type.override | conductor.elasticsearch.documentTypeOverride | + +Modified properties in the `grpc-server` module: + +| Old | New | +| --- | --- | +| conductor.grpc.server.port | conductor.grpc-server.port | +| conductor.grpc.server.reflectionEnabled | conductor.grpc-server.reflectionEnabled | + +Modified properties in the `mysql-persistence` module: + +| Old | New | +| --- | --- | +| jdbc.url | conductor.mysql.jdbcUrl | +| jdbc.username | conductor.mysql.jdbcUsername | +| jdbc.password | conductor.mysql.jdbcPassword | +| flyway.enabled | conductor.mysql.flywayEnabled | +| flyway.table | conductor.mysql.flywayTable | +| conductor.mysql.connection.pool.size.max | conductor.mysql.connectionPoolMaxSize | +| conductor.mysql.connection.pool.idle.min | conductor.mysql.connectionPoolMinIdle | +| conductor.mysql.connection.lifetime.max | conductor.mysql.connectionMaxLifetime | +| conductor.mysql.connection.idle.timeout | conductor.mysql.connectionIdleTimeout | +| conductor.mysql.connection.timeout | conductor.mysql.connectionTimeout | +| conductor.mysql.transaction.isolation.level | conductor.mysql.transactionIsolationLevel | +| conductor.mysql.autocommit | conductor.mysql.autoCommit | +| conductor.taskdef.cache.refresh.time.seconds | conductor.mysql.taskDefCacheRefreshTimeSecs | + +Modified properties in the `postgres-persistence` module: + +| Old | New | +| --- | --- | +| jdbc.url | conductor.postgres.jdbcUrl | +| jdbc.username | conductor.postgres.jdbcUsername | +| jdbc.password | conductor.postgres.jdbcPassword | +| flyway.enabled | conductor.postgres.flywayEnabled | +| flyway.table | conductor.postgres.flywayTable | +| conductor.postgres.connection.pool.size.max | conductor.postgres.connectionPoolMaxSize | +| conductor.postgres.connection.pool.idle.min | conductor.postgres.connectionPoolMinIdle | +| conductor.postgres.connection.lifetime.max | conductor.postgres.connectionMaxLifetime | +| conductor.postgres.connection.idle.timeout | conductor.postgres.connectionIdleTimeout | +| conductor.postgres.connection.timeout | conductor.postgres.connectionTimeout | +| conductor.postgres.transaction.isolation.level | conductor.postgres.transactionIsolationLevel | +| conductor.postgres.autocommit | conductor.postgres.autoCommit | +| conductor.taskdef.cache.refresh.time.seconds | conductor.postgres.taskDefCacheRefreshTimeSecs | + +Modified properties in the `redis-lock` module: + +| Old | New | +| --- | --- | +| workflow.redis.locking.server.type | conductor.redis-lock.serverType | +| workflow.redis.locking.server.address | conductor.redis-lock.serverAddress | +| workflow.redis.locking.server.password | conductor.redis-lock.serverPassword | +| workflow.redis.locking.server.master.name | conductor.redis-lock.serverMasterName | +| workflow.decider.locking.namespace | conductor.redis-lock.namespace | +| workflow.decider.locking.exceptions.ignore | conductor.redis-lock.ignoreLockingExceptions | + +Modified properties in the `redis-persistence` module: + +| Old | New | +| --- | --- | +| EC2_REGION | conductor.redis.dataCenterRegion | +| EC2_AVAILABILITY_ZONE | conductor.redis.availabilityZone | +| workflow.dynomite.cluster | _removed_ | +| workflow.dynomite.cluster.name | conductor.redis.clusterName | +| workflow.dynomite.cluster.hosts | conductor.redis.hosts | +| workflow.namespace.prefix | conductor.redis.workflowNamespacePrefix | +| workflow.namespace.queue.prefix | conductor.redis.queueNamespacePrefix | +| workflow.dyno.keyspace.domain | conductor.redis.keyspaceDomain | +| workflow.dynomite.connection.maxConnsPerHost | conductor.redis.maxConnectionsPerHost | +| queues.dynomite.nonQuorum.port | conductor.redis.queuesNonQuorumPort | +| workflow.dyno.queue.sharding.strategy | conductor.redis.queueShardingStrategy | +| conductor.taskdef.cache.refresh.time.seconds | conductor.redis.taskDefCacheRefreshTimeSecs | +| workflow.event.execution.persistence.ttl.seconds | conductor.redis.eventExecutionPersistenceTTLSecs | + +Modified properties in the `zookeeper-lock` module: + +| Old | New | +| --- | --- | +| workflow.zookeeper.lock.connection | conductor.zookeeper-lock.connectionString | +| workflow.zookeeper.lock.sessionTimeoutMs | conductor.zookeeper-lock.sessionTimeoutMs | +| workflow.zookeeper.lock.connectionTimeoutMs | conductor.zookeeper-lock.connectionTimeoutMs | +| workflow.decider.locking.namespace | conductor.zookeeper-lock.namespace | + +Modified properties that are used for configuring components: + +| Old | New | +| --- | --- | +| db | conductor.db.type | +| workflow.indexing.enabled | conductor.indexing.enabled | +| conductor.grpc.server.enabled | conductor.grpc-server.enabled | +| workflow.external.payload.storage | conductor.external-payload-storage.type | +| workflow.default.event.processor.enabled | conductor.default-event-processor.enabled | +| workflow.events.default.queue.type | conductor.default-event-queue.type | +| workflow.status.listener.type | conductor.workflow-status-listener.type | +| workflow.decider.locking.server | conductor.workflow-execution-lock.type | +| | | +| workflow.default.event.queue.enabled | conductor.event-queues.default.enabled | +| workflow.sqs.event.queue.enabled | conductor.event-queues.sqs.enabled | +| workflow.amqp.event.queue.enabled | conductor.event-queues.amqp.enabled | +| workflow.nats.event.queue.enabled | conductor.event-queues.nats.enabled | +| workflow.nats_stream.event.queue.enabled | conductor.event-queues.nats-stream.enabled | diff --git a/azureblob-storage/src/main/java/com/netflix/conductor/azureblob/config/AzureBlobConfiguration.java b/azureblob-storage/src/main/java/com/netflix/conductor/azureblob/config/AzureBlobConfiguration.java index fe30bac450..458eb8b57d 100644 --- a/azureblob-storage/src/main/java/com/netflix/conductor/azureblob/config/AzureBlobConfiguration.java +++ b/azureblob-storage/src/main/java/com/netflix/conductor/azureblob/config/AzureBlobConfiguration.java @@ -15,12 +15,13 @@ import com.netflix.conductor.azureblob.storage.AzureBlobPayloadStorage; import com.netflix.conductor.common.utils.ExternalPayloadStorage; import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; -@SuppressWarnings("SpringJavaInjectionPointsAutowiringInspection") -@Configuration -@ConditionalOnProperty(prefix = "workflow", name = "external.payload.storage", havingValue = "AZURE_BLOB") +@Configuration(proxyBeanMethods = false) +@EnableConfigurationProperties(AzureBlobProperties.class) +@ConditionalOnProperty(name = "conductor.external-payload-storage.type", havingValue = "azureblob") public class AzureBlobConfiguration { @Bean diff --git a/azureblob-storage/src/main/java/com/netflix/conductor/azureblob/config/AzureBlobProperties.java b/azureblob-storage/src/main/java/com/netflix/conductor/azureblob/config/AzureBlobProperties.java index a4a049b35b..55c8a2f1f7 100644 --- a/azureblob-storage/src/main/java/com/netflix/conductor/azureblob/config/AzureBlobProperties.java +++ b/azureblob-storage/src/main/java/com/netflix/conductor/azureblob/config/AzureBlobProperties.java @@ -12,77 +12,125 @@ */ package com.netflix.conductor.azureblob.config; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; -import org.springframework.stereotype.Component; +import org.springframework.boot.context.properties.ConfigurationProperties; -@Component -@ConditionalOnProperty(prefix = "workflow", name = "external.payload.storage", havingValue = "AZURE_BLOB") +@ConfigurationProperties("conductor.external-payload-storage.azureblob") public class AzureBlobProperties { - public static final String CONNECTION_STRING_PROPERTY_NAME = "workflow.external.payload.storage.azure_blob.connection_string"; - public static final String ENDPOINT_PROPERTY_NAME = "workflow.external.payload.storage.azure_blob.endpoint"; - - @Value("${" + CONNECTION_STRING_PROPERTY_NAME + ":#{null}}") - private String connectionString; - - @Value("${workflow.external.payload.storage.azure_blob.container_name:conductor-payloads}") - private String containerName; - - @Value("${" + ENDPOINT_PROPERTY_NAME + ":#{null}}") - private String endpoint; - - @Value("${workflow.external.payload.storage.azure_blob.sas_token:#{null}}") - private String sasToken; - - @Value("${workflow.external.payload.storage.azure_blob.signedurlexpirationseconds:5}") - private int signedUrlExpirationSeconds; - - @Value("${workflow.external.payload.storage.azure_blob.workflow_input_path:workflow/input/}") - private String workflowInputPath; - - @Value("${workflow.external.payload.storage.azure_blob.workflow_output_path:workflow/output/}") - private String workflowOutputPath; - - @Value("${workflow.external.payload.storage.azure_blob.task_input_path:task/input/}") - private String taskInputPath; - - @Value("${workflow.external.payload.storage.azure_blob.task_output_path:task/output/}") - private String taskOutputPath; + /** + * The connection string to be used to connect to Azure Blob storage + */ + private String connectionString = null; + + /** + * The name of the container where the payloads will be stored + */ + private String containerName = "conductor-payloads"; + + /** + * The endpoint to be used to connect to Azure Blob storage + */ + private String endpoint = null; + + /** + * The sas token to be used for authenticating requests + */ + private String sasToken = null; + + /** + * The time for which the shared access signature is valid + */ + private int signedUrlExpirationSeconds = 5; + + /** + * The path at which the workflow inputs will be stored + */ + private String workflowInputPath = "workflow/input/"; + + /** + * The path at which the workflow outputs will be stored + */ + private String workflowOutputPath = "workflow/output/"; + + /** + * The path at which the task inputs will be stored + */ + private String taskInputPath = "task/input/"; + + /** + * The path at which the task outputs will be stored + */ + private String taskOutputPath = "task/output/"; public String getConnectionString() { return connectionString; } + public void setConnectionString(String connectionString) { + this.connectionString = connectionString; + } + public String getContainerName() { return containerName; } + public void setContainerName(String containerName) { + this.containerName = containerName; + } + public String getEndpoint() { return endpoint; } + public void setEndpoint(String endpoint) { + this.endpoint = endpoint; + } + public String getSasToken() { return sasToken; } + public void setSasToken(String sasToken) { + this.sasToken = sasToken; + } + public int getSignedUrlExpirationSeconds() { return signedUrlExpirationSeconds; } + public void setSignedUrlExpirationSeconds(int signedUrlExpirationSeconds) { + this.signedUrlExpirationSeconds = signedUrlExpirationSeconds; + } + public String getWorkflowInputPath() { return workflowInputPath; } + public void setWorkflowInputPath(String workflowInputPath) { + this.workflowInputPath = workflowInputPath; + } + public String getWorkflowOutputPath() { return workflowOutputPath; } + public void setWorkflowOutputPath(String workflowOutputPath) { + this.workflowOutputPath = workflowOutputPath; + } + public String getTaskInputPath() { return taskInputPath; } + public void setTaskInputPath(String taskInputPath) { + this.taskInputPath = taskInputPath; + } + public String getTaskOutputPath() { return taskOutputPath; } + + public void setTaskOutputPath(String taskOutputPath) { + this.taskOutputPath = taskOutputPath; + } } diff --git a/azureblob-storage/src/main/java/com/netflix/conductor/azureblob/storage/AzureBlobPayloadStorage.java b/azureblob-storage/src/main/java/com/netflix/conductor/azureblob/storage/AzureBlobPayloadStorage.java index f04defa756..78d1592904 100644 --- a/azureblob-storage/src/main/java/com/netflix/conductor/azureblob/storage/AzureBlobPayloadStorage.java +++ b/azureblob-storage/src/main/java/com/netflix/conductor/azureblob/storage/AzureBlobPayloadStorage.java @@ -28,16 +28,15 @@ import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.netflix.conductor.core.exception.ApplicationException; import com.netflix.conductor.core.utils.IDGenerator; -import org.apache.commons.lang3.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.InputStream; import java.io.UncheckedIOException; import java.time.OffsetDateTime; import java.time.ZoneOffset; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * An implementation of {@link ExternalPayloadStorage} using Azure Blob for storing large JSON payload data. @@ -83,10 +82,7 @@ public AzureBlobPayloadStorage(AzureBlobProperties properties) { sasTokenCredential = null; } } else { - String msg = "Missing property " - + AzureBlobProperties.CONNECTION_STRING_PROPERTY_NAME - + " OR " - + AzureBlobProperties.ENDPOINT_PROPERTY_NAME; + String msg = "Missing property for connectionString OR endpoint"; LOGGER.error(msg); throw new ApplicationException(ApplicationException.Code.BACKEND_ERROR, msg); } diff --git a/build.gradle b/build.gradle index f8f65881f9..03f50af6df 100644 --- a/build.gradle +++ b/build.gradle @@ -25,6 +25,10 @@ def javaProjects = subprojects.findAll { it.name != "ui" } +configure(javaProjects) { + compileJava.inputs.files(processResources) +} + apply from: "$rootDir/dependencies.gradle" // change the ES version used by Spring Boot Dependency Management plugin diff --git a/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/config/CassandraConfiguration.java b/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/config/CassandraConfiguration.java index 1a3ed1db2c..21bdf8ba20 100644 --- a/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/config/CassandraConfiguration.java +++ b/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/config/CassandraConfiguration.java @@ -26,11 +26,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; @Configuration(proxyBeanMethods = false) -@ConditionalOnProperty(name = "db", havingValue = "cassandra") +@EnableConfigurationProperties(CassandraProperties.class) +@ConditionalOnProperty(name = "conductor.db.type", havingValue = "cassandra") public class CassandraConfiguration { private static final Logger LOGGER = LoggerFactory.getLogger(CassandraConfiguration.class); @@ -80,6 +82,6 @@ public EventHandlerDAO cassandraEventHandlerDAO(Session session, ObjectMapper ob @Bean public Statements statements(CassandraProperties cassandraProperties) { - return new Statements(cassandraProperties.getCassandraKeyspace()); + return new Statements(cassandraProperties.getKeyspace()); } } diff --git a/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/config/CassandraProperties.java b/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/config/CassandraProperties.java index fe23f190a3..6bf07e1db3 100644 --- a/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/config/CassandraProperties.java +++ b/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/config/CassandraProperties.java @@ -13,111 +13,178 @@ package com.netflix.conductor.cassandra.config; import com.datastax.driver.core.ConsistencyLevel; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; -import org.springframework.stereotype.Component; +import org.springframework.boot.context.properties.ConfigurationProperties; -@Component -@ConditionalOnProperty(name = "db", havingValue = "cassandra") +@ConfigurationProperties("conductor.cassandra") public class CassandraProperties { - @Value("${workflow.cassandra.host:127.0.0.1}") - private String hostAddress; + /** + * The address for the cassandra database host + */ + private String hostAddress = "127.0.0.1"; - @Value("${workflow.cassandra.port:9142}") - private int port; + /** + * The port to be used to connect to the cassandra database instance + */ + private int port = 9142; - @Value("${workflow.cassandra.cluster:}") - private String cluster; + /** + * The name of the cassandra cluster + */ + private String cluster = ""; - @Value("${workflow.cassandra.keyspace:conductor}") - private String keyspace; + /** + * The keyspace to be used in the cassandra datastore + */ + private String keyspace = "conductor"; - @Value("${workflow.cassandra.shard.size:100}") - private int shardSize; + /** + * The number of tasks to be stored in a single partition which will be used for sharding workflows in the + * datastore + */ + private int shardSize = 100; - @Value("${workflow.cassandra.replication.strategy:SimpleStrategy}") - private String replicationStrategy; + /** + * The replication strategy with which to configure the keyspace + */ + private String replicationStrategy = "SimpleStrategy"; - @Value("${workflow.cassandra.replication.factor.key:replication_factor}") - private String replicationFactorKey; + /** + * The key to be used while configuring the replication factor + */ + private String replicationFactorKey = "replication_factor"; - @Value("${workflow.cassandra.replication.factor.value:3}") - private int replicationFactorValue; + /** + * The replication factor value with which the keyspace is configured + */ + private int replicationFactorValue = 3; - @Value("${workflow.cassandra.read.consistency.level:#{T(com.datastax.driver.core.ConsistencyLevel).LOCAL_QUORUM.name()}}") - private String readConsistencyLevel; + /** + * The consistency level to be used for read operations + */ + private String readConsistencyLevel = ConsistencyLevel.LOCAL_QUORUM.name(); - @Value("${workflow.cassandra.write.consistency.level:#{T(com.datastax.driver.core.ConsistencyLevel).LOCAL_QUORUM.name()}}") - private String writeConsistencyLevel; + /** + * The consistency level to be used for write operations + */ + private String writeConsistencyLevel = ConsistencyLevel.LOCAL_QUORUM.name(); /** - * the refresh time for the in-memory task definition cache + * The time in seconds after which the in-memory task definitions cache will be refreshed */ - @Value("${conductor.taskdef.cache.refresh.time.seconds:60}") - private int taskDefCacheRefreshTimeSecs; + private int taskDefCacheRefreshTimeSecs = 60; /** - * the refresh time for the in-memory event handler cache + * The time in seconds after which the in-memory event handler cache will be refreshed */ - @Value("${conductor.eventhandler.cache.refresh.time.seconds:60}") - private int eventHandlerRefreshTimeSecs; + private int eventHandlerCacheRefreshTimeSecs = 60; /** - * The time to live in seconds of the event execution persisted + * The time to live in seconds for which the event execution will be persisted */ - @Value("${workflow.event.execution.persistence.ttl.seconds:0}") - private int eventExecutionPersistenceTTLSecs; + private int eventExecutionPersistenceTTLSecs = 0; public String getHostAddress() { return hostAddress; } + public void setHostAddress(String hostAddress) { + this.hostAddress = hostAddress; + } + public int getPort() { return port; } - public String getCassandraCluster() { + public void setPort(int port) { + this.port = port; + } + + public String getCluster() { return cluster; } - public String getCassandraKeyspace() { + public void setCluster(String cluster) { + this.cluster = cluster; + } + + public String getKeyspace() { return keyspace; } + public void setKeyspace(String keyspace) { + this.keyspace = keyspace; + } + public int getShardSize() { return shardSize; } + public void setShardSize(int shardSize) { + this.shardSize = shardSize; + } + public String getReplicationStrategy() { return replicationStrategy; } + public void setReplicationStrategy(String replicationStrategy) { + this.replicationStrategy = replicationStrategy; + } + public String getReplicationFactorKey() { return replicationFactorKey; } + public void setReplicationFactorKey(String replicationFactorKey) { + this.replicationFactorKey = replicationFactorKey; + } + public int getReplicationFactorValue() { return replicationFactorValue; } + public void setReplicationFactorValue(int replicationFactorValue) { + this.replicationFactorValue = replicationFactorValue; + } + public ConsistencyLevel getReadConsistencyLevel() { return ConsistencyLevel.valueOf(readConsistencyLevel); } + public void setReadConsistencyLevel(String readConsistencyLevel) { + this.readConsistencyLevel = readConsistencyLevel; + } + public ConsistencyLevel getWriteConsistencyLevel() { return ConsistencyLevel.valueOf(writeConsistencyLevel); } - public int getTaskDefRefreshTimeSecs() { + public void setWriteConsistencyLevel(String writeConsistencyLevel) { + this.writeConsistencyLevel = writeConsistencyLevel; + } + + public int getTaskDefCacheRefreshTimeSecs() { return taskDefCacheRefreshTimeSecs; } - public int getEventHandlerRefreshTimeSecs() { - return eventHandlerRefreshTimeSecs; + public void setTaskDefCacheRefreshTimeSecs(int taskDefCacheRefreshTimeSecs) { + this.taskDefCacheRefreshTimeSecs = taskDefCacheRefreshTimeSecs; } - public int getEventExecutionPersistenceTTL() { + public int getEventHandlerCacheRefreshTimeSecs() { + return eventHandlerCacheRefreshTimeSecs; + } + + public void setEventHandlerCacheRefreshTimeSecs(int eventHandlerCacheRefreshTimeSecs) { + this.eventHandlerCacheRefreshTimeSecs = eventHandlerCacheRefreshTimeSecs; + } + + public int getEventExecutionPersistenceTTLSecs() { return eventExecutionPersistenceTTLSecs; } + + public void setEventExecutionPersistenceTTLSecs(int eventExecutionPersistenceTTLSecs) { + this.eventExecutionPersistenceTTLSecs = eventExecutionPersistenceTTLSecs; + } } diff --git a/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/dao/CassandraBaseDAO.java b/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/dao/CassandraBaseDAO.java index 3f9fd90b43..8e93bf9a38 100644 --- a/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/dao/CassandraBaseDAO.java +++ b/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/dao/CassandraBaseDAO.java @@ -125,7 +125,7 @@ private void init() { } private String getCreateKeyspaceStatement() { - return SchemaBuilder.createKeyspace(properties.getCassandraKeyspace()) + return SchemaBuilder.createKeyspace(properties.getKeyspace()) .ifNotExists() .with() .replication( @@ -136,7 +136,7 @@ private String getCreateKeyspaceStatement() { } private String getCreateWorkflowsTableStatement() { - return SchemaBuilder.createTable(properties.getCassandraKeyspace(), TABLE_WORKFLOWS) + return SchemaBuilder.createTable(properties.getKeyspace(), TABLE_WORKFLOWS) .ifNotExists() .addPartitionKey(WORKFLOW_ID_KEY, DataType.uuid()) .addPartitionKey(SHARD_ID_KEY, DataType.cint()) @@ -149,7 +149,7 @@ private String getCreateWorkflowsTableStatement() { } private String getCreateTaskLookupTableStatement() { - return SchemaBuilder.createTable(properties.getCassandraKeyspace(), TABLE_TASK_LOOKUP) + return SchemaBuilder.createTable(properties.getKeyspace(), TABLE_TASK_LOOKUP) .ifNotExists() .addPartitionKey(TASK_ID_KEY, DataType.uuid()) .addColumn(WORKFLOW_ID_KEY, DataType.uuid()) @@ -157,7 +157,7 @@ private String getCreateTaskLookupTableStatement() { } private String getCreateTaskDefLimitTableStatement() { - return SchemaBuilder.createTable(properties.getCassandraKeyspace(), TABLE_TASK_DEF_LIMIT) + return SchemaBuilder.createTable(properties.getKeyspace(), TABLE_TASK_DEF_LIMIT) .ifNotExists() .addPartitionKey(TASK_DEF_NAME_KEY, DataType.text()) .addClusteringColumn(TASK_ID_KEY, DataType.uuid()) @@ -166,7 +166,7 @@ private String getCreateTaskDefLimitTableStatement() { } private String getCreateWorkflowDefsTableStatement() { - return SchemaBuilder.createTable(properties.getCassandraKeyspace(), TABLE_WORKFLOW_DEFS) + return SchemaBuilder.createTable(properties.getKeyspace(), TABLE_WORKFLOW_DEFS) .ifNotExists() .addPartitionKey(WORKFLOW_DEF_NAME_KEY, DataType.text()) .addClusteringColumn(WORKFLOW_VERSION_KEY, DataType.cint()) @@ -175,7 +175,7 @@ private String getCreateWorkflowDefsTableStatement() { } private String getCreateWorkflowDefsIndexTableStatement() { - return SchemaBuilder.createTable(properties.getCassandraKeyspace(), TABLE_WORKFLOW_DEFS_INDEX) + return SchemaBuilder.createTable(properties.getKeyspace(), TABLE_WORKFLOW_DEFS_INDEX) .ifNotExists() .addPartitionKey(WORKFLOW_DEF_INDEX_KEY, DataType.text()) .addClusteringColumn(WORKFLOW_DEF_NAME_VERSION_KEY, DataType.text()) @@ -184,7 +184,7 @@ private String getCreateWorkflowDefsIndexTableStatement() { } private String getCreateTaskDefsTableStatement() { - return SchemaBuilder.createTable(properties.getCassandraKeyspace(), TABLE_TASK_DEFS) + return SchemaBuilder.createTable(properties.getKeyspace(), TABLE_TASK_DEFS) .ifNotExists() .addPartitionKey(TASK_DEFS_KEY, DataType.text()) .addClusteringColumn(TASK_DEF_NAME_KEY, DataType.text()) @@ -193,7 +193,7 @@ private String getCreateTaskDefsTableStatement() { } private String getCreateEventHandlersTableStatement() { - return SchemaBuilder.createTable(properties.getCassandraKeyspace(), TABLE_EVENT_HANDLERS) + return SchemaBuilder.createTable(properties.getKeyspace(), TABLE_EVENT_HANDLERS) .ifNotExists() .addPartitionKey(HANDLERS_KEY, DataType.text()) .addClusteringColumn(EVENT_HANDLER_NAME_KEY, DataType.text()) @@ -202,7 +202,7 @@ private String getCreateEventHandlersTableStatement() { } private String getCreateEventExecutionsTableStatement() { - return SchemaBuilder.createTable(properties.getCassandraKeyspace(), TABLE_EVENT_EXECUTIONS) + return SchemaBuilder.createTable(properties.getKeyspace(), TABLE_EVENT_EXECUTIONS) .ifNotExists() .addPartitionKey(MESSAGE_ID_KEY, DataType.text()) .addPartitionKey(EVENT_HANDLER_NAME_KEY, DataType.text()) diff --git a/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/dao/CassandraEventHandlerDAO.java b/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/dao/CassandraEventHandlerDAO.java index 4504b3179a..c397d6b429 100644 --- a/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/dao/CassandraEventHandlerDAO.java +++ b/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/dao/CassandraEventHandlerDAO.java @@ -62,7 +62,7 @@ public CassandraEventHandlerDAO(Session session, ObjectMapper objectMapper, Cass deleteEventHandlerStatement = session.prepare(statements.getDeleteEventHandlerStatement()) .setConsistencyLevel(properties.getWriteConsistencyLevel()); - int cacheRefreshTime = properties.getEventHandlerRefreshTimeSecs(); + int cacheRefreshTime = properties.getEventHandlerCacheRefreshTimeSecs(); Executors.newSingleThreadScheduledExecutor().scheduleWithFixedDelay(this::refreshEventHandlersCache, 0, cacheRefreshTime, TimeUnit.SECONDS); } diff --git a/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/dao/CassandraExecutionDAO.java b/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/dao/CassandraExecutionDAO.java index be2c02cb44..5485c031b7 100644 --- a/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/dao/CassandraExecutionDAO.java +++ b/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/dao/CassandraExecutionDAO.java @@ -93,7 +93,7 @@ public CassandraExecutionDAO(Session session, ObjectMapper objectMapper, Cassand Statements statements) { super(session, objectMapper, properties); - eventExecutionsTTL = properties.getEventExecutionPersistenceTTL(); + eventExecutionsTTL = properties.getEventExecutionPersistenceTTLSecs(); this.insertWorkflowStatement = session.prepare(statements.getInsertWorkflowStatement()) .setConsistencyLevel(properties.getWriteConsistencyLevel()); diff --git a/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/dao/CassandraMetadataDAO.java b/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/dao/CassandraMetadataDAO.java index 0df88881e6..f9a9ee9e6e 100644 --- a/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/dao/CassandraMetadataDAO.java +++ b/cassandra-persistence/src/main/java/com/netflix/conductor/cassandra/dao/CassandraMetadataDAO.java @@ -106,7 +106,7 @@ public CassandraMetadataDAO(Session session, ObjectMapper objectMapper, Cassandr this.deleteTaskDefStatement = session.prepare(statements.getDeleteTaskDefStatement()) .setConsistencyLevel(properties.getWriteConsistencyLevel()); - int cacheRefreshTime = properties.getTaskDefRefreshTimeSecs(); + int cacheRefreshTime = properties.getTaskDefCacheRefreshTimeSecs(); Executors.newSingleThreadScheduledExecutor() .scheduleWithFixedDelay(this::refreshTaskDefsCache, 0, cacheRefreshTime, TimeUnit.SECONDS); } diff --git a/cassandra-persistence/src/test/java/com/netflix/conductor/cassandra/dao/CassandraDAOTest.java b/cassandra-persistence/src/test/java/com/netflix/conductor/cassandra/dao/CassandraDAOTest.java index 6ab577e294..766485ecc9 100644 --- a/cassandra-persistence/src/test/java/com/netflix/conductor/cassandra/dao/CassandraDAOTest.java +++ b/cassandra-persistence/src/test/java/com/netflix/conductor/cassandra/dao/CassandraDAOTest.java @@ -90,16 +90,16 @@ public void setUp() { properties = mock(CassandraProperties.class); when(properties.getHostAddress()).thenReturn("127.0.0.1"); when(properties.getPort()).thenReturn(9142); - when(properties.getCassandraKeyspace()).thenReturn(keyspaceName); + when(properties.getKeyspace()).thenReturn(keyspaceName); when(properties.getShardSize()).thenReturn(100); when(properties.getReplicationStrategy()).thenReturn("SimpleStrategy"); when(properties.getReplicationFactorKey()).thenReturn("replication_factor"); when(properties.getReplicationFactorValue()).thenReturn(1); when(properties.getReadConsistencyLevel()).thenReturn(ConsistencyLevel.LOCAL_ONE); when(properties.getWriteConsistencyLevel()).thenReturn(ConsistencyLevel.LOCAL_ONE); - when(properties.getTaskDefRefreshTimeSecs()).thenReturn(60); - when(properties.getEventHandlerRefreshTimeSecs()).thenReturn(60); - when(properties.getEventExecutionPersistenceTTL()).thenReturn(5); + when(properties.getTaskDefCacheRefreshTimeSecs()).thenReturn(60); + when(properties.getEventHandlerCacheRefreshTimeSecs()).thenReturn(60); + when(properties.getEventExecutionPersistenceTTLSecs()).thenReturn(5); Statements statements = new Statements(keyspaceName); metadataDAO = new CassandraMetadataDAO(session, objectMapper, properties, statements); executionDAO = new CassandraExecutionDAO(session, objectMapper, properties, statements); @@ -613,7 +613,7 @@ public void testEventExecutionCRUD() throws Exception { private void addWorkflowDefinition(WorkflowDef workflowDef) throws Exception { //INSERT INTO conductor.workflow_definitions (workflow_def_name,version,workflow_definition) VALUES (?,?,?); - String table = properties.getCassandraKeyspace() + "." + TABLE_WORKFLOW_DEFS; + String table = properties.getKeyspace() + "." + TABLE_WORKFLOW_DEFS; String queryString = "UPDATE " + table + " SET workflow_definition='" + objectMapper.writeValueAsString(workflowDef) + "' WHERE workflow_def_name='" + workflowDef.getName() @@ -624,7 +624,7 @@ private void addWorkflowDefinition(WorkflowDef workflowDef) throws Exception { private void addEventExecution(EventExecution eventExecution) throws Exception { //INSERT INTO junit.event_executions (message_id,event_handler_name,event_execution_id,payload) VALUES (?,?,?,?) - String table = properties.getCassandraKeyspace() + "." + TABLE_EVENT_EXECUTIONS; + String table = properties.getKeyspace() + "." + TABLE_EVENT_EXECUTIONS; String queryString = "INSERT INTO " + table + " (message_id, event_handler_name, event_execution_id, payload) " + "VALUES ('" + eventExecution.getMessageId() diff --git a/cassandra-persistence/src/test/resources/logback-test.xml b/cassandra-persistence/src/test/resources/logback-test.xml deleted file mode 100644 index 9547e53369..0000000000 --- a/cassandra-persistence/src/test/resources/logback-test.xml +++ /dev/null @@ -1,16 +0,0 @@ - - - - %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n - - - - - - - - - - - - diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/dao/index/NoopIndexDAOConfiguration.java b/contribs/src/main/java/com/netflix/conductor/contribs/dao/index/NoopIndexDAOConfiguration.java index 26df666e64..a075ac3d49 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/dao/index/NoopIndexDAOConfiguration.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/dao/index/NoopIndexDAOConfiguration.java @@ -18,7 +18,7 @@ import org.springframework.context.annotation.Configuration; @Configuration(proxyBeanMethods = false) -@ConditionalOnProperty(prefix = "workflow", name = "indexing.enabled", havingValue = "false") +@ConditionalOnProperty(name = "conductor.indexing.enabled", havingValue = "false") public class NoopIndexDAOConfiguration { @Bean diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/listener/archive/ArchivingWithTTLWorkflowStatusListener.java b/contribs/src/main/java/com/netflix/conductor/contribs/listener/archive/ArchivingWithTTLWorkflowStatusListener.java index cc8b95fb21..bd2d596cb1 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/listener/archive/ArchivingWithTTLWorkflowStatusListener.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/listener/archive/ArchivingWithTTLWorkflowStatusListener.java @@ -35,11 +35,11 @@ public class ArchivingWithTTLWorkflowStatusListener implements WorkflowStatusLis public ArchivingWithTTLWorkflowStatusListener(ExecutionDAOFacade executionDAOFacade, ArchivingWorkflowListenerProperties properties) { this.executionDAOFacade = executionDAOFacade; - this.archiveTTLSeconds = properties.getWorkflowArchivalTTL(); + this.archiveTTLSeconds = properties.getTtlSeconds(); this.delayArchiveSeconds = properties.getWorkflowArchivalDelay(); this.scheduledThreadPoolExecutor = new ScheduledThreadPoolExecutor( - properties.getWorkflowArchivalDelayQueueWorkerThreadCount(), + properties.getDelayQueueWorkerThreadCount(), (runnable, executor) -> { LOGGER.warn("Request {} to delay archiving index dropped in executor {}", runnable, executor); Monitors.recordDiscardedArchivalCount(); diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/listener/archive/ArchivingWorkflowListenerConfiguration.java b/contribs/src/main/java/com/netflix/conductor/contribs/listener/archive/ArchivingWorkflowListenerConfiguration.java index f7fe716104..22c588acf2 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/listener/archive/ArchivingWorkflowListenerConfiguration.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/listener/archive/ArchivingWorkflowListenerConfiguration.java @@ -15,17 +15,19 @@ import com.netflix.conductor.core.listener.WorkflowStatusListener; import com.netflix.conductor.core.orchestration.ExecutionDAOFacade; import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; @Configuration -@ConditionalOnProperty(prefix = "workflow", name = "status.listener.type", havingValue = "archive") +@EnableConfigurationProperties(ArchivingWorkflowListenerProperties.class) +@ConditionalOnProperty(name = "conductor.workflow-status-listener.type", havingValue = "archive") public class ArchivingWorkflowListenerConfiguration { @Bean public WorkflowStatusListener getWorkflowStatusListener(ExecutionDAOFacade executionDAOFacade, ArchivingWorkflowListenerProperties properties) { - if (properties.getWorkflowArchivalTTL() > 0) { + if (properties.getTtlSeconds() > 0) { return new ArchivingWithTTLWorkflowStatusListener(executionDAOFacade, properties); } else { return new ArchivingWorkflowStatusListener(executionDAOFacade); diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/listener/archive/ArchivingWorkflowListenerProperties.java b/contribs/src/main/java/com/netflix/conductor/contribs/listener/archive/ArchivingWorkflowListenerProperties.java index 6c3052589c..cc7ebf7a79 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/listener/archive/ArchivingWorkflowListenerProperties.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/listener/archive/ArchivingWorkflowListenerProperties.java @@ -12,17 +12,16 @@ */ package com.netflix.conductor.contribs.listener.archive; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.boot.context.properties.ConfigurationProperties; import org.springframework.core.env.Environment; -import org.springframework.stereotype.Component; -@Component -@ConditionalOnProperty(prefix = "workflow", name = "status.listener.type", havingValue = "archive") +@ConfigurationProperties("conductor.workflow-status-listener.archival") public class ArchivingWorkflowListenerProperties { private final Environment environment; + @Autowired public ArchivingWorkflowListenerProperties(Environment environment) { this.environment = environment; } @@ -30,28 +29,34 @@ public ArchivingWorkflowListenerProperties(Environment environment) { /** * The time to live in seconds for workflow archiving module. Currently, only RedisExecutionDAO supports this */ - @Value("${workflow.archival.ttl.seconds:0}") - private int workflowArchivalTTL; + private int ttlSeconds = 0; /** - * the number of threads to process the delay queue in workflow archival + * The number of threads to process the delay queue in workflow archival */ - @Value("${workflow.archival.delay.queue.worker.thread.count:5}") - private int workflowArchivalDelayQueueWorkerThreadCount; + private int delayQueueWorkerThreadCount = 5; - public int getWorkflowArchivalTTL() { - return workflowArchivalTTL; + public int getTtlSeconds() { + return ttlSeconds; + } + + public void setTtlSeconds(int ttlSeconds) { + this.ttlSeconds = ttlSeconds; + } + + public int getDelayQueueWorkerThreadCount() { + return delayQueueWorkerThreadCount; + } + + public void setDelayQueueWorkerThreadCount(int delayQueueWorkerThreadCount) { + this.delayQueueWorkerThreadCount = delayQueueWorkerThreadCount; } /** - * the time to delay the archival of workflow + * The time to delay the archival of workflow */ public int getWorkflowArchivalDelay() { - return environment.getProperty("workflow.archival.delay.seconds", Integer.class, + return environment.getProperty("conductor.workflow-status-listener.archival.delaySeconds", Integer.class, environment.getProperty("async.update.delay.seconds", Integer.class, 60)); } - - public int getWorkflowArchivalDelayQueueWorkerThreadCount() { - return workflowArchivalDelayQueueWorkerThreadCount; - } } diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/listener/conductorqueue/ConductorQueueStatusPublisher.java b/contribs/src/main/java/com/netflix/conductor/contribs/listener/conductorqueue/ConductorQueueStatusPublisher.java index 798d2bd8bb..4fb6cac57f 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/listener/conductorqueue/ConductorQueueStatusPublisher.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/listener/conductorqueue/ConductorQueueStatusPublisher.java @@ -19,11 +19,10 @@ import com.netflix.conductor.core.events.queue.Message; import com.netflix.conductor.core.listener.WorkflowStatusListener; import com.netflix.conductor.dao.QueueDAO; +import java.util.Collections; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collections; - /** * Publishes a {@link Message} containing a {@link WorkflowSummary} to the undlerying {@link QueueDAO} implementation on * a workflow completion or termination event. @@ -41,8 +40,8 @@ public ConductorQueueStatusPublisher(QueueDAO queueDAO, ObjectMapper objectMappe ConductorQueueStatusPublisherProperties properties) { this.queueDAO = queueDAO; this.objectMapper = objectMapper; - this.successStatusQueue = properties.getSuccessStatusQueue(); - this.failureStatusQueue = properties.getFailureStatusQueue(); + this.successStatusQueue = properties.getSuccessQueue(); + this.failureStatusQueue = properties.getFailureQueue(); } @Override diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/listener/conductorqueue/ConductorQueueStatusPublisherConfiguration.java b/contribs/src/main/java/com/netflix/conductor/contribs/listener/conductorqueue/ConductorQueueStatusPublisherConfiguration.java index 59ea9341bf..fd379d80b1 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/listener/conductorqueue/ConductorQueueStatusPublisherConfiguration.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/listener/conductorqueue/ConductorQueueStatusPublisherConfiguration.java @@ -16,11 +16,13 @@ import com.netflix.conductor.core.listener.WorkflowStatusListener; import com.netflix.conductor.dao.QueueDAO; import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; @Configuration -@ConditionalOnProperty(prefix = "workflow", name = "status.listener.type", havingValue = "queue_status_publisher") +@EnableConfigurationProperties(ConductorQueueStatusPublisherProperties.class) +@ConditionalOnProperty(name = "conductor.workflow-status-listener.type", havingValue = "queue_publisher") public class ConductorQueueStatusPublisherConfiguration { @Bean diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/listener/conductorqueue/ConductorQueueStatusPublisherProperties.java b/contribs/src/main/java/com/netflix/conductor/contribs/listener/conductorqueue/ConductorQueueStatusPublisherProperties.java index 54f27f144e..581ae6ed33 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/listener/conductorqueue/ConductorQueueStatusPublisherProperties.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/listener/conductorqueue/ConductorQueueStatusPublisherProperties.java @@ -12,25 +12,28 @@ */ package com.netflix.conductor.contribs.listener.conductorqueue; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; -import org.springframework.stereotype.Component; +import org.springframework.boot.context.properties.ConfigurationProperties; -@Component -@ConditionalOnProperty(prefix = "workflow", name = "status.listener.type", havingValue = "queue_status_publisher") +@ConfigurationProperties("conductor.workflow-status-listener.queue-publisher") public class ConductorQueueStatusPublisherProperties { - @Value("${workflowstatuslistener.publisher.success.queue:_callbackSuccessQueue}") - private String successStatusQueue; + private String successQueue = "_callbackSuccessQueue"; - @Value("${workflowstatuslistener.publisher.failure.queue:_callbackFailureQueue}") - private String failureStatusQueue; + private String failureQueue = "_callbackFailureQueue"; - public String getSuccessStatusQueue() { - return successStatusQueue; + public String getSuccessQueue() { + return successQueue; } - public String getFailureStatusQueue() { - return failureStatusQueue; + public void setSuccessQueue(String successQueue) { + this.successQueue = successQueue; + } + + public String getFailureQueue() { + return failureQueue; + } + + public void setFailureQueue(String failureQueue) { + this.failureQueue = failureQueue; } } diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/lock/LocalOnlyLockConfiguration.java b/contribs/src/main/java/com/netflix/conductor/contribs/lock/LocalOnlyLockConfiguration.java index d1f7eee527..f7cfcc060e 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/lock/LocalOnlyLockConfiguration.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/lock/LocalOnlyLockConfiguration.java @@ -18,7 +18,7 @@ import org.springframework.context.annotation.Configuration; @Configuration -@ConditionalOnProperty(prefix = "workflow", name = "decider.locking.server", havingValue = "LOCAL_ONLY") +@ConditionalOnProperty(name = "conductor.workflow-execution-lock.type", havingValue = "local_only") public class LocalOnlyLockConfiguration { @Bean diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/metrics/LoggingMetricsConfiguration.java b/contribs/src/main/java/com/netflix/conductor/contribs/metrics/LoggingMetricsConfiguration.java index 317877cfa7..f5e5e9c46e 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/metrics/LoggingMetricsConfiguration.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/metrics/LoggingMetricsConfiguration.java @@ -25,11 +25,11 @@ /** * Metrics logging reporter, dumping all metrics into an Slf4J logger. *

- * Enable in config: conductor.metrics.logger.enabled=true + * Enable in config: conductor.metrics-logger.enabled=true *

- * additional config: conductor.metrics.logger.reportPeriodSeconds=15 + * additional config: conductor.metrics-logger.reportPeriodSeconds=15 */ -@ConditionalOnProperty(value = "conductor.metrics.logger.enabled", havingValue = "true") +@ConditionalOnProperty(value = "conductor.metrics-logger.enabled", havingValue = "true") @Configuration public class LoggingMetricsConfiguration { @@ -39,7 +39,7 @@ public class LoggingMetricsConfiguration { // This way one can cleanly separate the metrics stream from rest of the logs private static final Logger METRICS_LOGGER = LoggerFactory.getLogger("ConductorMetrics"); - @Value("${conductor.metrics.logger.reportPeriodSeconds:30}") + @Value("${conductor.metrics-logger.reportPeriodSeconds:30}") private long reportInterval; @Bean diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/metrics/MetricsRegistryConfiguration.java b/contribs/src/main/java/com/netflix/conductor/contribs/metrics/MetricsRegistryConfiguration.java index dd898d1cde..d76445522f 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/metrics/MetricsRegistryConfiguration.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/metrics/MetricsRegistryConfiguration.java @@ -20,7 +20,7 @@ import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; -@ConditionalOnProperty(value = "conductor.metrics.logger.enabled", havingValue = "true") +@ConditionalOnProperty(value = "conductor.metrics-logger.enabled", havingValue = "true") @Configuration public class MetricsRegistryConfiguration { diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/queue/amqp/AMQPObservableQueue.java b/contribs/src/main/java/com/netflix/conductor/contribs/queue/amqp/AMQPObservableQueue.java index c17bfd91a6..6726b7aec3 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/queue/amqp/AMQPObservableQueue.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/queue/amqp/AMQPObservableQueue.java @@ -276,7 +276,7 @@ public Builder(AMQPEventQueueProperties properties) { this.factory = buildConnectionFactory(); // messages polling settings this.batchSize = properties.getBatchSize(); - this.pollTimeInMS = properties.getPollTimeMS(); + this.pollTimeInMS = properties.getPollTimeMs(); } private Address[] buildAddressesFromHosts() { diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/queue/amqp/config/AMQPEventQueueConfiguration.java b/contribs/src/main/java/com/netflix/conductor/contribs/queue/amqp/config/AMQPEventQueueConfiguration.java index 8a4468af33..d08d345de6 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/queue/amqp/config/AMQPEventQueueConfiguration.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/queue/amqp/config/AMQPEventQueueConfiguration.java @@ -22,11 +22,13 @@ import java.util.Map; import org.apache.commons.lang3.StringUtils; import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; @Configuration(proxyBeanMethods = false) -@ConditionalOnProperty(prefix = "workflow", name = "amqp.event.queue.enabled", havingValue = "true") +@EnableConfigurationProperties(AMQPEventQueueProperties.class) +@ConditionalOnProperty(name = "conductor.event-queues.amqp.enabled", havingValue = "true") public class AMQPEventQueueConfiguration { private enum QUEUE_TYPE { @@ -53,7 +55,7 @@ public EventQueueProvider amqpExchangeEventQueueProvider(AMQPEventQueuePropertie return new AMQPEventQueueProvider(properties, QUEUE_TYPE.AMQP_EXCHANGE.getType(), true); } - @ConditionalOnProperty(prefix = "workflow", name = "events.default.queue.type", havingValue = "amqp") + @ConditionalOnProperty(name = "conductor.default-event-queue.type", havingValue = "amqp") @Bean public Map getQueues(ConductorProperties conductorProperties, AMQPEventQueueProperties properties) { diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/queue/amqp/config/AMQPEventQueueProperties.java b/contribs/src/main/java/com/netflix/conductor/contribs/queue/amqp/config/AMQPEventQueueProperties.java index 8109ce86dc..a967f7d383 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/queue/amqp/config/AMQPEventQueueProperties.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/queue/amqp/config/AMQPEventQueueProperties.java @@ -12,137 +12,190 @@ */ package com.netflix.conductor.contribs.queue.amqp.config; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; -import org.springframework.stereotype.Component; +import com.rabbitmq.client.AMQP.PROTOCOL; +import com.rabbitmq.client.ConnectionFactory; +import org.springframework.boot.context.properties.ConfigurationProperties; -@Component -@ConditionalOnProperty(prefix = "workflow", name = "amqp.event.queue.enabled", havingValue = "true") +@ConfigurationProperties("conductor.event-queues.amqp") public class AMQPEventQueueProperties { - @Value("${workflow.event.queues.amqp.batchSize:1}") - private int batchSize; + private int batchSize = 1; - @Value("${workflow.event.queues.amqp.pollTimeInMs:100}") - private int pollTimeMS; + private int pollTimeMs = 100; - @Value("${workflow.event.queues.amqp.hosts:#{T(com.rabbitmq.client.ConnectionFactory).DEFAULT_HOST}}") - private String hosts; + private String hosts = ConnectionFactory.DEFAULT_HOST; - @Value("${workflow.event.queues.amqp.username:#{T(com.rabbitmq.client.ConnectionFactory).DEFAULT_USER}}") - private String username; + private String username = ConnectionFactory.DEFAULT_USER; - @Value("${workflow.event.queues.amqp.password:#{T(com.rabbitmq.client.ConnectionFactory).DEFAULT_PASS}}") - private String password; + private String password = ConnectionFactory.DEFAULT_PASS; - @Value("${workflow.event.queues.amqp.virtualHost:#{T(com.rabbitmq.client.ConnectionFactory).DEFAULT_VHOST}}") - private String virtualHost; + private String virtualHost = ConnectionFactory.DEFAULT_VHOST; - @Value("${workflow.event.queues.amqp.port:#{T(com.rabbitmq.client.AMQP.PROTOCOL).PORT}}") - private int port; + private int port = PROTOCOL.PORT; - @Value("${workflow.event.queues.amqp.connectionTimeout:#{T(com.rabbitmq.client.ConnectionFactory).DEFAULT_CONNECTION_TIMEOUT}}") - private int connectionTimeout; + private int connectionTimeout = ConnectionFactory.DEFAULT_CONNECTION_TIMEOUT; - @Value("${workflow.event.queues.amqp.useNio:false}") - private boolean useNio; + private boolean useNio = false; - @Value("${workflow.event.queues.amqp.durable:true}") - private boolean durable; + private boolean durable = true; - @Value("${workflow.event.queues.amqp.exclusive:false}") - private boolean exclusive; + private boolean exclusive = false; - @Value("${workflow.event.queues.amqp.autoDelete:false}") - private boolean autoDelete; + private boolean autoDelete = false; - @Value("${workflow.event.queues.amqp.contentType:application/json}") - private String contentType; + private String contentType = "application/json"; - @Value("${workflow.event.queues.amqp.contentEncoding:UTF-8}") - private String contentEncoding; + private String contentEncoding = "UTF-8"; - @Value("${workflow.event.queues.amqp.amqp_exchange:topic}") - private String exchangeType; + private String exchangeType = "topic"; - @Value("${workflow.event.queues.amqp.deliveryMode:2}") - private int deliveryMode; + private int deliveryMode = 2; - @Value("${workflow.listener.queue.useExchange:true}") - private boolean useExchange; + private boolean useExchange = true; - @Value("${workflow.listener.queue.prefix:}") - private String listenerQueuePrefix; + private String listenerQueuePrefix = ""; public int getBatchSize() { return batchSize; } - public int getPollTimeMS() { - return pollTimeMS; + public void setBatchSize(int batchSize) { + this.batchSize = batchSize; + } + + public int getPollTimeMs() { + return pollTimeMs; + } + + public void setPollTimeMs(int pollTimeMs) { + this.pollTimeMs = pollTimeMs; } public String getHosts() { return hosts; } + public void setHosts(String hosts) { + this.hosts = hosts; + } + public String getUsername() { return username; } + public void setUsername(String username) { + this.username = username; + } + public String getPassword() { return password; } + public void setPassword(String password) { + this.password = password; + } + public String getVirtualHost() { return virtualHost; } + public void setVirtualHost(String virtualHost) { + this.virtualHost = virtualHost; + } + public int getPort() { return port; } + public void setPort(int port) { + this.port = port; + } + public int getConnectionTimeout() { return connectionTimeout; } + public void setConnectionTimeout(int connectionTimeout) { + this.connectionTimeout = connectionTimeout; + } + public boolean isUseNio() { return useNio; } + public void setUseNio(boolean useNio) { + this.useNio = useNio; + } + public boolean isDurable() { return durable; } + public void setDurable(boolean durable) { + this.durable = durable; + } + public boolean isExclusive() { return exclusive; } + public void setExclusive(boolean exclusive) { + this.exclusive = exclusive; + } + public boolean isAutoDelete() { return autoDelete; } + public void setAutoDelete(boolean autoDelete) { + this.autoDelete = autoDelete; + } + public String getContentType() { return contentType; } + public void setContentType(String contentType) { + this.contentType = contentType; + } + public String getContentEncoding() { return contentEncoding; } + public void setContentEncoding(String contentEncoding) { + this.contentEncoding = contentEncoding; + } + public String getExchangeType() { return exchangeType; } + public void setExchangeType(String exchangeType) { + this.exchangeType = exchangeType; + } + public int getDeliveryMode() { return deliveryMode; } + public void setDeliveryMode(int deliveryMode) { + this.deliveryMode = deliveryMode; + } + public boolean isUseExchange() { return useExchange; } + public void setUseExchange(boolean useExchange) { + this.useExchange = useExchange; + } + public String getListenerQueuePrefix() { return listenerQueuePrefix; } + + public void setListenerQueuePrefix(String listenerQueuePrefix) { + this.listenerQueuePrefix = listenerQueuePrefix; + } } diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/queue/amqp/util/AMQPConstants.java b/contribs/src/main/java/com/netflix/conductor/contribs/queue/amqp/util/AMQPConstants.java index d2b48474dc..5127059464 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/queue/amqp/util/AMQPConstants.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/queue/amqp/util/AMQPConstants.java @@ -26,7 +26,7 @@ public class AMQPConstants { */ public static String AMQP_EXCHANGE_TYPE = "amqp_exchange"; - public static String PROPERTY_KEY_TEMPLATE = "workflow.event.queues.amqp.%s"; + public static String PROPERTY_KEY_TEMPLATE = "conductor.event-queues.amqp.%s"; /** * default content type for the message read from rabbitmq diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSConfiguration.java b/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSConfiguration.java index b83f027450..8a89600f5d 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSConfiguration.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSConfiguration.java @@ -20,7 +20,7 @@ import rx.Scheduler; @Configuration -@ConditionalOnProperty(prefix = "workflow", name = "nats.event.queue.enabled", havingValue = "true") +@ConditionalOnProperty(name = "conductor.event-queues.nats.enabled", havingValue = "true") public class NATSConfiguration { @Bean diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSProperties.java b/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSProperties.java deleted file mode 100644 index 05564b2606..0000000000 --- a/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSProperties.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Copyright 2020 Netflix, Inc. - *

- * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package com.netflix.conductor.contribs.queue.nats.config; - -import org.springframework.beans.factory.annotation.Value; -import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; -import org.springframework.stereotype.Component; - -@Component -@ConditionalOnProperty(prefix = "workflow", name = "nats_stream.event.queue.enabled", havingValue = "true") -public class NATSProperties { - - @Value("${io.nats.streaming.clusterId:test-cluster}") - private String clusterId; - - @Value("${io.nats.streaming.durableName:#{null}}") - private String durableName; - - @Value("${io.nats.streaming.url:#{T(io.nats.client.Nats).DEFAULT_URL}") - private String natsUrl; - - public String getClusterId() { - return clusterId; - } - - public String getDurableName() { - return durableName; - } - - public String getNatsUrl() { - return natsUrl; - } -} diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSStreamConfiguration.java b/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSStreamConfiguration.java index 965e63ec22..5affe7df29 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSStreamConfiguration.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSStreamConfiguration.java @@ -14,16 +14,18 @@ import com.netflix.conductor.core.events.EventQueueProvider; import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; import rx.Scheduler; @Configuration -@ConditionalOnProperty(prefix = "workflow", name = "nats_stream.event.queue.enabled", havingValue = "true") +@EnableConfigurationProperties(NATSStreamProperties.class) +@ConditionalOnProperty(name = "conductor.event-queues.nats-stream.enabled", havingValue = "true") public class NATSStreamConfiguration { @Bean - public EventQueueProvider natsEventQueueProvider(NATSProperties properties, Scheduler scheduler) { + public EventQueueProvider natsEventQueueProvider(NATSStreamProperties properties, Scheduler scheduler) { return new NATSStreamEventQueueProvider(properties, scheduler); } } diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSStreamEventQueueProvider.java b/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSStreamEventQueueProvider.java index 66f208a000..a10b253324 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSStreamEventQueueProvider.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSStreamEventQueueProvider.java @@ -34,14 +34,14 @@ public class NATSStreamEventQueueProvider implements EventQueueProvider { private final String natsUrl; private final Scheduler scheduler; - public NATSStreamEventQueueProvider(NATSProperties properties, Scheduler scheduler) { + public NATSStreamEventQueueProvider(NATSStreamProperties properties, Scheduler scheduler) { LOGGER.info("NATS Stream Event Queue Provider init"); this.scheduler = scheduler; // Get NATS Streaming options clusterId = properties.getClusterId(); durableName = properties.getDurableName(); - natsUrl = properties.getNatsUrl(); + natsUrl = properties.getUrl(); LOGGER.info("NATS Streaming clusterId=" + clusterId + ", natsUrl=" + natsUrl + ", durableName=" + durableName); diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSStreamProperties.java b/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSStreamProperties.java new file mode 100644 index 0000000000..f7c8ba8a33 --- /dev/null +++ b/contribs/src/main/java/com/netflix/conductor/contribs/queue/nats/config/NATSStreamProperties.java @@ -0,0 +1,59 @@ +/* + * Copyright 2020 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package com.netflix.conductor.contribs.queue.nats.config; + +import io.nats.client.Nats; +import org.springframework.boot.context.properties.ConfigurationProperties; + +@ConfigurationProperties("conductor.event-queues.nats-stream") +public class NATSStreamProperties { + + /** + * The cluster id of the STAN session + */ + private String clusterId = "test-cluster"; + + /** + * The durable subscriber name for the subscription + */ + private String durableName = null; + + /** + * The NATS connection url + */ + private String url = Nats.DEFAULT_URL; + + public String getClusterId() { + return clusterId; + } + + public void setClusterId(String clusterId) { + this.clusterId = clusterId; + } + + public String getDurableName() { + return durableName; + } + + public void setDurableName(String durableName) { + this.durableName = durableName; + } + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } +} diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/queue/sqs/config/SQSEventQueueConfiguration.java b/contribs/src/main/java/com/netflix/conductor/contribs/queue/sqs/config/SQSEventQueueConfiguration.java index 0461a3fedd..f7ce4d4d8e 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/queue/sqs/config/SQSEventQueueConfiguration.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/queue/sqs/config/SQSEventQueueConfiguration.java @@ -19,19 +19,20 @@ import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.core.events.EventQueueProvider; import com.netflix.conductor.core.events.queue.ObservableQueue; +import java.util.HashMap; +import java.util.Map; import org.apache.commons.lang3.StringUtils; import org.springframework.boot.autoconfigure.condition.ConditionalOnMissingBean; import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; import rx.Scheduler; -import java.util.HashMap; -import java.util.Map; - @SuppressWarnings("SpringJavaInjectionPointsAutowiringInspection") @Configuration -@ConditionalOnProperty(prefix = "workflow", name = "sqs.event.queue.enabled", havingValue = "true") +@EnableConfigurationProperties(SQSEventQueueProperties.class) +@ConditionalOnProperty(name = "conductor.event-queues.sqs.enabled", havingValue = "true") public class SQSEventQueueConfiguration { @ConditionalOnMissingBean @@ -46,7 +47,7 @@ public EventQueueProvider sqsEventQueueProvider(AmazonSQSClient sqsClient, SQSEv return new SQSEventQueueProvider(sqsClient, properties, scheduler); } - @ConditionalOnProperty(prefix = "workflow", name = "events.default.queue.type", havingValue = "sqs", matchIfMissing = true) + @ConditionalOnProperty(name = "conductor.default-event-queue.type", havingValue = "sqs", matchIfMissing = true) @Bean public Map getQueues(ConductorProperties conductorProperties, SQSEventQueueProperties properties, AmazonSQSClient sqsClient) { diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/queue/sqs/config/SQSEventQueueProperties.java b/contribs/src/main/java/com/netflix/conductor/contribs/queue/sqs/config/SQSEventQueueProperties.java index d47edb7a99..88347e307a 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/queue/sqs/config/SQSEventQueueProperties.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/queue/sqs/config/SQSEventQueueProperties.java @@ -12,46 +12,73 @@ */ package com.netflix.conductor.contribs.queue.sqs.config; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; -import org.springframework.stereotype.Component; +import org.springframework.boot.context.properties.ConfigurationProperties; -@Component -@ConditionalOnProperty(prefix = "workflow", name = "sqs.event.queue.enabled", havingValue = "true") +@ConfigurationProperties("conductor.event-queues.sqs") public class SQSEventQueueProperties { - @Value("${workflow.event.queues.sqs.batchSize:1}") - private int batchSize; + /** + * The maximum number of messages to be fetched from the queue in a single request + */ + private int batchSize = 1; - @Value("${workflow.event.queues.sqs.pollTimeInMS:100}") - private int pollTimeMS; + /** + * The polling interval (in milliseconds) + */ + private int pollTimeMs = 100; - @Value("${workflow.event.queues.sqs.visibilityTimeoutInSeconds:60}") - private int visibilityTimeoutSeconds; + /** + * The visibility timeout (in seconds) for the message on the queue + */ + private int visibilityTimeoutSeconds = 60; - @Value("${workflow.listener.queue.prefix:}") - private String listenerQueuePrefix; + /** + * The prefix to be used for the default listener queues + */ + private String listenerQueuePrefix = ""; - @Value("${workflow.listener.queue.authorizedAccounts:}") - private String authorizedAccounts; + /** + * The AWS account Ids authorized to send messages to the queues + */ + private String authorizedAccounts = ""; public int getBatchSize() { return batchSize; } - public int getPollTimeMS() { - return pollTimeMS; + public void setBatchSize(int batchSize) { + this.batchSize = batchSize; + } + + public int getPollTimeMs() { + return pollTimeMs; + } + + public void setPollTimeMs(int pollTimeMs) { + this.pollTimeMs = pollTimeMs; } public int getVisibilityTimeoutSeconds() { return visibilityTimeoutSeconds; } + public void setVisibilityTimeoutSeconds(int visibilityTimeoutSeconds) { + this.visibilityTimeoutSeconds = visibilityTimeoutSeconds; + } + public String getListenerQueuePrefix() { return listenerQueuePrefix; } + public void setListenerQueuePrefix(String listenerQueuePrefix) { + this.listenerQueuePrefix = listenerQueuePrefix; + } + public String getAuthorizedAccounts() { return authorizedAccounts; } + + public void setAuthorizedAccounts(String authorizedAccounts) { + this.authorizedAccounts = authorizedAccounts; + } } diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/queue/sqs/config/SQSEventQueueProvider.java b/contribs/src/main/java/com/netflix/conductor/contribs/queue/sqs/config/SQSEventQueueProvider.java index 08e7aa92d3..5b90a9d272 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/queue/sqs/config/SQSEventQueueProvider.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/queue/sqs/config/SQSEventQueueProvider.java @@ -33,7 +33,7 @@ public class SQSEventQueueProvider implements EventQueueProvider { public SQSEventQueueProvider(AmazonSQSClient client, SQSEventQueueProperties properties, Scheduler scheduler) { this.client = client; this.batchSize = properties.getBatchSize(); - this.pollTimeInMS = properties.getPollTimeMS(); + this.pollTimeInMS = properties.getPollTimeMs(); this.visibilityTimeoutInSeconds = properties.getVisibilityTimeoutSeconds(); this.scheduler = scheduler; } diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/storage/S3PayloadStorage.java b/contribs/src/main/java/com/netflix/conductor/contribs/storage/S3PayloadStorage.java index ab0808848f..2b23f8f3c0 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/storage/S3PayloadStorage.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/storage/S3PayloadStorage.java @@ -51,7 +51,7 @@ public class S3PayloadStorage implements ExternalPayloadStorage { public S3PayloadStorage(S3Properties properties) { bucketName = properties.getBucketName(); - expirationSec = properties.getExpirationSeconds(); + expirationSec = properties.getSignedUrlExpirationSeconds(); String region = properties.getRegion(); s3Client = AmazonS3ClientBuilder.standard().withRegion(region).build(); } diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/storage/config/S3Configuration.java b/contribs/src/main/java/com/netflix/conductor/contribs/storage/config/S3Configuration.java index cd6a50b49b..93c7dd1bfc 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/storage/config/S3Configuration.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/storage/config/S3Configuration.java @@ -15,11 +15,13 @@ import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.netflix.conductor.contribs.storage.S3PayloadStorage; import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; @Configuration -@ConditionalOnProperty(prefix = "workflow", name = "external.payload.storage", havingValue = "S3") +@EnableConfigurationProperties(S3Properties.class) +@ConditionalOnProperty(name = "conductor.external-payload-storage.type", havingValue = "s3") public class S3Configuration { @Bean diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/storage/config/S3Properties.java b/contribs/src/main/java/com/netflix/conductor/contribs/storage/config/S3Properties.java index 23a6d53d24..a03d3c467a 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/storage/config/S3Properties.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/storage/config/S3Properties.java @@ -12,32 +12,47 @@ */ package com.netflix.conductor.contribs.storage.config; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; -import org.springframework.stereotype.Component; +import org.springframework.boot.context.properties.ConfigurationProperties; -@Component -@ConditionalOnProperty(prefix = "workflow", name = "external.payload.storage", havingValue = "S3") +@ConfigurationProperties("conductor.external-payload-storage.s3") public class S3Properties { - @Value("${workflow.external.payload.storage.s3.bucket:conductor_payloads}") - private String bucketName; + /** + * The s3 bucket name where the payloads will be stored + */ + private String bucketName = "conductor_payloads"; - @Value("${workflow.external.payload.storage.s3.signedurlexpirationseconds:5}") - private int expirationSeconds; + /** + * The time (in seconds) for which the signed url will be valid + */ + private int signedUrlExpirationSeconds = 5; - @Value("${workflow.external.payload.storage.s3.region:us-east-1}") - private String region; + /** + * The AWS region of the s3 bucket + */ + private String region = "us-east-1"; public String getBucketName() { return bucketName; } - public int getExpirationSeconds() { - return expirationSeconds; + public void setBucketName(String bucketName) { + this.bucketName = bucketName; + } + + public int getSignedUrlExpirationSeconds() { + return signedUrlExpirationSeconds; + } + + public void setSignedUrlExpirationSeconds(int signedUrlExpirationSeconds) { + this.signedUrlExpirationSeconds = signedUrlExpirationSeconds; } public String getRegion() { return region; } + + public void setRegion(String region) { + this.region = region; + } } diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/tasks/http/DefaultRestTemplateProvider.java b/contribs/src/main/java/com/netflix/conductor/contribs/tasks/http/DefaultRestTemplateProvider.java index f36c9d08aa..a9661ae56e 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/tasks/http/DefaultRestTemplateProvider.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/tasks/http/DefaultRestTemplateProvider.java @@ -36,8 +36,8 @@ public class DefaultRestTemplateProvider implements RestTemplateProvider { private final int defaultConnectTimeout; @Autowired - public DefaultRestTemplateProvider(@Value("${http.task.read.timeout:150}") int readTimeout, - @Value("${http.task.connect.timeout:100}") int connectTimeout) { + public DefaultRestTemplateProvider(@Value("${conductor.tasks.http.readTimeout:150}") int readTimeout, + @Value("${conductor.tasks.http.connectTimeout:100}") int connectTimeout) { this.threadLocalRestTemplate = ThreadLocal.withInitial(RestTemplate::new); this.defaultReadTimeout = readTimeout; this.defaultConnectTimeout = connectTimeout; diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/tasks/http/RestTemplateProvider.java b/contribs/src/main/java/com/netflix/conductor/contribs/tasks/http/RestTemplateProvider.java index c54fa3648f..2fa7f6acf0 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/tasks/http/RestTemplateProvider.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/tasks/http/RestTemplateProvider.java @@ -1,3 +1,15 @@ +/* + * Copyright 2020 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package com.netflix.conductor.contribs.tasks.http; import org.springframework.web.client.RestTemplate; diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/tasks/kafka/KafkaProducerManager.java b/contribs/src/main/java/com/netflix/conductor/contribs/tasks/kafka/KafkaProducerManager.java index d265e37665..b524ce4ebf 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/tasks/kafka/KafkaProducerManager.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/tasks/kafka/KafkaProducerManager.java @@ -50,11 +50,11 @@ public class KafkaProducerManager { }; @Autowired - public KafkaProducerManager(@Value("${kafka.publish.request.timeout.ms: 100}") String publishRequestTimeoutMs, - @Value("${kafka.publish.max.block.ms:500}") String maxBlockMs, - @Value("${kafka.publish.producer.cache.size:10}") int cacheSize, - @Value("${kafka.publish.producer.cache.time.ms:120000}") int cacheTimeMs) { - this.requestTimeoutConfig = publishRequestTimeoutMs; + public KafkaProducerManager(@Value("${conductor.tasks.kafka-publish.requestTimeoutMs: 100}") String requestTimeoutMs, + @Value("${conductor.tasks.kafka-publish.maxBlockMs:500}") String maxBlockMs, + @Value("${conductor.tasks.kafka-publish.cacheSize:10}") int cacheSize, + @Value("${conductor.tasks.kafka-publish.cacheTimeMs:120000}") int cacheTimeMs) { + this.requestTimeoutConfig = requestTimeoutMs; this.maxBlockMsConfig = maxBlockMs; this.kafkaProducerCache = CacheBuilder.newBuilder().removalListener(LISTENER) .maximumSize(cacheSize).expireAfterAccess(cacheTimeMs, TimeUnit.MILLISECONDS) diff --git a/contribs/src/main/resources/META-INF/additional-spring-configuration-metadata.json b/contribs/src/main/resources/META-INF/additional-spring-configuration-metadata.json new file mode 100644 index 0000000000..4a30ef68b5 --- /dev/null +++ b/contribs/src/main/resources/META-INF/additional-spring-configuration-metadata.json @@ -0,0 +1,39 @@ +{ + "properties": [ + { + "name": "conductor.metrics-logger.reportPeriodSeconds", + "type": "java.lang.Long", + "description": "The interval (in seconds) at which the metrics will be reported into the log stream by the metrics-logger." + }, + { + "name": "conductor.tasks.http.readTimeout", + "type": "java.lang.Integer", + "description": "The read timeout of the underlying HttpClient used by the HTTP task." + }, + { + "name": "conductor.tasks.http.connectTimeout", + "type": "java.lang.Integer", + "description": "The connection timeout of the underlying HttpClient used by the HTTP task." + }, + { + "name": "conductor.tasks.kafka-publish.requestTimeoutMs", + "type": "java.lang.String", + "description": "The request.timeout.ms value that the kafka producer is configured with in the KAFKA_PUBLISH task." + }, + { + "name": "conductor.tasks.kafka-publish.maxBlockMs", + "type": "java.lang.String", + "description": "The max.block.ms value that the kafka producer is configured with in the KAFKA_PUBLISH task." + }, + { + "name": "conductor.tasks.kafka-publish.cacheSize", + "type": "java.lang.Integer", + "description": "The maximum number of entries permitted in the in-memory cache used by the KAFKA_PUBLISH task." + }, + { + "name": "conductor.tasks.kafka-publish.cacheTimeMs", + "type": "java.lang.Integer", + "description": "The duration after which a cached entry will be removed from the in-memory cache used by the KAFKA_PUBLISH task." + } + ] +} \ No newline at end of file diff --git a/contribs/src/test/java/com/netflix/conductor/contribs/lock/LocalOnlyLockTest.java b/contribs/src/test/java/com/netflix/conductor/contribs/lock/LocalOnlyLockTest.java index 64862b7e83..87a60258cb 100644 --- a/contribs/src/test/java/com/netflix/conductor/contribs/lock/LocalOnlyLockTest.java +++ b/contribs/src/test/java/com/netflix/conductor/contribs/lock/LocalOnlyLockTest.java @@ -107,7 +107,7 @@ public class LocalOnlyLockTest { // @Test // public void testLockConfiguration() { // new ApplicationContextRunner() -// .withPropertyValues("workflow.decider.locking.server=LOCAL_ONLY") +// .withPropertyValues("conductor.workflow-execution-lock.type=local_only") // .withUserConfiguration(LocalOnlyLockConfiguration.class) // .run(context -> { // LocalOnlyLock lock = context.getBean(LocalOnlyLock.class); diff --git a/contribs/src/test/java/com/netflix/conductor/contribs/queue/amqp/AMQPEventQueueProviderTest.java b/contribs/src/test/java/com/netflix/conductor/contribs/queue/amqp/AMQPEventQueueProviderTest.java index 406053626b..ef88fc859c 100644 --- a/contribs/src/test/java/com/netflix/conductor/contribs/queue/amqp/AMQPEventQueueProviderTest.java +++ b/contribs/src/test/java/com/netflix/conductor/contribs/queue/amqp/AMQPEventQueueProviderTest.java @@ -34,7 +34,7 @@ public class AMQPEventQueueProviderTest { public void setUp() { properties = mock(AMQPEventQueueProperties.class); when(properties.getBatchSize()).thenReturn(1); - when(properties.getPollTimeMS()).thenReturn(100); + when(properties.getPollTimeMs()).thenReturn(100); when(properties.getHosts()).thenReturn(ConnectionFactory.DEFAULT_HOST); when(properties.getUsername()).thenReturn(ConnectionFactory.DEFAULT_USER); when(properties.getPassword()).thenReturn(ConnectionFactory.DEFAULT_PASS); diff --git a/contribs/src/test/java/com/netflix/conductor/contribs/queue/amqp/AMQPObservableQueueTest.java b/contribs/src/test/java/com/netflix/conductor/contribs/queue/amqp/AMQPObservableQueueTest.java index f1478825f9..c053fdbfce 100644 --- a/contribs/src/test/java/com/netflix/conductor/contribs/queue/amqp/AMQPObservableQueueTest.java +++ b/contribs/src/test/java/com/netflix/conductor/contribs/queue/amqp/AMQPObservableQueueTest.java @@ -82,7 +82,7 @@ public class AMQPObservableQueueTest { public void setUp() { properties = mock(AMQPEventQueueProperties.class); when(properties.getBatchSize()).thenReturn(1); - when(properties.getPollTimeMS()).thenReturn(100); + when(properties.getPollTimeMs()).thenReturn(100); when(properties.getHosts()).thenReturn(ConnectionFactory.DEFAULT_HOST); when(properties.getUsername()).thenReturn(ConnectionFactory.DEFAULT_USER); when(properties.getPassword()).thenReturn(ConnectionFactory.DEFAULT_PASS); diff --git a/contribs/src/test/java/com/netflix/conductor/contribs/queue/amqp/AMQPSettingsTest.java b/contribs/src/test/java/com/netflix/conductor/contribs/queue/amqp/AMQPSettingsTest.java index a1c53bfec1..10b1664b48 100644 --- a/contribs/src/test/java/com/netflix/conductor/contribs/queue/amqp/AMQPSettingsTest.java +++ b/contribs/src/test/java/com/netflix/conductor/contribs/queue/amqp/AMQPSettingsTest.java @@ -33,7 +33,7 @@ public class AMQPSettingsTest { public void setUp() { properties = mock(AMQPEventQueueProperties.class); when(properties.getBatchSize()).thenReturn(1); - when(properties.getPollTimeMS()).thenReturn(100); + when(properties.getPollTimeMs()).thenReturn(100); when(properties.getHosts()).thenReturn(ConnectionFactory.DEFAULT_HOST); when(properties.getUsername()).thenReturn(ConnectionFactory.DEFAULT_USER); when(properties.getPassword()).thenReturn(ConnectionFactory.DEFAULT_PASS); diff --git a/core/src/main/java/com/netflix/conductor/core/config/ConductorCoreConfiguration.java b/core/src/main/java/com/netflix/conductor/core/config/ConductorCoreConfiguration.java index 82cbf568ca..40cb9690ca 100644 --- a/core/src/main/java/com/netflix/conductor/core/config/ConductorCoreConfiguration.java +++ b/core/src/main/java/com/netflix/conductor/core/config/ConductorCoreConfiguration.java @@ -33,28 +33,30 @@ import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Qualifier; import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; @Configuration(proxyBeanMethods = false) +@EnableConfigurationProperties(ConductorProperties.class) public class ConductorCoreConfiguration { private static final Logger LOGGER = LoggerFactory.getLogger(ConductorCoreConfiguration.class); - @ConditionalOnProperty(prefix = "workflow", name = "decider.locking.server", havingValue = "noop_lock", matchIfMissing = true) + @ConditionalOnProperty(name = "conductor.workflow-execution-lock.type", havingValue = "noop_lock", matchIfMissing = true) @Bean public Lock provideLock() { return new NoopLock(); } - @ConditionalOnProperty(prefix = "workflow", name = "external.payload.storage", havingValue = "DUMMY", matchIfMissing = true) + @ConditionalOnProperty(name = "conductor.external-payload-storage.type", havingValue = "dummy", matchIfMissing = true) @Bean public ExternalPayloadStorage dummyExternalPayloadStorage() { - LOGGER.info("Initialized dummy payload storage"); + LOGGER.info("Initialized dummy payload storage!"); return new DummyPayloadStorage(); } - @ConditionalOnProperty(prefix = "workflow", name = "status.listener.type", havingValue = "stub", matchIfMissing = true) + @ConditionalOnProperty(name = "conductor.workflow-status-listener.type", havingValue = "stub", matchIfMissing = true) @Bean public WorkflowStatusListener workflowStatusListener() { return new WorkflowStatusListenerStub(); @@ -66,7 +68,7 @@ public ExecutorService executorService(ConductorProperties conductorProperties) .setNameFormat("conductor-worker-%d") .setDaemon(true) .build(); - return Executors.newFixedThreadPool(conductorProperties.getExecutorServiceMaxThreads(), threadFactory); + return Executors.newFixedThreadPool(conductorProperties.getExecutorServiceMaxThreadCount(), threadFactory); } @Bean diff --git a/core/src/main/java/com/netflix/conductor/core/config/ConductorProperties.java b/core/src/main/java/com/netflix/conductor/core/config/ConductorProperties.java index b12df68c38..5a0cbac820 100644 --- a/core/src/main/java/com/netflix/conductor/core/config/ConductorProperties.java +++ b/core/src/main/java/com/netflix/conductor/core/config/ConductorProperties.java @@ -12,163 +12,146 @@ */ package com.netflix.conductor.core.config; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.stereotype.Component; - import java.util.HashMap; import java.util.Map; import java.util.Properties; +import org.springframework.boot.context.properties.ConfigurationProperties; -@Component +@ConfigurationProperties("conductor.app") public class ConductorProperties { /** - * Current environment. e.g. test, prod + * Name of the stack within which the app is running. e.g. devint, testintg, staging, prod etc. */ - @Value("${environment:test}") - private String environment; + private String stack = "test"; /** - * name of the stack under which the app is running. e.g. devint, testintg, staging, prod etc. + * The id with the app has been registered. */ - @Value("${STACK:test}") - private String stack; + private String appId = "conductor"; /** - * APP_ID + * The maximum number of threads to be allocated to the executor service threadpool. */ - @Value("${APP_ID:conductor}") - private String appId; + private int executorServiceMaxThreadCount = 50; /** - * maximum number of threads to be allocated to the executor service threadpool + * The frequency in seconds, at which the workflow sweeper should run to evaluate running workflows. */ - @Value("${workflow.executor.service.max.threads:50}") - private int executorServiceMaxThreads; + private int sweepFrequencySeconds = 30; /** - * time frequency in seconds, at which the workflow sweeper should run to evaluate running workflows + * Used to enable/disable the workflow sweeper. */ - @Value("${decider.sweep.frequency.seconds:30}") - private int sweepFrequency; + private boolean sweepDisabled = false; /** - * when set to true, the sweep is disabled + * The number of threads to configure the threadpool in the workflow sweeper. */ - @Value("${decider.sweep.disable:false}") - private boolean disableSweep; - - @Value("${workflow.sweeper.thread.count:5}") - private int sweeperThreadCount; + private int sweeperThreadCount = 5; /** - * Number of threads to be used by the event processor + * The number of threads to configure the threadpool in the event processor. */ - @Value("${workflow.event.processor.thread.count:2}") - private int eventProcessorThreadCount; + private int eventProcessorThreadCount = 2; /** - * when set to true, message from the event processing are indexed + * Used to enable/disable the indexing of messages within event payloads. */ - @Value("${workflow.event.message.indexing.enabled:true}") - private boolean eventMessageIndexingEnabled; + private boolean eventMessageIndexingEnabled = true; /** - * when set to true, event execution results are indexed + * Used to enable/disable the indexing of event execution results. */ - @Value("${workflow.event.execution.indexing.enabled:true}") - private boolean eventExecutionIndexingEnabled; + private boolean eventExecutionIndexingEnabled = true; - @Value("${workflow.decider.locking.enabled:false}") - private boolean enableWorkflowExecutionLock; - - @Value("${workflow.locking.lease.time.ms:60000}") - private long lockLeaseTimeMs; + /** + * Used to enable/disable the workflow execution lock. + */ + private boolean workflowExecutionLockEnabled = false; - @Value("${workflow.locking.time.to.try.ms:500}") - private long lockTimeToTryMs; + /** + * The time (in milliseconds) for which the lock is leased for. + */ + private long lockLeaseTimeMs = 60000; - @Value("${tasks.active.worker.lastpoll:10}") - private int activeWorkerLastPollSecs; + /** + * The time (in milliseconds) for which the thread will block in an attempt to acquire the lock. + */ + private long lockTimeToTryMs = 500; - @Value("${task.queue.message.postponeSeconds:60}") - private int queueTaskMessagePostponeSeconds; + /** + * The time (in seconds) that is used to consider if a worker is actively polling for a task. + */ + private int activeWorkerLastPollSecs = 10; - @Value("${task.requeue.timeout:60000}") - private int taskRequeueTimeout; + /** + * The time (in seconds) for which a task execution will be postponed if being rate limited or concurrent execution + * limited. + */ + private int taskExecutionPostponeSeconds = 60; /** - * if true(default), enables task execution log indexing + * Used to enable/disable the indexing of task execution logs. */ - @Value("${workflow.taskExecLog.indexing.enabled:true}") - private boolean taskExecLogIndexingEnabled; + private boolean taskExecLogIndexingEnabled = true; /** - * when set to true, the indexing operation to elasticsearch will be performed asynchronously + * Used to enable/disable asynchronous indexing to elasticsearch. */ - @Value("${async.indexing.enabled:false}") - private boolean enableAsyncIndexing; + private boolean asyncIndexingEnabled = false; /** - * the number of threads to be used within the threadpool for system task workers + * The number of threads to be used within the threadpool for system task workers. */ - @Value("${workflow.system.task.worker.thread.count:10}") - private int systemTaskWorkerThreadCount; + private int systemTaskWorkerThreadCount = 10; /** - * the interval (in seconds) after which a system task will be checked for completion + * The interval (in seconds) after which a system task will be checked by the system task worker for completion. */ - @Value("${workflow.system.task.worker.callback.seconds:30}") - private int systemTaskWorkerCallbackSeconds; + private int systemTaskWorkerCallbackSeconds = 30; /** - * the interval (in seconds) at which system task queues will be polled by the system task workers + * The interval (in seconds) at which system task queues will be polled by the system task workers. */ - @Value("${workflow.system.task.worker.poll.interval:50}") - private int systemTaskWorkerPollInterval; + private int systemTaskWorkerPollInterval = 50; /** - * the namespace for the system task workers to provide instance level isolation + * The namespace for the system task workers to provide instance level isolation. */ - @Value("${workflow.system.task.worker.executionNameSpace:}") - private String systemTaskWorkerExecutionNamespace; + private String systemTaskWorkerExecutionNamespace = ""; /** - * the number of threads to be used within the threadpool for system task workers in each isolation group + * The number of threads to be used within the threadpool for system task workers in each isolation group. */ - @Value("${workflow.isolated.system.task.worker.thread.count:1}") - private int systemTaskWorkerIsolatedThreadCount; + private int isolatedSystemTaskWorkerThreadCount = 1; /** - * the max number of system tasks to poll + * The max number of system tasks to be polled in a single request. */ - @Value("${workflow.system.task.queue.pollCount:1}") - private int systemTaskMaxPollCount; + private int systemTaskMaxPollCount = 1; /** - * when set to true, the background task workers executing async system tasks (eg HTTP) are disabled + * Used to enable/disable the system task workers that execute async system tasks like HTTP, etc. */ - @Value("${conductor.disable.async.workers:false}") - private boolean disableAsyncWorkers; + private boolean systemTaskWorkersDisabled = false; /** - * the duration of workflow execution which qualifies a workflow as a short-running workflow for async updating to - * the index + * The duration of workflow execution which qualifies a workflow as a short-running workflow when async indexing to + * elasticsearch is enabled. */ - @Value("${async.update.short.workflow.duration.seconds:30}") - private int asyncUpdateShortRunningWorkflowDuration; + private int asyncUpdateShortRunningWorkflowDuration = 30; /** - * the delay with which short-running workflows will be updated in the index + * The delay with which short-running workflows will be updated in the elasticsearch index when async indexing is + * enabled. */ - @Value("${async.update.delay.seconds:60}") - private int asyncUpdateDelay; + private int asyncUpdateDelay = 60; /** - * true if owner email is mandatory for task definitions and workflow definitions + * Used to control the validation for owner email field as mandatory within workflow and task definitions. */ - @Value("${workflow.owner.email.mandatory:true}") - private boolean ownerEmailMandatory; + private boolean ownerEmailMandatory = true; /** * Configuration to enable {@link com.netflix.conductor.core.execution.WorkflowRepairService}, that tries to keep @@ -176,257 +159,402 @@ public class ConductorProperties { *

* This is disabled by default; To enable, the Queueing layer must implement QueueDAO.containsMessage method. */ - @Value("${workflow.repairservice.enabled:false}") - private boolean workflowRepairServiceEnabled; + private boolean workflowRepairServiceEnabled = false; /** - * the number of threads to be use in Scheduler used for polling events from multiple event queues. By default, a + * The number of threads to be usde in Scheduler used for polling events from multiple event queues. By default, a * thread count equal to the number of CPU cores is chosen. */ - @Value("${workflow.event.queue.scheduler.poll.thread.count:#{T(java.lang.Runtime).getRuntime().availableProcessors()}}") - private int eventSchedulerPollThreadCount; + private int eventQueueSchedulerPollThreadCount = Runtime.getRuntime().availableProcessors(); - @Value("${workflow.dyno.queues.pollingInterval:100}") - private int eventQueuePollInterval; + /** + * The time interval (in milliseconds) at which the default event queues will be polled. + */ + private int eventQueuePollIntervalMs = 100; - @Value("${workflow.dyno.queues.pollCount:10}") - private int eventQueuePollCount; + /** + * The number of messages to be polled from a default event queue in a single operation. + */ + private int eventQueuePollCount = 10; - @Value("${workflow.dyno.queues.longPollTimeout:1000}") - private int eventQueueLongPollTimeout; + /** + * The timeout (in milliseconds) for the poll operation on the default event queue. + */ + private int eventQueueLongPollTimeout = 1000; /** * The threshold of the workflow input payload size in KB beyond which the payload will be stored in {@link - * com.netflix.conductor.common.utils.ExternalPayloadStorage} + * com.netflix.conductor.common.utils.ExternalPayloadStorage}. */ - @Value("${conductor.workflow.input.payload.threshold.kb:5120}") - private Long workflowInputPayloadSizeThresholdKB; + private Long workflowInputPayloadSizeThresholdKB = 5120L; /** * The maximum threshold of the workflow input payload size in KB beyond which input will be rejected and the - * workflow will be marked as FAILED + * workflow will be marked as FAILED. */ - @Value("${conductor.max.workflow.input.payload.threshold.kb:10240}") - private Long maxWorkflowInputPayloadSizeThresholdKB; + private Long maxWorkflowInputPayloadSizeThresholdKB = 10240L; /** * The threshold of the workflow output payload size in KB beyond which the payload will be stored in {@link - * com.netflix.conductor.common.utils.ExternalPayloadStorage} + * com.netflix.conductor.common.utils.ExternalPayloadStorage}. */ - @Value("${conductor.workflow.output.payload.threshold.kb:5120}") - private Long workflowOutputPayloadSizeThresholdKB; + private Long workflowOutputPayloadSizeThresholdKB = 5120L; /** * The maximum threshold of the workflow output payload size in KB beyond which output will be rejected and the - * workflow will be marked as FAILED + * workflow will be marked as FAILED. */ - @Value("${conductor.max.workflow.output.payload.threshold.kb:10240}") - private Long maxWorkflowOutputPayloadSizeThresholdKB; + private Long maxWorkflowOutputPayloadSizeThresholdKB = 10240L; /** * The threshold of the task input payload size in KB beyond which the payload will be stored in {@link - * com.netflix.conductor.common.utils.ExternalPayloadStorage} + * com.netflix.conductor.common.utils.ExternalPayloadStorage}. */ - @Value("${conductor.task.input.payload.threshold.kb:3072}") - private Long taskInputPayloadSizeThresholdKB; + private Long taskInputPayloadSizeThresholdKB = 3072L; /** * The maximum threshold of the task input payload size in KB beyond which the task input will be rejected and the - * task will be marked as FAILED_WITH_TERMINAL_ERROR + * task will be marked as FAILED_WITH_TERMINAL_ERROR. */ - @Value("${conductor.max.task.input.payload.threshold.kb:10240}") - private Long maxTaskInputPayloadSizeThresholdKB; + private Long maxTaskInputPayloadSizeThresholdKB = 10240L; /** * The threshold of the task output payload size in KB beyond which the payload will be stored in {@link - * com.netflix.conductor.common.utils.ExternalPayloadStorage} + * com.netflix.conductor.common.utils.ExternalPayloadStorage}. */ - @Value("${conductor.task.output.payload.threshold.kb:3072}") - private Long taskOutputPayloadSizeThresholdKB; + private Long taskOutputPayloadSizeThresholdKB = 3072L; /** * The maximum threshold of the task output payload size in KB beyond which the task input will be rejected and the - * task will be marked as FAILED_WITH_TERMINAL_ERROR + * task will be marked as FAILED_WITH_TERMINAL_ERROR. */ - @Value("${conductor.max.task.output.payload.threshold.kb:10240}") - private Long maxTaskOutputPayloadSizeThresholdKB; + private Long maxTaskOutputPayloadSizeThresholdKB = 10240L; /** * The maximum threshold of the workflow variables payload size in KB beyond which the task changes will be rejected - * and the task will be marked as FAILED_WITH_TERMINAL_ERROR + * and the task will be marked as FAILED_WITH_TERMINAL_ERROR. */ - @Value("${conductor.max.workflow.variables.payload.threshold.kb:256}") - private Long maxWorkflowVariablesPayloadSizeThresholdKB; - - public String getEnvironment() { - return environment; - } + private Long maxWorkflowVariablesPayloadSizeThresholdKB = 256L; public String getStack() { return stack; } + public void setStack(String stack) { + this.stack = stack; + } + public String getAppId() { return appId; } - public int getExecutorServiceMaxThreads() { - return executorServiceMaxThreads; + public void setAppId(String appId) { + this.appId = appId; + } + + public int getExecutorServiceMaxThreadCount() { + return executorServiceMaxThreadCount; } - public int getSweepFrequency() { - return sweepFrequency; + public void setExecutorServiceMaxThreadCount(int executorServiceMaxThreadCount) { + this.executorServiceMaxThreadCount = executorServiceMaxThreadCount; } - public boolean disableSweep() { - return disableSweep; + public int getSweepFrequencySeconds() { + return sweepFrequencySeconds; + } + + public void setSweepFrequencySeconds(int sweepFrequencySeconds) { + this.sweepFrequencySeconds = sweepFrequencySeconds; + } + + public boolean isSweepDisabled() { + return sweepDisabled; + } + + public void setSweepDisabled(boolean sweepDisabled) { + this.sweepDisabled = sweepDisabled; } public int getSweeperThreadCount() { return sweeperThreadCount; } + public void setSweeperThreadCount(int sweeperThreadCount) { + this.sweeperThreadCount = sweeperThreadCount; + } + public int getEventProcessorThreadCount() { return eventProcessorThreadCount; } + public void setEventProcessorThreadCount(int eventProcessorThreadCount) { + this.eventProcessorThreadCount = eventProcessorThreadCount; + } + public boolean isEventMessageIndexingEnabled() { return eventMessageIndexingEnabled; } + public void setEventMessageIndexingEnabled(boolean eventMessageIndexingEnabled) { + this.eventMessageIndexingEnabled = eventMessageIndexingEnabled; + } + public boolean isEventExecutionIndexingEnabled() { return eventExecutionIndexingEnabled; } + public void setEventExecutionIndexingEnabled(boolean eventExecutionIndexingEnabled) { + this.eventExecutionIndexingEnabled = eventExecutionIndexingEnabled; + } + public boolean isWorkflowExecutionLockEnabled() { - return enableWorkflowExecutionLock; + return workflowExecutionLockEnabled; + } + + public void setWorkflowExecutionLockEnabled(boolean workflowExecutionLockEnabled) { + this.workflowExecutionLockEnabled = workflowExecutionLockEnabled; } public long getLockLeaseTimeMs() { return lockLeaseTimeMs; } + public void setLockLeaseTimeMs(long lockLeaseTimeMs) { + this.lockLeaseTimeMs = lockLeaseTimeMs; + } + public long getLockTimeToTryMs() { return lockTimeToTryMs; } + public void setLockTimeToTryMs(long lockTimeToTryMs) { + this.lockTimeToTryMs = lockTimeToTryMs; + } + public int getActiveWorkerLastPollSecs() { return activeWorkerLastPollSecs; } - public int getQueueTaskMessagePostponeSeconds() { - return queueTaskMessagePostponeSeconds; + public void setActiveWorkerLastPollSecs(int activeWorkerLastPollSecs) { + this.activeWorkerLastPollSecs = activeWorkerLastPollSecs; + } + + public int getTaskExecutionPostponeSeconds() { + return taskExecutionPostponeSeconds; } - public int getTaskRequeueTimeout() { - return taskRequeueTimeout; + public void setTaskExecutionPostponeSeconds(int taskExecutionPostponeSeconds) { + this.taskExecutionPostponeSeconds = taskExecutionPostponeSeconds; } public boolean isTaskExecLogIndexingEnabled() { return taskExecLogIndexingEnabled; } - public boolean enableAsyncIndexing() { - return enableAsyncIndexing; + public void setTaskExecLogIndexingEnabled(boolean taskExecLogIndexingEnabled) { + this.taskExecLogIndexingEnabled = taskExecLogIndexingEnabled; + } + + public boolean isAsyncIndexingEnabled() { + return asyncIndexingEnabled; + } + + public void setAsyncIndexingEnabled(boolean asyncIndexingEnabled) { + this.asyncIndexingEnabled = asyncIndexingEnabled; } public int getSystemTaskWorkerThreadCount() { return systemTaskWorkerThreadCount; } + public void setSystemTaskWorkerThreadCount(int systemTaskWorkerThreadCount) { + this.systemTaskWorkerThreadCount = systemTaskWorkerThreadCount; + } + public int getSystemTaskWorkerCallbackSeconds() { return systemTaskWorkerCallbackSeconds; } + public void setSystemTaskWorkerCallbackSeconds(int systemTaskWorkerCallbackSeconds) { + this.systemTaskWorkerCallbackSeconds = systemTaskWorkerCallbackSeconds; + } + public int getSystemTaskWorkerPollInterval() { return systemTaskWorkerPollInterval; } + public void setSystemTaskWorkerPollInterval(int systemTaskWorkerPollInterval) { + this.systemTaskWorkerPollInterval = systemTaskWorkerPollInterval; + } + public String getSystemTaskWorkerExecutionNamespace() { return systemTaskWorkerExecutionNamespace; } - public int getSystemTaskWorkerIsolatedThreadCount() { - return systemTaskWorkerIsolatedThreadCount; + public void setSystemTaskWorkerExecutionNamespace(String systemTaskWorkerExecutionNamespace) { + this.systemTaskWorkerExecutionNamespace = systemTaskWorkerExecutionNamespace; + } + + public int getIsolatedSystemTaskWorkerThreadCount() { + return isolatedSystemTaskWorkerThreadCount; + } + + public void setIsolatedSystemTaskWorkerThreadCount(int isolatedSystemTaskWorkerThreadCount) { + this.isolatedSystemTaskWorkerThreadCount = isolatedSystemTaskWorkerThreadCount; } public int getSystemTaskMaxPollCount() { return systemTaskMaxPollCount; } - public boolean disableAsyncWorkers() { - return disableAsyncWorkers; + public void setSystemTaskMaxPollCount(int systemTaskMaxPollCount) { + this.systemTaskMaxPollCount = systemTaskMaxPollCount; + } + + public boolean isSystemTaskWorkersDisabled() { + return systemTaskWorkersDisabled; + } + + public void setSystemTaskWorkersDisabled(boolean systemTaskWorkersDisabled) { + this.systemTaskWorkersDisabled = systemTaskWorkersDisabled; } public int getAsyncUpdateShortRunningWorkflowDuration() { return asyncUpdateShortRunningWorkflowDuration; } + public void setAsyncUpdateShortRunningWorkflowDuration(int asyncUpdateShortRunningWorkflowDuration) { + this.asyncUpdateShortRunningWorkflowDuration = asyncUpdateShortRunningWorkflowDuration; + } + public int getAsyncUpdateDelay() { return asyncUpdateDelay; } + public void setAsyncUpdateDelay(int asyncUpdateDelay) { + this.asyncUpdateDelay = asyncUpdateDelay; + } + public boolean isOwnerEmailMandatory() { return ownerEmailMandatory; } + public void setOwnerEmailMandatory(boolean ownerEmailMandatory) { + this.ownerEmailMandatory = ownerEmailMandatory; + } + public boolean isWorkflowRepairServiceEnabled() { return workflowRepairServiceEnabled; } - public int getEventSchedulerPollThreadCount() { - return eventSchedulerPollThreadCount; + public void setWorkflowRepairServiceEnabled(boolean workflowRepairServiceEnabled) { + this.workflowRepairServiceEnabled = workflowRepairServiceEnabled; } - public int getEventQueuePollInterval() { - return eventQueuePollInterval; + public int getEventQueueSchedulerPollThreadCount() { + return eventQueueSchedulerPollThreadCount; + } + + public void setEventQueueSchedulerPollThreadCount(int eventQueueSchedulerPollThreadCount) { + this.eventQueueSchedulerPollThreadCount = eventQueueSchedulerPollThreadCount; + } + + public int getEventQueuePollIntervalMs() { + return eventQueuePollIntervalMs; + } + + public void setEventQueuePollIntervalMs(int eventQueuePollIntervalMs) { + this.eventQueuePollIntervalMs = eventQueuePollIntervalMs; } public int getEventQueuePollCount() { return eventQueuePollCount; } + public void setEventQueuePollCount(int eventQueuePollCount) { + this.eventQueuePollCount = eventQueuePollCount; + } + public int getEventQueueLongPollTimeout() { return eventQueueLongPollTimeout; } + public void setEventQueueLongPollTimeout(int eventQueueLongPollTimeout) { + this.eventQueueLongPollTimeout = eventQueueLongPollTimeout; + } + public Long getWorkflowInputPayloadSizeThresholdKB() { return workflowInputPayloadSizeThresholdKB; } + public void setWorkflowInputPayloadSizeThresholdKB(Long workflowInputPayloadSizeThresholdKB) { + this.workflowInputPayloadSizeThresholdKB = workflowInputPayloadSizeThresholdKB; + } + public Long getMaxWorkflowInputPayloadSizeThresholdKB() { return maxWorkflowInputPayloadSizeThresholdKB; } + public void setMaxWorkflowInputPayloadSizeThresholdKB(Long maxWorkflowInputPayloadSizeThresholdKB) { + this.maxWorkflowInputPayloadSizeThresholdKB = maxWorkflowInputPayloadSizeThresholdKB; + } + public Long getWorkflowOutputPayloadSizeThresholdKB() { return workflowOutputPayloadSizeThresholdKB; } + public void setWorkflowOutputPayloadSizeThresholdKB(Long workflowOutputPayloadSizeThresholdKB) { + this.workflowOutputPayloadSizeThresholdKB = workflowOutputPayloadSizeThresholdKB; + } + public Long getMaxWorkflowOutputPayloadSizeThresholdKB() { return maxWorkflowOutputPayloadSizeThresholdKB; } + public void setMaxWorkflowOutputPayloadSizeThresholdKB(Long maxWorkflowOutputPayloadSizeThresholdKB) { + this.maxWorkflowOutputPayloadSizeThresholdKB = maxWorkflowOutputPayloadSizeThresholdKB; + } + public Long getTaskInputPayloadSizeThresholdKB() { return taskInputPayloadSizeThresholdKB; } + public void setTaskInputPayloadSizeThresholdKB(Long taskInputPayloadSizeThresholdKB) { + this.taskInputPayloadSizeThresholdKB = taskInputPayloadSizeThresholdKB; + } + public Long getMaxTaskInputPayloadSizeThresholdKB() { return maxTaskInputPayloadSizeThresholdKB; } + public void setMaxTaskInputPayloadSizeThresholdKB(Long maxTaskInputPayloadSizeThresholdKB) { + this.maxTaskInputPayloadSizeThresholdKB = maxTaskInputPayloadSizeThresholdKB; + } + public Long getTaskOutputPayloadSizeThresholdKB() { return taskOutputPayloadSizeThresholdKB; } + public void setTaskOutputPayloadSizeThresholdKB(Long taskOutputPayloadSizeThresholdKB) { + this.taskOutputPayloadSizeThresholdKB = taskOutputPayloadSizeThresholdKB; + } + public Long getMaxTaskOutputPayloadSizeThresholdKB() { return maxTaskOutputPayloadSizeThresholdKB; } + public void setMaxTaskOutputPayloadSizeThresholdKB(Long maxTaskOutputPayloadSizeThresholdKB) { + this.maxTaskOutputPayloadSizeThresholdKB = maxTaskOutputPayloadSizeThresholdKB; + } + public Long getMaxWorkflowVariablesPayloadSizeThresholdKB() { return maxWorkflowVariablesPayloadSizeThresholdKB; } - //SBMTODO: is this needed (?) + public void setMaxWorkflowVariablesPayloadSizeThresholdKB(Long maxWorkflowVariablesPayloadSizeThresholdKB) { + this.maxWorkflowVariablesPayloadSizeThresholdKB = maxWorkflowVariablesPayloadSizeThresholdKB; + } /** * @return Returns all the configurations in a map. diff --git a/core/src/main/java/com/netflix/conductor/core/config/SchedulerConfiguration.java b/core/src/main/java/com/netflix/conductor/core/config/SchedulerConfiguration.java index 1cff380913..52dc8fcbfe 100644 --- a/core/src/main/java/com/netflix/conductor/core/config/SchedulerConfiguration.java +++ b/core/src/main/java/com/netflix/conductor/core/config/SchedulerConfiguration.java @@ -36,7 +36,7 @@ public Scheduler scheduler(ConductorProperties properties) { .setNameFormat("event-queue-poll-scheduler-thread-%d") .build(); Executor executorService = Executors - .newFixedThreadPool(properties.getEventSchedulerPollThreadCount(), threadFactory); + .newFixedThreadPool(properties.getEventQueueSchedulerPollThreadCount(), threadFactory); return Schedulers.from(executorService); } diff --git a/core/src/main/java/com/netflix/conductor/core/events/SimpleEventProcessor.java b/core/src/main/java/com/netflix/conductor/core/events/SimpleEventProcessor.java index 0c4f1ab5bb..c9b640e237 100644 --- a/core/src/main/java/com/netflix/conductor/core/events/SimpleEventProcessor.java +++ b/core/src/main/java/com/netflix/conductor/core/events/SimpleEventProcessor.java @@ -52,10 +52,10 @@ /** * Event Processor is used to dispatch actions based on the incoming events to execution queue. * - *

Set workflow.default.event.processor.enabled=false to disable event processing.

+ *

Set conductor.default-event-processor.enabled=false to disable event processing.

*/ @Component -@ConditionalOnProperty(prefix = "workflow", name = "default.event.processor.enabled", havingValue = "true", matchIfMissing = true) +@ConditionalOnProperty(name = "conductor.default-event-processor.enabled", havingValue = "true", matchIfMissing = true) public class SimpleEventProcessor implements EventProcessor { private static final Logger LOGGER = LoggerFactory.getLogger(SimpleEventProcessor.class); @@ -92,7 +92,7 @@ public SimpleEventProcessor(ExecutionService executionService, MetadataService m LOGGER.info("Event Processing is ENABLED. executorThreadCount set to {}", executorThreadCount); } else { LOGGER.warn("workflow.event.processor.thread.count={} must be greater than 0. " + - "To disable event processing, set workflow.default.event.processor.enabled=false", executorThreadCount); + "To disable event processing, set conductor.default-event-processor.enabled=false", executorThreadCount); throw new IllegalStateException("workflow.event.processor.thread.count must be greater than 0"); } } diff --git a/core/src/main/java/com/netflix/conductor/core/events/queue/ConductorEventQueueProvider.java b/core/src/main/java/com/netflix/conductor/core/events/queue/ConductorEventQueueProvider.java index 2be48ec0ee..31c888d336 100644 --- a/core/src/main/java/com/netflix/conductor/core/events/queue/ConductorEventQueueProvider.java +++ b/core/src/main/java/com/netflix/conductor/core/events/queue/ConductorEventQueueProvider.java @@ -15,23 +15,22 @@ import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.core.events.EventQueueProvider; import com.netflix.conductor.dao.QueueDAO; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; import org.springframework.stereotype.Component; import rx.Scheduler; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - /** - * Default provider for {@link com.netflix.conductor.core.events.queue.ObservableQueue} - * that listens on the conductor queue prefix. + * Default provider for {@link com.netflix.conductor.core.events.queue.ObservableQueue} that listens on the + * conductor queue prefix. * - *

Set workflow.default.event.queue.enabled=false to disable the default queue.

+ *

Set conductor.event-queues.default.enabled=false to disable the default queue.

* * @see ConductorObservableQueue */ @Component -@ConditionalOnProperty(prefix = "workflow", name = "default.event.queue.enabled", havingValue = "true", matchIfMissing = true) +@ConditionalOnProperty(name = "conductor.event-queues.default.enabled", havingValue = "true", matchIfMissing = true) public class ConductorEventQueueProvider implements EventQueueProvider { private final Map queues = new ConcurrentHashMap<>(); diff --git a/core/src/main/java/com/netflix/conductor/core/events/queue/ConductorObservableQueue.java b/core/src/main/java/com/netflix/conductor/core/events/queue/ConductorObservableQueue.java index 9846c70b66..8dcfd4235f 100644 --- a/core/src/main/java/com/netflix/conductor/core/events/queue/ConductorObservableQueue.java +++ b/core/src/main/java/com/netflix/conductor/core/events/queue/ConductorObservableQueue.java @@ -45,7 +45,7 @@ public class ConductorObservableQueue implements ObservableQueue { ConductorObservableQueue(String queueName, QueueDAO queueDAO, ConductorProperties properties, Scheduler scheduler) { this.queueName = queueName; this.queueDAO = queueDAO; - this.pollTimeInMS = properties.getEventQueuePollInterval(); + this.pollTimeInMS = properties.getEventQueuePollIntervalMs(); this.pollCount = properties.getEventQueuePollCount(); this.longPollTimeout = properties.getEventQueueLongPollTimeout(); this.scheduler = scheduler; diff --git a/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java b/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java index 7a91744e03..0b846b29ba 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java @@ -79,7 +79,7 @@ public class DeciderService { public DeciderService(ParametersUtils parametersUtils, MetadataDAO metadataDAO, ExternalPayloadStorageUtils externalPayloadStorageUtils, @Qualifier("taskProcessorsMap") Map taskMappers, - @Value("${workflow.task.pending.time.threshold.minutes:60}") long taskPendingTimeThresholdMins) { + @Value("${conductor.app.taskPendingTimeThresholdMins:60}") long taskPendingTimeThresholdMins) { this.metadataDAO = metadataDAO; this.parametersUtils = parametersUtils; this.taskMappers = taskMappers; diff --git a/core/src/main/java/com/netflix/conductor/core/execution/WorkflowExecutor.java b/core/src/main/java/com/netflix/conductor/core/execution/WorkflowExecutor.java index 60657e7e54..b50119c5eb 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/WorkflowExecutor.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/WorkflowExecutor.java @@ -113,7 +113,7 @@ public WorkflowExecutor(DeciderService deciderService, MetadataDAO metadataDAO, this.metadataMapperService = metadataMapperService; this.executionDAOFacade = executionDAOFacade; this.activeWorkerLastPollInSecs = properties.getActiveWorkerLastPollSecs(); - this.queueTaskMessagePostponeSeconds = properties.getQueueTaskMessagePostponeSeconds(); + this.queueTaskMessagePostponeSeconds = properties.getTaskExecutionPostponeSeconds(); this.workflowStatusListener = workflowStatusListener; this.executionLockService = executionLockService; this.parametersUtils = parametersUtils; @@ -346,7 +346,7 @@ public String startWorkflow( } private final Predicate validateLastPolledTime = pollData -> - pollData.getLastPollTime() > System.currentTimeMillis() - (activeWorkerLastPollInSecs * 1000); + pollData.getLastPollTime() > System.currentTimeMillis() - (activeWorkerLastPollInSecs * 1000L); private final Predicate isSystemTask = task -> SystemTaskType.is(task.getTaskType()); @@ -593,7 +593,7 @@ public void retry(String workflowId) { workflow.setStatus(WorkflowStatus.RUNNING); workflow.setLastRetriedTime(System.currentTimeMillis()); // Add to decider queue - queueDAO.push(DECIDER_QUEUE, workflow.getWorkflowId(), workflow.getPriority(), properties.getSweepFrequency()); + queueDAO.push(DECIDER_QUEUE, workflow.getWorkflowId(), workflow.getPriority(), properties.getSweepFrequencySeconds()); executionDAOFacade.updateWorkflow(workflow); // taskToBeRescheduled would set task `retried` to true, and hence it's important to updateTasks after obtaining task copy from taskToBeRescheduled. @@ -1619,7 +1619,7 @@ private boolean rerunWF(String workflowId, String taskId, Map ta } queueDAO - .push(DECIDER_QUEUE, workflow.getWorkflowId(), workflow.getPriority(), properties.getSweepFrequency()); + .push(DECIDER_QUEUE, workflow.getWorkflowId(), workflow.getPriority(), properties.getSweepFrequencySeconds()); executionDAOFacade.updateWorkflow(workflow); decide(workflowId); @@ -1662,7 +1662,7 @@ private boolean rerunWF(String workflowId, String taskId, Map ta } // Add to decider queue queueDAO - .push(DECIDER_QUEUE, workflow.getWorkflowId(), workflow.getPriority(), properties.getSweepFrequency()); + .push(DECIDER_QUEUE, workflow.getWorkflowId(), workflow.getPriority(), properties.getSweepFrequencySeconds()); executionDAOFacade.updateWorkflow(workflow); //update tasks in datastore to update workflow-tasks relationship for archived workflows executionDAOFacade.updateTasks(workflow.getTasks()); diff --git a/core/src/main/java/com/netflix/conductor/core/execution/WorkflowRepairService.java b/core/src/main/java/com/netflix/conductor/core/execution/WorkflowRepairService.java index 4efc8e4eef..6395a03cd4 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/WorkflowRepairService.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/WorkflowRepairService.java @@ -100,7 +100,7 @@ private boolean verifyAndRepairDeciderQueue(Workflow workflow) { if (!workflow.getStatus().isTerminal()) { String queueName = WorkflowExecutor.DECIDER_QUEUE; if (!queueDAO.containsMessage(queueName, workflow.getWorkflowId())) { - queueDAO.push(queueName, workflow.getWorkflowId(), properties.getSweepFrequency()); + queueDAO.push(queueName, workflow.getWorkflowId(), properties.getSweepFrequencySeconds()); Monitors.recordQueueMessageRepushFromRepairService(queueName); return true; } diff --git a/core/src/main/java/com/netflix/conductor/core/execution/WorkflowSweeper.java b/core/src/main/java/com/netflix/conductor/core/execution/WorkflowSweeper.java index 5b7f1a1a1e..a14c76d69d 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/WorkflowSweeper.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/WorkflowSweeper.java @@ -61,7 +61,7 @@ public void init(WorkflowExecutor workflowExecutor, WorkflowRepairService workfl ScheduledExecutorService deciderPool = Executors.newScheduledThreadPool(1); deciderPool.scheduleWithFixedDelay(() -> { try { - boolean disable = properties.disableSweep(); + boolean disable = properties.isSweepDisabled(); if (disable) { LOGGER.info("Workflow sweep is disabled."); return; @@ -103,7 +103,7 @@ public void sweep(List workflowIds, WorkflowExecutor workflowExecutor, boolean done = workflowExecutor.decide(workflowId); if (!done) { queueDAO.setUnackTimeout(WorkflowExecutor.DECIDER_QUEUE, workflowId, - properties.getSweepFrequency() * 1000); + properties.getSweepFrequencySeconds() * 1000); } else { queueDAO.remove(WorkflowExecutor.DECIDER_QUEUE, workflowId); } @@ -117,7 +117,7 @@ public void sweep(List workflowIds, WorkflowExecutor workflowExecutor, } catch (Exception e) { queueDAO .setUnackTimeout(WorkflowExecutor.DECIDER_QUEUE, workflowId, - properties.getSweepFrequency() * 1000); + properties.getSweepFrequencySeconds() * 1000); Monitors.error(CLASS_NAME, "sweep"); LOGGER.error("Error running sweep for " + workflowId, e); } diff --git a/core/src/main/java/com/netflix/conductor/core/execution/mapper/HTTPTaskMapper.java b/core/src/main/java/com/netflix/conductor/core/execution/mapper/HTTPTaskMapper.java index ecb901e14b..d891279def 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/mapper/HTTPTaskMapper.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/mapper/HTTPTaskMapper.java @@ -1,3 +1,15 @@ +/* + * Copyright 2020 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package com.netflix.conductor.core.execution.mapper; import com.netflix.conductor.common.metadata.tasks.Task; diff --git a/core/src/main/java/com/netflix/conductor/core/execution/mapper/SubWorkflowTaskMapper.java b/core/src/main/java/com/netflix/conductor/core/execution/mapper/SubWorkflowTaskMapper.java index ca1c20fbfb..f20f201472 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/mapper/SubWorkflowTaskMapper.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/mapper/SubWorkflowTaskMapper.java @@ -1,3 +1,15 @@ +/* + * Copyright 2020 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package com.netflix.conductor.core.execution.mapper; import com.google.common.annotations.VisibleForTesting; diff --git a/core/src/main/java/com/netflix/conductor/core/execution/tasks/IsolatedTaskQueueProducer.java b/core/src/main/java/com/netflix/conductor/core/execution/tasks/IsolatedTaskQueueProducer.java index f7a0bc4256..5809a45870 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/tasks/IsolatedTaskQueueProducer.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/tasks/IsolatedTaskQueueProducer.java @@ -16,18 +16,17 @@ import com.netflix.conductor.common.metadata.tasks.TaskDef; import com.netflix.conductor.core.utils.QueueUtils; import com.netflix.conductor.service.MetadataService; -import org.apache.commons.lang3.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.stereotype.Component; - import java.util.Collections; import java.util.List; import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.stereotype.Component; @Component public class IsolatedTaskQueueProducer { @@ -36,16 +35,16 @@ public class IsolatedTaskQueueProducer { private final MetadataService metadataService; public IsolatedTaskQueueProducer(MetadataService metadataService, - @Value("${workflow.isolated.system.task.enable:false}") boolean enableIsolatedSystemTask, - @Value("${workflow.isolated.system.task.poll.time.secs:10}") int isolatedSystemTaskPollInterval) { + @Value("${conductor.app.isolatedSystemTaskEnabled:false}") boolean isolatedSystemTaskEnabled, + @Value("${conductor.app.isolatedSystemTaskQueuePollIntervalSecs:10}") int isolatedSystemTaskQueuePollIntervalSecs) { this.metadataService = metadataService; - if (enableIsolatedSystemTask) { + if (isolatedSystemTaskEnabled) { LOGGER.info("Listening for isolation groups"); Executors.newSingleThreadScheduledExecutor().scheduleWithFixedDelay(this::addTaskQueues, 1000, - isolatedSystemTaskPollInterval, TimeUnit.SECONDS); + isolatedSystemTaskQueuePollIntervalSecs, TimeUnit.SECONDS); } else { LOGGER.info("Isolated System Task Worker DISABLED"); } @@ -59,9 +58,8 @@ private Set getIsolationExecutionNameSpaces() { .filter(taskDef -> StringUtils.isNotBlank(taskDef.getIsolationGroupId()) || StringUtils.isNotBlank(taskDef.getExecutionNameSpace())) .collect(Collectors.toSet()); - } catch (RuntimeException unknownException) { - LOGGER.error("Unknown exception received in getting isolation groups, sleeping and retrying", - unknownException); + } catch (RuntimeException e) { + LOGGER.error("Unknown exception received in getting isolation groups, sleeping and retrying", e); } return isolationExecutionNameSpaces; } diff --git a/core/src/main/java/com/netflix/conductor/core/execution/tasks/SystemTaskExecutor.java b/core/src/main/java/com/netflix/conductor/core/execution/tasks/SystemTaskExecutor.java index f5a891e9a2..c4c93fae1d 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/tasks/SystemTaskExecutor.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/tasks/SystemTaskExecutor.java @@ -139,7 +139,7 @@ ExecutionConfig getExecutionConfig(String taskQueue) { } private ExecutionConfig createExecutionConfig() { - int threadCount = properties.getSystemTaskWorkerIsolatedThreadCount(); + int threadCount = properties.getIsolatedSystemTaskWorkerThreadCount(); String threadNameFormat = "isolated-system-task-worker-%d"; return new ExecutionConfig(threadCount, threadNameFormat); } diff --git a/core/src/main/java/com/netflix/conductor/core/execution/tasks/SystemTaskWorkerCoordinator.java b/core/src/main/java/com/netflix/conductor/core/execution/tasks/SystemTaskWorkerCoordinator.java index 7148339b00..e1216c49c2 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/tasks/SystemTaskWorkerCoordinator.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/tasks/SystemTaskWorkerCoordinator.java @@ -114,7 +114,7 @@ private void listen(String queueName) { } private void pollAndExecute(String queueName) { - if (properties.disableAsyncWorkers()) { + if (properties.isSystemTaskWorkersDisabled()) { LOGGER.warn("System Task Worker is DISABLED. Not polling for system task in queue : {}", queueName); return; } diff --git a/core/src/main/java/com/netflix/conductor/core/orchestration/ExecutionDAOFacade.java b/core/src/main/java/com/netflix/conductor/core/orchestration/ExecutionDAOFacade.java index b96ef0edf5..c8190a3ad4 100644 --- a/core/src/main/java/com/netflix/conductor/core/orchestration/ExecutionDAOFacade.java +++ b/core/src/main/java/com/netflix/conductor/core/orchestration/ExecutionDAOFacade.java @@ -199,8 +199,8 @@ public String createWorkflow(Workflow workflow) { workflow.setCreateTime(System.currentTimeMillis()); executionDAO.createWorkflow(workflow); // Add to decider queue - queueDAO.push(DECIDER_QUEUE, workflow.getWorkflowId(), workflow.getPriority(), properties.getSweepFrequency()); - if (properties.enableAsyncIndexing()) { + queueDAO.push(DECIDER_QUEUE, workflow.getWorkflowId(), workflow.getPriority(), properties.getSweepFrequencySeconds()); + if (properties.isAsyncIndexingEnabled()) { indexDAO.asyncIndexWorkflow(workflow); } else { indexDAO.indexWorkflow(workflow); @@ -220,7 +220,7 @@ public String updateWorkflow(Workflow workflow) { workflow.setEndTime(System.currentTimeMillis()); } executionDAO.updateWorkflow(workflow); - if (properties.enableAsyncIndexing()) { + if (properties.isAsyncIndexingEnabled()) { if (workflow.getStatus().isTerminal() && workflow.getEndTime() - workflow.getStartTime() < properties.getAsyncUpdateShortRunningWorkflowDuration() * 1000) { final String workflowId = workflow.getWorkflowId(); @@ -323,7 +323,7 @@ public void resetWorkflow(String workflowId) { try { getWorkflowById(workflowId, true); executionDAO.removeWorkflow(workflowId); - if (properties.enableAsyncIndexing()) { + if (properties.isAsyncIndexingEnabled()) { indexDAO.asyncRemoveWorkflow(workflowId); } else { indexDAO.removeWorkflow(workflowId); @@ -384,7 +384,7 @@ public void updateTask(Task task) { * of tasks on a system failure. So only index for each update if async indexing is not enabled. * If it *is* enabled, tasks will be indexed only when a workflow is in terminal state. */ - if (!properties.enableAsyncIndexing()) { + if (!properties.isAsyncIndexingEnabled()) { indexDAO.indexTask(task); } } catch (Exception e) { @@ -454,7 +454,7 @@ public void updateEventExecution(EventExecution eventExecution) { private void indexEventExecution(EventExecution eventExecution) { if (properties.isEventExecutionIndexingEnabled()) { - if (properties.enableAsyncIndexing()) { + if (properties.isAsyncIndexingEnabled()) { indexDAO.asyncAddEventExecution(eventExecution); } else { indexDAO.addEventExecution(eventExecution); @@ -476,7 +476,7 @@ public boolean exceedsRateLimitPerFrequency(Task task, TaskDef taskDef) { public void addTaskExecLog(List logs) { if (properties.isTaskExecLogIndexingEnabled()) { - if (properties.enableAsyncIndexing()) { + if (properties.isAsyncIndexingEnabled()) { indexDAO.asyncAddTaskExecutionLogs(logs); } else { indexDAO.addTaskExecutionLogs(logs); @@ -485,7 +485,7 @@ public void addTaskExecLog(List logs) { } public void addMessage(String queue, Message message) { - if (properties.enableAsyncIndexing()) { + if (properties.isAsyncIndexingEnabled()) { indexDAO.asyncAddMessage(queue, message); } else { indexDAO.addMessage(queue, message); diff --git a/core/src/main/java/com/netflix/conductor/metrics/WorkflowMonitor.java b/core/src/main/java/com/netflix/conductor/metrics/WorkflowMonitor.java index a35f518361..4d85008498 100644 --- a/core/src/main/java/com/netflix/conductor/metrics/WorkflowMonitor.java +++ b/core/src/main/java/com/netflix/conductor/metrics/WorkflowMonitor.java @@ -45,13 +45,13 @@ public class WorkflowMonitor { private int refreshCounter = 0; public WorkflowMonitor(MetadataDAO metadataDAO, QueueDAO queueDAO, ExecutionDAOFacade executionDAOFacade, - @Value("${workflow.monitor.metadata.refresh.counter:10}") int metadataRefreshInterval, - @Value("${workflow.monitor.stats.freq.seconds:60}") int statsFrequencyInSeconds) { + @Value("${conductor.workflow-monitor.metadataRefreshInterval:10}") int metadataRefreshInterval, + @Value("${conductor.workflow-monitor.statsFrequencySeconds:60}") int statsFrequencySeconds) { this.metadataDAO = metadataDAO; this.queueDAO = queueDAO; this.executionDAOFacade = executionDAOFacade; this.metadataRefreshInterval = metadataRefreshInterval; - this.statsFrequencyInSeconds = statsFrequencyInSeconds; + this.statsFrequencyInSeconds = statsFrequencySeconds; init(); } diff --git a/core/src/main/java/com/netflix/conductor/service/AdminServiceImpl.java b/core/src/main/java/com/netflix/conductor/service/AdminServiceImpl.java index 8ad9ca82d0..6c44d5e291 100644 --- a/core/src/main/java/com/netflix/conductor/service/AdminServiceImpl.java +++ b/core/src/main/java/com/netflix/conductor/service/AdminServiceImpl.java @@ -23,10 +23,8 @@ import org.springframework.stereotype.Service; import javax.validation.constraints.NotEmpty; -import java.io.InputStream; import java.util.List; import java.util.Map; -import java.util.Properties; //@Audit //@Trace @@ -111,7 +109,7 @@ public boolean verifyAndRepairWorkflowConsistency( */ public String requeueSweep(String workflowId) { boolean pushed = queueDAO - .pushIfNotExists(WorkflowExecutor.DECIDER_QUEUE, workflowId, properties.getSweepFrequency()); + .pushIfNotExists(WorkflowExecutor.DECIDER_QUEUE, workflowId, properties.getSweepFrequencySeconds()); return pushed + "." + workflowId; } } diff --git a/core/src/main/java/com/netflix/conductor/service/ExecutionService.java b/core/src/main/java/com/netflix/conductor/service/ExecutionService.java index 1d4c69ab64..dbb4f1fd3e 100644 --- a/core/src/main/java/com/netflix/conductor/service/ExecutionService.java +++ b/core/src/main/java/com/netflix/conductor/service/ExecutionService.java @@ -66,7 +66,6 @@ public class ExecutionService { private final QueueDAO queueDAO; private final ExternalPayloadStorage externalPayloadStorage; - private final int taskRequeueTimeout; private final int queueTaskMessagePostponeSeconds; private static final int MAX_POLL_TIMEOUT_MS = 5000; @@ -82,8 +81,7 @@ public ExecutionService(WorkflowExecutor workflowExecutor, ExecutionDAOFacade ex this.queueDAO = queueDAO; this.externalPayloadStorage = externalPayloadStorage; - this.taskRequeueTimeout = properties.getTaskRequeueTimeout(); - this.queueTaskMessagePostponeSeconds = properties.getQueueTaskMessagePostponeSeconds(); + this.queueTaskMessagePostponeSeconds = properties.getTaskExecutionPostponeSeconds(); } public Task poll(String taskType, String workerId) { @@ -263,49 +261,6 @@ public void removeTaskfromQueue(String taskId) { queueDAO.remove(QueueUtils.getQueueName(task), taskId); } - public int requeuePendingTasks() { - long threshold = System.currentTimeMillis() - taskRequeueTimeout; - List workflowDefs = metadataDAO.getAllWorkflowDefs(); - int count = 0; - for (WorkflowDef workflowDef : workflowDefs) { - List workflows = workflowExecutor - .getRunningWorkflows(workflowDef.getName(), workflowDef.getVersion()); - for (Workflow workflow : workflows) { - count += requeuePendingTasks(workflow, threshold); - } - } - return count; - } - - private int requeuePendingTasks(Workflow workflow, long threshold) { - - int count = 0; - List tasks = workflow.getTasks(); - for (Task pending : tasks) { - if (SystemTaskType.is(pending.getTaskType())) { - continue; - } - if (pending.getStatus().isTerminal()) { - continue; - } - if (pending.getUpdateTime() < threshold) { - LOGGER.debug("Requeuing Task: {} of taskType: {} in Workflow: {}", pending.getTaskId(), - pending.getTaskType(), workflow.getWorkflowId()); - long callback = pending.getCallbackAfterSeconds(); - if (callback < 0) { - callback = 0; - } - boolean pushed = queueDAO - .pushIfNotExists(QueueUtils.getQueueName(pending), pending.getTaskId(), workflow.getPriority(), - callback); - if (pushed) { - count++; - } - } - } - return count; - } - public int requeuePendingTasks(String taskType) { int count = 0; diff --git a/core/src/main/java/com/netflix/conductor/service/WorkflowServiceImpl.java b/core/src/main/java/com/netflix/conductor/service/WorkflowServiceImpl.java index 1883e56fa8..3faa309c53 100644 --- a/core/src/main/java/com/netflix/conductor/service/WorkflowServiceImpl.java +++ b/core/src/main/java/com/netflix/conductor/service/WorkflowServiceImpl.java @@ -12,6 +12,8 @@ */ package com.netflix.conductor.service; +import com.netflix.conductor.annotations.Audit; +import com.netflix.conductor.annotations.Trace; import com.netflix.conductor.common.metadata.workflow.RerunWorkflowRequest; import com.netflix.conductor.common.metadata.workflow.SkipTaskRequest; import com.netflix.conductor.common.metadata.workflow.StartWorkflowRequest; @@ -24,19 +26,17 @@ import com.netflix.conductor.core.exception.ApplicationException; import com.netflix.conductor.core.execution.WorkflowExecutor; import com.netflix.conductor.core.utils.Utils; -import org.apache.commons.lang3.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.stereotype.Service; - import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.stereotype.Service; -//@Audit -//@Trace +@Audit +@Trace @Service public class WorkflowServiceImpl implements WorkflowService { @@ -44,14 +44,12 @@ public class WorkflowServiceImpl implements WorkflowService { private final WorkflowExecutor workflowExecutor; private final ExecutionService executionService; private final MetadataService metadataService; - private final int maxSearchSize; public WorkflowServiceImpl(WorkflowExecutor workflowExecutor, ExecutionService executionService, - MetadataService metadataService, @Value("${workflow.max.search.size:5000}") int maxSearchSize) { + MetadataService metadataService) { this.workflowExecutor = workflowExecutor; this.executionService = executionService; this.metadataService = metadataService; - this.maxSearchSize = maxSearchSize; } /** diff --git a/core/src/main/resources/META-INF/additional-spring-configuration-metadata.json b/core/src/main/resources/META-INF/additional-spring-configuration-metadata.json new file mode 100644 index 0000000000..a63f63d759 --- /dev/null +++ b/core/src/main/resources/META-INF/additional-spring-configuration-metadata.json @@ -0,0 +1,29 @@ +{ + "properties": [ + { + "name": "conductor.app.isolatedSystemTaskEnabled", + "type": "java.lang.Boolean", + "description": "Used to enable/disable use of isolation groups for system task workers." + }, + { + "name": "conductor.app.isolatedSystemTaskPollIntervalSecs", + "type": "java.lang.Integer", + "description": "The time interval (in seconds) at which new isolated task queues will be polled and added to the system task queue repository." + }, + { + "name": "conductor.app.taskPendingTimeThresholdMins", + "type": "java.lang.Long", + "description": "The time threshold (in minutes) beyond which a warning log will be emitted for a task if it stays in the same state for this duration." + }, + { + "name": "conductor.workflow-monitor.metadataRefreshInterval", + "type": "java.lang.Integer", + "description": "The interval (counter) after which the workflow monitor refreshes the metadata definitions from the datastore for reporting." + }, + { + "name": "conductor.workflow-monitor.statsFrequencySeconds", + "type": "java.lang.Integer", + "description": "The time (in seconds) at which the workflow monitor publishes metrics about the system." + } + ] +} \ No newline at end of file diff --git a/core/src/main/resources/META-INF/validation.xml b/core/src/main/resources/META-INF/validation.xml index a9ae5808fa..4c8ec2ce93 100644 --- a/core/src/main/resources/META-INF/validation.xml +++ b/core/src/main/resources/META-INF/validation.xml @@ -1,3 +1,20 @@ + + + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package com.netflix.conductor.core.execution.mapper; import com.netflix.conductor.common.metadata.tasks.Task; diff --git a/core/src/test/java/com/netflix/conductor/core/execution/tasks/TestSystemTaskExecutor.java b/core/src/test/java/com/netflix/conductor/core/execution/tasks/TestSystemTaskExecutor.java index 8c89571e5b..4e529aa134 100644 --- a/core/src/test/java/com/netflix/conductor/core/execution/tasks/TestSystemTaskExecutor.java +++ b/core/src/test/java/com/netflix/conductor/core/execution/tasks/TestSystemTaskExecutor.java @@ -63,7 +63,7 @@ public void setUp() { when(properties.getSystemTaskWorkerThreadCount()).thenReturn(10); when(properties.getSystemTaskWorkerCallbackSeconds()).thenReturn(30); when(properties.getSystemTaskMaxPollCount()).thenReturn(1); - when(properties.getSystemTaskWorkerIsolatedThreadCount()).thenReturn(1); + when(properties.getIsolatedSystemTaskWorkerThreadCount()).thenReturn(1); } @After @@ -83,7 +83,7 @@ public void testGetExecutionConfigForSystemTask() { @Test public void testGetExecutionConfigForIsolatedSystemTask() { - when(properties.getSystemTaskWorkerIsolatedThreadCount()).thenReturn(7); + when(properties.getIsolatedSystemTaskWorkerThreadCount()).thenReturn(7); systemTaskExecutor = new SystemTaskExecutor(queueDAO, workflowExecutor, properties, executionService); assertEquals(systemTaskExecutor.getExecutionConfig("test-iso").getSemaphoreUtil().availableSlots(), 7); } @@ -208,7 +208,7 @@ public void testBatchPollException() { @Test public void testMultipleQueuesExecution() { when(properties.getSystemTaskWorkerThreadCount()).thenReturn(1); - when(properties.getSystemTaskWorkerIsolatedThreadCount()).thenReturn(1); + when(properties.getIsolatedSystemTaskWorkerThreadCount()).thenReturn(1); String sysTask = "taskId"; String isolatedTask = "isolatedTaskId"; when(queueDAO.pop(TEST_TASK, 1, 200)).thenReturn(Collections.singletonList(sysTask)); diff --git a/core/src/test/java/com/netflix/conductor/core/orchestration/ExecutionDAOFacadeTest.java b/core/src/test/java/com/netflix/conductor/core/orchestration/ExecutionDAOFacadeTest.java index d10af92b4e..eb38cadce4 100644 --- a/core/src/test/java/com/netflix/conductor/core/orchestration/ExecutionDAOFacadeTest.java +++ b/core/src/test/java/com/netflix/conductor/core/orchestration/ExecutionDAOFacadeTest.java @@ -72,7 +72,7 @@ public void setUp() { PollDataDAO pollDataDAO = mock(PollDataDAO.class); ConductorProperties properties = mock(ConductorProperties.class); when(properties.isEventExecutionIndexingEnabled()).thenReturn(true); - when(properties.enableAsyncIndexing()).thenReturn(true); + when(properties.isAsyncIndexingEnabled()).thenReturn(true); executionDAOFacade = new ExecutionDAOFacade(executionDAO, queueDAO, indexDAO, rateLimitingDao, pollDataDAO, objectMapper, properties); } diff --git a/core/src/test/java/com/netflix/conductor/service/WorkflowServiceTest.java b/core/src/test/java/com/netflix/conductor/service/WorkflowServiceTest.java index 52fc812c32..e136d86b7f 100644 --- a/core/src/test/java/com/netflix/conductor/service/WorkflowServiceTest.java +++ b/core/src/test/java/com/netflix/conductor/service/WorkflowServiceTest.java @@ -12,30 +12,6 @@ */ package com.netflix.conductor.service; -import com.netflix.conductor.common.metadata.workflow.RerunWorkflowRequest; -import com.netflix.conductor.common.metadata.workflow.SkipTaskRequest; -import com.netflix.conductor.common.metadata.workflow.StartWorkflowRequest; -import com.netflix.conductor.common.metadata.workflow.WorkflowDef; -import com.netflix.conductor.common.run.SearchResult; -import com.netflix.conductor.common.run.Workflow; -import com.netflix.conductor.common.run.WorkflowSummary; -import com.netflix.conductor.core.exception.ApplicationException; -import com.netflix.conductor.core.execution.WorkflowExecutor; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.boot.autoconfigure.EnableAutoConfiguration; -import org.springframework.boot.test.context.TestConfiguration; -import org.springframework.context.annotation.Bean; -import org.springframework.test.context.junit4.SpringRunner; - -import javax.validation.ConstraintViolationException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - import static com.netflix.conductor.TestUtils.getConstraintViolationMessages; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -53,6 +29,29 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import com.netflix.conductor.common.metadata.workflow.RerunWorkflowRequest; +import com.netflix.conductor.common.metadata.workflow.SkipTaskRequest; +import com.netflix.conductor.common.metadata.workflow.StartWorkflowRequest; +import com.netflix.conductor.common.metadata.workflow.WorkflowDef; +import com.netflix.conductor.common.run.SearchResult; +import com.netflix.conductor.common.run.Workflow; +import com.netflix.conductor.common.run.WorkflowSummary; +import com.netflix.conductor.core.exception.ApplicationException; +import com.netflix.conductor.core.execution.WorkflowExecutor; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import javax.validation.ConstraintViolationException; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.boot.autoconfigure.EnableAutoConfiguration; +import org.springframework.boot.test.context.TestConfiguration; +import org.springframework.context.annotation.Bean; +import org.springframework.test.context.junit4.SpringRunner; + @SuppressWarnings("SpringJavaAutowiredMembersInspection") @RunWith(SpringRunner.class) @EnableAutoConfiguration @@ -79,7 +78,7 @@ public MetadataService metadataService() { @Bean public WorkflowService workflowService(WorkflowExecutor workflowExecutor, ExecutionService executionService, MetadataService metadataService) { - return new WorkflowServiceImpl(workflowExecutor, executionService, metadataService, 5000); + return new WorkflowServiceImpl(workflowExecutor, executionService, metadataService); } } diff --git a/docker/server/config/config-local.properties b/docker/server/config/config-local.properties index 31f3aad31d..c91fe4ed6c 100755 --- a/docker/server/config/config-local.properties +++ b/docker/server/config/config-local.properties @@ -1,54 +1,32 @@ # Servers. -conductor.jetty.server.enabled=true -conductor.grpc.server.enabled=false +conductor.grpc-server.enabled=false -# Database persistence model. Possible values are memory, redis, and dynomite. -# If ommitted, the persistence used is memory -# -# memory : The data is stored in memory and lost when the server dies. Useful for testing or demo -# redis : non-Dynomite based redis instance -# dynomite : Dynomite cluster. Use this for HA configuration. - -db=memory +# Database persistence model. +conductor.db.type=memory # Dynomite Cluster details. # format is host:port:rack separated by semicolon -workflow.dynomite.cluster.hosts=dyno1:8102:us-east-1c +conductor.redis.hosts=dyno1:8102:us-east-1c # Namespace for the keys stored in Dynomite/Redis -workflow.namespace.prefix=conductor +conductor.redis.workflowNamespacePrefix=conductor # Namespace prefix for the dyno queues -workflow.namespace.queue.prefix=conductor_queues +conductor.redis.queueNamespacePrefix=conductor_queues # No. of threads allocated to dyno-queues (optional) queues.dynomite.threads=10 # By default with dynomite, we want the repairservice enabled -workflow.repairservice.enabled=true - +conductor.app.workflowRepairServiceEnabled=true # Non-quorum port used to connect to local redis. Used by dyno-queues. # When using redis directly, set this to the same port as redis server # For Dynomite, this is 22122 by default or the local redis-server port used by Dynomite. -queues.dynomite.nonQuorum.port=22122 - -# Elastic search instance type. Possible values are memory and external. -# If not specified, the instance type will be embedded in memory -# -# memory: The instance is created in memory and lost when the server dies. Useful for development and testing. -# external: Elastic search instance runs outside of the server. Data is persisted and does not get lost when -# the server dies. Useful for more stable environments like staging or production. -workflow.elasticsearch.instanceType=memory - -# Transport address to elasticsearch -workflow.elasticsearch.url=localhost:9300 - -# Name of the elasticsearch cluster -workflow.elasticsearch.index.name=conductor +conductor.redis.queuesNonQuorumPort=22122 -# Additional modules (optional) -# conductor.additional.modules=class_extending_com.google.inject.AbstractModule +# Elastic search instance indexing is disabled. +conductor.indexing.enabled=false # Load sample kitchen sink workflow loadSample=true diff --git a/docker/server/config/config-mysql-grpc.properties b/docker/server/config/config-mysql-grpc.properties index 7275869223..b063299485 100755 --- a/docker/server/config/config-mysql-grpc.properties +++ b/docker/server/config/config-mysql-grpc.properties @@ -1,35 +1,17 @@ # Servers. -conductor.jetty.server.enabled=true -conductor.grpc.server.enabled=true +conductor.grpc-server.enabled=true -# Database persistence model. Possible values are memory, redis, and dynomite. -# If ommitted, the persistence used is memory -# -# memory : The data is stored in memory and lost when the server dies. Useful for testing or demo -# redis : non-Dynomite based redis instance -# dynomite : Dynomite cluster. Use this for HA configuration. +# Database persistence model. +conductor.db.type=mysql -db=mysql - -jdbc.url=jdbc:mysql://mysql:3306/conductor +conductor.mysql.jdbcUrl=jdbc:mysql://mysql:3306/conductor # Hikari pool sizes are -1 by default and prevent startup -conductor.mysql.connection.pool.size.max=10 -conductor.mysql.connection.pool.idle.min=2 - -# Elastic search instance type. Possible values are memory and external. -# If not specified, the instance type will be embedded in memory -# -# memory: The instance is created in memory and lost when the server dies. Useful for development and testing. -# external: Elastic search instance runs outside of the server. Data is persisted and does not get lost when -# the server dies. Useful for more stable environments like staging or production. -workflow.elasticsearch.instanceType=external +conductor.mysql.connectionPoolMaxSize=10 +conductor.mysql.connectionPoolMinIdle=2 # Transport address to elasticsearch -workflow.elasticsearch.url=elasticsearch:9300 +conductor.elasticsearch.url=elasticsearch:9300 # Name of the elasticsearch cluster -workflow.elasticsearch.index.name=conductor - -# Additional modules (optional) -# conductor.additional.modules=class_extending_com.google.inject.AbstractModule +conductor.elasticsearch.indexName=conductor diff --git a/docker/server/config/config-mysql.properties b/docker/server/config/config-mysql.properties index 8643d1cb85..8525a72858 100755 --- a/docker/server/config/config-mysql.properties +++ b/docker/server/config/config-mysql.properties @@ -1,38 +1,20 @@ # Servers. -conductor.jetty.server.enabled=true -conductor.grpc.server.enabled=false +conductor.grpc-server.enabled=false -# Database persistence model. Possible values are memory, redis, and dynomite. -# If ommitted, the persistence used is memory -# -# memory : The data is stored in memory and lost when the server dies. Useful for testing or demo -# redis : non-Dynomite based redis instance -# dynomite : Dynomite cluster. Use this for HA configuration. +# Database persistence type. +conductor.db.type=mysql -db=mysql - -jdbc.url=jdbc:mysql://mysql:3306/conductor +conductor.mysql.jdbcUrl=jdbc:mysql://mysql:3306/conductor # Hikari pool sizes are -1 by default and prevent startup -conductor.mysql.connection.pool.size.max=10 -conductor.mysql.connection.pool.idle.min=2 - -# Elastic search instance type. Possible values are memory and external. -# If not specified, the instance type will be embedded in memory -# -# memory: The instance is created in memory and lost when the server dies. Useful for development and testing. -# external: Elastic search instance runs outside of the server. Data is persisted and does not get lost when -# the server dies. Useful for more stable environments like staging or production. -workflow.elasticsearch.instanceType=external +conductor.mysql.connectionPoolMaxSize=10 +conductor.mysql.connectionPoolMinIdle=2 # Transport address to elasticsearch -workflow.elasticsearch.url=elasticsearch:9300 +conductor.elasticsearch.url=elasticsearch:9300 # Name of the elasticsearch cluster -workflow.elasticsearch.index.name=conductor - -# Additional modules (optional) -# conductor.additional.modules=class_extending_com.google.inject.AbstractModule +conductor.elasticsearch.indexName=conductor # Load sample kitchen sink workflow loadSample=true diff --git a/docker/server/config/config.properties b/docker/server/config/config.properties index 901afa413e..4637ee9c5f 100755 --- a/docker/server/config/config.properties +++ b/docker/server/config/config.properties @@ -1,60 +1,42 @@ # Servers. -conductor.jetty.server.enabled=true -conductor.grpc.server.enabled=false +conductor.grpc-server.enabled=false -# Database persistence model. Possible values are memory, redis, and dynomite. -# If ommitted, the persistence used is memory -# -# memory : The data is stored in memory and lost when the server dies. Useful for testing or demo -# redis : non-Dynomite based redis instance -# dynomite : Dynomite cluster. Use this for HA configuration. - -db=dynomite +# Database persistence type. +conductor.db.type=dynomite # Dynomite Cluster details. # format is host:port:rack separated by semicolon -workflow.dynomite.cluster.hosts=dyno1:8102:us-east-1c +conductor.redis.hosts=dyno1:8102:us-east-1c # Dynomite cluster name -workflow.dynomite.cluster.name=dyno1 +conductor.redis.clusterName=dyno1 # Namespace for the keys stored in Dynomite/Redis -workflow.namespace.prefix=conductor +conductor.redis.workflowNamespacePrefix=conductor # Namespace prefix for the dyno queues -workflow.namespace.queue.prefix=conductor_queues +conductor.redis.queueNamespacePrefix=conductor_queues # No. of threads allocated to dyno-queues (optional) queues.dynomite.threads=10 # By default with dynomite, we want the repairservice enabled -workflow.repairservice.enabled=true +conductor.app.workflowRepairServiceEnabled=true # Non-quorum port used to connect to local redis. Used by dyno-queues. # When using redis directly, set this to the same port as redis server # For Dynomite, this is 22122 by default or the local redis-server port used by Dynomite. -queues.dynomite.nonQuorum.port=22122 - -# Elastic search instance type. Possible values are memory and external. -# If not specified, the instance type will be embedded in memory -# -# memory: The instance is created in memory and lost when the server dies. Useful for development and testing. -# external: Elastic search instance runs outside of the server. Data is persisted and does not get lost when -# the server dies. Useful for more stable environments like staging or production. -workflow.elasticsearch.instanceType=external +conductor.redis.queuesNonQuorumPort=22122 # Transport address to elasticsearch -workflow.elasticsearch.url=es:9300 +conductor.elasticsearch.url=es:9300 # Name of the elasticsearch cluster -workflow.elasticsearch.index.name=conductor - -# Additional modules (optional) -# conductor.additional.modules=class_extending_com.google.inject.AbstractModule +conductor.elasticsearch.indexName=conductor # Additional modules for metrics collection (optional) -# conductor.additional.modules=com.netflix.conductor.contribs.metrics.MetricsRegistryModule,com.netflix.conductor.contribs.metrics.LoggingMetricsModule -# com.netflix.conductor.contribs.metrics.LoggingMetricsModule.reportPeriodSeconds=15 +# conductor.metrics-logger.enabled=true +# conductor.metrics-logger.reportPeriodSeconds=15 # Load sample kitchen sink workflow loadSample=true diff --git a/docker/serverAndUI/config/config-local.properties b/docker/serverAndUI/config/config-local.properties index 92b4c37e7b..d725130e89 100755 --- a/docker/serverAndUI/config/config-local.properties +++ b/docker/serverAndUI/config/config-local.properties @@ -1,43 +1,33 @@ -# Database persistence model. Possible values are memory, redis, and dynomite. -# If ommitted, the persistence used is memory -# -# memory : The data is stored in memory and lost when the server dies. Useful for testing or demo -# redis : non-Dynomite based redis instance -# dynomite : Dynomite cluster. Use this for HA configuration. - -db=memory +# Database persistence type. +conductor.db.type=memory # Dynomite Cluster details. # format is host:port:rack separated by semicolon -workflow.dynomite.cluster.hosts=dyno1:8102:us-east-1c +conductor.redis.hosts=dyno1:8102:us-east-1c # Namespace for the keys stored in Dynomite/Redis -workflow.namespace.prefix=conductor +conductor.redis.workflowNamespacePrefix=conductor # Namespace prefix for the dyno queues -workflow.namespace.queue.prefix=conductor_queues +conductor.redis.queueNamespacePrefix=conductor_queues # No. of threads allocated to dyno-queues (optional) queues.dynomite.threads=10 # By default with dynomite, we want the repairservice enabled -workflow.repairservice.enabled=true +conductor.app.workflowRepairServiceEnabled=true # Non-quorum port used to connect to local redis. Used by dyno-queues. # When using redis directly, set this to the same port as redis server # For Dynomite, this is 22122 by default or the local redis-server port used by Dynomite. -queues.dynomite.nonQuorum.port=22122 - +conductor.redis.queuesNonQuorumPort=22122 # Transport address to elasticsearch -workflow.elasticsearch.url=localhost:9300 +conductor.elasticsearch.url=localhost:9300 # Name of the elasticsearch cluster -workflow.elasticsearch.index.name=conductor - -# Additional modules (optional) -# conductor.additional.modules=class_extending_com.google.inject.AbstractModule +conductor.elasticsearch.indexName=conductor # Load sample kitchen sink workflow loadSample=true diff --git a/docker/serverAndUI/config/config.properties b/docker/serverAndUI/config/config.properties index 73a872bab2..c596c6f10f 100755 --- a/docker/serverAndUI/config/config.properties +++ b/docker/serverAndUI/config/config.properties @@ -1,46 +1,35 @@ -# Database persistence model. Possible values are memory, redis, and dynomite. -# If ommitted, the persistence used is memory -# -# memory : The data is stored in memory and lost when the server dies. Useful for testing or demo -# redis : non-Dynomite based redis instance -# dynomite : Dynomite cluster. Use this for HA configuration. - -db=dynomite +# Database persistence model. +conductor.db.type=dynomite # Dynomite Cluster details. # format is host:port:rack separated by semicolon -workflow.dynomite.cluster.hosts=dyno1:8102:us-east-1c +conductor.redis.hosts=dyno1:8102:us-east-1c # Dynomite cluster name -workflow.dynomite.cluster.name=dyno1 +conductor.redis.clusterName=dyno1 # Namespace for the keys stored in Dynomite/Redis -workflow.namespace.prefix=conductor +conductor.redis.workflowNamespacePrefix=conductor # Namespace prefix for the dyno queues -workflow.namespace.queue.prefix=conductor_queues +conductor.redis.queueNamespacePrefix=conductor_queues # No. of threads allocated to dyno-queues (optional) queues.dynomite.threads=10 # By default with dynomite, we want the repairservice enabled -workflow.repairservice.enabled=true - +conductor.app.workflowRepairServiceEnabled=true # Non-quorum port used to connect to local redis. Used by dyno-queues. # When using redis directly, set this to the same port as redis server # For Dynomite, this is 22122 by default or the local redis-server port used by Dynomite. -queues.dynomite.nonQuorum.port=22122 - +conductor.redis.queuesNonQuorumPort=22122 # Transport address to elasticsearch -workflow.elasticsearch.url=es:9300 +conductor.elasticsearch.url=es:9300 # Name of the elasticsearch cluster -workflow.elasticsearch.index.name=conductor - -# Additional modules (optional) -# conductor.additional.modules=class_extending_com.google.inject.AbstractModule +conductor.elasticsearch.indexName=conductor # Load sample kitchen sink workflow -loadSample=true \ No newline at end of file +loadSample=true diff --git a/docs/docs/extend.md b/docs/docs/extend.md index bd55fdd939..d4b7cd313a 100644 --- a/docs/docs/extend.md +++ b/docs/docs/extend.md @@ -56,4 +56,4 @@ for eg., with Dynomite and Redlock: * Implement ```Lock``` interface. * Add a binding similar to [this](https://github.com/Netflix/conductor/blob/master/server/src/main/java/com/netflix/conductor/bootstrap/ModulesProvider.java#L115-L129) -* Enable locking service: ```decider.locking.enabled: true``` +* Enable locking service: ```conductor.app.workflowExecutionLockEnabled: true``` diff --git a/docs/docs/server.md b/docs/docs/server.md index c5221194e8..fbea66956d 100644 --- a/docs/docs/server.md +++ b/docs/docs/server.md @@ -60,7 +60,7 @@ log4j.properties file path is optional and allows finer control over the logging # redis_cluster: AWS Elasticache Redis (cluster mode enabled).See [http://docs.aws.amazon.com/AmazonElastiCache/latest/UserGuide/Clusters.Create.CON.RedisCluster.html] # redis_sentinel: Redis HA with Redis Sentinel. See [https://redis.io/topics/sentinel] # dynomite : Dynomite cluster. Use this for HA configuration. -db=dynomite +conductor.db.type=dynomite # Dynomite Cluster details. # format is host:port:rack separated by semicolon @@ -114,7 +114,7 @@ Clients connects to the server via HTTP load balancer or using Discovery (on Net Conductor server can be used with a standlone Redis or ElastiCache server. To configure the server, change the config to use the following: ```properties -db=redis +conductor.db.type=redis # For AWS Elasticache Redis (cluster mode enabled) the format is configuration_endpoint:port:us-east-1e. # The region in this case does not matter @@ -130,7 +130,7 @@ See [Technical Details](../technicaldetails/#maintaining-workflow-consistency-wi Locking Service is disabled by default. Enable this by setting: -```decider.locking.enabled: true``` +```conductor.app.workflowExecutionLockEnabled: true``` Setup Zookeeper cluster connection string: diff --git a/es6-persistence/src/main/java/com/netflix/conductor/es6/config/ElasticSearchConditions.java b/es6-persistence/src/main/java/com/netflix/conductor/es6/config/ElasticSearchConditions.java index 100e7c51fb..7385379d06 100644 --- a/es6-persistence/src/main/java/com/netflix/conductor/es6/config/ElasticSearchConditions.java +++ b/es6-persistence/src/main/java/com/netflix/conductor/es6/config/ElasticSearchConditions.java @@ -27,13 +27,13 @@ public static class ElasticSearchV6Enabled extends AllNestedConditions { } @SuppressWarnings("unused") - @ConditionalOnProperty(prefix = "workflow", name = "indexing.enabled", havingValue = "true", matchIfMissing = true) + @ConditionalOnProperty(name = "conductor.indexing.enabled", havingValue = "true", matchIfMissing = true) static class enabledIndexing { } @SuppressWarnings("unused") - @ConditionalOnProperty(prefix = "workflow", name = "elasticsearch.version", havingValue = "6", matchIfMissing = true) + @ConditionalOnProperty(name = "conductor.elasticsearch.version", havingValue = "6", matchIfMissing = true) static class enabledES6 { } diff --git a/es6-persistence/src/main/java/com/netflix/conductor/es6/config/ElasticSearchProperties.java b/es6-persistence/src/main/java/com/netflix/conductor/es6/config/ElasticSearchProperties.java index 7b10b43e52..6638f51a0f 100644 --- a/es6-persistence/src/main/java/com/netflix/conductor/es6/config/ElasticSearchProperties.java +++ b/es6-persistence/src/main/java/com/netflix/conductor/es6/config/ElasticSearchProperties.java @@ -12,141 +12,192 @@ */ package com.netflix.conductor.es6.config; -import com.google.common.annotations.VisibleForTesting; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.context.annotation.Conditional; -import org.springframework.stereotype.Component; - import java.net.MalformedURLException; import java.net.URL; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; +import org.springframework.boot.context.properties.ConfigurationProperties; -@Component -@Conditional(ElasticSearchConditions.ElasticSearchV6Enabled.class) +@ConfigurationProperties("conductor.elasticsearch") public class ElasticSearchProperties { - @Value("${workflow.elasticsearch.url:localhost:9300}") - private String url; - - @Value("${workflow.elasticsearch.index.name:conductor}") - private String indexName; - - @Value("${workflow.elasticsearch.tasklog.index.name:task_log}") - private String taskLogIndexName; + /** + * The url for the elasticsearch cluster. Format -- host:port + */ + private String url = "localhost:9300"; - @Value("${workflow.elasticsearch.cluster.health.color:green}") - private String clusterHealthColor; + /** + * The index prefix to be used when creating indices + */ + private String indexPrefix = "conductor"; - @Value("${workflow.elasticsearch.archive.search.batchSize:5000}") - private int archiveSearchBatchSize; + /** + * The color of the elasticserach cluster to wait for to confirm healthy status + */ + private String clusterHealthColor = "green"; - @Value("${workflow.elasticsearch.index.batchSize:1}") - private int indexBatchSize; + /** + * The size of the batch to be used for bulk indexing in async mode + */ + private int indexBatchSize = 1; - @Value("${workflow.elasticsearch.async.dao.worker.queue.size:100}") - private int asyncWorkerQueueSize; + /** + * The size of the queue used for holding async indexing tasks + */ + private int asyncWorkerQueueSize = 100; - @Value("${workflow.elasticsearch.async.dao.max.pool.size:12}") - private int asyncMaxPoolSize; + /** + * The maximum number of threads allowed in the async pool + */ + private int asyncMaxPoolSize = 12; - @Value("${workflow.elasticsearch.async.buffer.flush.timeout.seconds:10}") - private int asyncBufferFlushTimeout; + /** + * The time in seconds after which the async buffers will be flushed (if no activity) to prevent data loss + */ + private int asyncBufferFlushTimeoutSecs = 10; - @Value("${workflow.elasticsearch.index.shard.count:5}") - private int indexShardCount; + /** + * The number of shards that the index will be created with + */ + private int indexShardCount = 5; - @Value("${workflow.elasticsearch.index.replicas.count:1}") - private int indexReplicasCount; + /** + * The number of replicas that the index will be configured to have + */ + private int indexReplicasCount = 1; - @Value("${tasklog.elasticsearch.query.size:10}") - private int taskLogResultLimit; + /** + * The number of task log results that will be returned in the response + */ + private int taskLogResultLimit = 10; - @Value("${workflow.elasticsearch.rest.client.connectionRequestTimeout.milliseconds:-1}") - private int restClientConnectionRequestTimeout; + /** + * The timeout in milliseconds used when requesting a connection from the connection manager + */ + private int restClientConnectionRequestTimeoutMs = -1; - @Value("${workflow.elasticsearch.auto.index.management.enabled:true}") - private boolean elasticSearchAutoIndexManagementEnabled; + /** + * Used to control if index management is to be enabled or will be controlled externally + */ + private boolean autoIndexManagementEnabled = true; /** * Document types are deprecated in ES6 and removed from ES7. This property can be used to disable the use of * specific document types with an override. This property is currently used in ES6 module. *

* Note that this property will only take effect if - * {@link ElasticSearchProperties#isElasticSearchAutoIndexManagementEnabled} is set to false and index management is - * handled outside of this module. + * {@link ElasticSearchProperties#isAutoIndexManagementEnabled} is set to false and index management is handled + * outside of this module. */ - @Value("${workflow.elasticsearch.document.type.override:}") - private String elasticSearchDocumentTypeOverride; + private String documentTypeOverride = ""; - public String getURL() { + public String getUrl() { return url; } - public String getIndexName() { - return indexName; + public void setUrl(String url) { + this.url = url; + } + + public String getIndexPrefix() { + return indexPrefix; } - public String getTaskLogIndexName() { - return taskLogIndexName; + public void setIndexPrefix(String indexPrefix) { + this.indexPrefix = indexPrefix; } public String getClusterHealthColor() { return clusterHealthColor; } - public int getArchiveSearchBatchSize() { - return archiveSearchBatchSize; + public void setClusterHealthColor(String clusterHealthColor) { + this.clusterHealthColor = clusterHealthColor; } public int getIndexBatchSize() { return indexBatchSize; } + public void setIndexBatchSize(int indexBatchSize) { + this.indexBatchSize = indexBatchSize; + } + public int getAsyncWorkerQueueSize() { return asyncWorkerQueueSize; } + public void setAsyncWorkerQueueSize(int asyncWorkerQueueSize) { + this.asyncWorkerQueueSize = asyncWorkerQueueSize; + } + public int getAsyncMaxPoolSize() { return asyncMaxPoolSize; } - public int getAsyncBufferFlushTimeout() { - return asyncBufferFlushTimeout; + public void setAsyncMaxPoolSize(int asyncMaxPoolSize) { + this.asyncMaxPoolSize = asyncMaxPoolSize; + } + + public int getAsyncBufferFlushTimeoutSecs() { + return asyncBufferFlushTimeoutSecs; + } + + public void setAsyncBufferFlushTimeoutSecs(int asyncBufferFlushTimeoutSecs) { + this.asyncBufferFlushTimeoutSecs = asyncBufferFlushTimeoutSecs; } - public int getElasticSearchIndexShardCount() { + public int getIndexShardCount() { return indexShardCount; } - public int getElasticSearchIndexReplicasCount() { + public void setIndexShardCount(int indexShardCount) { + this.indexShardCount = indexShardCount; + } + + public int getIndexReplicasCount() { return indexReplicasCount; } - public int getElasticSearchTasklogResultLimit() { + public void setIndexReplicasCount(int indexReplicasCount) { + this.indexReplicasCount = indexReplicasCount; + } + + public int getTaskLogResultLimit() { return taskLogResultLimit; } - public int getElasticsearchRestClientConnectionRequestTimeout() { - return restClientConnectionRequestTimeout; + public void setTaskLogResultLimit(int taskLogResultLimit) { + this.taskLogResultLimit = taskLogResultLimit; } - public boolean isElasticSearchAutoIndexManagementEnabled() { - return elasticSearchAutoIndexManagementEnabled; + public int getRestClientConnectionRequestTimeoutMs() { + return restClientConnectionRequestTimeoutMs; } - public String getElasticSearchDocumentTypeOverride() { - return elasticSearchDocumentTypeOverride; + public void setRestClientConnectionRequestTimeoutMs(int restClientConnectionRequestTimeoutMs) { + this.restClientConnectionRequestTimeoutMs = restClientConnectionRequestTimeoutMs; } - @VisibleForTesting - public void setURL(String url) { - this.url = url; + public boolean isAutoIndexManagementEnabled() { + return autoIndexManagementEnabled; + } + + public void setAutoIndexManagementEnabled(boolean autoIndexManagementEnabled) { + this.autoIndexManagementEnabled = autoIndexManagementEnabled; + } + + public String getDocumentTypeOverride() { + return documentTypeOverride; + } + + public void setDocumentTypeOverride(String documentTypeOverride) { + this.documentTypeOverride = documentTypeOverride; } public List getURLs() { - String clusterAddress = getURL(); + String clusterAddress = getUrl(); String[] hosts = clusterAddress.split(","); return Arrays.stream(hosts) .map(host -> diff --git a/es6-persistence/src/main/java/com/netflix/conductor/es6/config/ElasticSearchV6Configuration.java b/es6-persistence/src/main/java/com/netflix/conductor/es6/config/ElasticSearchV6Configuration.java index bd6650a603..69f7d668d9 100644 --- a/es6-persistence/src/main/java/com/netflix/conductor/es6/config/ElasticSearchV6Configuration.java +++ b/es6-persistence/src/main/java/com/netflix/conductor/es6/config/ElasticSearchV6Configuration.java @@ -26,6 +26,7 @@ import org.elasticsearch.transport.client.PreBuiltTransportClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Conditional; import org.springframework.context.annotation.Configuration; @@ -36,6 +37,7 @@ import java.util.Optional; @Configuration(proxyBeanMethods = false) +@EnableConfigurationProperties(ElasticSearchProperties.class) @Conditional(ElasticSearchConditions.ElasticSearchV6Enabled.class) public class ElasticSearchV6Configuration { @@ -70,9 +72,9 @@ public Client client(ElasticSearchProperties properties) { @Bean public RestClient restClient(ElasticSearchProperties properties) { RestClientBuilder restClientBuilder = RestClient.builder(convertToHttpHosts(properties.getURLs())); - if (properties.getElasticsearchRestClientConnectionRequestTimeout() > 0) { + if (properties.getRestClientConnectionRequestTimeoutMs() > 0) { restClientBuilder.setRequestConfigCallback(requestConfigBuilder -> requestConfigBuilder - .setConnectionRequestTimeout(properties.getElasticsearchRestClientConnectionRequestTimeout())); + .setConnectionRequestTimeout(properties.getRestClientConnectionRequestTimeoutMs())); } return restClientBuilder.build(); } @@ -85,7 +87,7 @@ public RestClientBuilder restClientBuilder(ElasticSearchProperties properties) { @Bean public IndexDAO es6IndexDAO(RestClientBuilder restClientBuilder, Client client, ElasticSearchProperties properties, ObjectMapper objectMapper) { - String url = properties.getURL(); + String url = properties.getUrl(); if (url.startsWith("http") || url.startsWith("https")) { return new ElasticSearchRestDAOV6(restClientBuilder, properties, objectMapper); } else { diff --git a/es6-persistence/src/main/java/com/netflix/conductor/es6/dao/index/ElasticSearchDAOV6.java b/es6-persistence/src/main/java/com/netflix/conductor/es6/dao/index/ElasticSearchDAOV6.java index acc8119e4a..1249066a1d 100644 --- a/es6-persistence/src/main/java/com/netflix/conductor/es6/dao/index/ElasticSearchDAOV6.java +++ b/es6-persistence/src/main/java/com/netflix/conductor/es6/dao/index/ElasticSearchDAOV6.java @@ -136,7 +136,7 @@ public ElasticSearchDAOV6(Client elasticSearchClient, ElasticSearchProperties pr ObjectMapper objectMapper) { this.objectMapper = objectMapper; this.elasticSearchClient = elasticSearchClient; - this.indexPrefix = properties.getIndexName(); + this.indexPrefix = properties.getIndexPrefix(); this.workflowIndexName = getIndexName(WORKFLOW_DOC_TYPE); this.taskIndexName = getIndexName(TASK_DOC_TYPE); this.logIndexPrefix = this.indexPrefix + "_" + LOG_DOC_TYPE; @@ -146,12 +146,12 @@ public ElasticSearchDAOV6(Client elasticSearchClient, ElasticSearchProperties pr int maximumPoolSize = properties.getAsyncMaxPoolSize(); this.bulkRequests = new ConcurrentHashMap<>(); this.indexBatchSize = properties.getIndexBatchSize(); - this.asyncBufferFlushTimeout = properties.getAsyncBufferFlushTimeout(); + this.asyncBufferFlushTimeout = properties.getAsyncBufferFlushTimeoutSecs(); this.properties = properties; - if (!properties.isElasticSearchAutoIndexManagementEnabled() && - StringUtils.isNotBlank(properties.getElasticSearchDocumentTypeOverride())) { - docTypeOverride = properties.getElasticSearchDocumentTypeOverride(); + if (!properties.isAutoIndexManagementEnabled() && + StringUtils.isNotBlank(properties.getDocumentTypeOverride())) { + docTypeOverride = properties.getDocumentTypeOverride(); } else { docTypeOverride = ""; } @@ -211,7 +211,7 @@ private void shutdownExecutorService(ExecutorService execService) { public void setup() throws Exception { waitForHealthyCluster(); - if (properties.isElasticSearchAutoIndexManagementEnabled()) { + if (properties.isAutoIndexManagementEnabled()) { createIndexesTemplates(); createWorkflowIndex(); createTaskIndex(); @@ -291,8 +291,8 @@ private void createIndex(String indexName) { try { CreateIndexRequest createIndexRequest = new CreateIndexRequest(indexName); createIndexRequest.settings(Settings.builder() - .put("index.number_of_shards", properties.getElasticSearchIndexShardCount()) - .put("index.number_of_replicas", properties.getElasticSearchIndexReplicasCount()) + .put("index.number_of_shards", properties.getIndexShardCount()) + .put("index.number_of_replicas", properties.getIndexReplicasCount()) ); elasticSearchClient.admin() @@ -450,7 +450,7 @@ public List getTaskExecutionLogs(String taskId) { final SearchRequestBuilder srb = elasticSearchClient.prepareSearch(logIndexPrefix + "*") .setQuery(query) .setTypes(docType) - .setSize(properties.getElasticSearchTasklogResultLimit()) + .setSize(properties.getTaskLogResultLimit()) .addSort(SortBuilders.fieldSort("createdTime").order(SortOrder.ASC)); return mapTaskExecLogsResponse(srb.execute().actionGet()); @@ -791,7 +791,7 @@ private List extractSearchIds(SearchRequestBuilder s) { /** * Flush the buffers if bulk requests have not been indexed for the past {@link - * ElasticSearchProperties#getAsyncBufferFlushTimeout()} seconds. This is to prevent data loss in case the instance + * ElasticSearchProperties#getAsyncBufferFlushTimeoutSecs()} seconds. This is to prevent data loss in case the instance * is terminated, while the buffer still holds documents to be indexed. */ private void flushBulkRequests() { diff --git a/es6-persistence/src/main/java/com/netflix/conductor/es6/dao/index/ElasticSearchRestDAOV6.java b/es6-persistence/src/main/java/com/netflix/conductor/es6/dao/index/ElasticSearchRestDAOV6.java index 0fa9438876..fc3ab23033 100644 --- a/es6-persistence/src/main/java/com/netflix/conductor/es6/dao/index/ElasticSearchRestDAOV6.java +++ b/es6-persistence/src/main/java/com/netflix/conductor/es6/dao/index/ElasticSearchRestDAOV6.java @@ -157,13 +157,13 @@ public ElasticSearchRestDAOV6(RestClientBuilder restClientBuilder, ElasticSearch this.clusterHealthColor = properties.getClusterHealthColor(); this.bulkRequests = new ConcurrentHashMap<>(); this.indexBatchSize = properties.getIndexBatchSize(); - this.asyncBufferFlushTimeout = properties.getAsyncBufferFlushTimeout(); + this.asyncBufferFlushTimeout = properties.getAsyncBufferFlushTimeoutSecs(); this.properties = properties; - this.indexPrefix = properties.getIndexName(); - if (!properties.isElasticSearchAutoIndexManagementEnabled() && - StringUtils.isNotBlank(properties.getElasticSearchDocumentTypeOverride())) { - docTypeOverride = properties.getElasticSearchDocumentTypeOverride(); + this.indexPrefix = properties.getIndexPrefix(); + if (!properties.isAutoIndexManagementEnabled() && + StringUtils.isNotBlank(properties.getDocumentTypeOverride())) { + docTypeOverride = properties.getDocumentTypeOverride(); } else { docTypeOverride = ""; } @@ -233,7 +233,7 @@ private void shutdownExecutorService(ExecutorService execService) { public void setup() throws Exception { waitForHealthyCluster(); - if (properties.isElasticSearchAutoIndexManagementEnabled()) { + if (properties.isAutoIndexManagementEnabled()) { createIndexesTemplates(); createWorkflowIndex(); createTaskIndex(); @@ -352,8 +352,8 @@ private void addIndex(final String index) throws IOException { ObjectNode setting = objectMapper.createObjectNode(); ObjectNode indexSetting = objectMapper.createObjectNode(); - indexSetting.put("number_of_shards", properties.getElasticSearchIndexShardCount()); - indexSetting.put("number_of_replicas", properties.getElasticSearchIndexReplicasCount()); + indexSetting.put("number_of_shards", properties.getIndexShardCount()); + indexSetting.put("number_of_replicas", properties.getIndexReplicasCount()); setting.set("index", indexSetting); @@ -547,7 +547,7 @@ public List getTaskExecutionLogs(String taskId) { SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); searchSourceBuilder.query(query); searchSourceBuilder.sort(new FieldSortBuilder("createdTime").order(SortOrder.ASC)); - searchSourceBuilder.size(properties.getElasticSearchTasklogResultLimit()); + searchSourceBuilder.size(properties.getTaskLogResultLimit()); // Generate the actual request to send to ES. String docType = StringUtils.isBlank(docTypeOverride) ? LOG_DOC_TYPE : docTypeOverride; @@ -974,7 +974,7 @@ private void indexWithRetry(final BulkRequest request, final String operationDes /** * Flush the buffers if bulk requests have not been indexed for the past {@link - * ElasticSearchProperties#getAsyncBufferFlushTimeout()} seconds. This is to prevent data loss in case the + * ElasticSearchProperties#getAsyncBufferFlushTimeoutSecs()} seconds. This is to prevent data loss in case the * instance is terminated, while the buffer still holds documents to be indexed. */ private void flushBulkRequests() { diff --git a/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/ElasticSearchDaoBaseTest.java b/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/ElasticSearchDaoBaseTest.java index 873b29552c..e4f5f136e4 100644 --- a/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/ElasticSearchDaoBaseTest.java +++ b/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/ElasticSearchDaoBaseTest.java @@ -1,3 +1,15 @@ +/* + * Copyright 2020 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package com.netflix.conductor.es6.dao.index; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; @@ -22,7 +34,7 @@ abstract class ElasticSearchDaoBaseTest extends ElasticSearchTest { @Before public void setup() throws Exception { int mappedPort = container.getMappedPort(9300); - properties.setURL("tcp://localhost:" + mappedPort); + properties.setUrl("tcp://localhost:" + mappedPort); Settings settings = Settings.builder() .put("client.transport.ignore_cluster_name", true) diff --git a/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/ElasticSearchRestDaoBaseTest.java b/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/ElasticSearchRestDaoBaseTest.java index 74134e6a2c..fc9c8845bc 100644 --- a/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/ElasticSearchRestDaoBaseTest.java +++ b/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/ElasticSearchRestDaoBaseTest.java @@ -1,3 +1,15 @@ +/* + * Copyright 2020 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package com.netflix.conductor.es6.dao.index; import org.apache.http.HttpHost; @@ -11,8 +23,6 @@ import java.io.IOException; import java.io.InputStreamReader; import java.io.Reader; -import java.util.HashMap; -import java.util.Map; abstract class ElasticSearchRestDaoBaseTest extends ElasticSearchTest { @@ -25,7 +35,7 @@ public void setup() throws Exception { String host = httpHostAddress.split(":")[0]; int port = Integer.parseInt(httpHostAddress.split(":")[1]); - properties.setURL("http://" + httpHostAddress); + properties.setUrl("http://" + httpHostAddress); RestClientBuilder restClientBuilder = RestClient.builder(new HttpHost(host, port, "http")); restClient = restClientBuilder.build(); diff --git a/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/ElasticSearchTest.java b/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/ElasticSearchTest.java index 93ea89db92..4145057f92 100644 --- a/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/ElasticSearchTest.java +++ b/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/ElasticSearchTest.java @@ -1,3 +1,15 @@ +/* + * Copyright 2020 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package com.netflix.conductor.es6.dao.index; import com.fasterxml.jackson.databind.ObjectMapper; @@ -17,7 +29,7 @@ @ContextConfiguration(classes = {ObjectMapperConfiguration.class, ElasticSearchTest.TestConfiguration.class}) @RunWith(SpringRunner.class) -@TestPropertySource(properties = {"workflow.indexing.enabled=true","workflow.elasticsearch.version=6"}) +@TestPropertySource(properties = {"conductor.indexing.enabled=true","conductor.elasticsearch.version=6"}) abstract class ElasticSearchTest { @Configuration diff --git a/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/TestElasticSearchDAOV6Batch.java b/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/TestElasticSearchDAOV6Batch.java index 0af460c2c9..283ff69a5c 100644 --- a/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/TestElasticSearchDAOV6Batch.java +++ b/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/TestElasticSearchDAOV6Batch.java @@ -1,3 +1,15 @@ +/* + * Copyright 2020 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package com.netflix.conductor.es6.dao.index; import com.netflix.conductor.common.metadata.tasks.Task; @@ -12,7 +24,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -@TestPropertySource(properties = "workflow.elasticsearch.index.batchSize=2") +@TestPropertySource(properties = "conductor.elasticsearch.indexBatchSize=2") public class TestElasticSearchDAOV6Batch extends ElasticSearchDaoBaseTest { @Test diff --git a/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/TestElasticSearchRestDAOV6Batch.java b/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/TestElasticSearchRestDAOV6Batch.java index 702711d189..d471aaba60 100644 --- a/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/TestElasticSearchRestDAOV6Batch.java +++ b/es6-persistence/src/test/java/com/netflix/conductor/es6/dao/index/TestElasticSearchRestDAOV6Batch.java @@ -1,3 +1,15 @@ +/* + * Copyright 2020 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package com.netflix.conductor.es6.dao.index; import com.netflix.conductor.common.metadata.tasks.Task; @@ -12,7 +24,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -@TestPropertySource(properties = "workflow.elasticsearch.index.batchSize=2") +@TestPropertySource(properties = "conductor.elasticsearch.indexBatchSize=2") public class TestElasticSearchRestDAOV6Batch extends ElasticSearchRestDaoBaseTest { @Test diff --git a/grpc-server/build.gradle b/grpc-server/build.gradle index 1e231d4bef..b099a5b00e 100644 --- a/grpc-server/build.gradle +++ b/grpc-server/build.gradle @@ -4,7 +4,6 @@ dependencies { implementation project(':conductor-grpc') implementation 'org.springframework.boot:spring-boot-starter' - annotationProcessor 'org.springframework.boot:spring-boot-configuration-processor' implementation "io.grpc:grpc-netty:${revGrpc}" implementation "io.grpc:grpc-services:${revGrpc}" diff --git a/grpc-server/src/main/java/com/netflix/conductor/grpc/server/GRPCServerProperties.java b/grpc-server/src/main/java/com/netflix/conductor/grpc/server/GRPCServerProperties.java index 760f209c67..414f63b08c 100644 --- a/grpc-server/src/main/java/com/netflix/conductor/grpc/server/GRPCServerProperties.java +++ b/grpc-server/src/main/java/com/netflix/conductor/grpc/server/GRPCServerProperties.java @@ -14,11 +14,17 @@ import org.springframework.boot.context.properties.ConfigurationProperties; -@ConfigurationProperties("conductor.grpc.server") +@ConfigurationProperties("conductor.grpc-server") public class GRPCServerProperties { + /** + * The port at which the gRPC server will serve requests + */ private int port = 8090; + /** + * Enables the reflection service for Protobuf services + */ private boolean reflectionEnabled = true; public int getPort() { diff --git a/grpc-server/src/main/java/com/netflix/conductor/grpc/server/GrpcConfiguration.java b/grpc-server/src/main/java/com/netflix/conductor/grpc/server/GrpcConfiguration.java index b4ae9ac443..7047b54840 100644 --- a/grpc-server/src/main/java/com/netflix/conductor/grpc/server/GrpcConfiguration.java +++ b/grpc-server/src/main/java/com/netflix/conductor/grpc/server/GrpcConfiguration.java @@ -21,9 +21,8 @@ import java.util.List; -@SuppressWarnings("SpringJavaInjectionPointsAutowiringInspection") @Configuration -@ConditionalOnProperty(name = "conductor.grpc.server.enabled", havingValue = "true") +@ConditionalOnProperty(name = "conductor.grpc-server.enabled", havingValue = "true") @EnableConfigurationProperties(GRPCServerProperties.class) public class GrpcConfiguration { diff --git a/grpc-server/src/test/java/com/netflix/conductor/grpc/server/service/HealthServiceImplTest.java b/grpc-server/src/test/java/com/netflix/conductor/grpc/server/service/HealthServiceImplTest.java index fa162bf03c..b65ef13db7 100644 --- a/grpc-server/src/test/java/com/netflix/conductor/grpc/server/service/HealthServiceImplTest.java +++ b/grpc-server/src/test/java/com/netflix/conductor/grpc/server/service/HealthServiceImplTest.java @@ -1,3 +1,15 @@ +/* + * Copyright 2020 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package com.netflix.conductor.grpc.server.service; public class HealthServiceImplTest { diff --git a/grpc/src/test/java/com/netflix/conductor/grpc/TestProtoMapper.java b/grpc/src/test/java/com/netflix/conductor/grpc/TestProtoMapper.java index 4d8bdef1cb..31286609a4 100644 --- a/grpc/src/test/java/com/netflix/conductor/grpc/TestProtoMapper.java +++ b/grpc/src/test/java/com/netflix/conductor/grpc/TestProtoMapper.java @@ -1,3 +1,15 @@ +/* + * Copyright 2020 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package com.netflix.conductor.grpc; import com.netflix.conductor.common.metadata.workflow.WorkflowTask; diff --git a/mysql-persistence/src/main/java/com/netflix/conductor/mysql/config/MySQLConfiguration.java b/mysql-persistence/src/main/java/com/netflix/conductor/mysql/config/MySQLConfiguration.java index 88b0ad72c9..8d5e34b05a 100644 --- a/mysql-persistence/src/main/java/com/netflix/conductor/mysql/config/MySQLConfiguration.java +++ b/mysql-persistence/src/main/java/com/netflix/conductor/mysql/config/MySQLConfiguration.java @@ -21,12 +21,14 @@ import com.netflix.conductor.mysql.dao.MySQLQueueDAO; import javax.sql.DataSource; import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; @SuppressWarnings("SpringJavaInjectionPointsAutowiringInspection") @Configuration(proxyBeanMethods = false) -@ConditionalOnProperty(name = "db", havingValue = "mysql") +@EnableConfigurationProperties(MySQLProperties.class) +@ConditionalOnProperty(name = "conductor.db.type", havingValue = "mysql") public class MySQLConfiguration { @Bean diff --git a/mysql-persistence/src/main/java/com/netflix/conductor/mysql/config/MySQLDataSourceProvider.java b/mysql-persistence/src/main/java/com/netflix/conductor/mysql/config/MySQLDataSourceProvider.java index 3d723ed16d..42f327b169 100644 --- a/mysql-persistence/src/main/java/com/netflix/conductor/mysql/config/MySQLDataSourceProvider.java +++ b/mysql-persistence/src/main/java/com/netflix/conductor/mysql/config/MySQLDataSourceProvider.java @@ -50,7 +50,7 @@ public DataSource getDataSource() { private HikariConfig createConfiguration() { HikariConfig hikariConfig = new HikariConfig(); hikariConfig.setJdbcUrl(properties.getJdbcUrl()); - hikariConfig.setUsername(properties.getJdbcUserName()); + hikariConfig.setUsername(properties.getJdbcUsername()); hikariConfig.setPassword(properties.getJdbcPassword()); hikariConfig.setAutoCommit(false); hikariConfig.setMaximumPoolSize(properties.getConnectionPoolMaxSize()); diff --git a/mysql-persistence/src/main/java/com/netflix/conductor/mysql/config/MySQLProperties.java b/mysql-persistence/src/main/java/com/netflix/conductor/mysql/config/MySQLProperties.java index e926480fb5..2f7692b29d 100644 --- a/mysql-persistence/src/main/java/com/netflix/conductor/mysql/config/MySQLProperties.java +++ b/mysql-persistence/src/main/java/com/netflix/conductor/mysql/config/MySQLProperties.java @@ -12,112 +12,183 @@ */ package com.netflix.conductor.mysql.config; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; -import org.springframework.stereotype.Component; - +import java.sql.Connection; import java.util.Optional; +import java.util.concurrent.TimeUnit; +import org.springframework.boot.context.properties.ConfigurationProperties; -@Component -@ConditionalOnProperty(name = "db", havingValue = "mysql") +@ConfigurationProperties("conductor.mysql") public class MySQLProperties { - @Value("${jdbc.url:jdbc:mysql://localhost:3306/conductor}") - private String jdbcUrl; + /** + * The jdbc url to be used for connecting to the database + */ + private String jdbcUrl = "jdbc:mysql://localhost:3306/conductor"; - @Value("${jdbc.username:conductor}") - private String jdbcUsername; + /** + * The username to be used for connections + */ + private String jdbcUsername = "conductor"; - @Value("${jdbc.password:password}") - private String jdbcPassword; + /** + * The password to be used for connections + */ + private String jdbcPassword = "password"; - @Value("${flyway.enabled:true}") - private boolean flywayEnabled; + /** + * Used to enable/disable flyway migrations + */ + private boolean flywayEnabled = true; - @Value("${flyway.table:#{null}}") - private String flywayTable; + /** + * Used to override the default flyway migration table + */ + private String flywayTable = null; // The defaults are currently in line with the HikariConfig defaults, which are unfortunately private. - @Value("${conductor.mysql.connection.pool.size.max:-1}") - private int connectionPoolMaxSize; + /** + * The maximum size that the connection pool is allowed to reach including idle and in-use connections + */ + private int connectionPoolMaxSize = -1; - @Value("${conductor.mysql.connection.pool.idle.min:-1}") - private int connectionPoolMinIdle; + /** + * The minimum number of idle connections that the connection pool tries to maintain in the pool + */ + private int connectionPoolMinIdle = -1; - @Value("${conductor.mysql.connection.lifetime.max:#{T(java.util.concurrent.TimeUnit).MINUTES.toMillis(30)}}") - private long connectionMaxLifetime; + /** + * The maximum lifetime of a connection (in milliseconds) in the pool + */ + private long connectionMaxLifetime = TimeUnit.MINUTES.toMillis(30); - @Value("${conductor.mysql.connection.idle.timeout:#{T(java.util.concurrent.TimeUnit).MINUTES.toMillis(10)}}") - private long connectionIdleTimeout; + /** + * The maximum amount of time (in milliseconds) that a connection is allowed to sit idle in the pool + */ + private long connectionIdleTimeout = TimeUnit.MINUTES.toMillis(10); - @Value("${conductor.mysql.connection.timeout:#{T(java.util.concurrent.TimeUnit).MINUTES.toMillis(30)}}") - private long connectionTimeout; + /** + * The maximum amount of time (in milliseconds) that a client will wait for a connection from the pool + */ + private long connectionTimeout = TimeUnit.MINUTES.toMillis(30); - @Value("${conductor.mysql.transaction.isolation.level:}") - private String transactionIsolationLevel; + /** + * The transaction isolation level as specified in {@link Connection} + */ + private String transactionIsolationLevel = ""; + //This is consistent with the current default when building the Hikari Client. /** - * This is consistent with the current default when building the Hikari Client. + * The auto-commit behavior of the connections in the pool */ - @Value("${conductor.mysql.autocommit:false}") - private boolean autoCommit; + private boolean autoCommit = false; /** - * the refresh time for the in-memory task definition cache + * The time in seconds after which the in-memory task definitions cache will be refreshed */ - @Value("${conductor.taskdef.cache.refresh.time.seconds:60}") - private int taskDefCacheRefreshTimeSecs; + private int taskDefCacheRefreshTimeSecs = 60; public String getJdbcUrl() { return jdbcUrl; } - public String getJdbcUserName() { + public void setJdbcUrl(String jdbcUrl) { + this.jdbcUrl = jdbcUrl; + } + + public String getJdbcUsername() { return jdbcUsername; } + public void setJdbcUsername(String jdbcUsername) { + this.jdbcUsername = jdbcUsername; + } + public String getJdbcPassword() { return jdbcPassword; } + public void setJdbcPassword(String jdbcPassword) { + this.jdbcPassword = jdbcPassword; + } + public boolean isFlywayEnabled() { return flywayEnabled; } + public void setFlywayEnabled(boolean flywayEnabled) { + this.flywayEnabled = flywayEnabled; + } + public Optional getFlywayTable() { return Optional.ofNullable(flywayTable); } + public void setFlywayTable(String flywayTable) { + this.flywayTable = flywayTable; + } + public int getConnectionPoolMaxSize() { return connectionPoolMaxSize; } + public void setConnectionPoolMaxSize(int connectionPoolMaxSize) { + this.connectionPoolMaxSize = connectionPoolMaxSize; + } + public int getConnectionPoolMinIdle() { return connectionPoolMinIdle; } + public void setConnectionPoolMinIdle(int connectionPoolMinIdle) { + this.connectionPoolMinIdle = connectionPoolMinIdle; + } + public long getConnectionMaxLifetime() { return connectionMaxLifetime; } + public void setConnectionMaxLifetime(long connectionMaxLifetime) { + this.connectionMaxLifetime = connectionMaxLifetime; + } + public long getConnectionIdleTimeout() { return connectionIdleTimeout; } + public void setConnectionIdleTimeout(long connectionIdleTimeout) { + this.connectionIdleTimeout = connectionIdleTimeout; + } + public long getConnectionTimeout() { return connectionTimeout; } + public void setConnectionTimeout(long connectionTimeout) { + this.connectionTimeout = connectionTimeout; + } + public String getTransactionIsolationLevel() { return transactionIsolationLevel; } + public void setTransactionIsolationLevel(String transactionIsolationLevel) { + this.transactionIsolationLevel = transactionIsolationLevel; + } + public boolean isAutoCommit() { return autoCommit; } - public int getTaskDefRefreshTimeSecs() { + public void setAutoCommit(boolean autoCommit) { + this.autoCommit = autoCommit; + } + + public int getTaskDefCacheRefreshTimeSecs() { return taskDefCacheRefreshTimeSecs; } + + public void setTaskDefCacheRefreshTimeSecs(int taskDefCacheRefreshTimeSecs) { + this.taskDefCacheRefreshTimeSecs = taskDefCacheRefreshTimeSecs; + } } diff --git a/mysql-persistence/src/main/java/com/netflix/conductor/mysql/dao/MySQLMetadataDAO.java b/mysql-persistence/src/main/java/com/netflix/conductor/mysql/dao/MySQLMetadataDAO.java index 4891fbd71b..9de32f9af2 100644 --- a/mysql-persistence/src/main/java/com/netflix/conductor/mysql/dao/MySQLMetadataDAO.java +++ b/mysql-persistence/src/main/java/com/netflix/conductor/mysql/dao/MySQLMetadataDAO.java @@ -42,7 +42,7 @@ public class MySQLMetadataDAO extends MySQLBaseDAO implements MetadataDAO, Event public MySQLMetadataDAO(ObjectMapper objectMapper, DataSource dataSource, MySQLProperties properties) { super(objectMapper, dataSource); - int cacheRefreshTime = properties.getTaskDefRefreshTimeSecs(); + int cacheRefreshTime = properties.getTaskDefCacheRefreshTimeSecs(); Executors.newSingleThreadScheduledExecutor() .scheduleWithFixedDelay(this::refreshTaskDefs, cacheRefreshTime, cacheRefreshTime, TimeUnit.SECONDS); } diff --git a/mysql-persistence/src/main/resources/db/migration/V1__initial_schema.sql b/mysql-persistence/src/main/resources/db/migration/V1__initial_schema.sql index 246b55ecd7..e5cd53ad81 100644 --- a/mysql-persistence/src/main/resources/db/migration/V1__initial_schema.sql +++ b/mysql-persistence/src/main/resources/db/migration/V1__initial_schema.sql @@ -1,3 +1,19 @@ +-- +-- Copyright 2020 Netflix, Inc. +-- +-- Licensed under the Apache License, Version 2.0 (the "License"); +-- you may not use this file except in compliance with the License. +-- You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + -- -------------------------------------------------------------------------------------------------------------- -- SCHEMA FOR METADATA DAO diff --git a/mysql-persistence/src/main/resources/db/migration/V2__queue_message_timestamps.sql b/mysql-persistence/src/main/resources/db/migration/V2__queue_message_timestamps.sql index ecf7956be9..5c96ec1174 100644 --- a/mysql-persistence/src/main/resources/db/migration/V2__queue_message_timestamps.sql +++ b/mysql-persistence/src/main/resources/db/migration/V2__queue_message_timestamps.sql @@ -1,2 +1,18 @@ +-- +-- Copyright 2020 Netflix, Inc. +-- +-- Licensed under the Apache License, Version 2.0 (the "License"); +-- you may not use this file except in compliance with the License. +-- You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + ALTER TABLE `queue_message` CHANGE `created_on` `created_on` TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP; ALTER TABLE `queue_message` CHANGE `deliver_on` `deliver_on` TIMESTAMP DEFAULT CURRENT_TIMESTAMP; diff --git a/mysql-persistence/src/main/resources/db/migration/V3__queue_add_priority.sql b/mysql-persistence/src/main/resources/db/migration/V3__queue_add_priority.sql index 2764df8b31..8ecf9134c2 100644 --- a/mysql-persistence/src/main/resources/db/migration/V3__queue_add_priority.sql +++ b/mysql-persistence/src/main/resources/db/migration/V3__queue_add_priority.sql @@ -1,3 +1,19 @@ +-- +-- Copyright 2020 Netflix, Inc. +-- +-- Licensed under the Apache License, Version 2.0 (the "License"); +-- you may not use this file except in compliance with the License. +-- You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + SET @dbname = DATABASE(); SET @tablename = "queue_message"; SET @columnname = "priority"; diff --git a/mysql-persistence/src/main/resources/db/migration/V4__1009_Fix_MySQLExecutionDAO_Index.sql b/mysql-persistence/src/main/resources/db/migration/V4__1009_Fix_MySQLExecutionDAO_Index.sql index 8787961a84..f555736c68 100644 --- a/mysql-persistence/src/main/resources/db/migration/V4__1009_Fix_MySQLExecutionDAO_Index.sql +++ b/mysql-persistence/src/main/resources/db/migration/V4__1009_Fix_MySQLExecutionDAO_Index.sql @@ -1,3 +1,19 @@ +-- +-- Copyright 2020 Netflix, Inc. +-- +-- Licensed under the Apache License, Version 2.0 (the "License"); +-- you may not use this file except in compliance with the License. +-- You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + # Drop the 'unique_event_execution' index if it exists SET @exist := (SELECT COUNT(INDEX_NAME) FROM information_schema.STATISTICS diff --git a/mysql-persistence/src/main/resources/db/migration/V5__correlation_id_index.sql b/mysql-persistence/src/main/resources/db/migration/V5__correlation_id_index.sql index 2f13789f37..ba2839aa6f 100644 --- a/mysql-persistence/src/main/resources/db/migration/V5__correlation_id_index.sql +++ b/mysql-persistence/src/main/resources/db/migration/V5__correlation_id_index.sql @@ -1,3 +1,19 @@ +-- +-- Copyright 2020 Netflix, Inc. +-- +-- Licensed under the Apache License, Version 2.0 (the "License"); +-- you may not use this file except in compliance with the License. +-- You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + # Drop the 'workflow_corr_id_index' index if it exists SET @exist := (SELECT COUNT(INDEX_NAME) FROM information_schema.STATISTICS diff --git a/mysql-persistence/src/main/resources/db/migration/V6__new_qm_index_with_priority.sql b/mysql-persistence/src/main/resources/db/migration/V6__new_qm_index_with_priority.sql index de591f972b..cd315f12af 100644 --- a/mysql-persistence/src/main/resources/db/migration/V6__new_qm_index_with_priority.sql +++ b/mysql-persistence/src/main/resources/db/migration/V6__new_qm_index_with_priority.sql @@ -1,3 +1,19 @@ +-- +-- Copyright 2020 Netflix, Inc. +-- +-- Licensed under the Apache License, Version 2.0 (the "License"); +-- you may not use this file except in compliance with the License. +-- You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + # Drop the 'combo_queue_message' index if it exists SET @exist := (SELECT COUNT(INDEX_NAME) FROM information_schema.STATISTICS diff --git a/mysql-persistence/src/test/java/com/netflix/conductor/mysql/dao/MySQLQueueDAOTest.java b/mysql-persistence/src/test/java/com/netflix/conductor/mysql/dao/MySQLQueueDAOTest.java index 499e1fd88c..d0b8b365e2 100644 --- a/mysql-persistence/src/test/java/com/netflix/conductor/mysql/dao/MySQLQueueDAOTest.java +++ b/mysql-persistence/src/test/java/com/netflix/conductor/mysql/dao/MySQLQueueDAOTest.java @@ -1,3 +1,15 @@ +/* + * Copyright 2020 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package com.netflix.conductor.mysql.dao; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/mysql-persistence/src/test/java/com/netflix/conductor/mysql/util/MySQLDAOTestUtil.java b/mysql-persistence/src/test/java/com/netflix/conductor/mysql/util/MySQLDAOTestUtil.java index fcc9f0b368..a14713cd08 100644 --- a/mysql-persistence/src/test/java/com/netflix/conductor/mysql/util/MySQLDAOTestUtil.java +++ b/mysql-persistence/src/test/java/com/netflix/conductor/mysql/util/MySQLDAOTestUtil.java @@ -40,9 +40,9 @@ public MySQLDAOTestUtil(ObjectMapper objectMapper, String dbName) { properties = mock(MySQLProperties.class); when(properties.getJdbcUrl()).thenReturn("jdbc:mysql://localhost:33307/" + dbName + "?useSSL=false&useUnicode=true&useJDBCCompliantTimezoneShift=true&useLegacyDatetimeCode=false&serverTimezone=UTC"); - when(properties.getJdbcUserName()).thenReturn("root"); + when(properties.getJdbcUsername()).thenReturn("root"); when(properties.getJdbcPassword()).thenReturn("root"); - when(properties.getTaskDefRefreshTimeSecs()).thenReturn(60); + when(properties.getTaskDefCacheRefreshTimeSecs()).thenReturn(60); createDatabase(dbName); this.objectMapper = objectMapper; this.dataSource = getDataSource(properties); @@ -71,7 +71,7 @@ private void createDatabase(String dbName) { private HikariDataSource getDataSource(MySQLProperties properties) { HikariDataSource dataSource = new HikariDataSource(); dataSource.setJdbcUrl(properties.getJdbcUrl()); - dataSource.setUsername(properties.getJdbcUserName()); + dataSource.setUsername(properties.getJdbcUsername()); dataSource.setPassword(properties.getJdbcPassword()); dataSource.setAutoCommit(false); diff --git a/postgres-persistence/src/main/java/com/netflix/conductor/postgres/config/PostgresConfiguration.java b/postgres-persistence/src/main/java/com/netflix/conductor/postgres/config/PostgresConfiguration.java index 1aa28ff197..32dfea5ef2 100644 --- a/postgres-persistence/src/main/java/com/netflix/conductor/postgres/config/PostgresConfiguration.java +++ b/postgres-persistence/src/main/java/com/netflix/conductor/postgres/config/PostgresConfiguration.java @@ -21,12 +21,14 @@ import com.netflix.conductor.postgres.dao.PostgresQueueDAO; import javax.sql.DataSource; import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; @SuppressWarnings("SpringJavaInjectionPointsAutowiringInspection") @Configuration(proxyBeanMethods = false) -@ConditionalOnProperty(name = "db", havingValue = "postgres") +@EnableConfigurationProperties(PostgresProperties.class) +@ConditionalOnProperty(name = "conductor.db.type", havingValue = "postgres") public class PostgresConfiguration { @Bean diff --git a/postgres-persistence/src/main/java/com/netflix/conductor/postgres/config/PostgresDataSourceProvider.java b/postgres-persistence/src/main/java/com/netflix/conductor/postgres/config/PostgresDataSourceProvider.java index 09716c43c9..f1e5e56c12 100644 --- a/postgres-persistence/src/main/java/com/netflix/conductor/postgres/config/PostgresDataSourceProvider.java +++ b/postgres-persistence/src/main/java/com/netflix/conductor/postgres/config/PostgresDataSourceProvider.java @@ -51,7 +51,7 @@ public DataSource getDataSource() { private HikariConfig createConfiguration() { HikariConfig hikariConfig = new HikariConfig(); hikariConfig.setJdbcUrl(properties.getJdbcUrl()); - hikariConfig.setUsername(properties.getJdbcUserName()); + hikariConfig.setUsername(properties.getJdbcUsername()); hikariConfig.setPassword(properties.getJdbcPassword()); hikariConfig.setAutoCommit(false); hikariConfig.setMaximumPoolSize(properties.getConnectionPoolMaxSize()); diff --git a/postgres-persistence/src/main/java/com/netflix/conductor/postgres/config/PostgresProperties.java b/postgres-persistence/src/main/java/com/netflix/conductor/postgres/config/PostgresProperties.java index de030785cf..6b450c077a 100644 --- a/postgres-persistence/src/main/java/com/netflix/conductor/postgres/config/PostgresProperties.java +++ b/postgres-persistence/src/main/java/com/netflix/conductor/postgres/config/PostgresProperties.java @@ -12,111 +12,182 @@ */ package com.netflix.conductor.postgres.config; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; -import org.springframework.stereotype.Component; - +import java.sql.Connection; import java.util.Optional; +import java.util.concurrent.TimeUnit; +import org.springframework.boot.context.properties.ConfigurationProperties; -@Component -@ConditionalOnProperty(name = "db", havingValue = "postgres") +@ConfigurationProperties("conductor.postgres") public class PostgresProperties { - @Value("${jdbc.url:jdbc:postgresql://localhost:5432/conductor}") - private String jdbcUrl; + /** + * The jdbc url to be used for connecting to the database + */ + private String jdbcUrl = "jdbc:postgresql://localhost:5432/conductor"; - @Value("${jdbc.username:conductor}") - private String jdbcUsername; + /** + * The username to be used for connections + */ + private String jdbcUsername = "conductor"; - @Value("${jdbc.password:password}") - private String jdbcPassword; + /** + * The password to be used for connections + */ + private String jdbcPassword = "password"; - @Value("${flyway.enabled:true}") - private boolean flywayEnabled; + /** + * Used to enable/disable flyway migrations + */ + private boolean flywayEnabled = true; - @Value("${flyway.table:#{null}}") - private String flywayTable; + /** + * Used to override the default flyway migration table + */ + private String flywayTable = null; // The defaults are currently in line with the HikariConfig defaults, which are unfortunately private. - @Value("${conductor.postgres.connection.pool.size.max:-1}") - private int connectionPoolMaxSize; + /** + * The maximum size that the connection pool is allowed to reach including idle and in-use connections + */ + private int connectionPoolMaxSize = -1; - @Value("${conductor.postgres.connection.pool.idle.min:-1}") - private int connectionPoolMinIdle; + /** + * The minimum number of idle connections that the connection pool tries to maintain in the pool + */ + private int connectionPoolMinIdle = -1; - @Value("${conductor.postgres.connection.lifetime.max:#{T(java.util.concurrent.TimeUnit).MINUTES.toMillis(30)}}") - private long connectionMaxLifetime; + /** + * The maximum lifetime (in milliseconds) of a connection in the pool + */ + private long connectionMaxLifetime = TimeUnit.MINUTES.toMillis(30); - @Value("${conductor.postgres.connection.idle.timeout:#{T(java.util.concurrent.TimeUnit).MINUTES.toMillis(10)}}") - private long connectionIdleTimeout; + /** + * The maximum amount of time (in milliseconds) that a connection is allowed to sit idle in the pool + */ + private long connectionIdleTimeout = TimeUnit.MINUTES.toMillis(30); - @Value("${conductor.postgres.connection.timeout:#{T(java.util.concurrent.TimeUnit).MINUTES.toMillis(30)}}") - private long connectionTimeout; + /** + * The maximum amount of time (in milliseconds) that a client will wait for a connection from the pool + */ + private long connectionTimeout = TimeUnit.MINUTES.toMillis(30); - @Value("${conductor.postgres.transaction.isolation.level:}") - private String transactionIsolationLevel; + /** + * The transaction isolation level as specified in {@link Connection} + */ + private String transactionIsolationLevel = ""; + // This is consistent with the current default when building the Hikari Client. /** - * This is consistent with the current default when building the Hikari Client. + * The auto-commit behavior of the connections in the pool */ - @Value("${conductor.postgres.autocommit:false}") - private boolean autoCommit; + private boolean autoCommit = false; /** - * the refresh time for the in-memory task definition cache + * The time in seconds after which the in-memory task definitions cache will be refreshed */ - @Value("${conductor.taskdef.cache.refresh.time.seconds:60}") - private int taskDefCacheRefreshTimeSecs; + private int taskDefCacheRefreshTimeSecs = 60; public String getJdbcUrl() { return jdbcUrl; } - public String getJdbcUserName() { + public void setJdbcUrl(String jdbcUrl) { + this.jdbcUrl = jdbcUrl; + } + + public String getJdbcUsername() { return jdbcUsername; } + public void setJdbcUsername(String jdbcUsername) { + this.jdbcUsername = jdbcUsername; + } + public String getJdbcPassword() { return jdbcPassword; } + public void setJdbcPassword(String jdbcPassword) { + this.jdbcPassword = jdbcPassword; + } + public boolean isFlywayEnabled() { return flywayEnabled; } + public void setFlywayEnabled(boolean flywayEnabled) { + this.flywayEnabled = flywayEnabled; + } + public Optional getFlywayTable() { return Optional.ofNullable(flywayTable); } + public void setFlywayTable(String flywayTable) { + this.flywayTable = flywayTable; + } + public int getConnectionPoolMaxSize() { return connectionPoolMaxSize; } + public void setConnectionPoolMaxSize(int connectionPoolMaxSize) { + this.connectionPoolMaxSize = connectionPoolMaxSize; + } + public int getConnectionPoolMinIdle() { return connectionPoolMinIdle; } + public void setConnectionPoolMinIdle(int connectionPoolMinIdle) { + this.connectionPoolMinIdle = connectionPoolMinIdle; + } + public long getConnectionMaxLifetime() { return connectionMaxLifetime; } + public void setConnectionMaxLifetime(long connectionMaxLifetime) { + this.connectionMaxLifetime = connectionMaxLifetime; + } + public long getConnectionIdleTimeout() { return connectionIdleTimeout; } + public void setConnectionIdleTimeout(long connectionIdleTimeout) { + this.connectionIdleTimeout = connectionIdleTimeout; + } + public long getConnectionTimeout() { return connectionTimeout; } + public void setConnectionTimeout(long connectionTimeout) { + this.connectionTimeout = connectionTimeout; + } + public String getTransactionIsolationLevel() { return transactionIsolationLevel; } + public void setTransactionIsolationLevel(String transactionIsolationLevel) { + this.transactionIsolationLevel = transactionIsolationLevel; + } + public boolean isAutoCommit() { return autoCommit; } - public int getTaskDefRefreshTimeSecs() { + public void setAutoCommit(boolean autoCommit) { + this.autoCommit = autoCommit; + } + + public int getTaskDefCacheRefreshTimeSecs() { return taskDefCacheRefreshTimeSecs; } + + public void setTaskDefCacheRefreshTimeSecs(int taskDefCacheRefreshTimeSecs) { + this.taskDefCacheRefreshTimeSecs = taskDefCacheRefreshTimeSecs; + } } diff --git a/postgres-persistence/src/main/java/com/netflix/conductor/postgres/dao/PostgresMetadataDAO.java b/postgres-persistence/src/main/java/com/netflix/conductor/postgres/dao/PostgresMetadataDAO.java index d64444b6b1..eb03edcad7 100644 --- a/postgres-persistence/src/main/java/com/netflix/conductor/postgres/dao/PostgresMetadataDAO.java +++ b/postgres-persistence/src/main/java/com/netflix/conductor/postgres/dao/PostgresMetadataDAO.java @@ -42,7 +42,7 @@ public class PostgresMetadataDAO extends PostgresBaseDAO implements MetadataDAO, public PostgresMetadataDAO(ObjectMapper objectMapper, DataSource dataSource, PostgresProperties properties) { super(objectMapper, dataSource); - int cacheRefreshTime = properties.getTaskDefRefreshTimeSecs(); + int cacheRefreshTime = properties.getTaskDefCacheRefreshTimeSecs(); Executors.newSingleThreadScheduledExecutor() .scheduleWithFixedDelay(this::refreshTaskDefs, cacheRefreshTime, cacheRefreshTime, TimeUnit.SECONDS); } diff --git a/postgres-persistence/src/main/resources/db/migration_postgres/V1__initial_schema.sql b/postgres-persistence/src/main/resources/db/migration_postgres/V1__initial_schema.sql index a76611b27d..ab066f2ae5 100644 --- a/postgres-persistence/src/main/resources/db/migration_postgres/V1__initial_schema.sql +++ b/postgres-persistence/src/main/resources/db/migration_postgres/V1__initial_schema.sql @@ -1,3 +1,19 @@ +-- +-- Copyright 2020 Netflix, Inc. +-- +-- Licensed under the Apache License, Version 2.0 (the "License"); +-- you may not use this file except in compliance with the License. +-- You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + -- -------------------------------------------------------------------------------------------------------------- -- SCHEMA FOR METADATA DAO diff --git a/postgres-persistence/src/main/resources/db/migration_postgres/V2__1009_Fix_PostgresExecutionDAO_Index.sql b/postgres-persistence/src/main/resources/db/migration_postgres/V2__1009_Fix_PostgresExecutionDAO_Index.sql index 03b132ab0d..8f383c7f0e 100644 --- a/postgres-persistence/src/main/resources/db/migration_postgres/V2__1009_Fix_PostgresExecutionDAO_Index.sql +++ b/postgres-persistence/src/main/resources/db/migration_postgres/V2__1009_Fix_PostgresExecutionDAO_Index.sql @@ -1,3 +1,19 @@ +-- +-- Copyright 2020 Netflix, Inc. +-- +-- Licensed under the Apache License, Version 2.0 (the "License"); +-- you may not use this file except in compliance with the License. +-- You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + DROP INDEX IF EXISTS unique_event_execution; CREATE UNIQUE INDEX unique_event_execution ON event_execution (event_handler_name,event_name,execution_id); \ No newline at end of file diff --git a/postgres-persistence/src/main/resources/db/migration_postgres/V3__correlation_id_index.sql b/postgres-persistence/src/main/resources/db/migration_postgres/V3__correlation_id_index.sql index 9ced890da0..33c120555f 100644 --- a/postgres-persistence/src/main/resources/db/migration_postgres/V3__correlation_id_index.sql +++ b/postgres-persistence/src/main/resources/db/migration_postgres/V3__correlation_id_index.sql @@ -1,3 +1,19 @@ +-- +-- Copyright 2020 Netflix, Inc. +-- +-- Licensed under the Apache License, Version 2.0 (the "License"); +-- you may not use this file except in compliance with the License. +-- You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + DROP INDEX IF EXISTS workflow_corr_id_index; CREATE INDEX workflow_corr_id_index ON workflow (correlation_id); \ No newline at end of file diff --git a/postgres-persistence/src/main/resources/db/migration_postgres/V4__new_qm_index_with_priority.sql b/postgres-persistence/src/main/resources/db/migration_postgres/V4__new_qm_index_with_priority.sql index 23d12a37c2..cc106cbf6e 100644 --- a/postgres-persistence/src/main/resources/db/migration_postgres/V4__new_qm_index_with_priority.sql +++ b/postgres-persistence/src/main/resources/db/migration_postgres/V4__new_qm_index_with_priority.sql @@ -1,3 +1,19 @@ +-- +-- Copyright 2020 Netflix, Inc. +-- +-- Licensed under the Apache License, Version 2.0 (the "License"); +-- you may not use this file except in compliance with the License. +-- You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + DROP INDEX IF EXISTS combo_queue_message; CREATE INDEX combo_queue_message ON queue_message (queue_name,priority,popped,deliver_on,created_on); \ No newline at end of file diff --git a/postgres-persistence/src/test/java/com/netflix/conductor/postgres/util/PostgresDAOTestUtil.java b/postgres-persistence/src/test/java/com/netflix/conductor/postgres/util/PostgresDAOTestUtil.java index 23d026b1e4..ad92382761 100644 --- a/postgres-persistence/src/test/java/com/netflix/conductor/postgres/util/PostgresDAOTestUtil.java +++ b/postgres-persistence/src/test/java/com/netflix/conductor/postgres/util/PostgresDAOTestUtil.java @@ -52,10 +52,10 @@ public PostgresDAOTestUtil(ObjectMapper objectMapper, String dbName) { createDb(dataSource, dbName); when(properties.getJdbcUrl()).thenReturn(JDBC_URL_PREFIX + dbName); - when(properties.getJdbcUserName()).thenReturn("postgres"); + when(properties.getJdbcUsername()).thenReturn("postgres"); when(properties.getJdbcPassword()).thenReturn("postgres"); when(properties.isFlywayEnabled()).thenReturn(true); - when(properties.getTaskDefRefreshTimeSecs()).thenReturn(60); + when(properties.getTaskDefCacheRefreshTimeSecs()).thenReturn(60); this.dataSource = getDataSource(properties); } @@ -64,7 +64,7 @@ private HikariDataSource getDataSource(PostgresProperties properties) { HikariDataSource dataSource = new HikariDataSource(); dataSource.setJdbcUrl(properties.getJdbcUrl()); - dataSource.setUsername(properties.getJdbcUserName()); + dataSource.setUsername(properties.getJdbcUsername()); dataSource.setPassword(properties.getJdbcPassword()); dataSource.setAutoCommit(false); diff --git a/redis-lock/src/main/java/com/netflix/conductor/redislock/config/RedisLockConfiguration.java b/redis-lock/src/main/java/com/netflix/conductor/redislock/config/RedisLockConfiguration.java index 1637b697db..effea8d429 100644 --- a/redis-lock/src/main/java/com/netflix/conductor/redislock/config/RedisLockConfiguration.java +++ b/redis-lock/src/main/java/com/netflix/conductor/redislock/config/RedisLockConfiguration.java @@ -15,19 +15,19 @@ import com.netflix.conductor.core.sync.Lock; import com.netflix.conductor.redislock.config.RedisLockProperties.REDIS_SERVER_TYPE; import com.netflix.conductor.redislock.lock.RedisLock; +import java.util.Arrays; import org.redisson.Redisson; import org.redisson.config.Config; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; -import java.util.Arrays; - -@SuppressWarnings("SpringJavaInjectionPointsAutowiringInspection") @Configuration -@ConditionalOnProperty(prefix = "workflow.decider", name = "locking.server", havingValue = "REDIS") +@EnableConfigurationProperties(RedisLockProperties.class) +@ConditionalOnProperty(name = "conductor.workflow-execution-lock.type", havingValue = "redis") public class RedisLockConfiguration { private static final Logger LOGGER = LoggerFactory.getLogger(RedisLockConfiguration.class); @@ -36,16 +36,16 @@ public class RedisLockConfiguration { public Redisson getRedisson(RedisLockProperties properties) { RedisLockProperties.REDIS_SERVER_TYPE redisServerType; try { - redisServerType = properties.getRedisServerType(); + redisServerType = properties.getServerType(); } catch (IllegalArgumentException ie) { - final String message = "Invalid Redis server type: " + properties.getRedisServerType() + final String message = "Invalid Redis server type: " + properties.getServerType() + ", supported values are: " + Arrays.toString(REDIS_SERVER_TYPE.values()); LOGGER.error(message); throw new RuntimeException(message, ie); } - String redisServerAddress = properties.getRedisServerAddress(); - String redisServerPassword = properties.getRedisServerPassword(); - String masterName = properties.getRedisServerMasterName(); + String redisServerAddress = properties.getServerAddress(); + String redisServerPassword = properties.getServerPassword(); + String masterName = properties.getServerMasterName(); Config redisConfig = new Config(); diff --git a/redis-lock/src/main/java/com/netflix/conductor/redislock/config/RedisLockProperties.java b/redis-lock/src/main/java/com/netflix/conductor/redislock/config/RedisLockProperties.java index 11144be7a4..8701858ca2 100644 --- a/redis-lock/src/main/java/com/netflix/conductor/redislock/config/RedisLockProperties.java +++ b/redis-lock/src/main/java/com/netflix/conductor/redislock/config/RedisLockProperties.java @@ -12,56 +12,89 @@ */ package com.netflix.conductor.redislock.config; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; -import org.springframework.stereotype.Component; +import org.springframework.boot.context.properties.ConfigurationProperties; -@Component -@ConditionalOnProperty(prefix = "workflow.decider", name = "locking.server", havingValue = "REDIS") +@ConfigurationProperties("conductor.redis-lock") public class RedisLockProperties { - @Value("${workflow.redis.locking.server.type:single}") - private String serverType; + /** + * The redis server configuration to be used. Valid configurations are listed in: {@link REDIS_SERVER_TYPE} + */ + private String serverType = "single"; - @Value("${workflow.redis.locking.server.address:redis://127.0.0.1:6379}") - private String serverAddress; + /** + * The address of the redis server following format -- host:port + */ + private String serverAddress = "redis://127.0.0.1:6379"; - @Value("${workflow.redis.locking.server.password:#{null}}") - private String serverPassword; + /** + * The password for redis authentication + */ + private String serverPassword = null; - @Value("${workflow.redis.locking.server.master.name:master}") - private String serverMasterName; + /** + * The master server name used by Redis Sentinel servers and master change monitoring task + */ + private String serverMasterName = "master"; - @Value("${workflow.decider.locking.namespace:}") - private String lockingNamespace; + /** + * The namespace to use to prepend keys used for locking in redis + */ + private String namespace = ""; - @Value("${workflow.decider.locking.exceptions.ignore:false}") - private boolean ignoreLockingExceptions; + /** + * Enable to otionally continue without a lock to not block executions until the locking service becomes available + */ + private boolean ignoreLockingExceptions = false; - public REDIS_SERVER_TYPE getRedisServerType() { + public REDIS_SERVER_TYPE getServerType() { return REDIS_SERVER_TYPE.valueOf(serverType.toUpperCase()); } - public String getRedisServerAddress() { + public void setServerType(String serverType) { + this.serverType = serverType; + } + + public String getServerAddress() { return serverAddress; } - public String getRedisServerPassword() { + public void setServerAddress(String serverAddress) { + this.serverAddress = serverAddress; + } + + public String getServerPassword() { return serverPassword; } - public String getRedisServerMasterName() { + public void setServerPassword(String serverPassword) { + this.serverPassword = serverPassword; + } + + public String getServerMasterName() { return serverMasterName; } - public String getLockingNamespace() { - return lockingNamespace; + public void setServerMasterName(String serverMasterName) { + this.serverMasterName = serverMasterName; + } + + public String getNamespace() { + return namespace; + } + + public void setNamespace(String namespace) { + this.namespace = namespace; } public boolean isIgnoreLockingExceptions() { return ignoreLockingExceptions; } + public void setIgnoreLockingExceptions(boolean ignoreLockingExceptions) { + this.ignoreLockingExceptions = ignoreLockingExceptions; + } + public enum REDIS_SERVER_TYPE { SINGLE, CLUSTER, SENTINEL } diff --git a/redis-lock/src/main/java/com/netflix/conductor/redislock/lock/RedisLock.java b/redis-lock/src/main/java/com/netflix/conductor/redislock/lock/RedisLock.java index cbcaf0a8db..997e8ba030 100644 --- a/redis-lock/src/main/java/com/netflix/conductor/redislock/lock/RedisLock.java +++ b/redis-lock/src/main/java/com/netflix/conductor/redislock/lock/RedisLock.java @@ -35,7 +35,7 @@ public class RedisLock implements Lock { public RedisLock(Redisson redisson, RedisLockProperties properties) { this.properties = properties; this.redisson = redisson; - LOCK_NAMESPACE = properties.getLockingNamespace(); + LOCK_NAMESPACE = properties.getNamespace(); } @Override diff --git a/redis-lock/src/test/java/com/netflix/conductor/redis/lock/RedisLockTest.java b/redis-lock/src/test/java/com/netflix/conductor/redis/lock/RedisLockTest.java index 5b9748fd3e..b8c5e9ea1d 100644 --- a/redis-lock/src/test/java/com/netflix/conductor/redis/lock/RedisLockTest.java +++ b/redis-lock/src/test/java/com/netflix/conductor/redis/lock/RedisLockTest.java @@ -48,10 +48,10 @@ public static void setUp() throws Exception { redisServer.start(); RedisLockProperties properties = mock(RedisLockProperties.class); - when(properties.getRedisServerType()).thenReturn(REDIS_SERVER_TYPE.SINGLE); - when(properties.getRedisServerAddress()).thenReturn(testServerAddress); - when(properties.getRedisServerMasterName()).thenReturn("master"); - when(properties.getLockingNamespace()).thenReturn(""); + when(properties.getServerType()).thenReturn(REDIS_SERVER_TYPE.SINGLE); + when(properties.getServerAddress()).thenReturn(testServerAddress); + when(properties.getServerMasterName()).thenReturn("master"); + when(properties.getNamespace()).thenReturn(""); when(properties.isIgnoreLockingExceptions()).thenReturn(false); Config redissonConfig = new Config(); diff --git a/redis-persistence/src/main/java/com/netflix/conductor/redis/config/AnyRedisCondition.java b/redis-persistence/src/main/java/com/netflix/conductor/redis/config/AnyRedisCondition.java index ed66879077..18e9a68d33 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/redis/config/AnyRedisCondition.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/redis/config/AnyRedisCondition.java @@ -21,22 +21,22 @@ public AnyRedisCondition() { super(ConfigurationPhase.PARSE_CONFIGURATION); } - @ConditionalOnProperty(name = "db", havingValue = "dynomite") + @ConditionalOnProperty(name = "conductor.db.type", havingValue = "dynomite") static class DynomiteClusterCondition { } - @ConditionalOnProperty(name = "db", havingValue = "memory") + @ConditionalOnProperty(name = "conductor.db.type", havingValue = "memory") static class InMemoryRedisCondition { } - @ConditionalOnProperty(name = "db", havingValue = "redis_cluster") + @ConditionalOnProperty(name = "conductor.db.type", havingValue = "redis_cluster") static class RedisClusterConfiguration { } - @ConditionalOnProperty(name = "db", havingValue = "redis_sentinel") + @ConditionalOnProperty(name = "conductor.db.type", havingValue = "redis_sentinel") static class RedisSentinelConfiguration { } diff --git a/redis-persistence/src/main/java/com/netflix/conductor/redis/config/DynomiteClusterConfiguration.java b/redis-persistence/src/main/java/com/netflix/conductor/redis/config/DynomiteClusterConfiguration.java index b62cc4a9b3..c6a2ce497c 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/redis/config/DynomiteClusterConfiguration.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/redis/config/DynomiteClusterConfiguration.java @@ -12,6 +12,7 @@ */ package com.netflix.conductor.redis.config; +import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.dyno.connectionpool.HostSupplier; import com.netflix.dyno.connectionpool.TokenMapSupplier; import com.netflix.dyno.connectionpool.impl.ConnectionPoolConfigurationImpl; @@ -21,27 +22,27 @@ import redis.clients.jedis.commands.JedisCommands; @Configuration(proxyBeanMethods = false) -@ConditionalOnProperty(name = "db", havingValue = "dynomite") +@ConditionalOnProperty(name = "conductor.db.type", havingValue = "dynomite") public class DynomiteClusterConfiguration extends JedisCommandsConfigurer { - protected JedisCommands createJedisCommands(RedisProperties properties, HostSupplier hostSupplier, - TokenMapSupplier tokenMapSupplier) { - ConnectionPoolConfigurationImpl connectionPoolConfiguration = - new ConnectionPoolConfigurationImpl(properties.getClusterName()) - .withTokenSupplier(tokenMapSupplier) - .setLocalRack(properties.getAvailabilityZone()) - .setLocalDataCenter(properties.getRegion()) - .setSocketTimeout(0) - .setConnectTimeout(0) - .setMaxConnsPerHost( - properties.getMaxConnectionsPerHost() - ); + protected JedisCommands createJedisCommands(RedisProperties properties, ConductorProperties conductorProperties, + HostSupplier hostSupplier, TokenMapSupplier tokenMapSupplier) { + ConnectionPoolConfigurationImpl connectionPoolConfiguration = + new ConnectionPoolConfigurationImpl(properties.getClusterName()) + .withTokenSupplier(tokenMapSupplier) + .setLocalRack(properties.getAvailabilityZone()) + .setLocalDataCenter(properties.getDataCenterRegion()) + .setSocketTimeout(0) + .setConnectTimeout(0) + .setMaxConnsPerHost( + properties.getMaxConnectionsPerHost() + ); - return new DynoJedisClient.Builder() - .withHostSupplier(hostSupplier) - .withApplicationName(properties.getAppId()) - .withDynomiteClusterName(properties.getClusterName()) - .withCPConfig(connectionPoolConfiguration) - .build(); + return new DynoJedisClient.Builder() + .withHostSupplier(hostSupplier) + .withApplicationName(conductorProperties.getAppId()) + .withDynomiteClusterName(properties.getClusterName()) + .withCPConfig(connectionPoolConfiguration) + .build(); } } diff --git a/redis-persistence/src/main/java/com/netflix/conductor/redis/config/InMemoryRedisConfiguration.java b/redis-persistence/src/main/java/com/netflix/conductor/redis/config/InMemoryRedisConfiguration.java index 624525b416..8a78d9fc91 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/redis/config/InMemoryRedisConfiguration.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/redis/config/InMemoryRedisConfiguration.java @@ -23,7 +23,7 @@ import org.springframework.context.annotation.Configuration; @Configuration(proxyBeanMethods = false) -@ConditionalOnProperty(name = "db", havingValue = "memory") +@ConditionalOnProperty(name = "conductor.db.type", havingValue = "memory") public class InMemoryRedisConfiguration { @Bean diff --git a/redis-persistence/src/main/java/com/netflix/conductor/redis/config/JedisCommandsConfigurer.java b/redis-persistence/src/main/java/com/netflix/conductor/redis/config/JedisCommandsConfigurer.java index b8daa8a513..461d307a38 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/redis/config/JedisCommandsConfigurer.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/redis/config/JedisCommandsConfigurer.java @@ -12,15 +12,16 @@ */ package com.netflix.conductor.redis.config; +import static com.netflix.conductor.redis.config.RedisCommonConfiguration.DEFAULT_CLIENT_INJECTION_NAME; +import static com.netflix.conductor.redis.config.RedisCommonConfiguration.READ_CLIENT_INJECTION_NAME; + +import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.redis.dynoqueue.ConfigurationHostSupplier; import com.netflix.dyno.connectionpool.HostSupplier; import com.netflix.dyno.connectionpool.TokenMapSupplier; import org.springframework.context.annotation.Bean; import redis.clients.jedis.commands.JedisCommands; -import static com.netflix.conductor.redis.config.RedisCommonConfiguration.DEFAULT_CLIENT_INJECTION_NAME; -import static com.netflix.conductor.redis.config.RedisCommonConfiguration.READ_CLIENT_INJECTION_NAME; - abstract class JedisCommandsConfigurer { @Bean @@ -29,17 +30,17 @@ public HostSupplier hostSupplier(RedisProperties properties) { } @Bean(name = DEFAULT_CLIENT_INJECTION_NAME) - public JedisCommands jedisCommands(RedisProperties properties, HostSupplier hostSupplier, - TokenMapSupplier tokenMapSupplier) { - return createJedisCommands(properties, hostSupplier, tokenMapSupplier); + public JedisCommands jedisCommands(RedisProperties properties, ConductorProperties conductorProperties, + HostSupplier hostSupplier, TokenMapSupplier tokenMapSupplier) { + return createJedisCommands(properties, conductorProperties, hostSupplier, tokenMapSupplier); } @Bean(name = READ_CLIENT_INJECTION_NAME) - public JedisCommands readJedisCommands(RedisProperties properties, HostSupplier hostSupplier, - TokenMapSupplier tokenMapSupplier) { - return createJedisCommands(properties, hostSupplier, tokenMapSupplier); + public JedisCommands readJedisCommands(RedisProperties properties, ConductorProperties conductorProperties, + HostSupplier hostSupplier, TokenMapSupplier tokenMapSupplier) { + return createJedisCommands(properties, conductorProperties, hostSupplier, tokenMapSupplier); } - protected abstract JedisCommands createJedisCommands(RedisProperties properties, HostSupplier hostSupplier, - TokenMapSupplier tokenMapSupplier); + protected abstract JedisCommands createJedisCommands(RedisProperties properties, + ConductorProperties conductorProperties, HostSupplier hostSupplier, TokenMapSupplier tokenMapSupplier); } diff --git a/redis-persistence/src/main/java/com/netflix/conductor/redis/config/RedisClusterConfiguration.java b/redis-persistence/src/main/java/com/netflix/conductor/redis/config/RedisClusterConfiguration.java index aad8a00d2a..23286c34ed 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/redis/config/RedisClusterConfiguration.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/redis/config/RedisClusterConfiguration.java @@ -12,6 +12,7 @@ */ package com.netflix.conductor.redis.config; +import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.redis.jedis.JedisCluster; import com.netflix.dyno.connectionpool.Host; import com.netflix.dyno.connectionpool.HostSupplier; @@ -23,16 +24,17 @@ import redis.clients.jedis.commands.JedisCommands; @Configuration(proxyBeanMethods = false) -@ConditionalOnProperty(name = "db", havingValue = "redis_cluster") +@ConditionalOnProperty(name = "conductor.db.type", havingValue = "redis_cluster") public class RedisClusterConfiguration extends JedisCommandsConfigurer { @Override - protected JedisCommands createJedisCommands(RedisProperties properties, HostSupplier hostSupplier, TokenMapSupplier tokenMapSupplier) { + protected JedisCommands createJedisCommands(RedisProperties properties, ConductorProperties conductorProperties, + HostSupplier hostSupplier, TokenMapSupplier tokenMapSupplier) { GenericObjectPoolConfig genericObjectPoolConfig = new GenericObjectPoolConfig(); genericObjectPoolConfig.setMaxTotal(properties.getMaxConnectionsPerHost()); Host host = hostSupplier.getHosts().get(0); return new JedisCluster( - new redis.clients.jedis.JedisCluster(new HostAndPort(host.getHostName(), host.getPort()), - genericObjectPoolConfig)); + new redis.clients.jedis.JedisCluster(new HostAndPort(host.getHostName(), host.getPort()), + genericObjectPoolConfig)); } } diff --git a/redis-persistence/src/main/java/com/netflix/conductor/redis/config/RedisCommonConfiguration.java b/redis-persistence/src/main/java/com/netflix/conductor/redis/config/RedisCommonConfiguration.java index 221330c475..f74d882c5f 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/redis/config/RedisCommonConfiguration.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/redis/config/RedisCommonConfiguration.java @@ -23,19 +23,20 @@ import com.netflix.dyno.queues.redis.RedisQueues; import com.netflix.dyno.queues.redis.sharding.ShardingStrategy; import com.netflix.dyno.queues.shard.DynoShardSupplier; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Qualifier; +import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Conditional; import org.springframework.context.annotation.Configuration; import redis.clients.jedis.commands.JedisCommands; -import java.util.ArrayList; -import java.util.List; -import java.util.Set; - @Configuration(proxyBeanMethods = false) +@EnableConfigurationProperties(RedisProperties.class) @Conditional(AnyRedisCondition.class) public class RedisCommonConfiguration { @@ -48,12 +49,12 @@ public class RedisCommonConfiguration { public ShardSupplier shardSupplier(HostSupplier hostSupplier, RedisProperties properties) { if (properties.getAvailabilityZone() == null) { throw new ProvisionException( - "Availability zone is not defined. Ensure Configuration.getAvailabilityZone() returns a non-null " + - "and non-empty value." + "Availability zone is not defined. Ensure Configuration.getAvailabilityZone() returns a non-null " + + "and non-empty value." ); } - String localDC = properties.getAvailabilityZone().replaceAll(properties.getRegion(), ""); - return new DynoShardSupplier(hostSupplier, properties.getRegion(), localDC); + String localDC = properties.getAvailabilityZone().replaceAll(properties.getDataCenterRegion(), ""); + return new DynoShardSupplier(hostSupplier, properties.getDataCenterRegion(), localDC); } @Bean @@ -85,10 +86,11 @@ public ShardingStrategy shardingStrategy(ShardSupplier shardSupplier, RedisPrope @Bean public RedisQueues redisQueues(@Qualifier(DEFAULT_CLIENT_INJECTION_NAME) JedisCommands jedisCommands, - @Qualifier(READ_CLIENT_INJECTION_NAME) JedisCommands jedisCommandsRead, - ShardSupplier shardSupplier, RedisProperties properties, ShardingStrategy shardingStrategy) { - RedisQueues queues = new RedisQueues(jedisCommands, jedisCommandsRead, properties.getQueuePrefix(), shardSupplier, - 60_000, 60_000, shardingStrategy); + @Qualifier(READ_CLIENT_INJECTION_NAME) JedisCommands jedisCommandsRead, + ShardSupplier shardSupplier, RedisProperties properties, ShardingStrategy shardingStrategy) { + RedisQueues queues = new RedisQueues(jedisCommands, jedisCommandsRead, properties.getQueuePrefix(), + shardSupplier, + 60_000, 60_000, shardingStrategy); LOGGER.info("DynoQueueDAO initialized with prefix " + properties.getQueuePrefix() + "!"); return queues; } diff --git a/redis-persistence/src/main/java/com/netflix/conductor/redis/config/RedisProperties.java b/redis-persistence/src/main/java/com/netflix/conductor/redis/config/RedisProperties.java index bdf94b4c26..f6bc49f742 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/redis/config/RedisProperties.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/redis/config/RedisProperties.java @@ -12,140 +12,182 @@ */ package com.netflix.conductor.redis.config; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.context.annotation.Conditional; -import org.springframework.stereotype.Component; +import com.netflix.conductor.core.config.ConductorProperties; +import com.netflix.conductor.redis.dynoqueue.RedisQueuesShardingStrategyProvider; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.boot.context.properties.ConfigurationProperties; -@Component -@Conditional(AnyRedisCondition.class) +@ConfigurationProperties("conductor.redis") public class RedisProperties { - // TODO Are cluster and cluster name really different things? + private final ConductorProperties conductorProperties; - /** - * name of the stack under which the app is running. e.g. devint, testintg, staging, prod etc. - */ - @Value("${STACK:test}") - private String stack; - - @Value("${APP_ID:conductor}") - private String appId; + @Autowired + public RedisProperties(ConductorProperties conductorProperties) { + this.conductorProperties = conductorProperties; + } /** * Data center region. If hosting on Amazon the value is something like us-east-1, us-west-2 etc. */ - @Value("${EC2_REGION:us-east-1}") - private String region; + private String dataCenterRegion = "us-east-1"; /** - * Availability zone / rack. For AWS deployments, the value is something like us-east-1a, etc. + * Local rack / availability zone. For AWS deployments, the value is something like us-east-1a, etc. */ - @Value("${EC2_AVAILABILITY_ZONE:us-east-1c}") - private String availabilityZone; - - @Value("${workflow.dynomite.cluster:#{null}}") - private String cluster; - - @Value("${workflow.dynomite.cluster.name:}") - private String clusterName; + private String availabilityZone = "us-east-1c"; - @Value("${workflow.dynomite.cluster.hosts:#{null}}") - private String hosts; + /** + * The name of the redis / dynomite cluster + */ + private String clusterName = ""; - @Value("${workflow.namespace.prefix:#{null}}") - private String workflowNamespacePrefix; + /** + * Dynomite Cluster details. Format is host:port:rack separated by semicolon + */ + private String hosts = null; - @Value("${workflow.namespace.queue.prefix:#{null}}") - private String queueNamespacePrefix; + /** + * The prefix used to prepend workflow data in redis + */ + private String workflowNamespacePrefix = null; - @Value("${workflow.dyno.keyspace.domain:#{null}}") - private String keyspaceDomain; + /** + * The prefix used to prepend keys for queues in redis + */ + private String queueNamespacePrefix = null; - @Value("${workflow.dynomite.connection.maxConnsPerHost:10}") - private int maxConnectionsPerHost; + /** + * The domain name to be used in the key prefix for logical separation of workflow data and queues in a shared redis + * setup + */ + private String keyspaceDomain = null; - @Value("${queues.dynomite.nonQuorum.port:22122}") - private int queuesNonQuorumPort; + /** + * The maximum number of connections that can be managed by the connection pool on a given instance + */ + private int maxConnectionsPerHost = 10; - @Value("${workflow.dyno.queue.sharding.strategy:#{T(com.netflix.conductor.redis.dynoqueue.RedisQueuesShardingStrategyProvider).ROUND_ROBIN_STRATEGY}}") - private String queueShardingStrategy; + /** + * The read connection port to be used for connecting to dyno-queues + */ + private int queuesNonQuorumPort = 22122; /** - * the refresh time for the in-memory task definition cache + * The sharding strategy to be used for the dyno queue configuration */ - @Value("${conductor.taskdef.cache.refresh.time.seconds:60}") - private int taskDefCacheRefreshTimeSecs; + private String queueShardingStrategy = RedisQueuesShardingStrategyProvider.ROUND_ROBIN_STRATEGY; /** - * The time to live in seconds of the event execution persisted + * The time in seconds after which the in-memory task definitions cache will be refreshed */ - @Value("${workflow.event.execution.persistence.ttl.seconds:0}") - private int eventExecutionPersistenceTTLSecs; + private int taskDefCacheRefreshTimeSecs = 60; - public String getStack() { - return stack; - } + /** + * The time to live in seconds for which the event execution will be persisted + */ + private int eventExecutionPersistenceTTLSecs = 60; - public String getAppId() { - return appId; + public String getDataCenterRegion() { + return dataCenterRegion; } - public String getRegion() { - return region; + public void setDataCenterRegion(String dataCenterRegion) { + this.dataCenterRegion = dataCenterRegion; } public String getAvailabilityZone() { return availabilityZone; } - public String getCluster() { - return cluster; + public void setAvailabilityZone(String availabilityZone) { + this.availabilityZone = availabilityZone; } public String getClusterName() { return clusterName; } + public void setClusterName(String clusterName) { + this.clusterName = clusterName; + } + public String getHosts() { return hosts; } + public void setHosts(String hosts) { + this.hosts = hosts; + } + public String getWorkflowNamespacePrefix() { return workflowNamespacePrefix; } + public void setWorkflowNamespacePrefix(String workflowNamespacePrefix) { + this.workflowNamespacePrefix = workflowNamespacePrefix; + } + public String getQueueNamespacePrefix() { return queueNamespacePrefix; } - public String getDomain() { + public void setQueueNamespacePrefix(String queueNamespacePrefix) { + this.queueNamespacePrefix = queueNamespacePrefix; + } + + public String getKeyspaceDomain() { return keyspaceDomain; } + public void setKeyspaceDomain(String keyspaceDomain) { + this.keyspaceDomain = keyspaceDomain; + } + public int getMaxConnectionsPerHost() { return maxConnectionsPerHost; } - public int getNonQuorumPort() { + public void setMaxConnectionsPerHost(int maxConnectionsPerHost) { + this.maxConnectionsPerHost = maxConnectionsPerHost; + } + + public int getQueuesNonQuorumPort() { return queuesNonQuorumPort; } + public void setQueuesNonQuorumPort(int queuesNonQuorumPort) { + this.queuesNonQuorumPort = queuesNonQuorumPort; + } + public String getQueueShardingStrategy() { return queueShardingStrategy; } - public int getTaskDefRefreshTimeSecs() { + public void setQueueShardingStrategy(String queueShardingStrategy) { + this.queueShardingStrategy = queueShardingStrategy; + } + + public int getTaskDefCacheRefreshTimeSecs() { return taskDefCacheRefreshTimeSecs; } - public int getEventExecutionPersistenceTTL() { + public void setTaskDefCacheRefreshTimeSecs(int taskDefCacheRefreshTimeSecs) { + this.taskDefCacheRefreshTimeSecs = taskDefCacheRefreshTimeSecs; + } + + public int getEventExecutionPersistenceTTLSecs() { return eventExecutionPersistenceTTLSecs; } + public void setEventExecutionPersistenceTTLSecs(int eventExecutionPersistenceTTLSecs) { + this.eventExecutionPersistenceTTLSecs = eventExecutionPersistenceTTLSecs; + } + public String getQueuePrefix() { - String prefix = getQueueNamespacePrefix() + "." + getStack(); - if (getDomain() != null) { - prefix = prefix + "." + getDomain(); + String prefix = getQueueNamespacePrefix() + "." + conductorProperties.getStack(); + if (getKeyspaceDomain() != null) { + prefix = prefix + "." + getKeyspaceDomain(); } return prefix; } diff --git a/redis-persistence/src/main/java/com/netflix/conductor/redis/config/RedisSentinelConfiguration.java b/redis-persistence/src/main/java/com/netflix/conductor/redis/config/RedisSentinelConfiguration.java index 855498f58f..55de3fda34 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/redis/config/RedisSentinelConfiguration.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/redis/config/RedisSentinelConfiguration.java @@ -12,10 +12,13 @@ */ package com.netflix.conductor.redis.config; +import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.redis.jedis.JedisSentinel; import com.netflix.dyno.connectionpool.Host; import com.netflix.dyno.connectionpool.HostSupplier; import com.netflix.dyno.connectionpool.TokenMapSupplier; +import java.util.HashSet; +import java.util.Set; import org.apache.commons.pool2.impl.GenericObjectPoolConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -24,18 +27,15 @@ import redis.clients.jedis.JedisSentinelPool; import redis.clients.jedis.commands.JedisCommands; -import java.util.HashSet; -import java.util.Set; - -@SuppressWarnings("SpringJavaInjectionPointsAutowiringInspection") @Configuration(proxyBeanMethods = false) -@ConditionalOnProperty(name = "db", havingValue = "redis_sentinel") +@ConditionalOnProperty(name = "conductor.db.type", havingValue = "redis_sentinel") public class RedisSentinelConfiguration extends JedisCommandsConfigurer { private static final Logger log = LoggerFactory.getLogger(RedisSentinelConfiguration.class); @Override - protected JedisCommands createJedisCommands(RedisProperties properties, HostSupplier hostSupplier, TokenMapSupplier tokenMapSupplier) { + protected JedisCommands createJedisCommands(RedisProperties properties, ConductorProperties conductorProperties, + HostSupplier hostSupplier, TokenMapSupplier tokenMapSupplier) { GenericObjectPoolConfig genericObjectPoolConfig = new GenericObjectPoolConfig(); genericObjectPoolConfig.setMinIdle(5); genericObjectPoolConfig.setMaxTotal(properties.getMaxConnectionsPerHost()); @@ -45,6 +45,6 @@ protected JedisCommands createJedisCommands(RedisProperties properties, HostSupp sentinels.add(host.getHostName() + ":" + host.getPort()); } return new JedisSentinel( - new JedisSentinelPool(properties.getClusterName(), sentinels, genericObjectPoolConfig)); + new JedisSentinelPool(properties.getClusterName(), sentinels, genericObjectPoolConfig)); } } diff --git a/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/BaseDynoDAO.java b/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/BaseDynoDAO.java index 2dcef36852..4421eacd70 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/BaseDynoDAO.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/BaseDynoDAO.java @@ -14,12 +14,12 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.metrics.Monitors; import com.netflix.conductor.redis.config.RedisProperties; import com.netflix.conductor.redis.jedis.JedisProxy; -import org.apache.commons.lang3.StringUtils; - import java.io.IOException; +import org.apache.commons.lang3.StringUtils; public class BaseDynoDAO { @@ -27,14 +27,17 @@ public class BaseDynoDAO { private static final String DAO_NAME = "redis"; private final String domain; private final RedisProperties properties; + private final ConductorProperties conductorProperties; protected JedisProxy jedisProxy; protected ObjectMapper objectMapper; - protected BaseDynoDAO(JedisProxy jedisProxy, ObjectMapper objectMapper, RedisProperties properties) { + protected BaseDynoDAO(JedisProxy jedisProxy, ObjectMapper objectMapper, + ConductorProperties conductorProperties, RedisProperties properties) { this.jedisProxy = jedisProxy; this.objectMapper = objectMapper; + this.conductorProperties = conductorProperties; this.properties = properties; - this.domain = properties.getDomain(); + this.domain = properties.getKeyspaceDomain(); } String nsKey(String... nsValues) { @@ -43,7 +46,7 @@ String nsKey(String... nsValues) { if (StringUtils.isNotBlank(rootNamespace)) { namespacedKey.append(rootNamespace).append(NAMESPACE_SEP); } - String stack = properties.getStack(); + String stack = conductorProperties.getStack(); if (StringUtils.isNotBlank(stack)) { namespacedKey.append(stack).append(NAMESPACE_SEP); } diff --git a/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisEventHandlerDAO.java b/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisEventHandlerDAO.java index 02a0a58609..0581e89057 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisEventHandlerDAO.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisEventHandlerDAO.java @@ -15,21 +15,21 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.netflix.conductor.common.metadata.events.EventHandler; +import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.core.exception.ApplicationException; import com.netflix.conductor.core.exception.ApplicationException.Code; import com.netflix.conductor.dao.EventHandlerDAO; import com.netflix.conductor.redis.config.AnyRedisCondition; import com.netflix.conductor.redis.config.RedisProperties; import com.netflix.conductor.redis.jedis.JedisProxy; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.springframework.context.annotation.Conditional; -import org.springframework.stereotype.Component; - import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.context.annotation.Conditional; +import org.springframework.stereotype.Component; @Component @Conditional(AnyRedisCondition.class) @@ -40,8 +40,9 @@ public class RedisEventHandlerDAO extends BaseDynoDAO implements EventHandlerDAO private final static String EVENT_HANDLERS = "EVENT_HANDLERS"; private final static String EVENT_HANDLERS_BY_EVENT = "EVENT_HANDLERS_BY_EVENT"; - public RedisEventHandlerDAO(JedisProxy jedisProxy, ObjectMapper objectMapper, RedisProperties properties) { - super(jedisProxy, objectMapper, properties); + public RedisEventHandlerDAO(JedisProxy jedisProxy, ObjectMapper objectMapper, + ConductorProperties conductorProperties, RedisProperties properties) { + super(jedisProxy, objectMapper, conductorProperties, properties); } @Override diff --git a/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisExecutionDAO.java b/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisExecutionDAO.java index ce90d40955..5ed98e46b2 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisExecutionDAO.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisExecutionDAO.java @@ -20,6 +20,7 @@ import com.netflix.conductor.common.metadata.tasks.Task.Status; import com.netflix.conductor.common.metadata.tasks.TaskDef; import com.netflix.conductor.common.run.Workflow; +import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.core.exception.ApplicationException; import com.netflix.conductor.core.exception.ApplicationException.Code; import com.netflix.conductor.dao.ExecutionDAO; @@ -65,10 +66,11 @@ public class RedisExecutionDAO extends BaseDynoDAO implements ExecutionDAO { private final static String EVENT_EXECUTION = "EVENT_EXECUTION"; private final int ttlEventExecutionSeconds; - public RedisExecutionDAO(JedisProxy jedisProxy, ObjectMapper objectMapper, RedisProperties properties) { - super(jedisProxy, objectMapper, properties); + public RedisExecutionDAO(JedisProxy jedisProxy, ObjectMapper objectMapper, ConductorProperties conductorProperties, + RedisProperties properties) { + super(jedisProxy, objectMapper, conductorProperties, properties); - ttlEventExecutionSeconds = properties.getEventExecutionPersistenceTTL(); + ttlEventExecutionSeconds = properties.getEventExecutionPersistenceTTLSecs(); } private static String dateStr(Long timeInMs) { diff --git a/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisMetadataDAO.java b/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisMetadataDAO.java index 5c2e26a305..94cc56bd6f 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisMetadataDAO.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisMetadataDAO.java @@ -16,6 +16,7 @@ import com.google.common.base.Preconditions; import com.netflix.conductor.common.metadata.tasks.TaskDef; import com.netflix.conductor.common.metadata.workflow.WorkflowDef; +import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.core.exception.ApplicationException; import com.netflix.conductor.core.exception.ApplicationException.Code; import com.netflix.conductor.dao.MetadataDAO; @@ -54,10 +55,11 @@ public class RedisMetadataDAO extends BaseDynoDAO implements MetadataDAO { private static final String className = RedisMetadataDAO.class.getSimpleName(); private Map taskDefCache = new HashMap<>(); - public RedisMetadataDAO(JedisProxy jedisProxy, ObjectMapper objectMapper, RedisProperties properties) { - super(jedisProxy, objectMapper, properties); + public RedisMetadataDAO(JedisProxy jedisProxy, ObjectMapper objectMapper, + ConductorProperties conductorProperties, RedisProperties properties) { + super(jedisProxy, objectMapper, conductorProperties, properties); refreshTaskDefs(); - int cacheRefreshTime = properties.getTaskDefRefreshTimeSecs(); + int cacheRefreshTime = properties.getTaskDefCacheRefreshTimeSecs(); Executors.newSingleThreadScheduledExecutor() .scheduleWithFixedDelay(this::refreshTaskDefs, cacheRefreshTime, cacheRefreshTime, TimeUnit.SECONDS); } diff --git a/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisPollDataDAO.java b/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisPollDataDAO.java index a09b4255c2..e2d62070f1 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisPollDataDAO.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisPollDataDAO.java @@ -15,6 +15,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.netflix.conductor.common.metadata.tasks.PollData; +import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.dao.PollDataDAO; import com.netflix.conductor.redis.jedis.JedisProxy; import com.netflix.conductor.redis.config.RedisProperties; @@ -33,8 +34,9 @@ public class RedisPollDataDAO extends BaseDynoDAO implements PollDataDAO { private final static String POLL_DATA = "POLL_DATA"; - public RedisPollDataDAO(JedisProxy jedisProxy, ObjectMapper objectMapper, RedisProperties properties) { - super(jedisProxy, objectMapper, properties); + public RedisPollDataDAO(JedisProxy jedisProxy, ObjectMapper objectMapper, + ConductorProperties conductorProperties, RedisProperties properties) { + super(jedisProxy, objectMapper, conductorProperties, properties); } @Override diff --git a/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisRateLimitingDAO.java b/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisRateLimitingDAO.java index 8ee997da17..0aff046c57 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisRateLimitingDAO.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/redis/dao/RedisRateLimitingDAO.java @@ -15,6 +15,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.netflix.conductor.common.metadata.tasks.Task; import com.netflix.conductor.common.metadata.tasks.TaskDef; +import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.dao.RateLimitingDAO; import com.netflix.conductor.metrics.Monitors; import com.netflix.conductor.redis.jedis.JedisProxy; @@ -36,8 +37,9 @@ public class RedisRateLimitingDAO extends BaseDynoDAO implements RateLimitingDAO private static final String TASK_RATE_LIMIT_BUCKET = "TASK_RATE_LIMIT_BUCKET"; - public RedisRateLimitingDAO(JedisProxy jedisProxy, ObjectMapper objectMapper, RedisProperties properties) { - super(jedisProxy, objectMapper, properties); + public RedisRateLimitingDAO(JedisProxy jedisProxy, ObjectMapper objectMapper, + ConductorProperties conductorProperties, RedisProperties properties) { + super(jedisProxy, objectMapper, conductorProperties, properties); } /** diff --git a/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/BaseDynoDAOTest.java b/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/BaseDynoDAOTest.java index 04346f99c6..fbde924ab7 100644 --- a/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/BaseDynoDAOTest.java +++ b/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/BaseDynoDAOTest.java @@ -13,6 +13,7 @@ package com.netflix.conductor.redis.dao; import com.fasterxml.jackson.databind.ObjectMapper; +import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.redis.jedis.JedisProxy; import com.netflix.conductor.redis.config.RedisProperties; import org.junit.Before; @@ -32,13 +33,15 @@ public class BaseDynoDAOTest { private ObjectMapper objectMapper; private RedisProperties properties; + private ConductorProperties conductorProperties; private BaseDynoDAO baseDynoDAO; @Before public void setUp() { properties = mock(RedisProperties.class); - this.baseDynoDAO = new BaseDynoDAO(jedisProxy, objectMapper, properties); + conductorProperties = mock(ConductorProperties.class); + this.baseDynoDAO = new BaseDynoDAO(jedisProxy, objectMapper, conductorProperties, properties); } @Test @@ -53,7 +56,7 @@ public void testNsKey() { assertEquals("test.key1.key2", baseDynoDAO.nsKey(keys)); - when(properties.getStack()).thenReturn("stack"); + when(conductorProperties.getStack()).thenReturn("stack"); assertEquals("test.stack.key1.key2", baseDynoDAO.nsKey(keys)); } } diff --git a/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisEventHandlerDAOTest.java b/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisEventHandlerDAOTest.java index 57337a53ef..f720f7d93d 100644 --- a/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisEventHandlerDAOTest.java +++ b/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisEventHandlerDAOTest.java @@ -12,15 +12,22 @@ */ package com.netflix.conductor.redis.dao; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.mockito.Mockito.mock; + import com.fasterxml.jackson.databind.ObjectMapper; import com.netflix.conductor.common.config.ObjectMapperConfiguration; import com.netflix.conductor.common.metadata.events.EventHandler; import com.netflix.conductor.common.metadata.events.EventHandler.Action; import com.netflix.conductor.common.metadata.events.EventHandler.Action.Type; import com.netflix.conductor.common.metadata.events.EventHandler.StartWorkflow; -import com.netflix.conductor.redis.jedis.JedisProxy; +import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.redis.config.RedisProperties; import com.netflix.conductor.redis.jedis.JedisMock; +import com.netflix.conductor.redis.jedis.JedisProxy; +import java.util.List; +import java.util.UUID; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -29,13 +36,6 @@ import org.springframework.test.context.junit4.SpringRunner; import redis.clients.jedis.commands.JedisCommands; -import java.util.List; -import java.util.UUID; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.mockito.Mockito.mock; - @ContextConfiguration(classes = {ObjectMapperConfiguration.class}) @RunWith(SpringRunner.class) public class RedisEventHandlerDAOTest { @@ -47,11 +47,12 @@ public class RedisEventHandlerDAOTest { @Before public void init() { + ConductorProperties conductorProperties = mock(ConductorProperties.class); RedisProperties properties = mock(RedisProperties.class); JedisCommands jedisMock = new JedisMock(); JedisProxy jedisProxy = new JedisProxy(jedisMock); - redisEventHandlerDAO = new RedisEventHandlerDAO(jedisProxy, objectMapper, properties); + redisEventHandlerDAO = new RedisEventHandlerDAO(jedisProxy, objectMapper, conductorProperties, properties); } @Test diff --git a/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisExecutionDAOTest.java b/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisExecutionDAOTest.java index 70a85bb9bc..1132d735ad 100644 --- a/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisExecutionDAOTest.java +++ b/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisExecutionDAOTest.java @@ -17,6 +17,7 @@ import com.netflix.conductor.common.metadata.tasks.Task; import com.netflix.conductor.common.metadata.tasks.Task.Status; import com.netflix.conductor.common.metadata.tasks.TaskDef; +import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.dao.ExecutionDAO; import com.netflix.conductor.dao.ExecutionDAOTest; import com.netflix.conductor.redis.jedis.JedisProxy; @@ -48,11 +49,12 @@ public class RedisExecutionDAOTest extends ExecutionDAOTest { @Before public void init() { + ConductorProperties conductorProperties = mock(ConductorProperties.class); RedisProperties properties = mock(RedisProperties.class); JedisCommands jedisMock = new JedisMock(); JedisProxy jedisProxy = new JedisProxy(jedisMock); - executionDAO = new RedisExecutionDAO(jedisProxy, objectMapper, properties); + executionDAO = new RedisExecutionDAO(jedisProxy, objectMapper, conductorProperties, properties); } @Test diff --git a/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisMetadataDAOTest.java b/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisMetadataDAOTest.java index f58d54f829..7a62d4d938 100644 --- a/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisMetadataDAOTest.java +++ b/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisMetadataDAOTest.java @@ -18,6 +18,7 @@ import com.netflix.conductor.common.metadata.tasks.TaskDef.RetryLogic; import com.netflix.conductor.common.metadata.tasks.TaskDef.TimeoutPolicy; import com.netflix.conductor.common.metadata.workflow.WorkflowDef; +import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.core.exception.ApplicationException; import com.netflix.conductor.redis.jedis.JedisProxy; import com.netflix.conductor.redis.config.RedisProperties; @@ -54,12 +55,13 @@ public class RedisMetadataDAOTest { @Before public void init() { + ConductorProperties conductorProperties = mock(ConductorProperties.class); RedisProperties properties = mock(RedisProperties.class); - when(properties.getTaskDefRefreshTimeSecs()).thenReturn(60); + when(properties.getTaskDefCacheRefreshTimeSecs()).thenReturn(60); JedisCommands jedisMock = new JedisMock(); JedisProxy jedisProxy = new JedisProxy(jedisMock); - redisMetadataDAO = new RedisMetadataDAO(jedisProxy, objectMapper, properties); + redisMetadataDAO = new RedisMetadataDAO(jedisProxy, objectMapper, conductorProperties, properties); } @Test(expected = ApplicationException.class) diff --git a/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisPollDataDAOTest.java b/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisPollDataDAOTest.java index 5d1fdf6d53..553d7a62b7 100644 --- a/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisPollDataDAOTest.java +++ b/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisPollDataDAOTest.java @@ -12,13 +12,16 @@ */ package com.netflix.conductor.redis.dao; +import static org.mockito.Mockito.mock; + import com.fasterxml.jackson.databind.ObjectMapper; import com.netflix.conductor.common.config.ObjectMapperConfiguration; +import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.dao.PollDataDAO; import com.netflix.conductor.dao.PollDataDAOTest; -import com.netflix.conductor.redis.jedis.JedisProxy; import com.netflix.conductor.redis.config.RedisProperties; import com.netflix.conductor.redis.jedis.JedisMock; +import com.netflix.conductor.redis.jedis.JedisProxy; import org.junit.Before; import org.junit.runner.RunWith; import org.springframework.beans.factory.annotation.Autowired; @@ -26,8 +29,6 @@ import org.springframework.test.context.junit4.SpringRunner; import redis.clients.jedis.commands.JedisCommands; -import static org.mockito.Mockito.mock; - @ContextConfiguration(classes = {ObjectMapperConfiguration.class}) @RunWith(SpringRunner.class) public class RedisPollDataDAOTest extends PollDataDAOTest { @@ -39,11 +40,12 @@ public class RedisPollDataDAOTest extends PollDataDAOTest { @Before public void init() { + ConductorProperties conductorProperties = mock(ConductorProperties.class); RedisProperties properties = mock(RedisProperties.class); JedisCommands jedisMock = new JedisMock(); JedisProxy jedisProxy = new JedisProxy(jedisMock); - redisPollDataDAO = new RedisPollDataDAO(jedisProxy, objectMapper, properties); + redisPollDataDAO = new RedisPollDataDAO(jedisProxy, objectMapper, conductorProperties, properties); } @Override diff --git a/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisRateLimitDAOTest.java b/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisRateLimitDAOTest.java index 285f86c8dc..40c55db7d6 100644 --- a/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisRateLimitDAOTest.java +++ b/redis-persistence/src/test/java/com/netflix/conductor/redis/dao/RedisRateLimitDAOTest.java @@ -16,6 +16,7 @@ import com.netflix.conductor.common.config.ObjectMapperConfiguration; import com.netflix.conductor.common.metadata.tasks.Task; import com.netflix.conductor.common.metadata.tasks.TaskDef; +import com.netflix.conductor.core.config.ConductorProperties; import com.netflix.conductor.redis.jedis.JedisProxy; import com.netflix.conductor.redis.config.RedisProperties; import com.netflix.conductor.redis.jedis.JedisMock; @@ -44,11 +45,12 @@ public class RedisRateLimitDAOTest { @Before public void init() { + ConductorProperties conductorProperties = mock(ConductorProperties.class); RedisProperties properties = mock(RedisProperties.class); JedisCommands jedisMock = new JedisMock(); JedisProxy jedisProxy = new JedisProxy(jedisMock); - rateLimitingDao = new RedisRateLimitingDAO(jedisProxy, objectMapper, properties); + rateLimitingDao = new RedisRateLimitingDAO(jedisProxy, objectMapper, conductorProperties, properties); } @Test diff --git a/rest/src/main/java/com/netflix/conductor/rest/config/RestConfiguration.java b/rest/src/main/java/com/netflix/conductor/rest/config/RestConfiguration.java index 5a3334cdad..b3326af067 100644 --- a/rest/src/main/java/com/netflix/conductor/rest/config/RestConfiguration.java +++ b/rest/src/main/java/com/netflix/conductor/rest/config/RestConfiguration.java @@ -1,3 +1,15 @@ +/* + * Copyright 2020 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package com.netflix.conductor.rest.config; import org.springframework.context.annotation.Configuration; diff --git a/rest/src/main/java/com/netflix/conductor/rest/startup/KitchenSinkInitializer.java b/rest/src/main/java/com/netflix/conductor/rest/startup/KitchenSinkInitializer.java index 8356adc899..52bdf1749d 100644 --- a/rest/src/main/java/com/netflix/conductor/rest/startup/KitchenSinkInitializer.java +++ b/rest/src/main/java/com/netflix/conductor/rest/startup/KitchenSinkInitializer.java @@ -1,3 +1,18 @@ +/** + * Copyright 2020 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.conductor.rest.startup; import com.netflix.conductor.common.config.ObjectMapperProvider; @@ -30,7 +45,7 @@ @Component public class KitchenSinkInitializer { - private static final Logger log = LoggerFactory.getLogger(KitchenSinkInitializer.class); + private static final Logger LOGGER = LoggerFactory.getLogger(KitchenSinkInitializer.class); private final RestTemplate restTemplate; @@ -62,11 +77,11 @@ public KitchenSinkInitializer(RestTemplateBuilder restTemplateBuilder) { public void setupKitchenSink() { try { if (loadSamples) { - log.info("Loading Kitchen Sink examples"); + LOGGER.info("Loading Kitchen Sink examples"); createKitchenSink(); } } catch (Exception e) { - log.error("Error initializing kitchen sink", e); + LOGGER.error("Error initializing kitchen sink", e); } } @@ -97,21 +112,21 @@ private void createKitchenSink() throws Exception { restTemplate.postForEntity(url("/api/metadata/workflow/"), request, Map.class); restTemplate.postForEntity(url("/api/workflow/kitchensink"), Collections.singletonMap("task2Name", "task_5"), String.class); - log.info("Kitchen sink workflow is created!"); + LOGGER.info("Kitchen sink workflow is created!"); /* * Kitchensink example with ephemeral workflow and stored tasks */ request = new HttpEntity<>(readToString(ephemeralWorkflowWithStoredTasks), headers); restTemplate.postForEntity(url("/api/workflow/"), request, String.class); - log.info("Ephemeral Kitchen sink workflow with stored tasks is created!"); + LOGGER.info("Ephemeral Kitchen sink workflow with stored tasks is created!"); /* * Kitchensink example with ephemeral workflow and ephemeral tasks */ request = new HttpEntity<>(readToString(ephemeralWorkflowWithEphemeralTasks), headers); restTemplate.postForEntity(url("/api/workflow/"), request, String.class); - log.info("Ephemeral Kitchen sink workflow with ephemeral tasks is created!"); + LOGGER.info("Ephemeral Kitchen sink workflow with ephemeral tasks is created!"); } private String readToString(Resource resource) throws IOException { diff --git a/rest/src/main/resources/static/index.html b/rest/src/main/resources/static/index.html index 67c78540c2..086d79a279 100644 --- a/rest/src/main/resources/static/index.html +++ b/rest/src/main/resources/static/index.html @@ -1,3 +1,20 @@ + diff --git a/server/src/main/resources/META-INF/additional-spring-configuration-metadata.json b/server/src/main/resources/META-INF/additional-spring-configuration-metadata.json new file mode 100644 index 0000000000..e5d9b2fc77 --- /dev/null +++ b/server/src/main/resources/META-INF/additional-spring-configuration-metadata.json @@ -0,0 +1,172 @@ +{ + "properties": [ + { + "name": "conductor.db.type", + "type": "java.lang.String", + "description": "The type of database to be used while running the Conductor application." + }, + { + "name": "conductor.indexing.enabled", + "type": "java.lang.Boolean", + "description": "Enable indexing to elasticsearch. If set to false, a no-op implementation will be used." + }, + { + "name": "conductor.grpc-server.enabled", + "type": "java.lang.Boolean", + "description": "Enable the gRPC server." + }, + { + "name": "conductor.external-payload-storage.type", + "type": "java.lang.String", + "description": "The type of payload storage to be used for externalizing large payloads." + }, + { + "name": "conductor.default-event-processor.enabled", + "type": "java.lang.Boolean", + "description": "If set to true, use the provided default event processor for handling events." + }, + { + "name": "conductor.default-event-queue.type", + "type": "java.lang.String", + "description": "The default event queue type to listen on for the WAIT task." + }, + { + "name": "conductor.workflow-status-listener.type", + "type": "java.lang.String", + "description": "The implementation of the workflow status listener to be used." + }, + { + "name": "conductor.workflow-execution-lock.type", + "type": "java.lang.String", + "description": "The implementation of the workflow execution lock to be used." + }, + { + "name": "conductor.event-queues.default.enabled", + "type": "java.lang.Boolean", + "description": "Enable the use of the underlying queue implementation to provide queues for consuming events." + }, + { + "name": "conductor.event-queues.sqs.enabled", + "type": "java.lang.Boolean", + "description": "Enable the use of AWS SQS implementation to provide queues for consuming events." + }, + { + "name": "conductor.event-queues.amqp.enabled", + "type": "java.lang.Boolean", + "description": "Enable the use of RabbitMQ implementation to provide queues for consuming events." + }, + { + "name": "conductor.event-queues.nats.enabled", + "type": "java.lang.Boolean", + "description": "Enable the use of NATS implementation to provide queues for consuming events." + }, + { + "name": "conductor.event-queues.nats-stream.enabled", + "type": "java.lang.Boolean", + "description": "Enable the use of NATS Streaming implementation to provide queues for consuming events." + } + ], + "hints": [ + { + "name": "conductor.db.type", + "values": [ + { + "value": "memory", + "description": "Use in-memory redis as the database implementation." + }, + { + "value": "cassandra", + "description": "Use cassandra as the database implementation." + }, + { + "value": "mysql", + "description": "Use MySQL as the database implementation." + }, + { + "value": "postgres", + "description": "Use Postgres as the database implementation." + }, + { + "value": "dynomite", + "description": "Use Dynomite as the database implementation." + }, + { + "value": "redis_cluster", + "description": "Use Redis Cluster configuration as the database implementation." + }, + { + "value": "redis_sentinel", + "description": "Use Redis Sentinel configuration as the database implementation." + } + ] + }, + { + "name": "conductor.external-payload-storage.type", + "values": [ + { + "value": "dummy", + "description": "Use the dummy no-op implementation as the external payload storage." + }, + { + "value": "azureblob", + "description": "Use Azure Blob as the external payload storage." + }, + { + "value": "s3", + "description": "Use AWS S3 as the external payload storage." + } + ] + }, + { + "name": "conductor.default-event-queue.type", + "values": [ + { + "value": "sqs", + "description": "Use AWS SQS as the event queue to listen on for the WAIT task." + }, + { + "value": "amqp", + "description": "Use RabbitMQ as the event queue to listen on for the WAIT task." + } + ] + }, + { + "name": "conductor.workflow-status-listener.type", + "values": [ + { + "value": "stub", + "description": "Use the no-op implementation of the workflow status listener." + }, + { + "value": "archive", + "description": "Use then archive implementation which immediately archives the workflow upon termination or completion as the workflow status listener." + }, + { + "value": "queue_publisher", + "description": "Use the publisher implementation which publishes a message to the underlying queue implementation upon termination or completion as the workflow status listener." + } + ] + }, + { + "name": "conductor.workflow-execution-lock.type", + "values": [ + { + "value": "noop_lock", + "description": "Use the no-op implementation as the lock provider." + }, + { + "value": "local_only", + "description": "Use the local in-memory cache based implementation as the lock provider." + }, + { + "value": "redis", + "description": "Use the redis-lock implementation as the lock provider." + }, + { + "value": "zookeeper", + "description": "Use the zookeeper-lock implementation as the lock provider." + } + ] + } + ] +} diff --git a/server/src/main/resources/application.properties b/server/src/main/resources/application.properties index 5969949542..ecfb151f51 100644 --- a/server/src/main/resources/application.properties +++ b/server/src/main/resources/application.properties @@ -17,87 +17,85 @@ spring.application.name=conductor springdoc.api-docs.path=/api-docs -db=memory +conductor.db.type=memory -workflow.indexing.enabled=false +conductor.indexing.enabled=false #Dynomite Cluster details. #format is host:port:rack separated by semicolon -workflow.dynomite.cluster.hosts=host1:port:rack;host2:port:rack:host3:port:rack +conductor.redis.hosts=host1:port:rack;host2:port:rack:host3:port:rack #namespace for the keys stored in Dynomite/Redis -workflow.namespace.prefix= +conductor.redis.workflowNamespacePrefix= #namespace prefix for the dyno queues -workflow.namespace.queue.prefix= +conductor.redis.queueNamespacePrefix= #no. of threads allocated to dyno-queues queues.dynomite.threads=10 # By default with dynomite, we want the repairservice enabled -workflow.repairservice.enabled=true +conductor.app.workflowRepairServiceEnabled=true #non-quorum port used to connect to local redis. Used by dyno-queues -queues.dynomite.nonQuorum.port=22122 +conductor.redis.queuesNonQuorumPort=22122 + +# For a single node dynomite or redis server, make sure the value below is set to same as rack specified in the "workflow.dynomite.cluster.hosts" property. +conductor.redis.availabilityZone=us-east-1c #Transport address to elasticsearch -workflow.elasticsearch.url=localhost:9300 +conductor.elasticsearch.url=localhost:9300 #Name of the elasticsearch cluster -workflow.elasticsearch.index.name=conductor +conductor.elasticsearch.indexName=conductor #Elasticsearch major release version. -workflow.elasticsearch.version=6 - -# For a single node dynomite or redis server, make sure the value below is set to same as rack specified in the "workflow.dynomite.cluster.hosts" property. -EC2_AVAILABILITY_ZONE=us-east-1c +conductor.elasticsearch.version=6 # Default event queue type to listen on for wait task -workflow.events.default.queue.type=sqs +conductor.default-event-queue.type=sqs #zookeeper -# workflow.zookeeper.lock.connection=host1.2181,host2:2181,host3:2181 -# workflow.zookeeper.lock.sessionTimeoutMs -# workflow.zookeeper.lock.connectionTimeoutMs +# conductor.zookeeper-lock.connectionString=host1.2181,host2:2181,host3:2181 +# conductor.zookeeper-lock.sessionTimeoutMs +# conductor.zookeeper-lock.connectionTimeoutMs +# conductor.zookeeper-lock.namespace -#enable locking during workflow execution -workflow.decider.locking.enabled=false -workflow.decider.locking.namespace= -workflow.decider.locking.server=noop_lock -workflow.decider.locking.leaseTimeInSeconds=60 +#disable locking during workflow execution +conductor.app.workflowExecutionLockEnabled=false +conductor.workflow-execution-lock.type=noop_lock #Redis cluster settings for locking module -# workflow.redis.locking.server.type=single +# conductor.redis-lock.serverType=single #Comma separated list of server nodes -# workflow.redis.locking.server.address=redis://127.0.0.1:6379 +# conductor.redis-lock.serverAddress=redis://127.0.0.1:6379 #Redis sentinel master name -# workflow.redis.locking.server.master.name=master +# conductor.redis-lock.serverMasterName=master +# conductor.redis-lock.namespace #Following properties set for using AMQP events and tasks with conductor: - -#conductor.additional.modules=com.netflix.conductor.contribs.AMQPModule(You must add module AMQModule to enable support of AMQP queues) +#(To enable support of AMQP queues) +#conductor.event-queues.amqp.enabled=true # Here are the settings with default values: -#workflow.event.queues.amqp.hosts= -#workflow.event.queues.amqp.username= -#workflow.event.queues.amqp.password= - - +#conductor.event-queues.amqp.hosts= +#conductor.event-queues.amqp.username= +#conductor.event-queues.amqp.password= -#workflow.event.queues.amqp.virtualHost=/ -#workflow.event.queues.amqp.port=5672 -#workflow.event.queues.amqp.useNio=false -#workflow.event.queues.amqp.batchSize=1 +#conductor.event-queues.amqp.virtualHost=/ +#conductor.event-queues.amqp.port.port=5672 +#conductor.event-queues.amqp.useNio=false +#conductor.event-queues.amqp.batchSize=1 -#workflow.event.queues.amqp.pollTimeInMs=100 +#conductor.event-queues.amqp.pollTimeMs=100 -#workflow.listener.queue.useExchange=true( exchange or queue) -#workflow.listener.queue.prefix=myqueue +#conductor.event-queues.amqp.useExchange=true( exchange or queue) +#conductor.event-queues.amqp.listenerQueuePrefix=myqueue # Use durable queue ? -#workflow.event.queues.amqp.durable=false +#conductor.event-queues.amqp.durable=false # Use exclusive queue ? -#workflow.event.queues.amqp.exclusive=false +#conductor.event-queues.amqp.exclusivee=false # Enable support of priorities on queue. Set the max priority on message. # Setting is ignored if the value is lower or equals to 0 -# workflow.event.queues.amqp.maxPriority=-1 +#conductor.event-queues.amqp.maxPriority=-1 diff --git a/server/src/main/resources/log4j2.xml b/server/src/main/resources/log4j2.xml index 55f7f42cff..cab346657d 100644 --- a/server/src/main/resources/log4j2.xml +++ b/server/src/main/resources/log4j2.xml @@ -1,3 +1,20 @@ + diff --git a/test-harness/.docker/mysqldb/docker-entrypoint-initdb.d/create-db.sql b/test-harness/.docker/mysqldb/docker-entrypoint-initdb.d/create-db.sql deleted file mode 100644 index de636fdc3f..0000000000 --- a/test-harness/.docker/mysqldb/docker-entrypoint-initdb.d/create-db.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE DATABASE IF NOT EXISTS conductor; \ No newline at end of file diff --git a/test-harness/.docker/postgresdb/docker-entrypoint-initdb.d/create-db.sql b/test-harness/.docker/postgresdb/docker-entrypoint-initdb.d/create-db.sql deleted file mode 100644 index 7cfa063932..0000000000 --- a/test-harness/.docker/postgresdb/docker-entrypoint-initdb.d/create-db.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE DATABASE conductor; \ No newline at end of file diff --git a/test-harness/build.gradle b/test-harness/build.gradle index 8d3646d1b2..d0ba2ad548 100644 --- a/test-harness/build.gradle +++ b/test-harness/build.gradle @@ -1,20 +1,6 @@ -buildscript { - - repositories { - jcenter() - } - - dependencies { - classpath "com.avast.gradle:gradle-docker-compose-plugin:${revDockerCompose}" - } -} - -apply plugin: 'docker-compose' apply plugin: 'groovy' dependencies { - - testImplementation project(':conductor-server') testImplementation project(':conductor-common') testImplementation project(':conductor-rest') diff --git a/test-harness/src/test/groovy/com/netflix/conductor/test/base/AbstractResiliencySpecification.groovy b/test-harness/src/test/groovy/com/netflix/conductor/test/base/AbstractResiliencySpecification.groovy index cc3abb78ce..f1e148b7f6 100644 --- a/test-harness/src/test/groovy/com/netflix/conductor/test/base/AbstractResiliencySpecification.groovy +++ b/test-harness/src/test/groovy/com/netflix/conductor/test/base/AbstractResiliencySpecification.groovy @@ -28,8 +28,8 @@ import redis.clients.jedis.commands.JedisCommands import spock.mock.DetachedMockFactory @TestPropertySource(properties = [ - "conductor.disable.async.workers=true", - "queue.spy.enabled=true" + "conductor.app.systemTaskWorkersDisabled=true", + "conductor.integ-test.queueSpyEnabled=true" ]) abstract class AbstractResiliencySpecification extends AbstractSpecification { @@ -38,7 +38,7 @@ abstract class AbstractResiliencySpecification extends AbstractSpecification { @Primary @Bean - @ConditionalOnProperty(name = "queue.spy.enabled", havingValue = "true") + @ConditionalOnProperty(name = "conductor.integ-test.queueSpyEnabled", havingValue = "true") QueueDAO SpyQueueDAO() { DetachedMockFactory detachedMockFactory = new DetachedMockFactory() JedisCommands jedisMock = new JedisMock() diff --git a/test-harness/src/test/groovy/com/netflix/conductor/test/integration/ForkJoinSpec.groovy b/test-harness/src/test/groovy/com/netflix/conductor/test/integration/ForkJoinSpec.groovy index e3c3fd436c..5dcf7529b8 100644 --- a/test-harness/src/test/groovy/com/netflix/conductor/test/integration/ForkJoinSpec.groovy +++ b/test-harness/src/test/groovy/com/netflix/conductor/test/integration/ForkJoinSpec.groovy @@ -1,3 +1,15 @@ +/* + * Copyright 2020 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package com.netflix.conductor.test.integration import com.netflix.conductor.common.metadata.tasks.Task @@ -512,7 +524,7 @@ class ForkJoinSpec extends AbstractSpecification { 'fork_join_nested_test', input, null, null, null) - then: "The workflow is in the running template" + then: "The workflow is in the running state" with(workflowExecutionService.getExecutionStatus(workflowInstanceId, true)) { status == Workflow.WorkflowStatus.RUNNING tasks.size() == 8 diff --git a/test-harness/src/test/groovy/com/netflix/conductor/test/integration/KafkaTaskSpec.groovy b/test-harness/src/test/groovy/com/netflix/conductor/test/integration/KafkaPublishTaskSpec.groovy similarity index 91% rename from test-harness/src/test/groovy/com/netflix/conductor/test/integration/KafkaTaskSpec.groovy rename to test-harness/src/test/groovy/com/netflix/conductor/test/integration/KafkaPublishTaskSpec.groovy index 6db0826861..9539c342d8 100644 --- a/test-harness/src/test/groovy/com/netflix/conductor/test/integration/KafkaTaskSpec.groovy +++ b/test-harness/src/test/groovy/com/netflix/conductor/test/integration/KafkaPublishTaskSpec.groovy @@ -1,3 +1,15 @@ +/* + * Copyright 2020 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package com.netflix.conductor.test.integration import com.fasterxml.jackson.databind.ObjectMapper @@ -11,7 +23,7 @@ import com.netflix.conductor.test.base.AbstractSpecification import org.springframework.beans.factory.annotation.Autowired import spock.lang.Shared -class KafkaTaskSpec extends AbstractSpecification { +class KafkaPublishTaskSpec extends AbstractSpecification { @Autowired ObjectMapper objectMapper diff --git a/test-harness/src/test/groovy/com/netflix/conductor/test/integration/SystemTaskSpec.groovy b/test-harness/src/test/groovy/com/netflix/conductor/test/integration/SystemTaskSpec.groovy index 5e9b292e95..e16ee3dc4b 100644 --- a/test-harness/src/test/groovy/com/netflix/conductor/test/integration/SystemTaskSpec.groovy +++ b/test-harness/src/test/groovy/com/netflix/conductor/test/integration/SystemTaskSpec.groovy @@ -17,10 +17,10 @@ import com.netflix.conductor.common.metadata.tasks.TaskResult import com.netflix.conductor.common.run.Workflow import com.netflix.conductor.core.execution.WorkflowRepairService import com.netflix.conductor.core.execution.WorkflowSweeper -import com.netflix.conductor.core.execution.tasks.SystemTaskWorkerCoordinator import com.netflix.conductor.core.execution.tasks.WorkflowSystemTask import com.netflix.conductor.dao.QueueDAO import com.netflix.conductor.test.base.AbstractSpecification +import com.netflix.conductor.test.utils.UserTask import org.springframework.beans.factory.annotation.Autowired import spock.lang.Shared @@ -86,10 +86,10 @@ class SystemTaskSpec extends AbstractSpecification { when: "the system task is started by issuing a system task call" List polledTaskIds = queueDAO.pop("USER_TASK", 1, 200) - WorkflowSystemTask systemTask = SystemTaskWorkerCoordinator.taskNameWorkflowTaskMapping.get("USER_TASK") + WorkflowSystemTask systemTask = WorkflowSystemTask.get(UserTask.NAME) workflowExecutor.executeSystemTask(systemTask, polledTaskIds.get(0), 30) - then: "verify that the system task is in a state" + then: "verify that the system task is in IN_PROGRESS state" with(workflowExecutionService.getExecutionStatus(workflowInstanceId, true)) { status == Workflow.WorkflowStatus.RUNNING tasks.size() == 2 diff --git a/test-harness/src/test/java/com/netflix/conductor/test/integration/AbstractEndToEndTest.java b/test-harness/src/test/java/com/netflix/conductor/test/integration/AbstractEndToEndTest.java index d796fe9e1e..92be2c013b 100644 --- a/test-harness/src/test/java/com/netflix/conductor/test/integration/AbstractEndToEndTest.java +++ b/test-harness/src/test/java/com/netflix/conductor/test/integration/AbstractEndToEndTest.java @@ -41,7 +41,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; -@TestPropertySource(properties = {"workflow.indexing.enabled=true", "workflow.elasticsearch.version=6"}) +@TestPropertySource(properties = {"conductor.indexing.enabled=true", "conductor.elasticsearch.version=6"}) public abstract class AbstractEndToEndTest { private static final Logger log = LoggerFactory.getLogger(AbstractEndToEndTest.class); @@ -65,7 +65,7 @@ public abstract class AbstractEndToEndTest { static { container.start(); String httpHostAddress = container.getHttpHostAddress(); - System.setProperty("workflow.elasticsearch.url", "http://" + httpHostAddress); + System.setProperty("conductor.elasticsearch.url", "http://" + httpHostAddress); log.info("Initialized Elasticsearch {}", container.getContainerId()); } diff --git a/test-harness/src/test/java/com/netflix/conductor/test/integration/grpc/AbstractGrpcEndToEndTest.java b/test-harness/src/test/java/com/netflix/conductor/test/integration/grpc/AbstractGrpcEndToEndTest.java index f48658fd21..6d0ad27b4f 100644 --- a/test-harness/src/test/java/com/netflix/conductor/test/integration/grpc/AbstractGrpcEndToEndTest.java +++ b/test-harness/src/test/java/com/netflix/conductor/test/integration/grpc/AbstractGrpcEndToEndTest.java @@ -42,8 +42,8 @@ @RunWith(SpringRunner.class) @SpringBootTest(properties = { - "conductor.grpc.server.enabled=true", - "conductor.grpc.server.port=8092" + "conductor.grpc-server.enabled=true", + "conductor.grpc-server.port=8092" }) @TestPropertySource(locations = "classpath:application-integrationtest.properties") public abstract class AbstractGrpcEndToEndTest extends AbstractEndToEndTest { diff --git a/test-harness/src/test/java/com/netflix/conductor/test/integration/grpc/GrpcEndToEndTest.java b/test-harness/src/test/java/com/netflix/conductor/test/integration/grpc/GrpcEndToEndTest.java index 4b2392b73e..8917d9394b 100644 --- a/test-harness/src/test/java/com/netflix/conductor/test/integration/grpc/GrpcEndToEndTest.java +++ b/test-harness/src/test/java/com/netflix/conductor/test/integration/grpc/GrpcEndToEndTest.java @@ -17,11 +17,10 @@ import com.netflix.conductor.client.grpc.WorkflowClient; import org.junit.Before; -@SuppressWarnings("SpringJavaInjectionPointsAutowiringInspection") public class GrpcEndToEndTest extends AbstractGrpcEndToEndTest { @Before - public void init() throws Exception { + public void init() { taskClient = new TaskClient("localhost", 8092); workflowClient = new WorkflowClient("localhost", 8092); metadataClient = new MetadataClient("localhost", 8092); diff --git a/test-harness/src/test/java/com/netflix/conductor/test/integration/grpc/mysql/MySQLGrpcEndToEndTest.java b/test-harness/src/test/java/com/netflix/conductor/test/integration/grpc/mysql/MySQLGrpcEndToEndTest.java index 955b6153f9..72d793e891 100644 --- a/test-harness/src/test/java/com/netflix/conductor/test/integration/grpc/mysql/MySQLGrpcEndToEndTest.java +++ b/test-harness/src/test/java/com/netflix/conductor/test/integration/grpc/mysql/MySQLGrpcEndToEndTest.java @@ -21,23 +21,21 @@ import org.springframework.test.context.TestPropertySource; import org.springframework.test.context.junit4.SpringRunner; -@SuppressWarnings("SpringJavaInjectionPointsAutowiringInspection") @RunWith(SpringRunner.class) @TestPropertySource(properties = { - "db=mysql", - "conductor.grpc.server.port=8094", - "jdbc.url=jdbc:tc:mysql:///conductor", // "tc" prefix starts the MySql container - "jdbc.username=root", - "jdbc.password=root", - "conductor.mysql.connection.pool.size.min=8", - "conductor.mysql.connection.pool.size.max=8", - "conductor.mysql.connection.pool.idle.min=300000", + "conductor.db.type=mysql", + "conductor.grpc-server.port=8094", + "conductor.mysql.jdbcUrl=jdbc:tc:mysql:///conductor", // "tc" prefix starts the MySql container + "conductor.mysql.jdbcUsername=root", + "conductor.mysql.jdbcPassword=root", + "conductor.mysql.connectionPoolMaxSize=8", + "conductor.mysql.connectionPoolMinIdle=300000", "spring.flyway.enabled=false" }) public class MySQLGrpcEndToEndTest extends AbstractGrpcEndToEndTest { @Before - public void init() throws Exception { + public void init() { taskClient = new TaskClient("localhost", 8094); workflowClient = new WorkflowClient("localhost", 8094); metadataClient = new MetadataClient("localhost", 8094); diff --git a/test-harness/src/test/java/com/netflix/conductor/test/integration/grpc/postgres/PostgresGrpcEndToEndTest.java b/test-harness/src/test/java/com/netflix/conductor/test/integration/grpc/postgres/PostgresGrpcEndToEndTest.java index 1ea834d910..0bd65d32ef 100644 --- a/test-harness/src/test/java/com/netflix/conductor/test/integration/grpc/postgres/PostgresGrpcEndToEndTest.java +++ b/test-harness/src/test/java/com/netflix/conductor/test/integration/grpc/postgres/PostgresGrpcEndToEndTest.java @@ -21,23 +21,21 @@ import org.springframework.test.context.TestPropertySource; import org.springframework.test.context.junit4.SpringRunner; -@SuppressWarnings("SpringJavaInjectionPointsAutowiringInspection") @RunWith(SpringRunner.class) @TestPropertySource(properties = { - "db=postgres", - "conductor.grpc.server.port=8098", - "jdbc.url=jdbc:tc:postgresql:///conductor", // "tc" prefix starts the Postgres container - "jdbc.username=postgres", - "jdbc.password=postgres", - "conductor.postgres.connection.pool.size.min=8", - "conductor.postgres.connection.pool.size.max=8", - "conductor.postgres.connection.pool.idle.min=300000", + "conductor.db.type=postgres", + "conductor.grpc-server.port=8098", + "conductor.postgres.jdbcUrl=jdbc:tc:postgresql:///conductor", // "tc" prefix starts the Postgres container + "conductor.postgres.jdbcUsername=postgres", + "conductor.postgres.jdbcPassword=postgres", + "conductor.postgres.connectionPoolMaxSize=8", + "conductor.postgres.connectionPoolMinIdle=300000", "spring.flyway.enabled=false" }) public class PostgresGrpcEndToEndTest extends AbstractGrpcEndToEndTest { @Before - public void init() throws Exception { + public void init() { taskClient = new TaskClient("localhost", 8098); workflowClient = new WorkflowClient("localhost", 8098); metadataClient = new MetadataClient("localhost", 8098); diff --git a/test-harness/src/test/java/com/netflix/conductor/test/integration/http/HttpEndToEndTest.java b/test-harness/src/test/java/com/netflix/conductor/test/integration/http/HttpEndToEndTest.java index cc6bb246e3..5c4a3e4ab3 100644 --- a/test-harness/src/test/java/com/netflix/conductor/test/integration/http/HttpEndToEndTest.java +++ b/test-harness/src/test/java/com/netflix/conductor/test/integration/http/HttpEndToEndTest.java @@ -17,11 +17,10 @@ import com.netflix.conductor.client.http.WorkflowClient; import org.junit.Before; -@SuppressWarnings("SpringJavaInjectionPointsAutowiringInspection") public class HttpEndToEndTest extends AbstractHttpEndToEndTest { @Before - public void init() throws Exception { + public void init() { apiRoot = String.format("http://localhost:%d/api/", port); taskClient = new TaskClient(); diff --git a/test-harness/src/test/java/com/netflix/conductor/test/listener/WorkflowStatusPublisherIntegrationTest.java b/test-harness/src/test/java/com/netflix/conductor/test/listener/WorkflowStatusPublisherIntegrationTest.java index e0033d15fb..d461ee973c 100644 --- a/test-harness/src/test/java/com/netflix/conductor/test/listener/WorkflowStatusPublisherIntegrationTest.java +++ b/test-harness/src/test/java/com/netflix/conductor/test/listener/WorkflowStatusPublisherIntegrationTest.java @@ -12,6 +12,9 @@ */ package com.netflix.conductor.test.listener; +import static com.netflix.conductor.common.metadata.tasks.Task.Status.COMPLETED; +import static org.junit.Assert.assertEquals; + import com.fasterxml.jackson.databind.ObjectMapper; import com.netflix.conductor.common.metadata.tasks.Task; import com.netflix.conductor.common.metadata.tasks.TaskDef; @@ -24,6 +27,13 @@ import com.netflix.conductor.dao.QueueDAO; import com.netflix.conductor.service.ExecutionService; import com.netflix.conductor.service.MetadataService; +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -33,22 +43,11 @@ import org.springframework.test.context.TestPropertySource; import org.springframework.test.context.junit4.SpringRunner; -import java.io.IOException; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import static com.netflix.conductor.common.metadata.tasks.Task.Status.COMPLETED; -import static org.junit.Assert.assertEquals; - @RunWith(SpringRunner.class) @SpringBootTest(properties = { - "workflow.status.listener.type=queue_status_publisher", - "workflowstatuslistener.publisher.success.queue=dummy", - "workflowstatuslistener.publisher.failure.queue=dummy" + "conductor.workflow-status-listener.type=queue_publisher", + "conductor.workflow-status-listener.queue-publisher.successQueue=dummy", + "conductor.workflow-status-listener.queue-publisher.failureQueue=dummy" }) @TestPropertySource(locations = "classpath:application-integrationtest.properties") public class WorkflowStatusPublisherIntegrationTest { diff --git a/test-harness/src/test/java/com/netflix/conductor/test/utils/MockExternalPayloadStorage.java b/test-harness/src/test/java/com/netflix/conductor/test/utils/MockExternalPayloadStorage.java index 9d78a53f1e..9744f94113 100644 --- a/test-harness/src/test/java/com/netflix/conductor/test/utils/MockExternalPayloadStorage.java +++ b/test-harness/src/test/java/com/netflix/conductor/test/utils/MockExternalPayloadStorage.java @@ -30,7 +30,7 @@ import java.util.HashMap; import java.util.Map; -@ConditionalOnProperty(prefix = "workflow", name = "external.payload.storage", havingValue = "mock") +@ConditionalOnProperty(name = "conductor.external-payload-storage.type", havingValue = "mock") @Component public class MockExternalPayloadStorage implements ExternalPayloadStorage { diff --git a/test-harness/src/test/resources/application-integrationtest.properties b/test-harness/src/test/resources/application-integrationtest.properties index a7244eaf99..976c34a408 100644 --- a/test-harness/src/test/resources/application-integrationtest.properties +++ b/test-harness/src/test/resources/application-integrationtest.properties @@ -1,46 +1,40 @@ -environment=test -STACK=test -APP_ID=conductor -EC2_AVAILABILITY_ZONE=us-east-1c -EC2_REGION=us-east-1 +conductor.db.type=memory +conductor.workflow-execution-lock.type=local_only +conductor.external-payload-storage.type=mock +conductor.indexing.enabled=false -db=memory +conductor.app.stack=test +conductor.app.appId=conductor -decider.sweep.disable=false -decider.sweep.frequency.seconds=30 +conductor.app.sweepDisabled=false +conductor.app.sweepFrequencySeconds=30 -conductor.disable.async.workers=false +conductor.app.systemTaskWorkersDisabled=true +conductor.app.systemTaskWorkerCallbackSeconds=0 +conductor.app.systemTaskWorkerThreadCount=0 -workflow.system.task.worker.callback.seconds=0 -workflow.system.task.worker.queue.size=10000 -workflow.system.task.worker.thread.count=10 +conductor.app.eventMessageIndexingEnabled=true +conductor.app.eventExecutionIndexingEnabled=true -workflow.event.message.indexing.enabled=true -workflow.event.execution.indexing.enabled=true +conductor.app.workflowRepairServiceEnabled=false -conductor.workflow.input.payload.threshold.kb=10 -conductor.max.workflow.input.payload.threshold.kb=10240 -conductor.workflow.output.payload.threshold.kb=10 -conductor.max.workflow.output.payload.threshold.kb=10240 -conductor.task.input.payload.threshold.kb=10 -conductor.max.task.input.payload.threshold.kb=10240 -conductor.task.output.payload.threshold.kb=10 -conductor.max.task.output.payload.threshold.kb=10240 -conductor.max.workflow.variables.payload.threshold.kb=2 +conductor.app.workflowExecutionLockEnabled=false -workflow.decider.locking.server=LOCAL_ONLY +conductor.app.workflowInputPayloadSizeThresholdKB=10 +conductor.app.maxWorkflowInputPayloadSizeThresholdKB=10240 +conductor.app.workflowOutputPayloadSizeThresholdKB=10 +conductor.app.maxWorkflowOutputPayloadSizeThresholdKB=10240 +conductor.app.taskInputPayloadSizeThresholdKB=10 +conductor.app.maxTaskInputPayloadSizeThresholdKB=10240 +conductor.app.taskOutputPayloadSizeThresholdKB=10 +conductor.app.maxTaskOutputPayloadSizeThresholdKB=10240 +conductor.app.maxWorkflowVariablesPayloadSizeThresholdKB=2 -workflow.indexing.enabled=false +conductor.redis.availabilityZone=us-east-1c +conductor.redis.dataCenterRegion=us-east-1 +conductor.redis.workflowNamespacePrefix=integration-test +conductor.redis.queueNamespacePrefix=integtest -workflow.repairservice.enabled=false - -workflow.decider.locking.enabled=false - -workflow.external.payload.storage=mock - -workflow.namespace.prefix=integration-test -workflow.namespace.queue.prefix=integtest - -workflow.elasticsearch.index.name=conductor -workflow.elasticsearch.cluster.health.color=yellow +conductor.elasticsearch.indexName=conductor +conductor.elasticsearch.clusterHealthColor=yellow diff --git a/zookeeper-lock/src/main/java/com/netflix/conductor/zookeeper/config/ZookeeperLockConfiguration.java b/zookeeper-lock/src/main/java/com/netflix/conductor/zookeeper/config/ZookeeperLockConfiguration.java index e4f2540908..b65c990e5f 100644 --- a/zookeeper-lock/src/main/java/com/netflix/conductor/zookeeper/config/ZookeeperLockConfiguration.java +++ b/zookeeper-lock/src/main/java/com/netflix/conductor/zookeeper/config/ZookeeperLockConfiguration.java @@ -15,12 +15,13 @@ import com.netflix.conductor.core.sync.Lock; import com.netflix.conductor.zookeeper.lock.ZookeeperLock; import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; -@SuppressWarnings("SpringJavaInjectionPointsAutowiringInspection") @Configuration -@ConditionalOnProperty(prefix = "workflow.decider", name = "locking.server", havingValue = "ZOOKEEPER") +@EnableConfigurationProperties(ZookeeperProperties.class) +@ConditionalOnProperty(name = "conductor.workflow-execution-lock.type", havingValue = "zookeeper") public class ZookeeperLockConfiguration { @Bean diff --git a/zookeeper-lock/src/main/java/com/netflix/conductor/zookeeper/config/ZookeeperProperties.java b/zookeeper-lock/src/main/java/com/netflix/conductor/zookeeper/config/ZookeeperProperties.java index 6e1a5af1cd..b35d6b4d6e 100644 --- a/zookeeper-lock/src/main/java/com/netflix/conductor/zookeeper/config/ZookeeperProperties.java +++ b/zookeeper-lock/src/main/java/com/netflix/conductor/zookeeper/config/ZookeeperProperties.java @@ -12,37 +12,61 @@ */ package com.netflix.conductor.zookeeper.config; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.stereotype.Component; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.springframework.boot.context.properties.ConfigurationProperties; -@Component +@ConfigurationProperties("conductor.zookeeper-lock") public class ZookeeperProperties { - @Value("${workflow.zookeeper.lock.connection:localhost:2181}") - private String connection; + /** + * The connection string to be used to connect to the Zookeeper cluster + */ + private String connectionString = "localhost:2181"; - @Value("${workflow.zookeeper.lock.sessionTimeoutMs:#{T(org.apache.curator.framework.CuratorFrameworkFactory).builder().getSessionTimeoutMs()}}") - private int sessionTimeoutMs; + /** + * The session timeout for the curator + */ + private int sessionTimeoutMs = CuratorFrameworkFactory.builder().getSessionTimeoutMs(); - @Value("${workflow.zookeeper.lock.connectionTimeoutMs#{T(org.apache.curator.framework.CuratorFrameworkFactory).builder().getConnectionTimeoutMs()}}") - private int connectionTimeoutMs; + /** + * The connection timeout for the curator + */ + private int connectionTimeoutMs = CuratorFrameworkFactory.builder().getConnectionTimeoutMs(); - @Value("${workflow.decider.locking.namespace:}") - private String lockingNamespace; + /** + * The namespace to use within the zookeeper cluster + */ + private String namespace = ""; - public String getZkConnection() { - return connection; + public String getConnectionString() { + return connectionString; } - public int getZkSessiontimeoutMs() { + public void setConnectionString(String connectionString) { + this.connectionString = connectionString; + } + + public int getSessionTimeoutMs() { return sessionTimeoutMs; } - public int getZkConnectiontimeoutMs() { + public void setSessionTimeoutMs(int sessionTimeoutMs) { + this.sessionTimeoutMs = sessionTimeoutMs; + } + + public int getConnectionTimeoutMs() { return connectionTimeoutMs; } - public String getLockingNamespace() { - return lockingNamespace; + public void setConnectionTimeoutMs(int connectionTimeoutMs) { + this.connectionTimeoutMs = connectionTimeoutMs; + } + + public String getNamespace() { + return namespace; + } + + public void setNamespace(String namespace) { + this.namespace = namespace; } } diff --git a/zookeeper-lock/src/main/java/com/netflix/conductor/zookeeper/lock/ZookeeperLock.java b/zookeeper-lock/src/main/java/com/netflix/conductor/zookeeper/lock/ZookeeperLock.java index 19806dede5..b789677ff7 100644 --- a/zookeeper-lock/src/main/java/com/netflix/conductor/zookeeper/lock/ZookeeperLock.java +++ b/zookeeper-lock/src/main/java/com/netflix/conductor/zookeeper/lock/ZookeeperLock.java @@ -40,12 +40,12 @@ public class ZookeeperLock implements Lock { private final String zkPath; public ZookeeperLock(ZookeeperProperties properties) { - String lockNamespace = properties.getLockingNamespace(); + String lockNamespace = properties.getNamespace(); RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3); client = CuratorFrameworkFactory.newClient( - properties.getZkConnection(), - properties.getZkSessiontimeoutMs(), - properties.getZkConnectiontimeoutMs(), + properties.getConnectionString(), + properties.getSessionTimeoutMs(), + properties.getConnectionTimeoutMs(), retryPolicy ); client.start(); diff --git a/zookeeper-lock/src/test/java/com/netflix/conductor/zookeeper/lock/ZookeeperLockTest.java b/zookeeper-lock/src/test/java/com/netflix/conductor/zookeeper/lock/ZookeeperLockTest.java index d67719fc55..09398c087a 100644 --- a/zookeeper-lock/src/test/java/com/netflix/conductor/zookeeper/lock/ZookeeperLockTest.java +++ b/zookeeper-lock/src/test/java/com/netflix/conductor/zookeeper/lock/ZookeeperLockTest.java @@ -41,11 +41,11 @@ public class ZookeeperLockTest { public void setUp() throws Exception { zkServer = new TestingServer(2181); properties = mock(ZookeeperProperties.class); - when(properties.getZkConnection()).thenReturn("localhost:2181"); - when(properties.getZkSessiontimeoutMs()).thenReturn(CuratorFrameworkFactory.builder().getSessionTimeoutMs()); - when(properties.getZkConnectiontimeoutMs()) + when(properties.getConnectionString()).thenReturn("localhost:2181"); + when(properties.getSessionTimeoutMs()).thenReturn(CuratorFrameworkFactory.builder().getSessionTimeoutMs()); + when(properties.getConnectionTimeoutMs()) .thenReturn(CuratorFrameworkFactory.builder().getConnectionTimeoutMs()); - when(properties.getLockingNamespace()).thenReturn(""); + when(properties.getNamespace()).thenReturn(""); } @After