mirror of https://github.com/apache/druid.git
commit
e7e5a87d68
|
@ -1,22 +1,32 @@
|
|||
[{
|
||||
"schema" : { "dataSource":"randseq",
|
||||
"aggregators":[ {"type":"count", "name":"events"},
|
||||
{"type":"doubleSum","name":"outColumn","fieldName":"inColumn"} ],
|
||||
"indexGranularity":"minute",
|
||||
"shardSpec" : { "type": "none" } },
|
||||
"config" : { "maxRowsInMemory" : 50000,
|
||||
"intermediatePersistPeriod" : "PT10m" },
|
||||
|
||||
"firehose" : { "type" : "rand",
|
||||
"sleepUsec": 100000,
|
||||
"maxGeneratedRows" : 5000000,
|
||||
"seed" : 0,
|
||||
"nTokens" : 19,
|
||||
"nPerSleep" : 3
|
||||
},
|
||||
"schema": {
|
||||
"dataSource": "randseq",
|
||||
"aggregators": [
|
||||
{"type": "count", "name": "events"},
|
||||
{"type": "doubleSum", "name": "outColumn", "fieldName": "inColumn"}
|
||||
],
|
||||
"indexGranularity": "minute",
|
||||
"shardSpec": {"type": "none"}
|
||||
},
|
||||
|
||||
"plumber" : { "type" : "realtime",
|
||||
"windowPeriod" : "PT5m",
|
||||
"segmentGranularity":"hour",
|
||||
"basePersistDirectory" : "/tmp/realtime/basePersist" }
|
||||
"config": {
|
||||
"maxRowsInMemory": 50000,
|
||||
"intermediatePersistPeriod": "PT10m"
|
||||
},
|
||||
|
||||
"firehose": {
|
||||
"type": "rand",
|
||||
"sleepUsec": 100000,
|
||||
"maxGeneratedRows": 5000000,
|
||||
"seed": 0,
|
||||
"nTokens": 19,
|
||||
"nPerSleep": 3
|
||||
},
|
||||
|
||||
"plumber": {
|
||||
"type": "realtime",
|
||||
"windowPeriod": "PT5m",
|
||||
"segmentGranularity": "hour",
|
||||
"basePersistDirectory": "/tmp/rand_realtime/basePersist"
|
||||
}
|
||||
}]
|
||||
|
|
|
@ -1,5 +1,5 @@
|
|||
#!/usr/bin/env bash
|
||||
echo " run RealtimeStandaloneMain service in background"
|
||||
echo "This will run RealtimeStandaloneMain service in background"
|
||||
set +u
|
||||
shopt -s xpg_echo
|
||||
shopt -s expand_aliases
|
||||
|
@ -7,18 +7,18 @@ trap "exit 1" 1 2 3 15
|
|||
|
||||
# props are set in src/main/resources/runtime.properties
|
||||
|
||||
echo "cleaning up previous run, if any"
|
||||
/bin/rm -fr /tmp/rand_realtime
|
||||
[ -d /tmp/rand_realtime ] && echo "cleaning up from previous run.." && /bin/rm -fr /tmp/rand_realtime
|
||||
|
||||
# check rand_realtime.spec exists
|
||||
# check spec file exists
|
||||
[ ! -e rand_realtime.spec ] && echo "expecting file rand_realtime.spec (as specified by property druid.realtime.specFile) to be in current directory" && exit 3
|
||||
|
||||
# start RealtimeNode process
|
||||
#
|
||||
java -Xmx400m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath target/druid-examples-rand-*-selfcontained.jar druid.examples.RealtimeStandaloneMain >RealtimeNode.out 2>&1 &
|
||||
echo "Log output of service can be found in ./RealtimeNode.out"
|
||||
java -Xmx600m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath target/druid-examples-rand-*-selfcontained.jar druid.examples.RealtimeStandaloneMain >RealtimeNode.out 2>&1 &
|
||||
PID=$!
|
||||
|
||||
trap "kill $PID" 1 2 3 15
|
||||
trap "kill $PID ; exit 1" 1 2 3 15
|
||||
sleep 4
|
||||
grep druid.examples.RandomFirehoseFactory RealtimeNode.out | awk '{ print $7,$8,$9,$10,$11,$12,$13,$14,$15 }'
|
||||
wait $PID
|
||||
|
|
|
@ -1,4 +1,5 @@
|
|||
# Properties for demo of Realtime Node in standalone mode.
|
||||
# To Use This: copy this file to runtime.properties and put directory containing it in classpath.
|
||||
#
|
||||
comment.origin=druid/examples/rand/src/main/resources/runtime.properties
|
||||
|
||||
|
@ -40,6 +41,14 @@ druid.paths.segmentInfoCache=/tmp/rand_realtime/segmentInfoCache
|
|||
# Path to schema definition file
|
||||
druid.request.logging.dir=/tmp/rand_realtime/log
|
||||
|
||||
# TODO: have these moved to spec file?
|
||||
# unknown # druid.realtime.dataSources=
|
||||
# unknown # druid.realtime.index.maxSize=500000
|
||||
# unknown # druid.realtime.persistPeriod=PT600S
|
||||
# unknown # druid.realtime.scheduledExec.threads=1
|
||||
# unknown # druid.realtime.uploadPeriod=PT3600S
|
||||
# unknown # druid.realtime.windowPeriod=PT600S
|
||||
|
||||
#druid.server.maxSize=0
|
||||
druid.server.maxSize=300000000000
|
||||
# =realtime or =historical (default)
|
||||
|
@ -91,7 +100,6 @@ druid.server.type=realtime
|
|||
# Default is value of druid.zk.paths.base + /properties
|
||||
#druid.zk.paths.propertiesPath=/druid/properties
|
||||
|
||||
|
||||
druid.host=127.0.0.1
|
||||
druid.port=8080
|
||||
|
||||
|
@ -100,6 +108,8 @@ druid.http.numThreads=10
|
|||
# default is 5 min. (300000)
|
||||
#druid.http.maxIdleTimeMillis=300000
|
||||
|
||||
# unknown # com.metamx.service=compute
|
||||
com.metamx.emitter.http=true
|
||||
com.metamx.emitter.logging=true
|
||||
com.metamx.emitter.logging.level=info
|
||||
com.metamx.metrics.emitter.period=PT60S
|
||||
|
@ -119,7 +129,6 @@ druid.processing.numThreads=3
|
|||
# other properties found
|
||||
#
|
||||
druid.computation.buffer.size=10000000
|
||||
|
||||
druid.merger.threads=1
|
||||
druid.merger.runner=remote
|
||||
druid.merger.whitelist.enabled=false
|
||||
|
@ -139,5 +148,6 @@ druid.realtime.specFile=rand_realtime.spec
|
|||
#
|
||||
# Integration-Test Related
|
||||
#
|
||||
# is this for RAM? which process?
|
||||
# is this for RAM? which process?
|
||||
druid.bard.cache.sizeInBytes=40000000
|
||||
#ignore#druid.bard.host=
|
||||
|
|
|
@ -1,11 +1,11 @@
|
|||
#!/usr/bin/env bash
|
||||
echo " This will run RealtimeStandaloneMain service in background"
|
||||
echo "This will run RealtimeStandaloneMain service in background"
|
||||
set +u
|
||||
shopt -s xpg_echo
|
||||
shopt -s expand_aliases
|
||||
|
||||
PF=./twitter4j.properties
|
||||
|
||||
# check twitter_realtime.spec exists
|
||||
[ ! -e twitter_realtime.spec ] && echo "expecting file twitter_realtime.spec (as specified by property druid.realtime.specFile) to be in current directory" && exit 3
|
||||
|
||||
# if twitter4j.properties already exists, then user is okay with having twitter pw in file.
|
||||
# Otherwise a twitter4j.properties file in curr. dir. is made temporarily for twitter login.
|
||||
if [ -e "$PF" ]; then
|
||||
|
@ -29,14 +29,16 @@ trap "${PF_CLEANUP} ; exit 1" 1 2 3 15
|
|||
|
||||
[ -d /tmp/twitter_realtime ] && echo "cleaning up from previous run.." && /bin/rm -fr /tmp/twitter_realtime
|
||||
|
||||
echo "Log output of service can be found in ./RealtimeNode.out"
|
||||
# check spec file exists
|
||||
[ ! -e twitter_realtime.spec ] && echo "expecting file twitter_realtime.spec (as specified by property druid.realtime.specFile) to be in current directory" && exit 3
|
||||
|
||||
OPT_PROPS=""
|
||||
#OPT_PROPS="-Dtwitter4j.debug=true -Dtwitter4j.http.prettyDebug=true"
|
||||
|
||||
# start RealtimeNode process
|
||||
#
|
||||
java -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Xmx600m -classpath target/druid-examples-twitter-*-selfcontained.jar $OPT_PROPS -Dtwitter4j.http.useSSL=true -Ddruid.realtime.specFile=twitter_realtime.spec druid.examples.RealtimeStandaloneMain >RealtimeNode.out 2>&1 &
|
||||
echo "Log output of service can be found in ./RealtimeNode.out"
|
||||
java -Xmx600m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath target/druid-examples-twitter-*-selfcontained.jar $OPT_PROPS -Dtwitter4j.http.useSSL=true druid.examples.RealtimeStandaloneMain >RealtimeNode.out 2>&1 &
|
||||
PID=$!
|
||||
|
||||
trap "${PF_CLEANUP} ; kill ${PID} ; exit 1" 1 2 3 15
|
||||
|
|
|
@ -25,9 +25,12 @@ druid.database.segmentTable=prod_segments
|
|||
druid.emitter.period=PT60S
|
||||
|
||||
druid.master.host
|
||||
# ToDo: description?
|
||||
# Poll period the master runs on
|
||||
druid.master.period=PT60S
|
||||
# Number of poll periods to wait for a node to come back before believing it is really gone
|
||||
druid.master.removedSegmentLifetime=1
|
||||
# Delay for the master to start its work, this should be sufficiently high so that the master can get all of the
|
||||
# information it needs from ZK before starting. It's a hack, but it works until we re-work our ZK integration.
|
||||
druid.master.startDelay=PT600S
|
||||
|
||||
# Path on local FS for storage of segments; dir. will be created if needed
|
||||
|
@ -36,6 +39,8 @@ druid.paths.indexCache=/tmp/twitter_realtime/indexCache
|
|||
druid.paths.segmentInfoCache=/tmp/twitter_realtime/segmentInfoCache
|
||||
|
||||
# Path to schema definition file
|
||||
druid.request.logging.dir=/tmp/twitter_realtime/log
|
||||
|
||||
# TODO: have these moved to spec file?
|
||||
# unknown # druid.realtime.dataSources=
|
||||
# unknown # druid.realtime.index.maxSize=500000
|
||||
|
@ -44,8 +49,6 @@ druid.paths.segmentInfoCache=/tmp/twitter_realtime/segmentInfoCache
|
|||
# unknown # druid.realtime.uploadPeriod=PT3600S
|
||||
# unknown # druid.realtime.windowPeriod=PT600S
|
||||
|
||||
druid.request.logging.dir=/tmp/twitter_realtime/log
|
||||
|
||||
#druid.server.maxSize=0
|
||||
druid.server.maxSize=300000000000
|
||||
# =realtime or =historical (default)
|
||||
|
@ -106,10 +109,10 @@ druid.http.numThreads=10
|
|||
#druid.http.maxIdleTimeMillis=300000
|
||||
|
||||
# unknown # com.metamx.service=compute
|
||||
com.metamx.emitter.logging=true
|
||||
com.metamx.emitter.http=true
|
||||
# unknown # com.metamx.emitter.logging.level=info
|
||||
# unknown # com.metamx.metrics.emitter.period=PT60S
|
||||
com.metamx.emitter.logging=true
|
||||
com.metamx.emitter.logging.level=info
|
||||
com.metamx.metrics.emitter.period=PT60S
|
||||
|
||||
# ZK quorum IPs; ZK coordinates in the form host1:port1[,host2:port2[, ...]]
|
||||
# if =none then do not contact zookeeper (only for RealtimeStandaloneMain examples)
|
||||
|
@ -145,6 +148,6 @@ druid.realtime.specFile=twitter_realtime.spec
|
|||
#
|
||||
# Integration-Test Related
|
||||
#
|
||||
# is this for RAM? which process?
|
||||
# is this for RAM? which process?
|
||||
druid.bard.cache.sizeInBytes=40000000
|
||||
#ignore#druid.bard.host=
|
||||
|
|
|
@ -1,36 +1,44 @@
|
|||
[{
|
||||
"schema" : { "dataSource":"twitterstream",
|
||||
"aggregators":[
|
||||
{"type":"count", "name":"tweets"},
|
||||
{"type":"doubleSum","fieldName":"follower_count","name":"total_follower_count"},
|
||||
{"type":"doubleSum","fieldName":"retweet_count","name":"total_retweet_count"},
|
||||
{"type":"doubleSum","fieldName":"friends_count","name":"total_friends_count"},
|
||||
{"type":"doubleSum","fieldName":"statuses_count","name":"total_statuses_count"},
|
||||
"schema": {
|
||||
"dataSource": "twitterstream",
|
||||
"aggregators": [
|
||||
{"type": "count", "name": "tweets"},
|
||||
{"type": "doubleSum", "fieldName": "follower_count", "name": "total_follower_count"},
|
||||
{"type": "doubleSum", "fieldName": "retweet_count", "name": "total_retweet_count" },
|
||||
{"type": "doubleSum", "fieldName": "friends_count", "name": "total_friends_count" },
|
||||
{"type": "doubleSum", "fieldName": "statuses_count", "name": "total_statuses_count"},
|
||||
|
||||
{"type":"min","fieldName":"follower_count","name":"min_follower_count"},
|
||||
{"type":"max","fieldName":"follower_count","name":"max_follower_count"},
|
||||
{"type": "min", "fieldName": "follower_count", "name": "min_follower_count"},
|
||||
{"type": "max", "fieldName": "follower_count", "name": "max_follower_count"},
|
||||
|
||||
{"type":"min","fieldName":"friends_count","name":"min_friends_count"},
|
||||
{"type":"max","fieldName":"friends_count","name":"max_friends_count"},
|
||||
{"type": "min", "fieldName": "friends_count", "name": "min_friends_count"},
|
||||
{"type": "max", "fieldName": "friends_count", "name": "max_friends_count"},
|
||||
|
||||
{"type":"min","fieldName":"statuses_count","name":"min_statuses_count"},
|
||||
{"type":"max","fieldName":"statuses_count","name":"max_statuses_count"},
|
||||
{"type": "min", "fieldName": "statuses_count", "name": "min_statuses_count"},
|
||||
{"type": "max", "fieldName": "statuses_count", "name": "max_statuses_count"},
|
||||
|
||||
{"type":"min","fieldName":"retweet_count","name":"min_retweet_count"},
|
||||
{"type":"max","fieldName":"retweet_count","name":"max_retweet_count"}
|
||||
],
|
||||
"indexGranularity":"minute",
|
||||
"shardSpec" : { "type": "none" } },
|
||||
"config" : { "maxRowsInMemory" : 50000,
|
||||
"intermediatePersistPeriod" : "PT2m" },
|
||||
|
||||
"firehose" : { "type" : "twitzer",
|
||||
"maxEventCount": 50000,
|
||||
"maxRunMinutes" : 10
|
||||
},
|
||||
{"type": "min", "fieldName": "retweet_count", "name": "min_retweet_count"},
|
||||
{"type": "max", "fieldName": "retweet_count", "name": "max_retweet_count"}
|
||||
],
|
||||
"indexGranularity": "minute",
|
||||
"shardSpec": {"type": "none"}
|
||||
},
|
||||
|
||||
"plumber" : { "type" : "realtime",
|
||||
"windowPeriod" : "PT3m",
|
||||
"segmentGranularity":"hour",
|
||||
"basePersistDirectory" : "/tmp/twitter_realtime/basePersist" }
|
||||
"config": {
|
||||
"maxRowsInMemory": 50000,
|
||||
"intermediatePersistPeriod": "PT2m"
|
||||
},
|
||||
|
||||
"firehose": {
|
||||
"type": "twitzer",
|
||||
"maxEventCount": 50000,
|
||||
"maxRunMinutes": 10
|
||||
},
|
||||
|
||||
"plumber": {
|
||||
"type": "realtime",
|
||||
"windowPeriod": "PT3m",
|
||||
"segmentGranularity": "hour",
|
||||
"basePersistDirectory": "/tmp/twitter_realtime/basePersist"
|
||||
}
|
||||
}]
|
||||
|
|
Loading…
Reference in New Issue