merged with master head

This commit is contained in:
jboner 2009-10-19 21:17:04 +02:00
commit 00b606b3e8
37 changed files with 2688 additions and 123 deletions

2
.gitignore vendored
View file

@ -6,7 +6,7 @@ reports
dist dist
build build
target target
deploy deploy/*.jar
data data
out out
logs logs

View file

@ -29,7 +29,6 @@ object Annotations {
} }
/** /**
* Factory object for Active Objects.
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
@ -62,49 +61,49 @@ object ActiveObject {
def newInstance[T](target: Class[T], timeout: Long, dispatcher: MessageDispatcher): T = { def newInstance[T](target: Class[T], timeout: Long, dispatcher: MessageDispatcher): T = {
val actor = new Dispatcher(None) val actor = new Dispatcher(None)
actor.dispatcher = dispatcher actor.messageDispatcher = dispatcher
newInstance(target, actor, None, timeout) newInstance(target, actor, None, timeout)
} }
def newInstance[T](target: Class[T], timeout: Long, dispatcher: MessageDispatcher, restartCallbacks: Option[RestartCallbacks]): T = { def newInstance[T](target: Class[T], timeout: Long, dispatcher: MessageDispatcher, restartCallbacks: Option[RestartCallbacks]): T = {
val actor = new Dispatcher(restartCallbacks) val actor = new Dispatcher(restartCallbacks)
actor.dispatcher = dispatcher actor.messageDispatcher = dispatcher
newInstance(target, actor, None, timeout) newInstance(target, actor, None, timeout)
} }
def newInstance[T](intf: Class[T], target: AnyRef, timeout: Long, dispatcher: MessageDispatcher): T = { def newInstance[T](intf: Class[T], target: AnyRef, timeout: Long, dispatcher: MessageDispatcher): T = {
val actor = new Dispatcher(None) val actor = new Dispatcher(None)
actor.dispatcher = dispatcher actor.messageDispatcher = dispatcher
newInstance(intf, target, actor, None, timeout) newInstance(intf, target, actor, None, timeout)
} }
def newInstance[T](intf: Class[T], target: AnyRef, timeout: Long, dispatcher: MessageDispatcher, restartCallbacks: Option[RestartCallbacks]): T = { def newInstance[T](intf: Class[T], target: AnyRef, timeout: Long, dispatcher: MessageDispatcher, restartCallbacks: Option[RestartCallbacks]): T = {
val actor = new Dispatcher(restartCallbacks) val actor = new Dispatcher(restartCallbacks)
actor.dispatcher = dispatcher actor.messageDispatcher = dispatcher
newInstance(intf, target, actor, None, timeout) newInstance(intf, target, actor, None, timeout)
} }
def newRemoteInstance[T](target: Class[T], timeout: Long, dispatcher: MessageDispatcher, hostname: String, port: Int): T = { def newRemoteInstance[T](target: Class[T], timeout: Long, dispatcher: MessageDispatcher, hostname: String, port: Int): T = {
val actor = new Dispatcher(None) val actor = new Dispatcher(None)
actor.dispatcher = dispatcher actor.messageDispatcher = dispatcher
newInstance(target, actor, Some(new InetSocketAddress(hostname, port)), timeout) newInstance(target, actor, Some(new InetSocketAddress(hostname, port)), timeout)
} }
def newRemoteInstance[T](target: Class[T], timeout: Long, dispatcher: MessageDispatcher, hostname: String, port: Int, restartCallbacks: Option[RestartCallbacks]): T = { def newRemoteInstance[T](target: Class[T], timeout: Long, dispatcher: MessageDispatcher, hostname: String, port: Int, restartCallbacks: Option[RestartCallbacks]): T = {
val actor = new Dispatcher(restartCallbacks) val actor = new Dispatcher(restartCallbacks)
actor.dispatcher = dispatcher actor.messageDispatcher = dispatcher
newInstance(target, actor, Some(new InetSocketAddress(hostname, port)), timeout) newInstance(target, actor, Some(new InetSocketAddress(hostname, port)), timeout)
} }
def newRemoteInstance[T](intf: Class[T], target: AnyRef, timeout: Long, dispatcher: MessageDispatcher, hostname: String, port: Int): T = { def newRemoteInstance[T](intf: Class[T], target: AnyRef, timeout: Long, dispatcher: MessageDispatcher, hostname: String, port: Int): T = {
val actor = new Dispatcher(None) val actor = new Dispatcher(None)
actor.dispatcher = dispatcher actor.messageDispatcher = dispatcher
newInstance(intf, target, actor, Some(new InetSocketAddress(hostname, port)), timeout) newInstance(intf, target, actor, Some(new InetSocketAddress(hostname, port)), timeout)
} }
def newRemoteInstance[T](intf: Class[T], target: AnyRef, timeout: Long, dispatcher: MessageDispatcher, hostname: String, port: Int, restartCallbacks: Option[RestartCallbacks]): T = { def newRemoteInstance[T](intf: Class[T], target: AnyRef, timeout: Long, dispatcher: MessageDispatcher, hostname: String, port: Int, restartCallbacks: Option[RestartCallbacks]): T = {
val actor = new Dispatcher(restartCallbacks) val actor = new Dispatcher(restartCallbacks)
actor.dispatcher = dispatcher actor.messageDispatcher = dispatcher
newInstance(intf, target, actor, Some(new InetSocketAddress(hostname, port)), timeout) newInstance(intf, target, actor, Some(new InetSocketAddress(hostname, port)), timeout)
} }

View file

@ -108,7 +108,7 @@ trait Actor extends Logging with TransactionManagement {
* .buildThreadPool * .buildThreadPool
* </pre> * </pre>
*/ */
protected[akka] var dispatcher: MessageDispatcher = { protected[akka] var messageDispatcher: MessageDispatcher = {
val dispatcher = Dispatchers.newEventBasedThreadPoolDispatcher(getClass.getName) val dispatcher = Dispatchers.newEventBasedThreadPoolDispatcher(getClass.getName)
mailbox = dispatcher.messageQueue mailbox = dispatcher.messageQueue
dispatcher.registerHandler(this, new ActorMessageInvoker(this)) dispatcher.registerHandler(this, new ActorMessageInvoker(this))
@ -214,7 +214,7 @@ trait Actor extends Logging with TransactionManagement {
*/ */
def start = synchronized { def start = synchronized {
if (!isRunning) { if (!isRunning) {
dispatcher.start messageDispatcher.start
isRunning = true isRunning = true
//if (isTransactional) this !! TransactionalInit //if (isTransactional) this !! TransactionalInit
} }
@ -298,14 +298,16 @@ trait Actor extends Logging with TransactionManagement {
case Some(future) => future.completeWithResult(message) case Some(future) => future.completeWithResult(message)
} }
def dispatcher = messageDispatcher
/** /**
* Sets the dispatcher for this actor. Needs to be invoked before the actor is started. * Sets the dispatcher for this actor. Needs to be invoked before the actor is started.
*/ */
def setDispatcher(disp: MessageDispatcher) = synchronized { def dispatcher_=(dispatcher: MessageDispatcher): Unit = synchronized {
if (!isRunning) { if (!isRunning) {
dispatcher = disp messageDispatcher = dispatcher
mailbox = dispatcher.messageQueue mailbox = messageDispatcher.messageQueue
dispatcher.registerHandler(this, new ActorMessageInvoker(this)) messageDispatcher.registerHandler(this, new ActorMessageInvoker(this))
} else throw new IllegalArgumentException("Can not swap dispatcher for " + toString + " after it has been started") } else throw new IllegalArgumentException("Can not swap dispatcher for " + toString + " after it has been started")
} }
@ -625,9 +627,9 @@ trait Actor extends Logging with TransactionManagement {
private[akka] def swapDispatcher(disp: MessageDispatcher) = synchronized { private[akka] def swapDispatcher(disp: MessageDispatcher) = synchronized {
dispatcher = disp messageDispatcher = disp
mailbox = dispatcher.messageQueue mailbox = messageDispatcher.messageQueue
dispatcher.registerHandler(this, new ActorMessageInvoker(this)) messageDispatcher.registerHandler(this, new ActorMessageInvoker(this))
} }
private def serializeMessage(message: AnyRef): AnyRef = if (Actor.SERIALIZE_MESSAGES) { private def serializeMessage(message: AnyRef): AnyRef = if (Actor.SERIALIZE_MESSAGES) {

View file

@ -7,7 +7,7 @@ package se.scalablesolutions.akka.dispatch
import java.util.concurrent.LinkedBlockingQueue import java.util.concurrent.LinkedBlockingQueue
import java.util.Queue import java.util.Queue
import actor.{Actor, ActorMessageInvoker} import se.scalablesolutions.akka.actor.{Actor, ActorMessageInvoker}
/** /**
* Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue. * Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue.
@ -44,8 +44,8 @@ class ThreadBasedDispatcher private[akka] (val name: String, val messageHandler:
selectorThread.interrupt selectorThread.interrupt
} }
def registerHandler(key: AnyRef, handler: MessageInvoker) = throw new UnsupportedOperationException def registerHandler(key: AnyRef, handler: MessageInvoker) = {}
def unregisterHandler(key: AnyRef) = throw new UnsupportedOperationException def unregisterHandler(key: AnyRef) = {}
} }
class BlockingMessageQueue(name: String) extends MessageQueue { class BlockingMessageQueue(name: String) extends MessageQueue {

View file

@ -19,13 +19,6 @@ import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory
import org.jboss.netty.handler.codec.frame.{LengthFieldBasedFrameDecoder, LengthFieldPrepender} import org.jboss.netty.handler.codec.frame.{LengthFieldBasedFrameDecoder, LengthFieldPrepender}
import org.jboss.netty.handler.codec.protobuf.{ProtobufDecoder, ProtobufEncoder} import org.jboss.netty.handler.codec.protobuf.{ProtobufDecoder, ProtobufEncoder}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class RemoteServer extends Logging {
def start = RemoteServer.start(None)
}
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
@ -34,8 +27,9 @@ object RemoteServer extends Logging {
val PORT = config.getInt("akka.remote.port", 9999) val PORT = config.getInt("akka.remote.port", 9999)
val CONNECTION_TIMEOUT_MILLIS = config.getInt("akka.remote.connection-timeout", 1000) val CONNECTION_TIMEOUT_MILLIS = config.getInt("akka.remote.connection-timeout", 1000)
val name = "RemoteServer@" + HOSTNAME private var hostname = HOSTNAME
private var port = PORT
@volatile private var isRunning = false @volatile private var isRunning = false
@volatile private var isConfigured = false @volatile private var isConfigured = false
@ -45,19 +39,26 @@ object RemoteServer extends Logging {
private val bootstrap = new ServerBootstrap(factory) private val bootstrap = new ServerBootstrap(factory)
def name = "RemoteServer@" + hostname + ":" + port
def start: Unit = start(None) def start: Unit = start(None)
def start(loader: Option[ClassLoader]): Unit = start(HOSTNAME, PORT)
def start(loader: Option[ClassLoader]): Unit = start(HOSTNAME, PORT, loader)
def start(hostname: String, port: Int): Unit = start(hostname, port, None) def start(hostname: String, port: Int): Unit = start(hostname, port, None)
def start(hostname: String, port: Int, loader: Option[ClassLoader]): Unit = synchronized {
def start(_hostname: String, _port: Int, loader: Option[ClassLoader]): Unit = synchronized {
if (!isRunning) { if (!isRunning) {
log.info("Starting remote server at [%s:%s]", HOSTNAME, PORT) hostname = _hostname
port = _port
log.info("Starting remote server at [%s:%s]", hostname, port)
bootstrap.setPipelineFactory(new RemoteServerPipelineFactory(name, loader)) bootstrap.setPipelineFactory(new RemoteServerPipelineFactory(name, loader))
// FIXME make these RemoteServer options configurable // FIXME make these RemoteServer options configurable
bootstrap.setOption("child.tcpNoDelay", true) bootstrap.setOption("child.tcpNoDelay", true)
bootstrap.setOption("child.keepAlive", true) bootstrap.setOption("child.keepAlive", true)
bootstrap.setOption("child.reuseAddress", true) bootstrap.setOption("child.reuseAddress", true)
bootstrap.setOption("child.connectTimeoutMillis", CONNECTION_TIMEOUT_MILLIS) bootstrap.setOption("child.connectTimeoutMillis", CONNECTION_TIMEOUT_MILLIS)
bootstrap.bind(new InetSocketAddress(HOSTNAME, PORT)) bootstrap.bind(new InetSocketAddress(hostname, port))
isRunning = true isRunning = true
} }
} }

View file

@ -37,6 +37,12 @@ object Serializer {
val EMPTY_CLASS_ARRAY = Array[Class[_]]() val EMPTY_CLASS_ARRAY = Array[Class[_]]()
val EMPTY_ANY_REF_ARRAY = Array[AnyRef]() val EMPTY_ANY_REF_ARRAY = Array[AnyRef]()
object NOOP extends Serializer {
def deepClone(obj: AnyRef): AnyRef = obj
def out(obj: AnyRef): Array[Byte] = obj.asInstanceOf[Array[Byte]]
def in(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = bytes
}
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */

View file

@ -4,8 +4,8 @@ import junit.framework.Test
import junit.framework.TestCase import junit.framework.TestCase
import junit.framework.TestSuite import junit.framework.TestSuite
import actor.{ActorSpec, RemoteActorSpec, InMemoryActorSpec, SupervisorSpec, RemoteSupervisorSpec,SchedulerSpec} import se.scalablesolutions.akka.actor.{RemoteActorSpec, InMemoryActorSpec, ThreadBasedActorSpec, SupervisorSpec, RemoteSupervisorSpec, SchedulerSpec}
import dispatch.{EventBasedSingleThreadDispatcherTest, EventBasedThreadPoolDispatcherTest} import se.scalablesolutions.akka.dispatch.{EventBasedSingleThreadDispatcherTest, EventBasedThreadPoolDispatcherTest}
object AllTest extends TestCase { object AllTest extends TestCase {
def suite(): Test = { def suite(): Test = {
@ -14,7 +14,9 @@ object AllTest extends TestCase {
suite.addTestSuite(classOf[RemoteSupervisorSpec]) suite.addTestSuite(classOf[RemoteSupervisorSpec])
suite.addTestSuite(classOf[EventBasedSingleThreadDispatcherTest]) suite.addTestSuite(classOf[EventBasedSingleThreadDispatcherTest])
suite.addTestSuite(classOf[EventBasedThreadPoolDispatcherTest]) suite.addTestSuite(classOf[EventBasedThreadPoolDispatcherTest])
suite.addTestSuite(classOf[ActorSpec]) suite.addTestSuite(classOf[ThreadBasedActorSpec])
suite.addTestSuite(classOf[EventBasedSingleThreadDispatcherTest])
suite.addTestSuite(classOf[EventBasedThreadPoolDispatcherTest])
suite.addTestSuite(classOf[RemoteActorSpec]) suite.addTestSuite(classOf[RemoteActorSpec])
suite.addTestSuite(classOf[InMemoryActorSpec]) suite.addTestSuite(classOf[InMemoryActorSpec])
suite.addTestSuite(classOf[SchedulerSpec]) suite.addTestSuite(classOf[SchedulerSpec])

View file

@ -0,0 +1,69 @@
package se.scalablesolutions.akka.actor
import java.util.concurrent.TimeUnit
import junit.framework.Assert._
import se.scalablesolutions.akka.dispatch.Dispatchers
class EventBasedSingleThreadActorSpec extends junit.framework.TestCase {
private val unit = TimeUnit.MILLISECONDS
class TestActor extends Actor {
dispatcher = Dispatchers.newEventBasedSingleThreadDispatcher(name)
def receive: PartialFunction[Any, Unit] = {
case "Hello" =>
reply("World")
case "Failure" =>
throw new RuntimeException("expected")
}
}
def testSendOneWay = {
implicit val timeout = 5000L
var oneWay = "nada"
val actor = new Actor {
def receive: PartialFunction[Any, Unit] = {
case "OneWay" => oneWay = "received"
}
}
actor.start
val result = actor ! "OneWay"
Thread.sleep(100)
assertEquals("received", oneWay)
actor.stop
}
def testSendReplySync = {
implicit val timeout = 5000L
val actor = new TestActor
actor.start
val result: String = actor !? "Hello"
assertEquals("World", result)
actor.stop
}
def testSendReplyAsync = {
implicit val timeout = 5000L
val actor = new TestActor
actor.start
val result = actor !! "Hello"
assertEquals("World", result.get.asInstanceOf[String])
actor.stop
}
def testSendReceiveException = {
implicit val timeout = 5000L
val actor = new TestActor
actor.start
try {
actor !! "Failure"
fail("Should have thrown an exception")
} catch {
case e =>
assertEquals("expected", e.getMessage())
}
actor.stop
}
}

View file

@ -5,9 +5,11 @@ import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.locks.Lock import java.util.concurrent.locks.Lock
import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.locks.ReentrantLock
import org.junit.{Test, Before} import org.junit.{Test, Before}
import org.junit.Assert._ import org.junit.Assert._
import junit.framework.TestCase import junit.framework.TestCase
import se.scalablesolutions.akka.actor.Actor import se.scalablesolutions.akka.actor.Actor
class EventBasedSingleThreadDispatcherTest extends TestCase { class EventBasedSingleThreadDispatcherTest extends TestCase {
@ -108,8 +110,8 @@ class EventBasedSingleThreadDispatcherTest extends TestCase {
}) })
dispatcher.start dispatcher.start
for (i <- 0 until 100) { for (i <- 0 until 100) {
dispatcher.messageQueue.append(new MessageInvocation(key1, new Integer(i), None, None)) dispatcher.messageQueue.append(new MessageInvocation(key1, new java.lang.Integer(i), None, None))
dispatcher.messageQueue.append(new MessageInvocation(key2, new Integer(i), None, None)) dispatcher.messageQueue.append(new MessageInvocation(key2, new java.lang.Integer(i), None, None))
} }
assertTrue(handleLatch.await(5, TimeUnit.SECONDS)) assertTrue(handleLatch.await(5, TimeUnit.SECONDS))
assertFalse(threadingIssueDetected.get) assertFalse(threadingIssueDetected.get)

View file

@ -2,9 +2,9 @@ package se.scalablesolutions.akka.actor
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import org.junit.Assert._ import junit.framework.Assert._
class ActorSpec extends junit.framework.TestCase { class EventBasedThreadPoolActorSpec extends junit.framework.TestCase {
private val unit = TimeUnit.MILLISECONDS private val unit = TimeUnit.MILLISECONDS
class TestActor extends Actor { class TestActor extends Actor {
@ -15,7 +15,7 @@ class ActorSpec extends junit.framework.TestCase {
throw new RuntimeException("expected") throw new RuntimeException("expected")
} }
} }
def testSendOneWay = { def testSendOneWay = {
implicit val timeout = 5000L implicit val timeout = 5000L
var oneWay = "nada" var oneWay = "nada"

View file

@ -151,8 +151,8 @@ class EventBasedThreadPoolDispatcherTest extends TestCase {
}) })
dispatcher.start dispatcher.start
for (i <- 0 until 100) { for (i <- 0 until 100) {
dispatcher.messageQueue.append(new MessageInvocation(key1, new Integer(i), None, None)) dispatcher.messageQueue.append(new MessageInvocation(key1, new java.lang.Integer(i), None, None))
dispatcher.messageQueue.append(new MessageInvocation(key2, new Integer(i), None, None)) dispatcher.messageQueue.append(new MessageInvocation(key2, new java.lang.Integer(i), None, None))
} }
assertTrue(handleLatch.await(5, TimeUnit.SECONDS)) assertTrue(handleLatch.await(5, TimeUnit.SECONDS))
assertFalse(threadingIssueDetected.get) assertFalse(threadingIssueDetected.get)

View file

@ -34,7 +34,7 @@ case class User(val usernamePassword: Tuple2[String, String],
def toBytes: Array[Byte] = toByteArray(this) def toBytes: Array[Byte] = toByteArray(this)
} }
case class RemotePing extends TestMessage case object RemotePing extends TestMessage
case object RemotePong extends TestMessage case object RemotePong extends TestMessage
case object RemoteOneWay extends TestMessage case object RemoteOneWay extends TestMessage
case object RemoteDie extends TestMessage case object RemoteDie extends TestMessage

View file

@ -30,8 +30,7 @@ class RemoteActorSpec extends TestCase {
akka.Config.config akka.Config.config
new Thread(new Runnable() { new Thread(new Runnable() {
def run = { def run = {
val server = new RemoteServer RemoteServer.start
server.start
} }
}).start }).start
Thread.sleep(1000) Thread.sleep(1000)

View file

@ -25,8 +25,7 @@ class RemoteSupervisorSpec extends junit.framework.TestCase with Suite {
akka.Config.config akka.Config.config
new Thread(new Runnable() { new Thread(new Runnable() {
def run = { def run = {
val server = new RemoteServer RemoteServer.start
server.start
} }
}).start }).start
Thread.sleep(1000) Thread.sleep(1000)

View file

@ -0,0 +1,69 @@
package se.scalablesolutions.akka.actor
import java.util.concurrent.TimeUnit
import junit.framework.Assert._
import se.scalablesolutions.akka.dispatch.Dispatchers
class ThreadBasedActorSpec extends junit.framework.TestCase {
private val unit = TimeUnit.MILLISECONDS
class TestActor extends Actor {
dispatcher = Dispatchers.newThreadBasedDispatcher(this)
def receive: PartialFunction[Any, Unit] = {
case "Hello" =>
reply("World")
case "Failure" =>
throw new RuntimeException("expected")
}
}
def testSendOneWay = {
implicit val timeout = 5000L
var oneWay = "nada"
val actor = new Actor {
def receive: PartialFunction[Any, Unit] = {
case "OneWay" => oneWay = "received"
}
}
actor.start
val result = actor ! "OneWay"
Thread.sleep(100)
assertEquals("received", oneWay)
actor.stop
}
def testSendReplySync = {
implicit val timeout = 5000L
val actor = new TestActor
actor.start
val result: String = actor !? "Hello"
assertEquals("World", result)
actor.stop
}
def testSendReplyAsync = {
implicit val timeout = 5000L
val actor = new TestActor
actor.start
val result = actor !! "Hello"
assertEquals("World", result.get.asInstanceOf[String])
actor.stop
}
def testSendReceiveException = {
implicit val timeout = 5000L
val actor = new TestActor
actor.start
try {
actor !! "Failure"
fail("Should have thrown an exception")
} catch {
case e =>
assertEquals("expected", e.getMessage())
}
actor.stop
}
}

View file

@ -41,32 +41,31 @@ class AkkaServlet extends ServletContainer with Logging {
} }
class AkkaCometServlet extends org.atmosphere.cpr.AtmosphereServlet { class AkkaCometServlet extends org.atmosphere.cpr.AtmosphereServlet {
override def init(sconf: ServletConfig) = {
val servlet = new AkkaServlet with AtmosphereServletProcessor { val servlet = new AkkaServlet with AtmosphereServletProcessor {
//Delegate to implement the behavior for AtmosphereHandler //Delegate to implement the behavior for AtmosphereHandler
private val handler = new AbstractReflectorAtmosphereHandler { private val handler = new AbstractReflectorAtmosphereHandler {
override def onRequest(event: AtmosphereResource[HttpServletRequest, HttpServletResponse]): Unit = { override def onRequest(event: AtmosphereResource[HttpServletRequest, HttpServletResponse]) {
event.getRequest.setAttribute(ReflectorServletProcessor.ATMOSPHERE_RESOURCE, event) if(event ne null)
event.getRequest.setAttribute(ReflectorServletProcessor.ATMOSPHERE_HANDLER, this) {
service(event.getRequest, event.getResponse) event.getRequest.setAttribute(ReflectorServletProcessor.ATMOSPHERE_RESOURCE, event)
event.getRequest.setAttribute(ReflectorServletProcessor.ATMOSPHERE_HANDLER, this)
service(event.getRequest, event.getResponse)
}
}
}
override def onStateChange(event : AtmosphereResourceEvent[HttpServletRequest, HttpServletResponse] ) {
if(event ne null)
handler onStateChange event
}
override def onRequest(resource: AtmosphereResource[HttpServletRequest, HttpServletResponse]) {
handler onRequest resource
}
} }
}
override def onStateChange(event: AtmosphereResourceEvent[HttpServletRequest, HttpServletResponse]) { override def loadConfiguration(sc : ServletConfig) {
handler onStateChange event atmosphereHandlers.put("/*", new AtmosphereHandlerWrapper(servlet, new JerseyBroadcaster))
}
override def onRequest(resource: AtmosphereResource[HttpServletRequest, HttpServletResponse]) {
handler onRequest resource
}
} }
config = new AtmosphereConfig {ah = servlet} }
atmosphereHandlers.put("/*", new AtmosphereHandlerWrapper(servlet, new JerseyBroadcaster))
setCometSupport(new GrizzlyCometSupport(config))
getCometSupport.init(sconf)
servlet.init(sconf)
}
override def loadAtmosphereDotXml(is: InputStream, urlc: URLClassLoader) = () //Hide it
}

View file

@ -33,7 +33,6 @@ object Kernel extends Logging {
// FIXME add API to shut server down gracefully // FIXME add API to shut server down gracefully
@volatile private var hasBooted = false @volatile private var hasBooted = false
private var remoteServer: RemoteServer = _
private var jerseySelectorThread: SelectorThread = _ private var jerseySelectorThread: SelectorThread = _
private val startTime = System.currentTimeMillis private val startTime = System.currentTimeMillis
private var applicationLoader: Option[ClassLoader] = None private var applicationLoader: Option[ClassLoader] = None

View file

@ -49,11 +49,12 @@ object CassandraStorage extends MapStorage
*/ */
private[this] val serializer: Serializer = { private[this] val serializer: Serializer = {
config.getString("akka.storage.cassandra.storage-format", "java") match { config.getString("akka.storage.cassandra.storage-format", "manual") match {
case "scala-json" => Serializer.ScalaJSON case "scala-json" => Serializer.ScalaJSON
case "java-json" => Serializer.JavaJSON case "java-json" => Serializer.JavaJSON
case "protobuf" => Serializer.Protobuf case "protobuf" => Serializer.Protobuf
case "java" => Serializer.Java case "java" => Serializer.Java
case "manual" => Serializer.NOOP
case "sbinary" => throw new UnsupportedOperationException("SBinary serialization protocol is not yet supported for storage") case "sbinary" => throw new UnsupportedOperationException("SBinary serialization protocol is not yet supported for storage")
case "avro" => throw new UnsupportedOperationException("Avro serialization protocol is not yet supported for storage") case "avro" => throw new UnsupportedOperationException("Avro serialization protocol is not yet supported for storage")
case unknown => throw new UnsupportedOperationException("Unknown storage serialization protocol [" + unknown + "]") case unknown => throw new UnsupportedOperationException("Unknown storage serialization protocol [" + unknown + "]")

View file

@ -13,10 +13,10 @@ class NoTransactionInScopeException extends RuntimeException
sealed abstract class PersistentStateConfig sealed abstract class PersistentStateConfig
abstract class PersistentStorageConfig extends PersistentStateConfig abstract class PersistentStorageConfig extends PersistentStateConfig
case class CassandraStorageConfig extends PersistentStorageConfig case class CassandraStorageConfig() extends PersistentStorageConfig
case class TerracottaStorageConfig extends PersistentStorageConfig case class TerracottaStorageConfig() extends PersistentStorageConfig
case class TokyoCabinetStorageConfig extends PersistentStorageConfig case class TokyoCabinetStorageConfig() extends PersistentStorageConfig
case class MongoStorageConfig extends PersistentStorageConfig case class MongoStorageConfig() extends PersistentStorageConfig
/** /**
* Example Scala usage: * Example Scala usage:
@ -24,7 +24,6 @@ case class MongoStorageConfig extends PersistentStorageConfig
* val myMap = PersistentState.newMap(CassandraStorageConfig) * val myMap = PersistentState.newMap(CassandraStorageConfig)
* </pre> * </pre>
* <p/> * <p/>
*
* Example Java usage: * Example Java usage:
* <pre> * <pre>
* TransactionalMap myMap = PersistentState.newMap(new CassandraStorageConfig()); * TransactionalMap myMap = PersistentState.newMap(new CassandraStorageConfig());
@ -252,7 +251,7 @@ class CassandraPersistentVector extends PersistentVector {
val storage = CassandraStorage val storage = CassandraStorage
} }
/** /**
* Implements a persistent transactional vector based on the MongoDB distributed P2P key-value storage. * Implements a persistent transactional vector based on the MongoDB distributed P2P key-value storage.
* *
* @author <a href="http://debasishg.blogspot.com">Debaissh Ghosh</a> * @author <a href="http://debasishg.blogspot.com">Debaissh Ghosh</a>

View file

@ -13,7 +13,7 @@ import se.scalablesolutions.akka.annotation.prerestart;
import se.scalablesolutions.akka.annotation.postrestart; import se.scalablesolutions.akka.annotation.postrestart;
import se.scalablesolutions.akka.state.PersistentMap; import se.scalablesolutions.akka.state.PersistentMap;
import se.scalablesolutions.akka.state.PersistentState; import se.scalablesolutions.akka.state.PersistentState;
import se.scalablesolutions.akka.state.TransactionalMap; import se.scalablesolutions.akka.state.PersistentMap;
import se.scalablesolutions.akka.state.CassandraStorageConfig; import se.scalablesolutions.akka.state.CassandraStorageConfig;
/** /**
@ -26,7 +26,7 @@ import se.scalablesolutions.akka.state.CassandraStorageConfig;
@Path("/persistentjavacount") @Path("/persistentjavacount")
@transactionrequired @transactionrequired
public class PersistentSimpleService { public class PersistentSimpleService {
private String KEY = "COUNTER"; private Object KEY = "COUNTER";
private boolean hasStartedTicking = false; private boolean hasStartedTicking = false;
private PersistentMap storage = PersistentState.newMap(new CassandraStorageConfig()); private PersistentMap storage = PersistentState.newMap(new CassandraStorageConfig());

View file

@ -5,6 +5,7 @@ import se.scalablesolutions.akka.actor.{SupervisorFactory, Actor}
import se.scalablesolutions.akka.config.ScalaConfig._ import se.scalablesolutions.akka.config.ScalaConfig._
import se.scalablesolutions.akka.util.Logging import se.scalablesolutions.akka.util.Logging
import java.lang.Integer
import javax.ws.rs.core.MultivaluedMap import javax.ws.rs.core.MultivaluedMap
import javax.ws.rs.{GET, POST, Path, Produces, WebApplicationException, Consumes} import javax.ws.rs.{GET, POST, Path, Produces, WebApplicationException, Consumes}

View file

@ -9,6 +9,7 @@ import se.scalablesolutions.akka.actor.{SupervisorFactory, Actor}
import se.scalablesolutions.akka.config.ScalaConfig._ import se.scalablesolutions.akka.config.ScalaConfig._
import se.scalablesolutions.akka.util.Logging import se.scalablesolutions.akka.util.Logging
import java.lang.Integer
import javax.ws.rs.core.MultivaluedMap import javax.ws.rs.core.MultivaluedMap
import javax.ws.rs.{GET, POST, Path, Produces, WebApplicationException, Consumes} import javax.ws.rs.{GET, POST, Path, Produces, WebApplicationException, Consumes}
@ -112,7 +113,13 @@ class Chat extends Actor with Logging {
@Suspend @Suspend
@GET @GET
@Produces(Array("text/html")) @Produces(Array("text/html"))
def suspend = <!-- Comet is a programming technique that enables web servers to send data to the client without having any need for the client to request it. --> def suspend = {
val s = new StringBuilder
s append "<!-- "
for(i <- 1 to 10) s append "Comet is a programming technique that enables web servers to send data to the client without having any need for the client to request it. "
s append " -->"
s toString
}
override def receive: PartialFunction[Any, Unit] = { override def receive: PartialFunction[Any, Unit] = {
case Chat(who, what, msg) => { case Chat(who, what, msg) => {

View file

@ -1,5 +1,5 @@
include "akka-reference.conf" include "akka-reference.conf"
# This config import the Akka reference configuration. # This config import the Akka reference configuration.
# In this file you can override any option defined in the 'akka-reference.conf' file. # In this file you can override any option defined in the 'akka-reference.conf' file.

1
config/jndi.properties Normal file
View file

@ -0,0 +1 @@
java.naming.factory.initial=com.sun.jndi.fscontext.RefFSContextFactory

View file

@ -1,9 +0,0 @@
<atmosphere-handlers>
<!-- Associate a context-root (servlet-mapping) to an AtmosphereHandler.
Request sent using that context-root will be mapped to its associated AtmosphereHandler
-->
<atmosphere-handler context-root="" class-name="se.scalablesolutions.akka.kernel.rest.AkkaServlet" broadcaster="org.atmosphere.core.JerseyBroadcaster">
<!-- Define some AtmosphereHandler properties -->
<property name="com.sun.jersey.spi.container.ResourceFilters" value="org.atmosphere.core.AtmosphereFilter"/>
</atmosphere-handler>
</atmosphere-handlers>

View file

@ -1,5 +0,0 @@
<?xml version="1.0" encoding="UTF-8"?>
<Context>
<!-- <Loader className="org.atmosphere.util.AtmosphereClassloader"/> -->
<Loader delegate="true"/>
</Context>

Binary file not shown.

After

Width:  |  Height:  |  Size: 2.3 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 20 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 7.8 KiB

45
deploy/root/index.html Normal file
View file

@ -0,0 +1,45 @@
<?xml version="1.0" encoding="UTF-8" ?>
<!DOCTYPE html
PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
"http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
<html xmlns="http://www.w3.org/1999/xhtml" xml:lang="en" lang="en">
<head>
<meta http-equiv="Content-Type" content="text/html; charset=UTF-8" />
<title>Atmosphere Scala REST Chat</title>
<link rel="stylesheet" href="stylesheets/default.css" type="text/css" />
<script type="text/javascript" src="javascripts/prototype.js"></script>
<script type="text/javascript" src="javascripts/behaviour.js"></script>
<script type="text/javascript" src="javascripts/moo.fx.js"></script>
<script type="text/javascript" src="javascripts/moo.fx.pack.js"></script>
<script type="text/javascript" src="javascripts/application.js"></script>
</head>
<body>
<div id="container">
<div id="container-inner">
<div id="header">
<h1>Atmosphere REST Chat</h1>
</div>
<div id="main">
<div id="display">
</div>
<div id="form">
<div id="system-message">Please input your name:</div>
<div id="login-form">
<input id="login-name" type="text" />
<br />
<input id="login-button" type="button" value="Login" />
</div>
<div id="message-form" style="display: none;">
<div>
<textarea id="message" name="message" rows="2" cols="40"></textarea>
<br />
<input id="post-button" type="button" value="Post Message" />
</div>
</div>
</div>
</div>
</div>
</div>
<iframe id="comet-frame" style="display: none;"></iframe>
</body>
</html>

View file

@ -0,0 +1,90 @@
var count = 0;
var app = {
url: '/chat',
initialize: function() {
$('login-name').focus();
app.listen();
},
listen: function() {
$('comet-frame').src = app.url + '?' + count;
count ++;
},
login: function() {
var name = $F('login-name');
if(! name.length > 0) {
$('system-message').style.color = 'red';
$('login-name').focus();
return;
}
$('system-message').style.color = '#2d2b3d';
$('system-message').innerHTML = name + ':';
$('login-button').disabled = true;
$('login-form').style.display = 'none';
$('message-form').style.display = '';
var query =
'action=login' +
'&name=' + encodeURI($F('login-name'));
new Ajax.Request(app.url, {
postBody: query,
onSuccess: function() {
$('message').focus();
}
});
},
post: function() {
var message = $F('message');
if(!message > 0) {
return;
}
$('message').disabled = true;
$('post-button').disabled = true;
var query =
'action=post' +
'&name=' + encodeURI($F('login-name')) +
'&message=' + encodeURI(message);
new Ajax.Request(app.url, {
postBody: query,
onComplete: function() {
$('message').disabled = false;
$('post-button').disabled = false;
$('message').focus();
$('message').value = '';
}
});
},
update: function(data) {
var p = document.createElement('p');
p.innerHTML = data.name + ':<br/>' + data.message;
$('display').appendChild(p);
new Fx.Scroll('display').down();
}
};
var rules = {
'#login-name': function(elem) {
Event.observe(elem, 'keydown', function(e) {
if(e.keyCode == 13) {
$('login-button').focus();
}
});
},
'#login-button': function(elem) {
elem.onclick = app.login;
},
'#message': function(elem) {
Event.observe(elem, 'keydown', function(e) {
if(e.shiftKey && e.keyCode == 13) {
$('post-button').focus();
}
});
},
'#post-button': function(elem) {
elem.onclick = app.post;
}
};
Behaviour.addLoadEvent(app.initialize);
Behaviour.register(rules);

View file

@ -0,0 +1,254 @@
/*
Behaviour v1.1 by Ben Nolan, June 2005. Based largely on the work
of Simon Willison (see comments by Simon below).
Description:
Uses css selectors to apply javascript behaviours to enable
unobtrusive javascript in html documents.
Usage:
var myrules = {
'b.someclass' : function(element){
element.onclick = function(){
alert(this.innerHTML);
}
},
'#someid u' : function(element){
element.onmouseover = function(){
this.innerHTML = "BLAH!";
}
}
};
Behaviour.register(myrules);
// Call Behaviour.apply() to re-apply the rules (if you
// update the dom, etc).
License:
This file is entirely BSD licensed.
More information:
http://ripcord.co.nz/behaviour/
*/
var Behaviour = {
list : new Array,
register : function(sheet){
Behaviour.list.push(sheet);
},
start : function(){
Behaviour.addLoadEvent(function(){
Behaviour.apply();
});
},
apply : function(){
for (h=0;sheet=Behaviour.list[h];h++){
for (selector in sheet){
list = document.getElementsBySelector(selector);
if (!list){
continue;
}
for (i=0;element=list[i];i++){
sheet[selector](element);
}
}
}
},
addLoadEvent : function(func){
var oldonload = window.onload;
if (typeof window.onload != 'function') {
window.onload = func;
} else {
window.onload = function() {
oldonload();
func();
}
}
}
}
Behaviour.start();
/*
The following code is Copyright (C) Simon Willison 2004.
document.getElementsBySelector(selector)
- returns an array of element objects from the current document
matching the CSS selector. Selectors can contain element names,
class names and ids and can be nested. For example:
elements = document.getElementsBySelect('div#main p a.external')
Will return an array of all 'a' elements with 'external' in their
class attribute that are contained inside 'p' elements that are
contained inside the 'div' element which has id="main"
New in version 0.4: Support for CSS2 and CSS3 attribute selectors:
See http://www.w3.org/TR/css3-selectors/#attribute-selectors
Version 0.4 - Simon Willison, March 25th 2003
-- Works in Phoenix 0.5, Mozilla 1.3, Opera 7, Internet Explorer 6, Internet Explorer 5 on Windows
-- Opera 7 fails
*/
function getAllChildren(e) {
// Returns all children of element. Workaround required for IE5/Windows. Ugh.
return e.all ? e.all : e.getElementsByTagName('*');
}
document.getElementsBySelector = function(selector) {
// Attempt to fail gracefully in lesser browsers
if (!document.getElementsByTagName) {
return new Array();
}
// Split selector in to tokens
var tokens = selector.split(' ');
var currentContext = new Array(document);
for (var i = 0; i < tokens.length; i++) {
token = tokens[i].replace(/^\s+/,'').replace(/\s+$/,'');;
if (token.indexOf('#') > -1) {
// Token is an ID selector
var bits = token.split('#');
var tagName = bits[0];
var id = bits[1];
var element = document.getElementById(id);
if (tagName && element.nodeName.toLowerCase() != tagName) {
// tag with that ID not found, return false
return new Array();
}
// Set currentContext to contain just this element
currentContext = new Array(element);
continue; // Skip to next token
}
if (token.indexOf('.') > -1) {
// Token contains a class selector
var bits = token.split('.');
var tagName = bits[0];
var className = bits[1];
if (!tagName) {
tagName = '*';
}
// Get elements matching tag, filter them for class selector
var found = new Array;
var foundCount = 0;
for (var h = 0; h < currentContext.length; h++) {
var elements;
if (tagName == '*') {
elements = getAllChildren(currentContext[h]);
} else {
elements = currentContext[h].getElementsByTagName(tagName);
}
for (var j = 0; j < elements.length; j++) {
found[foundCount++] = elements[j];
}
}
currentContext = new Array;
var currentContextIndex = 0;
for (var k = 0; k < found.length; k++) {
if (found[k].className && found[k].className.match(new RegExp('\\b'+className+'\\b'))) {
currentContext[currentContextIndex++] = found[k];
}
}
continue; // Skip to next token
}
// Code to deal with attribute selectors
if (token.match(/^(\w*)\[(\w+)([=~\|\^\$\*]?)=?"?([^\]"]*)"?\]$/)) {
var tagName = RegExp.$1;
var attrName = RegExp.$2;
var attrOperator = RegExp.$3;
var attrValue = RegExp.$4;
if (!tagName) {
tagName = '*';
}
// Grab all of the tagName elements within current context
var found = new Array;
var foundCount = 0;
for (var h = 0; h < currentContext.length; h++) {
var elements;
if (tagName == '*') {
elements = getAllChildren(currentContext[h]);
} else {
elements = currentContext[h].getElementsByTagName(tagName);
}
for (var j = 0; j < elements.length; j++) {
found[foundCount++] = elements[j];
}
}
currentContext = new Array;
var currentContextIndex = 0;
var checkFunction; // This function will be used to filter the elements
switch (attrOperator) {
case '=': // Equality
checkFunction = function(e) { return (e.getAttribute(attrName) == attrValue); };
break;
case '~': // Match one of space seperated words
checkFunction = function(e) { return (e.getAttribute(attrName).match(new RegExp('\\b'+attrValue+'\\b'))); };
break;
case '|': // Match start with value followed by optional hyphen
checkFunction = function(e) { return (e.getAttribute(attrName).match(new RegExp('^'+attrValue+'-?'))); };
break;
case '^': // Match starts with value
checkFunction = function(e) { return (e.getAttribute(attrName).indexOf(attrValue) == 0); };
break;
case '$': // Match ends with value - fails with "Warning" in Opera 7
checkFunction = function(e) { return (e.getAttribute(attrName).lastIndexOf(attrValue) == e.getAttribute(attrName).length - attrValue.length); };
break;
case '*': // Match ends with value
checkFunction = function(e) { return (e.getAttribute(attrName).indexOf(attrValue) > -1); };
break;
default :
// Just test for existence of attribute
checkFunction = function(e) { return e.getAttribute(attrName); };
}
currentContext = new Array;
var currentContextIndex = 0;
for (var k = 0; k < found.length; k++) {
if (checkFunction(found[k])) {
currentContext[currentContextIndex++] = found[k];
}
}
// alert('Attribute Selector: '+tagName+' '+attrName+' '+attrOperator+' '+attrValue);
continue; // Skip to next token
}
if (!currentContext[0]){
return;
}
// If we get here, token is JUST an element (not a class or ID selector)
tagName = token;
var found = new Array;
var foundCount = 0;
for (var h = 0; h < currentContext.length; h++) {
var elements = currentContext[h].getElementsByTagName(tagName);
for (var j = 0; j < elements.length; j++) {
found[foundCount++] = elements[j];
}
}
currentContext = found;
}
return currentContext;
}
/* That revolting regular expression explained
/^(\w+)\[(\w+)([=~\|\^\$\*]?)=?"?([^\]"]*)"?\]$/
\---/ \---/\-------------/ \-------/
| | | |
| | | The value
| | ~,|,^,$,* or =
| Attribute
Tag
*/

View file

@ -0,0 +1,136 @@
//(c) 2006 Valerio Proietti (http://mad4milk.net). MIT-style license.
//moo.fx.js - depends on prototype.js OR prototype.lite.js
//version 2.0
var Fx = fx = {};
Fx.Base = function(){};
Fx.Base.prototype = {
setOptions: function(options){
this.options = Object.extend({
onStart: function(){},
onComplete: function(){},
transition: Fx.Transitions.sineInOut,
duration: 500,
unit: 'px',
wait: true,
fps: 50
}, options || {});
},
step: function(){
var time = new Date().getTime();
if (time < this.time + this.options.duration){
this.cTime = time - this.time;
this.setNow();
} else {
setTimeout(this.options.onComplete.bind(this, this.element), 10);
this.clearTimer();
this.now = this.to;
}
this.increase();
},
setNow: function(){
this.now = this.compute(this.from, this.to);
},
compute: function(from, to){
var change = to - from;
return this.options.transition(this.cTime, from, change, this.options.duration);
},
clearTimer: function(){
clearInterval(this.timer);
this.timer = null;
return this;
},
_start: function(from, to){
if (!this.options.wait) this.clearTimer();
if (this.timer) return;
setTimeout(this.options.onStart.bind(this, this.element), 10);
this.from = from;
this.to = to;
this.time = new Date().getTime();
this.timer = setInterval(this.step.bind(this), Math.round(1000/this.options.fps));
return this;
},
custom: function(from, to){
return this._start(from, to);
},
set: function(to){
this.now = to;
this.increase();
return this;
},
hide: function(){
return this.set(0);
},
setStyle: function(e, p, v){
if (p == 'opacity'){
if (v == 0 && e.style.visibility != "hidden") e.style.visibility = "hidden";
else if (e.style.visibility != "visible") e.style.visibility = "visible";
if (window.ActiveXObject) e.style.filter = "alpha(opacity=" + v*100 + ")";
e.style.opacity = v;
} else e.style[p] = v+this.options.unit;
}
};
Fx.Style = Class.create();
Fx.Style.prototype = Object.extend(new Fx.Base(), {
initialize: function(el, property, options){
this.element = $(el);
this.setOptions(options);
this.property = property.camelize();
},
increase: function(){
this.setStyle(this.element, this.property, this.now);
}
});
Fx.Styles = Class.create();
Fx.Styles.prototype = Object.extend(new Fx.Base(), {
initialize: function(el, options){
this.element = $(el);
this.setOptions(options);
this.now = {};
},
setNow: function(){
for (p in this.from) this.now[p] = this.compute(this.from[p], this.to[p]);
},
custom: function(obj){
if (this.timer && this.options.wait) return;
var from = {};
var to = {};
for (p in obj){
from[p] = obj[p][0];
to[p] = obj[p][1];
}
return this._start(from, to);
},
increase: function(){
for (var p in this.now) this.setStyle(this.element, p, this.now[p]);
}
});
//Transitions (c) 2003 Robert Penner (http://www.robertpenner.com/easing/), BSD License.
Fx.Transitions = {
linear: function(t, b, c, d) { return c*t/d + b; },
sineInOut: function(t, b, c, d) { return -c/2 * (Math.cos(Math.PI*t/d) - 1) + b; }
};

View file

@ -0,0 +1,83 @@
//by Valerio Proietti (http://mad4milk.net). MIT-style license.
//moo.fx.pack.js - depends on prototype.js or prototype.lite.js + moo.fx.js
//version 2.0
Fx.Scroll = Class.create();
Fx.Scroll.prototype = Object.extend(new Fx.Base(), {
initialize: function(el, options) {
this.element = $(el);
this.setOptions(options);
this.element.style.overflow = 'hidden';
},
down: function(){
return this.custom(this.element.scrollTop, this.element.scrollHeight-this.element.offsetHeight);
},
up: function(){
return this.custom(this.element.scrollTop, 0);
},
increase: function(){
this.element.scrollTop = this.now;
}
});
//fx.Color, originally by Tom Jensen (http://neuemusic.com) MIT-style LICENSE.
Fx.Color = Class.create();
Fx.Color.prototype = Object.extend(new Fx.Base(), {
initialize: function(el, property, options){
this.element = $(el);
this.setOptions(options);
this.property = property.camelize();
this.now = [];
},
custom: function(from, to){
return this._start(from.hexToRgb(true), to.hexToRgb(true));
},
setNow: function(){
[0,1,2].each(function(i){
this.now[i] = Math.round(this.compute(this.from[i], this.to[i]));
}.bind(this));
},
increase: function(){
this.element.style[this.property] = "rgb("+this.now[0]+","+this.now[1]+","+this.now[2]+")";
}
});
Object.extend(String.prototype, {
rgbToHex: function(array){
var rgb = this.match(new RegExp('([\\d]{1,3})', 'g'));
if (rgb[3] == 0) return 'transparent';
var hex = [];
for (var i = 0; i < 3; i++){
var bit = (rgb[i]-0).toString(16);
hex.push(bit.length == 1 ? '0'+bit : bit);
}
var hexText = '#'+hex.join('');
if (array) return hex;
else return hexText;
},
hexToRgb: function(array){
var hex = this.match(new RegExp('^[#]{0,1}([\\w]{1,2})([\\w]{1,2})([\\w]{1,2})$'));
var rgb = [];
for (var i = 1; i < hex.length; i++){
if (hex[i].length == 1) hex[i] += hex[i];
rgb.push(parseInt(hex[i], 16));
}
var rgbText = 'rgb('+rgb.join(',')+')';
if (array) return rgb;
else return rgbText;
}
});

1781
deploy/root/javascripts/prototype.js vendored Normal file

File diff suppressed because it is too large Load diff

File diff suppressed because one or more lines are too long

View file

@ -0,0 +1,54 @@
body {
background-image: url(../images/body-background.png);
background-repeat: repeat-x;
background-color: #5c8098;
}
html, body, h1, h2 {
margin: 0px;
padding: 0px;
}
body, textarea, input {
font-size: 12px;
font-family: Verdana, Helvetica, Arial, sans-serif;
color: #2d2b3d;
}
#container {
text-align:center;
}
#container-inner {
margin-left: auto;
margin-right: auto;
text-align: justify;
width: 820px;
}
#header {
width: 820px;
height: 100px;
background-image: url(../images/header-background.png);
background-repeat: no-repeat;
}
#header h1 {
display: none;
}
#main {
height: 610px;
background-image: url(../images/main-background.png);
background-repeat: no-repeat;
text-align: left;
width: 740px;
padding: 30px 40px 20px 40px;
}
#display {
border: 1px solid #5c8098;
width: 740px;
height: 400px;
margin-bottom: 10px;
overflow-y: scroll;
}
#login-name {
width: 200px;
}
#message {
width: 740px;
height: 50px;
}