Skip to content
This repository was archived by the owner on Sep 18, 2021. It is now read-only.

Commit 4105f0b

Browse files
author
Robey Pointer
committed
Merge branch 'master' into site
2 parents 6a5d083 + 9cf4d58 commit 4105f0b

19 files changed

+364
-148
lines changed

ChangeLog

Lines changed: 13 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,18 @@
1+
2+
2.1.5
3+
-----
4+
release: 12 January 2012
5+
6+
- Only sync to disk when there's data to be written. [Stephan Zuercher]
7+
- Track latency measurements better, and track the percentage of java heap
8+
reserved for in-memory queues. [Stephan Zuercher]
9+
- Fix the startup script to work in "dash". [Matt Parlane]
10+
- Send the correct reponse back for the memcached protocol for delete. [Matt
11+
Erkkila]
12+
113
2.1.4
214
-----
15+
release: 21 November 2011
316

417
- Separate timers for journal fsync operations from those used for request
518
timeouts and queue expiry
@@ -9,7 +22,6 @@
922
bugs)
1023
- Creating a queue with an illegal name causes an error
1124

12-
1325
2.1.3
1426
-----
1527
release: 13 October 2011

config/production.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ new KestrelConfig {
2121
default.defaultJournalSize = 16.megabytes
2222
default.maxMemorySize = 128.megabytes
2323
default.maxJournalSize = 1.gigabyte
24-
default.syncJournal = 20.milliseconds
24+
default.syncJournal = 100.milliseconds
2525

2626
admin.httpPort = 2223
2727

project/build.properties

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1,9 +1,9 @@
11
#Project properties
2-
#Mon Nov 21 15:08:49 PST 2011
2+
#Thu Jan 12 16:30:20 PST 2012
33
project.organization=net.lag
44
project.name=kestrel
55
sbt.version=0.7.4
6-
project.version=2.1.5-SNAPSHOT
6+
project.version=2.1.6-SNAPSHOT
77
def.scala.version=2.7.7
88
build.scala.versions=2.8.1
99
project.initialize=false

project/build/KestrelProject.scala

Lines changed: 18 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -58,7 +58,24 @@ class KestrelProject(info: ProjectInfo) extends StandardServiceProject(info) wit
5858
lazy val packageLoadTests = packageLoadTestsAction
5959
override def packageDistTask = packageLoadTestsAction && super.packageDistTask
6060

61-
// override def fork = forkRun(List("-Xmx1024m", "-verbosegc", "-XX:+PrintGCDetails"))
61+
// generate a distribution zip for release.
62+
def releaseDistTask = task {
63+
val releaseDistPath = "dist-release" / distName ##
64+
65+
releaseDistPath.asFile.mkdirs()
66+
(releaseDistPath / "libs").asFile.mkdirs()
67+
(releaseDistPath / "config").asFile.mkdirs()
68+
69+
FileUtilities.copyFlat(List(jarPath), releaseDistPath, log).left.toOption orElse
70+
FileUtilities.copyFlat(List(outputPath / loadTestJarFilename), releaseDistPath, log).left.toOption orElse
71+
FileUtilities.copyFlat(dependentJars.get, releaseDistPath / "libs", log).left.toOption orElse
72+
FileUtilities.copy(((configPath ***) --- (configPath ** "*.class")).get, releaseDistPath / "config", log).left.toOption orElse
73+
FileUtilities.copy((scriptsOutputPath ***).get, releaseDistPath, log).left.toOption orElse
74+
FileUtilities.zip((("dist-release" ##) / distName).get, "dist-release" / (distName + ".zip"), true, log)
75+
}
76+
val ReleaseDistDescription = "Creates a deployable zip file with dependencies, config, and scripts."
77+
lazy val releaseDist = releaseDistTask.dependsOn(`package`, makePom, copyScripts).describedAs(ReleaseDistDescription)
78+
6279

6380
lazy val putMany = task { args =>
6481
runTask(Some("net.lag.kestrel.load.PutMany"), testClasspath, args).dependsOn(testCompile)

project/release.properties

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,4 @@
11
#Automatically generated by ReleaseManagement
2-
#Mon Nov 21 15:08:49 PST 2011
3-
version=2.1.4
4-
sha1=10d84a9dee3c23813c8fd76b55ce95d958154423
2+
#Thu Jan 12 16:30:20 PST 2012
3+
version=2.1.5
4+
sha1=459ec824963e31e0da78dbc425be0c7084a3e298

src/main/scala/net/lag/kestrel/Journal.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -20,14 +20,14 @@ package net.lag.kestrel
2020
import java.io._
2121
import java.nio.{ByteBuffer, ByteOrder}
2222
import java.nio.channels.FileChannel
23-
import java.util.concurrent.LinkedBlockingQueue
23+
import java.util.concurrent.{LinkedBlockingQueue, ScheduledExecutorService}
2424
import java.util.concurrent.atomic.AtomicInteger
2525
import scala.annotation.tailrec
2626
import com.twitter.conversions.storage._
2727
import com.twitter.conversions.time._
2828
import com.twitter.logging.Logger
2929
import com.twitter.ostrich.admin.BackgroundProcess
30-
import com.twitter.util.{Future, Duration, Timer, Time}
30+
import com.twitter.util.{Future, Duration, Time}
3131

3232
case class BrokenItemException(lastValidPosition: Long, cause: Throwable) extends IOException(cause)
3333

@@ -51,7 +51,7 @@ object JournalItem {
5151
/**
5252
* Codes for working with the journal file for a PersistentQueue.
5353
*/
54-
class Journal(queuePath: File, queueName: String, syncTimer: Timer, syncJournal: Duration) {
54+
class Journal(queuePath: File, queueName: String, syncScheduler: ScheduledExecutorService, syncJournal: Duration) {
5555
import Journal._
5656

5757
private val log = Logger.get(getClass)
@@ -97,7 +97,7 @@ class Journal(queuePath: File, queueName: String, syncTimer: Timer, syncJournal:
9797
def this(fullPath: String) = this(fullPath, Duration.MaxValue)
9898

9999
private def open(file: File) {
100-
writer = new PeriodicSyncFile(file, syncTimer, syncJournal)
100+
writer = new PeriodicSyncFile(file, syncScheduler, syncJournal)
101101
}
102102

103103
def open() {

src/main/scala/net/lag/kestrel/Kestrel.scala

Lines changed: 19 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -18,9 +18,10 @@
1818
package net.lag.kestrel
1919

2020
import java.net.InetSocketAddress
21-
import java.util.concurrent.{Executors, ExecutorService, TimeUnit}
21+
import java.util.concurrent._
2222
import java.util.concurrent.atomic.AtomicInteger
2323
import scala.collection.{immutable, mutable}
24+
import com.twitter.concurrent.NamedPoolThreadFactory
2425
import com.twitter.conversions.time._
2526
import com.twitter.logging.Logger
2627
import com.twitter.naggati.codec.MemcacheCodec
@@ -72,7 +73,7 @@ class Kestrel(defaultQueueConfig: QueueConfig, builders: List[QueueBuilder],
7273

7374
var queueCollection: QueueCollection = null
7475
var timer: Timer = null
75-
var journalSyncTimer: Timer = null
76+
var journalSyncScheduler: ScheduledExecutorService = null
7677
var executor: ExecutorService = null
7778
var channelFactory: ChannelFactory = null
7879
var memcacheAcceptor: Option[Channel] = None
@@ -93,15 +94,25 @@ class Kestrel(defaultQueueConfig: QueueConfig, builders: List[QueueBuilder],
9394
listenAddress, memcacheListenPort, textListenPort, queuePath, protocol,
9495
expirationTimerFrequency, clientTimeout, maxOpenTransactions)
9596

96-
// this means no timeout will be at better granularity than N ms.
97-
journalSyncTimer = new HashedWheelTimer(10, TimeUnit.MILLISECONDS)
97+
// this means no timeout will be at better granularity than 100 ms.
9898
timer = new HashedWheelTimer(100, TimeUnit.MILLISECONDS)
9999

100-
queueCollection = new QueueCollection(queuePath, new NettyTimer(timer), new NettyTimer(journalSyncTimer), defaultQueueConfig, builders)
100+
journalSyncScheduler =
101+
new ScheduledThreadPoolExecutor(
102+
Runtime.getRuntime.availableProcessors,
103+
new NamedPoolThreadFactory("journal-sync", true),
104+
new RejectedExecutionHandler {
105+
override def rejectedExecution(r: Runnable, executor: ThreadPoolExecutor) {
106+
log.warning("Rejected journal fsync")
107+
}
108+
})
109+
110+
queueCollection = new QueueCollection(queuePath, new NettyTimer(timer), journalSyncScheduler, defaultQueueConfig, builders)
101111
queueCollection.loadQueues()
102112

103113
Stats.addGauge("items") { queueCollection.currentItems.toDouble }
104114
Stats.addGauge("bytes") { queueCollection.currentBytes.toDouble }
115+
Stats.addGauge("reserved_memory_ratio") { queueCollection.reservedMemoryRatio }
105116

106117
// netty setup:
107118
executor = Executors.newCachedThreadPool()
@@ -161,8 +172,9 @@ class Kestrel(defaultQueueConfig: QueueConfig, builders: List[QueueBuilder],
161172
executor.awaitTermination(5, TimeUnit.SECONDS)
162173
timer.stop()
163174
timer = null
164-
journalSyncTimer.stop()
165-
journalSyncTimer = null
175+
journalSyncScheduler.shutdown()
176+
journalSyncScheduler.awaitTermination(5, TimeUnit.SECONDS)
177+
journalSyncScheduler = null
166178
log.info("Goodbye.")
167179
}
168180

src/main/scala/net/lag/kestrel/KestrelHandler.scala

Lines changed: 0 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -194,21 +194,6 @@ abstract class KestrelHandler(val queues: QueueCollection, val maxOpenTransactio
194194
waitingFor = Some(future)
195195
future.map { itemOption =>
196196
waitingFor = None
197-
timeout match {
198-
case None => {
199-
val usec = (Time.now - startTime).inMicroseconds.toInt
200-
val statName = if (itemOption.isDefined) "get_hit_latency_usec" else "get_miss_latency_usec"
201-
Stats.addMetric(statName, usec)
202-
Stats.addMetric("q/" + key + "/" + statName, usec)
203-
}
204-
case Some(_) => {
205-
if (!itemOption.isDefined) {
206-
val msec = (Time.now - startTime).inMilliseconds.toInt
207-
Stats.addMetric("get_timeout_msec", msec)
208-
Stats.addMetric("q/" + key + "/get_timeout_msec", msec)
209-
}
210-
}
211-
}
212197
itemOption.foreach { item =>
213198
log.debug("get <- %s", item)
214199
if (opening) pendingTransactions.add(key, item.xid)

src/main/scala/net/lag/kestrel/MemcacheHandler.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -85,7 +85,7 @@ extends NettyHandler[MemcacheRequest](channelGroup, queueCollection, maxOpenTran
8585
dumpStats(request.line.drop(1))
8686
case "delete" =>
8787
delete(request.line(1))
88-
channel.write(new MemcacheResponse("END"))
88+
channel.write(new MemcacheResponse("DELETED"))
8989
case "flush_expired" =>
9090
channel.write(new MemcacheResponse(flushExpired(request.line(1)).toString))
9191
case "flush_all_expired" =>
@@ -189,6 +189,7 @@ extends NettyHandler[MemcacheRequest](channelGroup, queueCollection, maxOpenTran
189189
report += (("curr_items", queues.currentItems.toString))
190190
report += (("total_items", Stats.getCounter("total_items")().toString))
191191
report += (("bytes", queues.currentBytes.toString))
192+
report += (("reserved_memory_ratio", "%.3f".format(queues.reservedMemoryRatio)))
192193
report += (("curr_connections", Kestrel.sessions.get().toString))
193194
report += (("total_connections", Stats.getCounter("total_connections")().toString))
194195
report += (("cmd_get", Stats.getCounter("cmd_get")().toString))

src/main/scala/net/lag/kestrel/PeriodicSyncFile.scala

Lines changed: 53 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -1,47 +1,86 @@
11
package net.lag.kestrel
22

3-
import java.nio.ByteBuffer
4-
import java.util.concurrent.ConcurrentLinkedQueue
53
import com.twitter.conversions.time._
4+
import com.twitter.ostrich.stats.Stats
65
import com.twitter.util._
76
import java.io.{IOException, FileOutputStream, File}
7+
import java.nio.ByteBuffer
8+
import java.util.concurrent.{ConcurrentLinkedQueue, ScheduledExecutorService, ScheduledFuture, TimeUnit}
9+
10+
abstract class PeriodicSyncTask(val scheduler: ScheduledExecutorService, initialDelay: Duration, period: Duration)
11+
extends Runnable {
12+
@volatile private[this] var scheduledFsync: Option[ScheduledFuture[_]] = None
13+
14+
def start() {
15+
synchronized {
16+
if (scheduledFsync.isEmpty && period > 0.seconds) {
17+
val handle = scheduler.scheduleWithFixedDelay(this, initialDelay.inMilliseconds, period.inMilliseconds,
18+
TimeUnit.MILLISECONDS)
19+
scheduledFsync = Some(handle)
20+
}
21+
}
22+
}
23+
24+
def stop() {
25+
synchronized { _stop() }
26+
}
27+
28+
def stopIf(f: => Boolean) {
29+
synchronized {
30+
if (f) _stop()
31+
}
32+
}
33+
34+
private[this] def _stop() {
35+
scheduledFsync.foreach { _.cancel(false) }
36+
scheduledFsync = None
37+
}
38+
}
839

940
/**
1041
* Open a file for writing, and fsync it on a schedule. The period may be 0 to force an fsync
1142
* after every write, or `Duration.MaxValue` to never fsync.
1243
*/
13-
class PeriodicSyncFile(file: File, timer: Timer, period: Duration) {
44+
class PeriodicSyncFile(file: File, scheduler: ScheduledExecutorService, period: Duration) {
1445
// pre-completed future for writers who are behaving synchronously.
1546
private final val DONE = Future(())
1647

17-
val writer = new FileOutputStream(file, true).getChannel
18-
val promises = new ConcurrentLinkedQueue[Promise[Unit]]()
19-
20-
@volatile var closed = false
48+
case class TimestampedPromise(val promise: Promise[Unit], val time: Time)
2149

22-
if (period > 0.seconds && period < Duration.MaxValue) {
23-
timer.schedule(Time.now, period) {
50+
val writer = new FileOutputStream(file, true).getChannel
51+
val promises = new ConcurrentLinkedQueue[TimestampedPromise]()
52+
val periodicSyncTask = new PeriodicSyncTask(scheduler, period, period) {
53+
override def run() {
2454
if (!closed && !promises.isEmpty) fsync()
2555
}
2656
}
2757

58+
@volatile var closed = false
59+
2860
private def fsync() {
2961
synchronized {
3062
// race: we could underestimate the number of completed writes. that's okay.
3163
val completed = promises.size
64+
val fsyncStart = Time.now
3265
try {
3366
writer.force(false)
3467
} catch {
3568
case e: IOException =>
3669
for (i <- 0 until completed) {
37-
promises.poll().setException(e)
70+
promises.poll().promise.setException(e)
3871
}
3972
return;
4073
}
4174

4275
for (i <- 0 until completed) {
43-
promises.poll().setValue(())
76+
val timestampedPromise = promises.poll()
77+
timestampedPromise.promise.setValue(())
78+
val delaySinceWrite = fsyncStart - timestampedPromise.time
79+
val durationBehind = if (delaySinceWrite > period) delaySinceWrite - period else 0.seconds
80+
Stats.addMetric("fsync_delay_usec", durationBehind.inMicroseconds.toInt)
4481
}
82+
83+
periodicSyncTask.stopIf { promises.isEmpty }
4584
}
4685
}
4786

@@ -62,7 +101,8 @@ class PeriodicSyncFile(file: File, timer: Timer, period: Duration) {
62101
DONE
63102
} else {
64103
val promise = new Promise[Unit]()
65-
promises.add(promise)
104+
promises.add(TimestampedPromise(promise, Time.now))
105+
periodicSyncTask.start()
66106
promise
67107
}
68108
}
@@ -73,6 +113,7 @@ class PeriodicSyncFile(file: File, timer: Timer, period: Duration) {
73113
*/
74114
def close() {
75115
closed = true
116+
periodicSyncTask.stop()
76117
fsync()
77118
writer.close()
78119
}

0 commit comments

Comments
 (0)