Skip to content

Databus 2.0 Configuration Properties

phanindraganti edited this page Feb 25, 2013 · 18 revisions

Introduction

The following document describes the configuration properties available for the various Databus components: the relay, the bootstrap producer, the bootstrap server, and the bootstrap client library.

Configuration properties are generally split in static configuration properties and dynamic configuration properties.

Static configuration

Static configuration properties are initialized at start up and cannot be changed later. Static configuration properties can be set using

  • Java property files
  • JSON
  • Spring

Dynamic (runtime) configuration

Dynamic (runtime) configuration properties can also be initialized at start up, but they can also be changed at runtime. They can be changed using

  • a /config call
  • JMX configuration beans

Property File Syntax

  • All property names must be prefixed with “databus.”, for example “databus.relay.container.id=666”
  • Properties for nested configurations are separated with a dot (“.”), for example: “fileSystem.refreshPeriodMs = 60000”
  • Array-type properties are set using the notation ~”propertyName[index]”
  • Map-type properties are set using the notation “propertyName(key)”, for example: “databus.relay.sourceName(1)=databus.test.source1”

Example

databus.relay.container.id = 9786
databus.relay.container.port = 8080
databus.relay.runtime.statsCollector.enabled = false
databus.relay.runtime.container.requestProcessingBudgetMs = 60000
databus.relay.runtime.container.defaultExecutor.maxThreadsNum = 200
databus.relay.runtime.container.defaultExecutor.coreThreasdNum = 100
databus.relay.runtime.container.statsCollector.enabled = false
databus.relay.sourceName(1001)=com.linkedin.events.liar.jobrelay.LiarJobRelay
databus.relay.sourceName(1002)=com.linkedin.events.liar.jobrelay.LiarMemberRelay

JSON notation

  • Nested configurations are represented as nested objects
  • Array-type properties are set using JSON arrays
  • Map-type properties are set using a nested object where the attribute names are the keys

Example

{"databus":
    {"relay":
        {"container": {"id": 9786, "port": 8080}},
        {"runtime":
            {"statsCollector": {"enabled": "false"}},
            {"container":
                {"requestProcessingBudgetMs": 60000,
                 "defaultExecutor": {"maxThreadsNum": 200, "coreThreasdNum": 100},
                 "statsCollector": {"enabled": "false"}},
        {"sourceName": {"1001": "com.linkedin.events.liar.jobrelay.LiarJobRelay",
                        "1002": "com.linkedin.events.liar.jobrelay.LiarMemberRelay"}}
    }
}

Command-line

Sample Java properties configuration (can be used with the “-p” command-line option): example properties file

#CUSTOM_ID: <<relayConfiguration>>

Relay Configuration

#CUSTOM_ID: <<relayStaticConf>>

Relay Static Configuration

Property prefix: “databus.relay”

Property name Type Description Default value
container [Netty Serving Container Static Configuration#serverContainerStaticConf] Configuration options for the relay netty container
eventBuffer [Event Buffer Static Configuration#eventBufferStaticConf] Configuration options for the relay event buffer
eventLogReaderConfig [Event Reader Static Configuration#eventLogReaderStaticConf] Configuration options for the event log reader
eventLogWriterConfig [Event Writer Static Configuration#eventLogWriterStaticConf] Configuration options for the event log writer
httpStatsCollector [HTTP Calls Statistics Collector Static Configuration#relayHttpStatsCollectorStaticConf] Configuration options for the HTTP calls statistics collector
randomProducer [Random Event Producer Static Configuration#randomProducerStaticConf] Configuration options for the relay random events producer (testing)
runtime [Relay Runtime Configuration#relayRuntimeConf] Relay runtime configuration options
schemaRegistry [Schema Registry Static Configuration#schemaRegistryStaticConf] Configuration options for the schema registry
sourceName Map from source id to Databus source name Databus sources registered in the relay
startDbPuller Boolean Configuration option for starting db puller thread on startup false
dataSources [DataSources#dataSources] Configuration for DataSources
databus.relay.container.id = 9786
databus.relay.container.jmx.jmxServicePort = 9988
databus.relay.runtime.statsCollector.enabled = false
databus.relay.runtime.container.requestProcessingBudgetMs = 60000
databus.relay.runtime.container.defaultExecutor.maxThreadsNum = 200
databus.relay.runtime.container.defaultExecutor.coreThreasdNum = 100
databus.relay.runtime.container.statsCollector.enabled = false
databus.relay.sourceName(1001)=com.linkedin.events.liar.jobrelay.LiarJobRelay
databus.relay.sourceName(1002)=com.linkedin.events.liar.jobrelay.LiarMemberRelay
databus.relay.dataSources.sequenceNumbersHandler.file.scnDir=<dir>

#CUSTOM_ID: <<relayRuntimeConf>>

Relay Runtime Configuration

Property prefix: “databus.relay.runtime”

Property name Type Description Default value
httpStatsCollector Relay Statistics Collector Runtime Configuration#relayHttpStatsCollectorRuntimeConf Runtime configuration options for the HTTP calls statistics collector
container Serving Container Runtime Configuration#serverContainerRuntimeConf Runtime configuration for the Netty serving container
databus.relay.runtime.statsCollector.enabled = false
databus.relay.runtime.container.requestProcessingBudgetMs = 60000
databus.relay.runtime.container.defaultExecutor.maxThreadsNum = 200
databus.relay.runtime.container.defaultExecutor.coreThreasdNum = 100
databus.relay.runtime.container.statsCollector.enabled = false

#CUSTOM_ID: eventLogReaderStaticConf

Event Log Reader Static Configuration

Property name Type Description Default value
enabled Boolean A flag that indicates if the event log reader is to be invoked on relay startup false
readSessionDir String The directory where the event logs are read for the current session
topLevelLogDir String The parent directory for all session directories

#CUSTOM_ID: <<eventLogWriterStaticConf>>

Event Log Writer Static Configuration

Property name Type Description Default value
blockOnWrite Boolean A flag that indicates if the log writer should hold locks in the buffer until logs are written to disk false
enabled Boolean A flag that indicates if the event log writer is to be invoked on relay startup false
encoding JSON, BINARY The encoding of the events BINARY
fsyncIntervalInMillis Integer The number of milliseconds between fsyncs to ensure logs are securely written to disk 10000
individualFileMaxBytes Integer The max size of a file with event logs in the write session directory 500000000
maxFiles Integer Max number of files in the write session directory 5
maxPendingWrites Integer Max number of events that can be buffered before the event log writer gives up trying to keep up with the event stream 20
topLevelLogDir String The parent directory for all session directories
writeSessionDir String The directory where the event logs are written for the current session
writeBatchSizeInBytes Integer The number of bytes to buffer before writing the logs 64000

Schema Registry Configuration

#CUSTOM_ID: <<schemaRegistryStaticConf>>

Schema Registry Static Configuration

Property name Type Description Default value
type FILE_SYSTEM, REMOTE, EXISTING The type of the schema registry service to use FILE_SYSTEM
fileSystem File System Schema Registry Static Configuration fsSchemaRegistryStaticConf] Configuration for the file system registry service
existingService com.linkedin.schemaregistry.SchemaRegistryService Dependency injection to an already instantiated registry service (NOTE: Only meaningful in Spring configurations) none

Example

databus.relay.schemaRegistry.type = FILE_SYSTEM

#CUSTOM_ID: <<fsSchemaRegistryStaticConf>>

File System Schema Registry Static Configuration

Property name Type Description Default value
schemaDir String The path to the schema registry directory; if relative, the default start directory is used ./schemas_registry
refreshPeriodMs Long The time in milliseconds between checks for changes in the schema registry directory 3600000

Example

databus.relay.schemaRegistry.fileSystem.schemaDir = /var/share/databus/schemas_registry
databus.relay.schemaRegistry.fileSystem.refreshPeriodMs = 60000

Relay DataSources Configuration

#CUSTOM_ID: <<dataSources>>

Relay DataSources Static Configuration

Property name Type Description Default value
sequenceNumbersHandler SCN Handler Static Configuration sequenceNumberHandler Configuration for the SCN Handler

Example

databus.relay.dataSources.sequenceNumbersHandler.file.scnDir=<dir>
#END_EXAMPLE

** SequenceNumberHandler Configuration

#CUSTOM_ID: <<sequenceNumberHandler>>

*** MaxSequenceNumberHandler Static Configuration

| Property name | Type | Description | Default value |
| file | [MaxScnStorage#scnNumberLocation] | Configuration for the location to store max scnHandler | |

Example

databus.relay.dataSources.sequenceNumbersHandler.file.scnDir=<dir>

SCN storage Configuration

#CUSTOM_ID: <<scnNumberLocation>>

Max SCN storage Static Configuration

Property name Type Description Default value
key String Name(prefix) of the file used for store the SCN “MaxSCN”
scnDir String Directory where the SCN will be stored “databus2-maxscn”
initVal Long Initial scn value(on start) 0
flushItvl Long Number of SCN updates before the SCN is persisted to disk 1

Example

databus.relay.dataSources.sequenceNumbersHandler.file.scnDir=<dir>
databus.relay.dataSources.sequenceNumbersHandler.file.key="max_scn"
databus.relay.dataSources.sequenceNumbersHandler.file.initVal=100
databus.relay.dataSources.sequenceNumbersHandler.file.flushItvl=1

#CUSTOM_ID: <<randomProducerStaticConf>>

Random Producer Static Configuration

#CUSTOM_ID: <<DatabaseSourcesStaticConf>>

Database Sources Static Configuration

Property name Type Description Default value
name String The name of the source database required
uri String Database JDBC connection URI required
slowSourceQueryThreshold Integer The max time in ms with no updates after which the database event puller switches to slow source mode. 3000
restartScnOffset Integer The delta to rollback the last scn read on relay restart. This is meant to not start the relay with an empty buffer. The value is ignored if it is <= 0. 0
sources an array of [Logical Source Static Configuration#logicalSrcStaticConf] Configuration of the logical Databus sources available in the physical database.
maxScnHandler [MaxSCN Handler Static Configuration#maxScnHandlerStaticConf] Configuration for MaxSCN handler which persists the last scn processed and reads it on relay restart.
chunkingType Enum { NO_CHUNKING, TXN_CHUNKING } Type of Chunking used by Relay. NO_CHUNKING
chunkedScnThreshold long Threshold for enabling chunking. Chunking (if configured) will be enabled If relay’s current SCN is lower than DB’s max SCN (as cached by the relay) by more than this threshold. 20000
txnsPerChunk long Number of Txns to be fetched for each relay query during TXN based chunking 20000
maxScnDelayMs long Delay between successive maxScn queries by the relay 300000

#CUSTOM_ID: <<maxScnHandlerStaticConf>>

MaxSCN Handler Static Configuration

Property name Type Description Default value
type DISABLED, FILE, EXISTING Type of of the MaxSCN handler FILE
file [File-Based MaxSCN Handler Static Configuration#fileMaxScnHandlerStaticConf] The configuration for the file-system based MaxSCN handler; used only if getType() returns ‘FILE’.
existing Wired MaxSCN handler; used only if getType() returns ‘EXISTING’. This setting makes sense only of Spring-based configuration.

#{anchor:fileMaxScnHandlerStaticConf}

File-Based MaxSCN Handler Static Configuration

Property name Type Description Default value
key String the name of the file used for storing the SCN MaxSCN
scnDir String the directory where the SCN will be saved databus2-maxscn
initVal Long the initial scn value if none is persisted on disk 0
flushItvl Long the number of SCN updates before the SCN is persisted to disk 1

#{anchor:logicalSrcStaticConf}

Logical Source Static Configuration

Property name Type Description Default value
id Short Globally unique source id required
name String Fully qualified source name as specified by the Avro schema for the source required
uri String DB connection URI (table name) required
partitionFunction String Partitioning function spec empty

Bootstrap Drop Source Configuration

BootstrapDB Configuration

Property prefix: “bootstrap.”

Property name Type Description Default value
bootstrapDBHostname String The hostname of the Bootstrap DB server localhost
bootstrapDBPassword String The password for access to the Bootstrap DB bootstrap
bootstrapDBUsername String The user name for access to the Bootstrap DB bootstrap

Bootstrap Reader Configuration

BootstrapDB and Schema Configuration

Property prefix: “databus.reader”

Property name Type Description Default value
bootstrap.bootstrapDBHostname String The hostname of the Bootstrap DB server localhost
bootstrap.bootstrapDBPassword String The password for access to the Bootstrap DB bootstrap
bootstrap.bootstrapDBUsername String The user name for access to the Bootstrap DB bootstrap
bootstrap.bootstrapLogSize Integer The maximum number of rows in a bootstrap DB log_\* table. 10000
schemaRegistry.type String/enum Schema Source Type FILE_SYSTEM
schemaRegistry.fileSystem.schemaDir String Schema Location ./schemas_registry

Query Configuration

Property prefix: “databus.query””

Property name Type Description Default value
SourceId Integer The SourceId to query 0
SourceName String The SourceName to query empty
tablePrefix String Log Table (log_) or Source Table (tab_) empty
field String The Column Name to query empty
rangeQuery Boolean Is it a range Query false
logId Integer LogId of the log table to be queries ( used only when tablePrefix = “log_” 0
value Integer Value of the point Query ( isRangeQuery = false) 0
minValue Integer Min Value of the range Query ( isRangeQuery = true) 0
maxValue Integer Max Value of the range Query ( isRangeQuery = true) 0

Bootsrap Utils Configuration

#{anchor:bootstrapSeederStaticConf}

BootstrapDBType

#{anchor:bootstrapDBType}

Type Definition
BOOTSTRAP_CATCHUP_APPLIER_RUNNING This is a bootstrap DB which has not been seeded for the source and is run in catchup mode. Both log tables and snapshot tables for the source are being populated by bootstrap producer and applier respectively.
BOOTSTRAP_CATCHUP_APPLIER_NOT_RUNNING This is a bootstrap DB which has not been seeded for the source and is run in catchup mode. Only log tables for the source are being populated by bootstrap producer and applier is not running
BOOTSTRAP_FULL This is the regular bootstrap DB which has been seeded for the source and producer/applier is running

Bootstrap Cleaner Static Configuration

Property prefix: “databus.bootstrap.cleaner”

Property name Type Description Default value
enable Boolean Enable bootstrap Cleaner true
bootstrapTypeDefault [BootstrapDBType#bootstrapDBType] Bootstrap DB Type BOOTSTRAP_FULL
bootstrapTypeOverride Map<String,[BootstrapDBType#bootstrapDBType] > Source specific override of bootstrapDB types empty
retentionDefault [retentionConfig#retentionConfig] Default Retention Config
retentionOverride Map<String,[retentionConfig#retentionConfig]> Source specific overrides for retention configs empty
enableOptimizeTableDefault Boolean Enable Optimize table when tab table is cleaned up false
enableOptimizeTable Map<String,Boolean> Source specific overrides to enable Optimize table empty
enableForceTabTableCleanupDefault Boolean Force tab table to cleanup even if applier is not specified false
enableForceTabTableCleanup Map<String,Boolean> Source specific overrides to force tab table cleanup empty
diskSpaceTrigger [DiskSpaceTriggerConfig#diskSpaceTriggerConfig] DiskSpaceTrigger Configuration
periodicTrigger [PeriodicTriggerConfig#periodicTriggerConfig] PeriodicTrigger Configuration

DiskSpaceTrigger Config

#{anchor:diskSpaceTriggerConfig}

Property name Type Description Default value
enable boolean Enable disk-space trigger true
runIntervalSeconds long Delay between successive disk space utilization checks 1 hour
availableThresholdPercent double Threshold percentage of available space 25.0 ( If disk space is more than 75%, then cleaner will run
bootstrapDBDrive String MySQL installation drive /mnt/u001

Periodic Config

#{anchor:periodicTriggerConfig}

Property name Type Description Default value
enable boolean Enable periodic trigger true
runIntervalSeconds long Delay between successive cleanup 1 day
runOnStart boolean Run cleaner immediately on startup of bootstrap producer false

Retention Config

#{anchor:retentionConfig}

Property name Type Description Default value
retentionType [retentionType#retentionType] Retention type RETENTION_SECONDS
retentionQuantity long For Retention_Seconds, this is retain_seconds, For retention_logs, this is numLogsToRetain 2 weeks

RetentionType

#{anchor:retentionType}

Type Definition
NO_CLEANUP Dont run cleaner
RETENTION_SECONDS Retain based on timestamps
RETENTION_LOGS Retain based on number of log tables

Bootstrap Seeder Static Configuration

Property prefix: “databus.seed”

Property name Type Description Default value
bootstrap.bootstrapDBHostname String The hostname of the Bootstrap DB server localhost
bootstrap.bootstrapDBPassword String The password for access to the Bootstrap DB bootstrap
bootstrap.bootstrapDBUsername String The user name for access to the Bootstrap DB bootstrap
bootstrap.bootstrapLogSize Integer The maximum number of rows in a bootstrap DB log_* table. 10000
schemaRegistry.type String/enum Schema Source Type FILE_SYSTEM
schemaRegistry.fileSystem.schemaDir String Schema Location ./schemas_registry
controller.commitInterval Integer Checkpoint Interval for MySQL Write 10000
controller.numRowsPrefetch Integer Oracle JDBC NumRowsPrefetch Argument 10
controller.LOBPrefetchSize Integer Oracle JDBC LOBPrefetch Size Argument 4000
controller.numRowsPerQuery Integer Oracle Read Chunk/Page Size 100000
controller.keyTxnMapFile(SOURCE) String KeyTxn File generated during Seeding and used by Audit to track if rows got changed Empty
controller.keyTxnFileBufferSize(SOURCE) Integer Buffer Size for Writing KeyTxn file during Seeding 0
controller.pKeyName(SOURCE) String Primary Key for the Source key
controller.pKeyIndex(SOURCE) String Primary Key Index for the Source SOURCE_PK
controller.endSrcKey(SOURCE) String/Long End Source Key for the Source. If the seeding script completes seeding the row corresponding to this key, it will move on to the next source for seeding. Empty (full seeding)
controller.beginSrcKey(SOURCE) String/Long Starting Primary Key for Seeding. Empty(Use Oracle’s minKey())

#{anchor:bootstrapProducerConfiguration}

Bootstrap Producer Configuration

#{anchor:bootstrapProducerStaticConf}

Bootstrap Producer Static Configuration

Property prefix: “databus.bootstrap.producer”

Property name Type Description Default value
bootstrapDBHostname String The hostname of the Bootstrap DB server localhost
bootstrapDBPassword String The password for access to the Bootstrap DB bootstrap
bootstrapDBUsername String The user name for access to the Bootstrap DB bootstrap
bootstrapLogSize Integer The maximum number of rows in a bootstrap DB log_* table. 10000
client [Client static configuration#clientStaticConf] The Databus client static configuration for the bootstrap producer
retryTimer [BootstrapDB Retry Configuration https://github.com/linkedin/databus/wiki/Databus-2.0-Configuration-Properties#Databusv2.0ConfigurationProperties-RetriesConfiguration] The retry configuration for MySQL DB errors seen by Producer/Applier

Example

databus.bootstrap.bootstrapLogSize=360000
databus.bootstrap.producer.client.container.httpPort=4567

#{anchor:bootstrapServerConfiguration}

Bootstrap Server Configuration

#{anchor:bootstrapServerStaticConf}

Bootstrap Server Static Configuration

Property prefix: “databus.bootstrap.server”

Property name Type Description Default value
defaultRowsThresholdForSnapshotBypass LONG Default NumRows threshold for bypassing snapshot phase 0
rowsThresholdForSnapshotBypass(SOURCE) LONG Source specific NumRowsThreshold override for bypassing snapshot phase 0
disableSnapshotBypass(SOURCE) BOOLEAN Source specific Flag to disable bypassing
db.bootstrapBatchSize Integer The maximum number of rows per bootstrap snapshot or catchup call 1000
db.bootstrapDBHostname String The hostname of the Bootstrap DB server localhost
db.bootstrapDBPassword String The password for access to the Bootstrap DB bootstrap
db.bootstrapDBUsername String The user name for access to the Bootstrap DB bootstrap
db.bootstrapHttpPort String The HTTP port of the bootstrap server. Deprecated: Use {{container.httpPort}} instead 6060
db.container [Netty Serving Container Static Configuration#serverContainerStaticConf] The Netty serving container static configuration

Example

databus.bootstrap.server.db.bootstrapBatchSize=10000
databus.bootstrap.server.db.container.httpPort=666

#{anchor:clientConfiguration}

Client Library Configuration

#{anchor:clientStaticConf}

Client Library Static Configuration

Property prefix for standalone use: “databus.client”

Property name Type Description Default value
checkpointPersistence [Checkpoint Persistent Provider Static Configuration#cpPersistenceStaticConf] The checkpoint persistent provider static configuration
connection [Databus Sources Connection Static Configuration#sourcesConnStaticConf] A map from a comma-separated Databus source list to the connection static configuration for that list of sources
connectionDefaults [Databus Sources Connection Static Configuration#sourcesConnStaticConf] The default static configuration for all Databus sources connections
container [Netty Serving Container Static Configuration#serverContainerStaticConf] The Netty container static configuration
cluster [Shared state zookeeper cluster config#clusterStaticConf] Zookeeper cluster configuration
httpStatsCollector [HTTP Calls Statistics Collector Configuration#relayHttpStatsCollectorStaticConf] The static configuration for statistics collector for all HTTP calls to the relays
loggingListener [Logging Consumer Static Configuration#loggingConsumerStaticConf] Static configuration for the logging consumer
runtime [Client Library Runtime Configuration#clientRuntimeConf] The client runtime configuration
clientCluster [Databus ClientCluster static configuration#clientClusterStaticConf] Client Cluster configuration for load-balanced client clusters. More than one cluster can be configured. Each cluster is identified using an id as key

#{anchor:clientRuntimeConf}

Client Library Runtime Configuration

Property prefix for standalone use: “databus.client.runtime”

Property name Type Description Default value
bootstrap [Client Bootstrapping Runtime Configuration#clientBootstrappingRuntimeConf] Runtime configuration for bootstrapping
checkpointPersistence [Checkpoint Persistent Provider Runtime Configuration#cpPersistenceRuntimeConf] Runtime configuration for the checkpoint persistence provider
container [Netty Serving Container Runtime Configuration#serverContainerRuntimeConf] Runtime configuration for the Netty serving container
httpStatsCollector [HTTP Calls Statistics Collector Runtime Configuration#relayHttpStatsCollectorRuntimeConf] Runtime configuration for statistics collector for the HTTP calls to the relays
loggingListener [Logging Consumer Runtime Configuration#loggingConsumerRuntimeConf] Runtime configuration for the logging consumer
relay A map from a relay ID to a Relay [Server Configuration#serverConf] Server configuration for the available relays empty

#{anchor:clientBootstrappingRuntimeConf}

Client Bootstrapping Runtime Configuration

Property name Type Description Default value
enabled Boolean A flag that indicates if bootstrapping is enabled false
service A map from a bootstrap server ID to a Bootstrap [Server Configuration#serverConf] Server configuration for the bootstrap servers empty

Server Side Filtering Configuration

There will be per source configurations for Server Side Filtering. The following are the parameters:

Property name Type Description Validation Default value Example
serversidefilter.filter(source).type String (RANGE, MOD, NONE) The filter type for this source Must be one of the enum values mentioned NONE RANGE
serversidefilter.filter(source).range.size Long The Size of each range. Applicable only for Range Partitioning Must be positive 0 5000000
serversidefilter.filter(source).range.partitions String The list of partitionIds to filte.Applicable only for Range Partitioning Within a range “a-b” , a must be less than or equal to 0. Also, Both a and b must not be negative. The ranges used here follow the pattern [a-b) NONE [1,3-6] (With the size set to 5M, the ranges 5M to(10M - 1) and 15M to (30M - 1) will be filtered)
serversidefilter.filter(source).mod.numBuckets Long The number of buckets. Applicable only for MOD partitioning Must be positive NONE 2
serversidefilter.filter(source).mod.buckets String The list of bucketIds to filter. Applicable only for MOD partitioning For bucket range “a-b”, both a and b must be less than numBuckets, “a” must be less than or equal to “b”. Also, Both a and b must not be negative.The ranges used here follow the pattern [a-b) NONE [0] (With numBuckets set to 2, all the even number keys will be filtered)

#{anchor:clusterStaticConf}

Shared State Cluster Configuration

Top level property name: databus.client.cluster

Property name Type Description Default value
enabled boolean Enable cluster failover, shared persistence of checkpoint false
clusterServerList string Comma separated list of server:port that specifies zookeeper servers empty
sessionTimeoutMillis int Zookeeper session timeout in milliseconds 5000
connectionTimeoutMillis int Zookeeper connection timeout in milliseconds 10000
group string Group to which this databus client belongs. e.g. all members of failover cluster will have the same group, will see the same shared-data default-databus-group
domain string Top level zookeeper path that defines the namespace for the family of applications e.g. all databus2 client groups will be under /databus2 /Databus-Client-Domain
name string Name of this databus client , ideally this should be unique within a group default-name+randomInt

Checkpoint Persistent Provider Configuration

#{anchor:cpPersistenceStaticConf}

Static Configuration

Property name Type Description Default value
existing com.linkedin.databus.client.pub.CheckpointPersistenceProvider A spring-wired existing checkpoint persistence provider if one has been specified. ‘type’ needs to be ‘EXISTING’ for this property to have an effect. NOTE: This is meaningful only for Spring-wired configurations. null
fileSystem [File-system Checkpoint Persistence Provider Static Configuration#fsCpPersistenceStaticConf] The static configuration for the file system checkpoint persistence provider. ‘type’ needs to be ‘FILE_SYSTEM’ for this property to have an effect.
shared [Shared-state Checkpoint Persistence Provider Static Configuration#shCpPersitenceStaticConf] The static configuration for the shared state checkpoint persistence provider. ‘type’ needs to be ‘SHARED’ for this property to have an effect. [Shared State Cluster#clusterStaticConf] must be appropriately configured. Note that if mode=SHARED and databus.client.cluster.enabled=false or if databus.client.cluster.clusterServerList is empty, a config exception is thrown.
runtime [Checkpoint Persistent Provider Runtime Configuration#cpPersistenceRuntimeConf] The runtime configuration for the checkpoint persistence provider
type FILE_SYSTEM, SHARED, EXISTING, NONE The type of the checkpoint persistence provider. FILE_SYSTEM specifies to store the checkpoints in a directory on a local file system. SHARED specifies to store the checkpoints in shared state; in the current implementation Zookeeper provides the shared-state storage. EXISTING specifies to use a Spring-wired checkpoint persistence provider. NONE specifies not to persist checkpoints FILE_SYSTEM
clearBeforeUse Boolean A flag that indicates whether to clear any existing persisted checkpoints on first use false

#{anchor:cpPersistenceRuntimeConf}

Runtime Configuration

Property name Type Description Default value
fileSystem [File-system Checkpoint Persistence Provider Runtime Configuration#fsCpPersistenceRuntimeConf] Runtime configuration for the file system checkpoint persistence provider

File System Checkpoint Persistence Provider Configuration

#{anchor:fsCpPersistenceStaticConf}

Static Configuration

Property name Type Description Default value
rootDirectory String The root directory for all checkpoint files. ./databus2-checkpoints
runtime [File-system Checkpoint Persistence Provider Runtime Configuration#fsCpPersistenceRuntimeConf] The runtime configuration properties

#{anchor:fsCpPersistenceRuntimeConf}

Runtime Configuration

Property name Type Description Default value
historySize Integer The number of archived checkpoints to store 5
historyEnabled Boolean A flag that indicates if the provider should archive previously-persisted checkpoints true

#{anchor:shCpPersitenceStaticConf}

Shared Checkpoint Persistence Provider Configuration

Static Configuration

Property name Type Description Default value
maxNumWritesSkipped int Specifies the number of checkpoint persistence skipped before committing to shared storage 0
sharedDirectory (not available) string Specifies the directory within zookeeper where shared data is written shareddata

#{anchor:sourcesConnStaticConf}

Databus Sources Connection Static Configuration

Property name Type Description Default value
checkpointThresholdPct Double The percentage of event buffer occupancy that will trigger a checkpoint attempt. This is to ensure that we can make progress in large event windows without having to reprocess them entirely in case of a failure. 90.0
consumerParallelism Integer Max number of consumers that can be called in parallel to process an event 1
consumerTimeBudgetMs Integer Max time that a consumer should use to process an event before it is considered failed 300000
eventBuffer [Event Buffer Static Configuration#eventBufferStaticConf] The relay event buffer static configuration. Currently, this is also used for the bootstrapping event buffer.
pullerRetries [Retries Static Configuration#retriesStaticConf] Pull requests and error retries configuration when talking to the relays or bootstrap servers. NOTE: {{initSleep}} is used to determine the sleep between subsequent pull requests from the relay initSleep=100
bsPullerRetriesBeforeCkptCleanup [Retries Static Configuration#Databusv2.0ConfigurationProperties-retriesStaticConf] Retry configuration for bootstrap Puller to try connecting to the same server and preserving the checkpoint. If the retry expires, then bootstrapPuller will try connecting to other servers and reset bootstrap info in the checkpoint numRetries=3
bsPullerRetriesBeforeCkptCleanup [Retries Static Configuration#retriesStaticConf] Bootstrap Puller retry on the same server before switching to a new server and restarting bootstrap initSleep=1sec,numRetries=25,sleepIncFactor=1,SleepIncDelta=1000 (1 sec)
dispatcherRetries [Retries Static Configuration#retriesStaticConf] Error retries configuration calling the consumer code
freeBufferThreshold Integer The minimum amount of free space in client buffer (specified in bytes) required for the Relay and/or Bootstrap Puller threads to initiate requests to the relay /bootstrap servers.

Note: freeBufferThreshold + checkPointThresholdPct/100*bufferCapacityInBytes&nbsp; < bufferCapacityInBytes | 10000 |

enablePullerMessageQueueLogging Boolean Enable Logging of State transitions for the relay and bootstrap puller threads. This is useful to validate State transitions issues in the relay and bootstrap puller threads. (Note: This could be excessive logging ) false

Databus Client Cluster Static Configuration

#{anchor:clientClusterStaticConf}

Property name Type Description Default value
clusterName String Name of the client Cluster. ””
zkAddr String ZK HostPort config seperated by colon ””
numPartitions long Total number of partitions for this cluster 1
quorum long Minimum number of nodes to be active before partitions can be allocated 1
maxCkptWritesSkipped int Number of checkpoints that can skipped before persisting the progress in ZooKeeper. 0
checkpointIntervalMs long Minimum time interval in ms that between successive checkpoint persistence 5*60*1000

Logging Consumer Configuration

#{anchor:loggingConsumerStaticConf}

Static Configuration

Property name Type Description Default value
runtime [Logging Consumer Runtime Configuration#loggingConsumerRuntimeConf] Runtime configuration properties
logTypedValue Boolean A flag that indicates if the consumer should log the typed value of data events as JSON false

#{anchor:loggingConsumerRuntimeConf}

Runtime Configuration

Property name Type Description Default value
enabled Boolean A flag that indicates if the consumer is enabled true
logLevel FATAL, ERROR, WARN, INFO, DEBUG, TRACE, OFF The logging level to be used by the consumer INFO
verbosity EVENTS_ONLY, EVENT_WINDOWS, ALL The verbosity of the consumer, that is, which events are to be logged. EVENTS_ONLY - only onEvent() and onBootstrapEvent() calls are logged, EVENT_WINDOWS - all calls in EVENTS_ONLY plus start/endEventSequence() and start/endBootstrapSequence() are logged. ALL - all calls are logged EVENT_WINDOWS
validityCheckEnabled Boolean A flag that indicates if the consumer should check the validity of the events it gets true

Common Configuration Options

Netty Serving Container Configuration

#{anchor:serverContainerStaticConf}

Static Configuration

Property name Type Description Default value
httpPort Integer HTTP port to listen on 8080
id Integer Container ID Hash(hostname) + port
inboundEventsStatsCollector [Event traffic statistics collector static configuration#eventsStatsCollectorStaticConf] Static configuration of the statistics collector for the inbound event traffic
jmx [JMX static configuration#jmxStaticConf] JMX static configuration
outboundEventsStatsCollector [Event traffic statistics collector static configuration#eventsStatsCollectorStaticConf] Static configuration of the statistics collector for the outbound event traffic
runtime [Netty serving container runtime configuration#serverContainerRuntimeConf] Runtime configuration
statsCollector [Network traffic statistics collector static configuration#nettyStatsCollectorStaticConf] Static configuration for the netty container statistics collector
healthcheckPath String Path on which the healthcheck will respond (sans the initial /) admin
readTimeoutMs Long Timeout for reading parts of HTTP requests (in milliseconds) 15000
writeTimeoutMs Long Timeout for confirmation from the peer when sending a response 15000
tcp [TCP Static Config#tcpStaticConf] Static configuration for the TCP command interface

#{anchor:serverContainerRuntimeConf}

Runtime Configuration

Property name Type Description Default value
defaultExecutor [Executor Runtime Configuration#executorRuntimeConfig] Runtime configuration for the request executor threads pool
ioExecutor [Executor Runtime Configuration#executorRuntimeConfig] Runtime configuration for the io threads pool
statsCollector [Netty Statistics Collector Runtime Configuration#nettyStatsCollectorRuntimeConf] Runtime configuration for the statistics collector
inboundEventsStatsCollector [Events statistics collector runtime configuration#eventsStatsCollectorRuntimeConf] Runtime configuration for the inbound event traffic statistics collector
outboundEventsStatsCollector [Events statistics collector runtime configuration#eventsStatsCollectorRuntimeConf] Runtime configuration for the outbound event traffic statistics collector
requestProcessingBudgetMs Integer Maximum time in milliseconds to process a request before interrupting the request processing 100
statsCollector [Netty statistics collector runtime configuration#nettyStatsCollectorRuntimeConf] Runtime configuration for the netty statistics collector

Example

databus.relay.container.requestProcessingBudgetMs = 60000
databus.relay.container.defaultExecutor.maxThreadsNum = 200
databus.relay.container.defaultExecutor.coreThreasdNum = 100
databus.client.container.statsCollector.enabled = false

#{anchor:executorRuntimeConfig}

Executor Runtime Configuration

Property name Type Description Default value
maxThreadsNum Integer The maximum number of threads 100
coreThreadsNum Integer The default number of threads 50
keepAliveMs Integer The time in milliseconds to keep an idle thread above the {{coreThreadsNum}} threshold idle before killing them 60000

Netty Statistics Collector Configuration

#{anchor:nettyStatsCollectorStaticConf}

Static Configuration

Property name Type Description Default value
runtime [Netty Statistics Collector Runtime Configuration#nettyStatsCollectorRuntimeConf] Runtime configuration for container statistics collection

#{anchor:nettyStatsCollectorRuntimeConf}

Runtime Configuration

Property name Type Description Default value
enabled Boolean Enables/disables container statistics collection true

#{anchor:jmxStaticConf}

JMX Static Configuration

Property name Type Description Default value
rmiEnabled Boolean A flag if the RMI connector is to be enabled true
jmxServicePort Integer The port for the JMX service 9999
jmxServiceHost String The hostname for the JMX service localhost
rmiRegistryPort Integer The port of the RMI registry where the JMX server will be registered 1099
rmiRegistryHost String The hostname of the RMI registry where the JMX server will be registered localhost

#{anchor:eventBufferStaticConf}

Event Buffer Static Configuration

Property name Type Description Default value
allocationPolicy HEAP_MEMORY, DIRECT_MEMORY, MMAPPED_MEMORY Allocation policy for the eventBuffer. Controls if the event buffer should be allocated as a jvm heap buffer, direct buffer (not GC-ed) or mmapped buffer. MMAPPED_MEMORY if maxSize > 10000; HEAP_MEMORY otherwise
defaultMemUsage Double The fraction of available memory to be used by the event buffer if no {{maxSize}} has been explicitly specified 0.75
maxIndividualBufferSize Integer The maximum size of one single sub-buffer in the event buffer for data 500000000
maxSize Long The maximum size of the buffer 0.8 * {{defaultMemUsage}} \* (available memory)
mmapDirectory String Top-level directory for mmapped files. Session directories are located under the mmapDirectory and cleaned up on jvm exit. Ideally mmapDirectory and eventLogWriter.topLevelDir should be located on different disks for optimum performance. mmapDir
queuePolicy BLOCK_ON_WRITE, OVERWRITE_ON_WRITE The queueing policy for the event buffer. A flag if the event buffer should overwrite the oldest events if there is no space (OVERWRITE_ON_WRITE) for new events or whether it should block (BLOCK_ON_WRITE). OVERWRITE_ON_WRITE
readBufferSize Integer The amount of memory to be used for the event buffer for read buffering 0.1 * {{defaultMemUsage}} \* (available memory)
scnIndexSize Integer The amount of memory to be used for the event buffer for the SCN index 0.1 * {{defaultMemUsage}} \* (available memory)
existingBuffer com.linkedin.databus.core.DbusEventBuffer Dependency injection to an existing event buffer. NOTE: Only meaningful in Spring configurations none
databus.relay.eventBuffer.maxSize=2048000000
databus.client.eventBuffer.maxSize=10000000

Events Statistics Collector Configuration

#{anchor:eventsStatsCollectorStaticConf}

Static configuration

Property name Type Description Default value
runtime [Events Statistics Collector Runtime Configuration#eventsStatsCollectorRuntimeConf] Runtime configuration for container statistics collection

#{anchor:eventsStatsCollectorRuntimeConf}

Runtime configuration

Property name Type Description Default value
enabled Boolean Enables/disables events statistics collection true

HTTP Calls Statistics Collector Configuration

#{anchor:relayHttpStatsCollectorStaticConf}

Static Configuration

Property name Type Description Default value
runtime [HTTP Calls Statistics Collector#relayHttpStatsCollectorRuntimeConf] Runtime configuration

#{anchor:relayHttpStatsCollectorRuntimeConf}

Runtime Configuration

Property name Type Description Default value
enabled Boolean A flag that indicates if the statistics collector is enabled and if it will update the stats counters false

#{anchor:tcpStaticConf}

TCP Static Configuration

Property name Type Description Default value
enabled Boolean A flag if the TCP command interface is to be enabled false
port int The port for the TCP command interface 8180

#{anchor:serverConf}

#{anchor:retriesStaticConf}

Retries Configuration

Property name Type Description Default value
initSleep Long base sleep (when no errors) in milliseconds 0
maxSleep Long the maximum sleep between retries in milliseconds 60000
sleepIncFactor Double The exponential increase part in the sleep 2.0
sleepIncDelta Long The linear increase part in the sleep 1
maxRetryNum Integer Max number of error retries before giving up; < 0 infinite retries 10

Server Configuration

Property name Type Description Default value
host String Host name or IP address of the server localhost
id Integer Container ID of the server -1
name String A name that identifies the server {{host_id}}]
port Integer The HTTP port on which the server listens 8080
sources String Comma-separated list of sources supported by the server empty
Clone this wiki locally