Added: 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/java/org/apache/kahadb/replication/transport/KDBRTransportTest.java
URL: 
http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/java/org/apache/kahadb/replication/transport/KDBRTransportTest.java?rev=889781&view=auto
==============================================================================
--- 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/java/org/apache/kahadb/replication/transport/KDBRTransportTest.java
 (added)
+++ 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/java/org/apache/kahadb/replication/transport/KDBRTransportTest.java
 Fri Dec 11 19:39:58 2009
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kahadb.replication.transport;
+
+import java.io.ByteArrayInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import junit.framework.TestCase;
+
+import org.apache.activemq.transport.Transport;
+import org.apache.activemq.transport.TransportAcceptListener;
+import org.apache.activemq.transport.TransportFactory;
+import org.apache.activemq.transport.TransportListener;
+import org.apache.activemq.transport.TransportServer;
+import org.apache.kahadb.replication.ReplicationFrame;
+import org.apache.kahadb.replication.pb.PBHeader;
+import org.apache.kahadb.replication.pb.PBJournalLocation;
+import org.apache.kahadb.replication.pb.PBSlaveInit;
+import org.apache.kahadb.replication.pb.PBType;
+
+public class KDBRTransportTest extends TestCase {
+
+       private static final String KDBR_URI = "kdbr://localhost:61618";
+       private List<Object> serverQueue;
+       private List<Object> clientQueue;
+       private List<Transport> serverTransports;
+       private TransportServer server;
+       private Transport client;
+       
+       private Object commandLatchMutex = new Object();
+       private CountDownLatch commandLatch;
+       
+       protected void releaseCommandLatch() {
+               synchronized( commandLatchMutex ) {
+                       if( commandLatch == null ) {
+                               return;
+                       }
+                       commandLatch.countDown();
+                       commandLatch=null;
+               }
+       }
+       
+       protected CountDownLatch getCommandLatch() {
+               synchronized( commandLatchMutex ) {
+                       if( commandLatch == null ) {
+                               commandLatch = new CountDownLatch(1);
+                       }
+                       return commandLatch;
+               }
+       }
+       
+       @Override
+       protected void setUp() throws Exception {
+               serverQueue = Collections.synchronizedList(new 
ArrayList<Object>()); 
+               clientQueue = Collections.synchronizedList(new 
ArrayList<Object>()); 
+               serverTransports = Collections.synchronizedList(new 
ArrayList<Transport>()); 
+               
+               // Setup a server
+               server = TransportFactory.bind(new URI(KDBR_URI));
+               server.setAcceptListener(new TransportAcceptListener() {
+                       public void onAccept(Transport transport) {
+                               try {
+                                       transport.setTransportListener(new 
TransportListener() {
+                                               public void onCommand(Object 
command) {
+                                                       try {
+                                                               
serverQueue.add(command);
+                                                               
process(command);
+                                                               
releaseCommandLatch();
+                                                       } catch (IOException e) 
{
+                                                               onException(e);
+                                                       }
+                                               }
+
+                                               public void 
onException(IOException error) {
+                                                       serverQueue.add(error);
+                                                       
serverTransports.remove(this);
+                                                       releaseCommandLatch();
+                                               }
+
+                                               public void 
transportInterupted() {
+                                               }
+
+                                               public void transportResumed() {
+                                               }
+                                       });
+                                       transport.start();
+                                       serverTransports.add(transport);
+                               } catch (Exception e) {
+                                       onAcceptError(e);
+                               }
+                       }
+
+                       public void onAcceptError(Exception error) {
+                               error.printStackTrace();
+                       }
+               });
+               server.start();
+
+               // Connect a client.
+               client = TransportFactory.connect(new URI(KDBR_URI));
+               client.setTransportListener(new TransportListener() {
+                       public void onCommand(Object command) {
+                               clientQueue.add(command);
+                               releaseCommandLatch();
+                       }
+
+                       public void onException(IOException error) {
+                               clientQueue.add(error);
+                               releaseCommandLatch();
+                       }
+
+                       public void transportInterupted() {
+                       }
+
+                       public void transportResumed() {
+                       }
+               });
+               client.start(); 
+       }
+       
+       @Override
+       protected void tearDown() throws Exception {
+               client.stop();
+               server.stop();
+       }
+
+       private void process(Object command) throws IOException {               
+               ReplicationFrame frame = (ReplicationFrame) command;
+               // Since we are processing the commands async in this test case 
we need to full read the stream before
+               // returning since will be be used to read the next command 
once we return.
+               if( frame.getHeader().getType() == 
PBType.FILE_TRANSFER_RESPONSE ) {
+                       InputStream ais = (InputStream) frame.getPayload();
+                       byte actualPayload[] = new 
byte[(int)frame.getHeader().getPayloadSize()];
+                       readFully(ais, actualPayload);
+                       frame.setPayload(actualPayload);
+               }
+       }
+       
+       /**
+        * Test a frame that has a streaming payload.
+        * 
+        * @throws Exception
+        */
+       public void testFileTransferResponse() throws Exception {
+
+               byte expectedPayload[] = {1,2,3,4,5,6,7,8,9,10}; 
+
+               ReplicationFrame expected = new ReplicationFrame();
+               expected.setHeader(new 
PBHeader().setType(PBType.FILE_TRANSFER_RESPONSE).setPayloadSize(expectedPayload.length));
+               ByteArrayInputStream is = new 
ByteArrayInputStream(expectedPayload);
+               expected.setPayload(is);
+               
+               CountDownLatch latch = getCommandLatch();
+               client.oneway(expected);
+               is.close();
+               latch.await(2, TimeUnit.SECONDS);
+               
+               assertEquals(1, serverQueue.size());
+               ReplicationFrame actual = (ReplicationFrame) 
serverQueue.remove(0);
+               
+               assertEquals(expected.getHeader(), actual.getHeader());         
+               assertTrue(Arrays.equals(expectedPayload, 
(byte[])actual.getPayload()));
+               
+       }
+
+       
+       /**
+        * Test out sending a frame that has a PB payload.
+        * 
+        * @throws Exception
+        */
+       public void testPBSlaveInitFrame() throws Exception {
+
+
+               ReplicationFrame expected = new ReplicationFrame();
+               expected.setHeader(new PBHeader().setType(PBType.SLAVE_INIT));
+               expected.setPayload(new PBSlaveInit().setNodeId("foo"));
+               
+               CountDownLatch latch = getCommandLatch();
+               client.oneway(expected);
+               latch.await(2, TimeUnit.SECONDS);
+               
+               assertEquals(1, serverQueue.size());
+               ReplicationFrame actual = (ReplicationFrame) 
serverQueue.remove(0);
+               
+               assertEquals(expected.getHeader(), actual.getHeader());
+               assertEquals(expected.getPayload(), actual.getPayload());
+               
+       }
+
+
+       private void readFully(InputStream ais, byte[] actualPayload) throws 
IOException {
+               int pos = 0;
+               int c;
+               while( pos < actualPayload.length && (c=ais.read(actualPayload, 
pos, actualPayload.length-pos))>=0 ) {
+                       pos += c;
+               }
+               if( pos  < actualPayload.length ) {
+                       throw new EOFException();
+               }
+       }
+}

Added: 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/java/org/apache/kahadb/replication/zk/ZooKeeperClusterStateManagerTest.java
URL: 
http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/java/org/apache/kahadb/replication/zk/ZooKeeperClusterStateManagerTest.java?rev=889781&view=auto
==============================================================================
--- 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/java/org/apache/kahadb/replication/zk/ZooKeeperClusterStateManagerTest.java
 (added)
+++ 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/java/org/apache/kahadb/replication/zk/ZooKeeperClusterStateManagerTest.java
 Fri Dec 11 19:39:58 2009
@@ -0,0 +1,220 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kahadb.replication.zk;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.net.Socket;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import junit.framework.TestCase;
+
+import org.apache.activemq.util.Callback;
+import org.apache.kahadb.replication.ClusterListener;
+import org.apache.kahadb.replication.ClusterState;
+import org.apache.kahadb.replication.pb.PBClusterNodeStatus;
+import org.apache.kahadb.replication.pb.PBJournalLocation;
+import org.apache.kahadb.replication.pb.PBClusterNodeStatus.State;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooKeeper.States;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerStats;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.zookeeper.server.NIOServerCnxn.Factory;
+import org.apache.zookeeper.server.persistence.FileTxnLog;
+
+public class ZooKeeperClusterStateManagerTest extends TestCase {
+
+    private static final int PORT = 2181;
+    private ZooKeeperClusterStateManager zkcsm1;
+    private ZooKeeper zk;
+    private Factory serverFactory;
+
+    @Override
+    protected void setUp() throws Exception {
+
+        ServerStats.registerAsConcrete();
+        File tmpDir = new File("target/test-data/zookeeper");
+        tmpDir.mkdirs();
+
+        // Reduces startup time..
+        System.setProperty("zookeeper.preAllocSize", "100");
+        FileTxnLog.setPreallocSize(100);
+
+        ZooKeeperServer zs = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+        
+        serverFactory = new NIOServerCnxn.Factory(PORT);
+        serverFactory.startup(zs);
+
+        zkcsm1 = new ZooKeeperClusterStateManager();
+        zk = zkcsm1.createZooKeeperConnection();
+        
+        // Wait till the ZK client gets connected..
+        States state;
+        while( (state = zk.getState()) != States.CONNECTED ) {
+            Thread.sleep(100);
+        }
+        
+        // Cleanup after previous run...
+        zkRecusiveDelete(zkcsm1.getPath());
+    }
+    
+    private void zkRecusiveDelete(String path) throws KeeperException, 
InterruptedException {
+        Stat stat = zk.exists(path, false);
+        if( stat!=null ) {
+            if( stat.getNumChildren() > 0 ) {
+                List<String> children = zk.getChildren(path, false);
+                for (String node : children) {
+                    zkRecusiveDelete(path+"/"+node);
+                }
+            }
+            zk.delete(path, stat.getVersion());
+        }
+    }
+
+    @Override
+    protected void tearDown() throws Exception {
+        zk.close();
+        serverFactory.shutdown();
+        ServerStats.unregister();
+    }
+    
+    public void testTwoNodesGoingOnline() throws Exception {
+        final LinkedBlockingQueue<ClusterState> stateEvents1 = new 
LinkedBlockingQueue<ClusterState>();
+        final LinkedBlockingQueue<ClusterState> stateEvents2 = new 
LinkedBlockingQueue<ClusterState>();
+        
+        zkcsm1.addListener(new ClusterListener() {
+            public void onClusterChange(ClusterState config) {
+                stateEvents1.add(config);
+            }
+        });
+        zkcsm1.start();
+        zkcsm1.addMember("kdbr://localhost:60001");
+        
+        final ZooKeeperClusterStateManager zkcsm2 = new 
ZooKeeperClusterStateManager();
+        zkcsm2.addListener(new ClusterListener() {
+            public void onClusterChange(ClusterState config) {
+                stateEvents2.add(config);
+            }
+        });
+        zkcsm2.start();
+        zkcsm2.addMember("kdbr://localhost:60002");
+        
+        // Drain the events..
+        while( stateEvents1.poll(100, TimeUnit.MILLISECONDS)!=null ) {
+        }
+        while( stateEvents2.poll(100, TimeUnit.MILLISECONDS)!=null ) {
+        }
+        
+        // Bring node 1 online
+        final PBClusterNodeStatus status1 = new PBClusterNodeStatus();
+        status1.setConnectUri("kdbr://localhost:60001");
+        status1.setLastUpdate(new 
PBJournalLocation().setFileId(1).setOffset(50));
+        status1.setState(State.SLAVE_UNCONNECTED);
+
+        executeAsync(new Callback() {
+            public void execute() throws Exception {
+                zkcsm1.setMemberStatus(status1);
+            }
+         });
+
+        // Bring node 2 online
+        final PBClusterNodeStatus status2 = new PBClusterNodeStatus();
+        status2.setConnectUri("kdbr://localhost:60002");
+        status2.setLastUpdate(new 
PBJournalLocation().setFileId(2).setOffset(20));
+        status2.setState(State.SLAVE_UNCONNECTED);
+        
+        executeAsync(new Callback() {
+            public void execute() throws Exception {
+                Thread.sleep(1000);
+                zkcsm2.setMemberStatus(status2);
+            }
+         });
+
+        ClusterState state = stateEvents1.poll(10, TimeUnit.SECONDS);
+        assertNotNull(state);
+        assertNotNull(state.getMaster());
+        assertEquals("kdbr://localhost:60002", state.getMaster());
+        assertTrue(state.getSlaves().size()==1);
+
+        state = stateEvents2.poll(1, TimeUnit.SECONDS);
+        assertNotNull(state);
+        assertNotNull(state.getMaster());
+        assertEquals("kdbr://localhost:60002", state.getMaster());
+        assertTrue(state.getSlaves().size()==1);
+
+        zkcsm2.stop();
+        zkcsm1.stop();
+    }
+
+    private void executeAsync(final Callback callback) {
+        new Thread("Async Test Task") {
+            @Override
+            public void run() {
+                try {
+                    callback.execute();
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        }.start();
+    }
+    
+    public void testOneNodeGoingOnline() throws Exception {
+        final LinkedBlockingQueue<ClusterState> stateEvents1 = new 
LinkedBlockingQueue<ClusterState>();
+        zkcsm1.addListener(new ClusterListener() {
+            public void onClusterChange(ClusterState config) {
+                stateEvents1.add(config);
+            }
+        });
+        zkcsm1.start();
+        
+        // Drain the events..
+        while( stateEvents1.poll(100, TimeUnit.MILLISECONDS)!=null ) {
+        }
+        
+        // Let node1 join the cluster.
+        zkcsm1.addMember("kdbr://localhost:60001");
+
+        ClusterState state = stateEvents1.poll(1, TimeUnit.SECONDS);
+        assertNotNull(state);
+        assertNull(state.getMaster());
+        assertTrue(state.getSlaves().size()==1);
+        
+        // Let the cluster know that node1 is online..
+        PBClusterNodeStatus status = new PBClusterNodeStatus();
+        status.setConnectUri("kdbr://localhost:60001");
+        status.setLastUpdate(new 
PBJournalLocation().setFileId(0).setOffset(0));
+        status.setState(State.SLAVE_UNCONNECTED);
+        zkcsm1.setMemberStatus(status);
+
+        state = stateEvents1.poll(10, TimeUnit.SECONDS);
+        assertNotNull(state);
+        assertNotNull(state.getMaster());
+        assertEquals("kdbr://localhost:60001", state.getMaster());
+        assertTrue(state.getSlaves().isEmpty());
+
+        zkcsm1.stop();
+    }
+}

Added: 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/broker1/ha-broker.xml
URL: 
http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/broker1/ha-broker.xml?rev=889781&view=auto
==============================================================================
--- 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/broker1/ha-broker.xml
 (added)
+++ 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/broker1/ha-broker.xml
 Fri Dec 11 19:39:58 2009
@@ -0,0 +1,36 @@
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+   
+    http://www.apache.org/licenses/LICENSE-2.0
+   
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+-->
+<!-- START SNIPPET: example -->
+<beans
+  xmlns="http://www.springframework.org/schema/beans";
+  xmlns:amq="http://activemq.apache.org/schema/core";
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+  xsi:schemaLocation="http://www.springframework.org/schema/beans 
http://www.springframework.org/schema/beans/spring-beans-2.0.xsd
+  http://activemq.apache.org/schema/core 
http://activemq.apache.org/schema/core/activemq-core.xsd   
+  http://activemq.apache.org/camel/schema/spring 
http://activemq.apache.org/camel/schema/spring/camel-spring.xsd";>
+
+    <broker xmlns="http://activemq.apache.org/schema/core"; start="false" 
brokerName="localhost" dataDirectory="target/data" useJmx="false">
+
+        <!-- The transport connectors ActiveMQ will listen to -->
+        <transportConnectors>
+            <transportConnector name="openwire" uri="tcp://localhost:61616"/>
+        </transportConnectors>
+
+    </broker>
+
+</beans>
+<!-- END SNIPPET: example -->

Added: 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/broker1/ha.xml
URL: 
http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/broker1/ha.xml?rev=889781&view=auto
==============================================================================
--- 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/broker1/ha.xml
 (added)
+++ 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/broker1/ha.xml
 Fri Dec 11 19:39:58 2009
@@ -0,0 +1,47 @@
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+   
+    http://www.apache.org/licenses/LICENSE-2.0
+   
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+-->
+<!-- START SNIPPET: example -->
+<beans
+  xmlns="http://www.springframework.org/schema/beans";
+  xmlns:amq="http://activemq.apache.org/schema/core";
+  xmlns:kdb="http://activemq.apache.org/schema/kahadb";
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+  xsi:schemaLocation="http://www.springframework.org/schema/beans 
http://www.springframework.org/schema/beans/spring-beans-2.0.xsd
+  http://activ emq.apache.org/schema/core 
http://activemq.apache.org/schema/core/activemq-core.xsd   
+  http://activemq.apache.org/camel/schema/spring 
http://activemq.apache.org/camel/schema/spring/camel-spring.xsd
+  http://activemq.apache.org/schema/kahadb 
http://activemq.apache.org/schema/kahadb/kahadb.xsd";>
+
+  <bean 
class="org.springframework.beans.factory.config.PropertyPlaceholderConfigurer"/>
+
+  <kahadbReplicationBroker xmlns="http://activemq.apache.org/schema/kahadb";>
+       <replicationService>
+         <kahadbReplication
+       directory="target/kaha-data/broker1" 
+       brokerURI="xbean:broker1/ha-broker.xml" 
+       uri="kdbr://localhost:6001"
+       minimumReplicas="0">
+       
+       <cluster>
+               <zookeeperCluster 
uri="zk://localhost:2181/activemq/ha-cluster/mygroup" userid="activemq" 
password=""/>
+       </cluster>
+       
+      </kahadbReplication>
+       </replicationService>
+  </kahadbReplicationBroker>
+  
+</beans>
+<!-- END SNIPPET: example -->

Added: 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/broker2/ha-broker.xml
URL: 
http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/broker2/ha-broker.xml?rev=889781&view=auto
==============================================================================
--- 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/broker2/ha-broker.xml
 (added)
+++ 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/broker2/ha-broker.xml
 Fri Dec 11 19:39:58 2009
@@ -0,0 +1,36 @@
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+   
+    http://www.apache.org/licenses/LICENSE-2.0
+   
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+-->
+<!-- START SNIPPET: example -->
+<beans
+  xmlns="http://www.springframework.org/schema/beans";
+  xmlns:amq="http://activemq.apache.org/schema/core";
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+  xsi:schemaLocation="http://www.springframework.org/schema/beans 
http://www.springframework.org/schema/beans/spring-beans-2.0.xsd
+  http://activemq.apache.org/schema/core 
http://activemq.apache.org/schema/core/activemq-core.xsd   
+  http://activemq.apache.org/camel/schema/spring 
http://activemq.apache.org/camel/schema/spring/camel-spring.xsd";>
+
+    <broker xmlns="http://activemq.apache.org/schema/core"; start="false" 
brokerName="localhost" dataDirectory="target/data" useJmx="false">
+
+        <!-- The transport connectors ActiveMQ will listen to -->
+        <transportConnectors>
+            <transportConnector name="openwire" uri="tcp://localhost:61617"/>
+        </transportConnectors>
+
+    </broker>
+
+</beans>
+<!-- END SNIPPET: example -->

Added: 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/broker2/ha.xml
URL: 
http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/broker2/ha.xml?rev=889781&view=auto
==============================================================================
--- 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/broker2/ha.xml
 (added)
+++ 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/broker2/ha.xml
 Fri Dec 11 19:39:58 2009
@@ -0,0 +1,47 @@
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+   
+    http://www.apache.org/licenses/LICENSE-2.0
+   
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+-->
+<!-- START SNIPPET: example -->
+<beans
+  xmlns="http://www.springframework.org/schema/beans";
+  xmlns:amq="http://activemq.apache.org/schema/core";
+  xmlns:kdb="http://activemq.apache.org/schema/kahadb";
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+  xsi:schemaLocation="http://www.springframework.org/schema/beans 
http://www.springframework.org/schema/beans/spring-beans-2.0.xsd
+  http://activ emq.apache.org/schema/core 
http://activemq.apache.org/schema/core/activemq-core.xsd   
+  http://activemq.apache.org/camel/schema/spring 
http://activemq.apache.org/camel/schema/spring/camel-spring.xsd
+  http://activemq.apache.org/schema/kahadb 
http://activemq.apache.org/schema/kahadb/kahadb.xsd";>
+
+  <bean 
class="org.springframework.beans.factory.config.PropertyPlaceholderConfigurer"/>
+
+  <kahadbReplicationBroker xmlns="http://activemq.apache.org/schema/kahadb";>
+       <replicationService>
+         <kahadbReplication
+       directory="target/kaha-data-broker2" 
+       brokerURI="xbean:broker2/ha-broker.xml" 
+       uri="kdbr://localhost:6002"
+       minimumReplicas="0">
+       
+       <cluster>
+               <zookeeperCluster 
uri="zk://localhost:2181/activemq/ha-cluster/mygroup" userid="activemq" 
password=""/>
+       </cluster>
+       
+      </kahadbReplication>
+       </replicationService>
+  </kahadbReplicationBroker>
+  
+</beans>
+<!-- END SNIPPET: example -->

Added: 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/log4j.properties
URL: 
http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/log4j.properties?rev=889781&view=auto
==============================================================================
--- 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/log4j.properties
 (added)
+++ 
activemq/sandbox/activemq-apollo/activemq-kahadb-replication/src/test/resources/log4j.properties
 Fri Dec 11 19:39:58 2009
@@ -0,0 +1,36 @@
+## ---------------------------------------------------------------------------
+## Licensed to the Apache Software Foundation (ASF) under one or more
+## contributor license agreements.  See the NOTICE file distributed with
+## this work for additional information regarding copyright ownership.
+## The ASF licenses this file to You under the Apache License, Version 2.0
+## (the "License"); you may not use this file except in compliance with
+## the License.  You may obtain a copy of the License at
+## 
+## http://www.apache.org/licenses/LICENSE-2.0
+## 
+## Unless required by applicable law or agreed to in writing, software
+## distributed under the License is distributed on an "AS IS" BASIS,
+## WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+## See the License for the specific language governing permissions and
+## limitations under the License.
+## ---------------------------------------------------------------------------
+
+#
+# The logging properties used during tests..
+#
+log4j.rootLogger=INFO, out
+
+log4j.logger.org.apache.activemq.spring=WARN
+log4j.logger.org.apache.zookeeper=WARN
+
+# CONSOLE appender not used by default
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d [%-15.15t] %-5p %-30.30c{1} 
- %m%n
+
+# File appender
+log4j.appender.out=org.apache.log4j.FileAppender
+log4j.appender.out.layout=org.apache.log4j.PatternLayout
+log4j.appender.out.layout.ConversionPattern=%d [%-15.15t] %-5p %-30.30c{1} - 
%m%n
+log4j.appender.out.file=target/activemq-test.log
+log4j.appender.out.append=true


Reply via email to