merged with master head
This commit is contained in:
commit
00b606b3e8
37 changed files with 2688 additions and 123 deletions
2
.gitignore
vendored
2
.gitignore
vendored
|
|
@ -6,7 +6,7 @@ reports
|
||||||
dist
|
dist
|
||||||
build
|
build
|
||||||
target
|
target
|
||||||
deploy
|
deploy/*.jar
|
||||||
data
|
data
|
||||||
out
|
out
|
||||||
logs
|
logs
|
||||||
|
|
|
||||||
|
|
@ -29,7 +29,6 @@ object Annotations {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Factory object for Active Objects.
|
|
||||||
*
|
*
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Boné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)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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) {
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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ér</a>
|
|
||||||
*/
|
|
||||||
class RemoteServer extends Logging {
|
|
||||||
def start = RemoteServer.start(None)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Boné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
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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ér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
|
|
|
||||||
|
|
@ -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])
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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"
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
69
akka-actors/src/test/scala/ThreadBasedActorSpec.scala
Normal file
69
akka-actors/src/test/scala/ThreadBasedActorSpec.scala
Normal 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
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -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
|
|
||||||
}
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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 + "]")
|
||||||
|
|
|
||||||
|
|
@ -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>
|
||||||
|
|
|
||||||
|
|
@ -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());
|
||||||
|
|
|
||||||
|
|
@ -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}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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) => {
|
||||||
|
|
|
||||||
|
|
@ -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
1
config/jndi.properties
Normal file
|
|
@ -0,0 +1 @@
|
||||||
|
java.naming.factory.initial=com.sun.jndi.fscontext.RefFSContextFactory
|
||||||
|
|
@ -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>
|
|
||||||
|
|
@ -1,5 +0,0 @@
|
||||||
<?xml version="1.0" encoding="UTF-8"?>
|
|
||||||
<Context>
|
|
||||||
<!-- <Loader className="org.atmosphere.util.AtmosphereClassloader"/> -->
|
|
||||||
<Loader delegate="true"/>
|
|
||||||
</Context>
|
|
||||||
BIN
deploy/root/images/body-background.png
Normal file
BIN
deploy/root/images/body-background.png
Normal file
Binary file not shown.
|
After Width: | Height: | Size: 2.3 KiB |
BIN
deploy/root/images/header-background.png
Normal file
BIN
deploy/root/images/header-background.png
Normal file
Binary file not shown.
|
After Width: | Height: | Size: 20 KiB |
BIN
deploy/root/images/main-background.png
Normal file
BIN
deploy/root/images/main-background.png
Normal file
Binary file not shown.
|
After Width: | Height: | Size: 7.8 KiB |
45
deploy/root/index.html
Normal file
45
deploy/root/index.html
Normal 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>
|
||||||
90
deploy/root/javascripts/application.js
Normal file
90
deploy/root/javascripts/application.js
Normal 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);
|
||||||
254
deploy/root/javascripts/behaviour.js
Normal file
254
deploy/root/javascripts/behaviour.js
Normal 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
|
||||||
|
*/
|
||||||
136
deploy/root/javascripts/moo.fx.js
Normal file
136
deploy/root/javascripts/moo.fx.js
Normal 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; }
|
||||||
|
};
|
||||||
83
deploy/root/javascripts/moo.fx.pack.js
Normal file
83
deploy/root/javascripts/moo.fx.pack.js
Normal 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
1781
deploy/root/javascripts/prototype.js
vendored
Normal file
File diff suppressed because it is too large
Load diff
19
deploy/root/jquery-1.3.2.min.js
vendored
19
deploy/root/jquery-1.3.2.min.js
vendored
File diff suppressed because one or more lines are too long
54
deploy/root/stylesheets/default.css
Normal file
54
deploy/root/stylesheets/default.css
Normal 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;
|
||||||
|
}
|
||||||
Loading…
Add table
Add a link
Reference in a new issue