Merge pull request #39 from darxriggs/evaluation

Examples Cleanup
This commit is contained in:
cheddar 2012-12-06 14:40:41 -08:00
commit e7e5a87d68
6 changed files with 103 additions and 70 deletions

View File

@ -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"
}
}]

View File

@ -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

View File

@ -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=

View File

@ -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

View File

@ -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=

View File

@ -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"
}
}]