It sounds like your master cluster resolves bk1-4 as ds1-4. Could you
check that by doing a ping on those hostnames from those machines?
Else... I can't see what could be the error at the moment...

J-D

On Mon, Dec 13, 2010 at 3:55 PM, Nathaniel Cook
<[email protected]> wrote:
> Running the 'ls /hbase/rs' cmd through zkcli  on the master I get:
>
> [ds2.internal,60020,1292278767510, ds3.internal,60020,1292278776930,
> ds1.internal,60020,1292278759087, ds4.internal,60020,1292278792724
>
> On my slave cluster I get:
>
> [bk1.internal,60020,1292278881467, bk3.internal,60020,1292278895189,
> bk2.internal,60020,1292278888034, bk4.internal,60020,1292278905096]
>
> But as I mentioned the peer it chooses is ds4 from the master cluster.
>
> Could it be that for some reason the Configuration passed to the
> ZooKeeperWrapper.createInstance for the slave cluster isn't honored
> and is defaulting to the local connection settings? I am running a
> QuorumPeer on the same machine as the RegionServers for these test
> clusters. Could it be finding the zoo.cfg file on that machine that
> points to the local quorum?
>
> To test this i wrote a quick jruby script...
> #------------------------------------------------------
> include Java
> import org.apache.hadoop.hbase.HBaseConfiguration
> import org.apache.hadoop.hbase.HConstants
> import org.apache.hadoop.conf.Configuration
> import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper
>
>
> parts1 = ARGV[0].split(":")
>
> c1 = HBaseConfiguration.create()
> c1.set(HConstants::ZOOKEEPER_QUORUM, parts1[0])
> c1.set("hbase.zookeeper.property.clientPort", parts1[1])
> c1.set(HConstants::ZOOKEEPER_ZNODE_PARENT, parts1[2])
>
>
> zkw = ZooKeeperWrapper.createInstance(c1, "ZK")
>
> zkw.writeZNode(parts1[2], "test", "")
>
> #------------------------------------------------------------
>
> I ran it from the master cluster and gave it the address of the slave
> quorum with this command:
>
> hbase org.jruby.Main testZK.rb bk1,bk2,bk3:2181:/hbase
>
> The slave ZK quorum didn't have the '/hbase/test' node but the master
> ZK quorum did. The script didn't honor the specified configuration.
> Any thoughts?
>
>
> On Mon, Dec 13, 2010 at 4:04 PM, Jean-Daniel Cryans <[email protected]> 
> wrote:
>> Interesting... the fact that it says that it's connecting to
>> bk1,bk2,bk3 means that it's looking at the right zookeeper ensemble.
>> What it does next is reading all the znodes in /hbase/rs/ (which is
>> the list of live region servers) and chooses a subset of it.
>>
>> Using the zcli utility, could you check the value of those znodes and
>> see if it makes sense? You can run it like that:
>>
>> bin/hbase zkcli
>>
>> And it will be run against the ensemble that that cluster is using.
>>
>> J-D
>>
>> On Mon, Dec 13, 2010 at 2:03 PM, Nathaniel Cook
>> <[email protected]> wrote:
>>> When the master cluster chooses a peer it is supposed to choose a peer
>>> from the slave cluster correct?
>>>
>>> This is what I am seeing in the master cluster logs.
>>>
>>>
>>> Added new peer cluster bk1,bk2,bk3,2181,/hbase
>>> Getting 1 rs from peer cluster # test
>>> Choosing peer 192.168.1.170:60020
>>>
>>> But 192.168.1.170 is an address in the master cluster. I think this
>>> may be related to the problem I had while running the add_peer.rb
>>> script. When I ran that script it would only talk to the ZK quorum
>>> running on that machine and would not talk to the slave ZK quorum .
>>> Could it be that when it is trying to choose a peer, instead of going
>>> to the slave ZK quorum  running on a different machine it is talking
>>> only to the ZK quorum running on its localhost?
>>>
>>>
>>>
>>> On Mon, Dec 13, 2010 at 2:51 PM, Nathaniel Cook
>>> <[email protected]> wrote:
>>>> Thanks for looking into this with me.
>>>>
>>>> Ok so on the master region servers I am getting the two statements
>>>> 'Replicating x' and 'Replicated in total: y'
>>>>
>>>> Nothing on the slave cluster.
>>>>
>>>> On Mon, Dec 13, 2010 at 12:28 PM, Jean-Daniel Cryans
>>>> <[email protected]> wrote:
>>>>> Hi Nathaniel,
>>>>>
>>>>> Thanks for trying out replication, let's make it work for you.
>>>>>
>>>>> So on the master-side there's 2 lines that are important to make sure
>>>>> that replication works, first it has to say:
>>>>>
>>>>> Replicating x
>>>>>
>>>>> Where x is the number of edits it's going to ship, and then
>>>>>
>>>>> Replicated in total: y
>>>>>
>>>>> Where y is the total number it replicated. Seeing the second line
>>>>> means that replication was successful, at least from the master point
>>>>> of view.
>>>>>
>>>>> On the slave, one node should have:
>>>>>
>>>>> Total replicated: z
>>>>>
>>>>> And that z is the number of edits that that region server applied on
>>>>> it's cluster. It could be on any region server, since the sink for
>>>>> replication is chose at random.
>>>>>
>>>>> Do you see those? Any exceptions around those logs apart from EOFs?
>>>>>
>>>>> Thx,
>>>>>
>>>>> J-D
>>>>>
>>>>> On Mon, Dec 13, 2010 at 10:52 AM, Nathaniel Cook
>>>>> <[email protected]> wrote:
>>>>>> Hi,
>>>>>>
>>>>>> I am trying to setup replication for my HBase clusters. I have two
>>>>>> small clusters for testing each with 4 machines. The setup for the two
>>>>>> clusters is identical. Each machine runs a DataNode, and
>>>>>> HRegionServer. Three of the machines run a ZK peer and one machine
>>>>>> runs the HMaster and NameNode. The cluster master machines have
>>>>>> hostnames (ds1,ds2 ...) and the slave cluster is (bk1, bk2 ...). I set
>>>>>> the replication  scope to 1 for my test table column families and set
>>>>>> the hbase.replication property to true for both clusters. Next I ran
>>>>>> the add_peer.rb script with the following command on the ds1 machine:
>>>>>>
>>>>>> hbase org.jruby.Main /usr/lib/hbase/bin/replication/add_peer.rb
>>>>>> ds1:2181:/hbase bk1:2181:/hbase
>>>>>>
>>>>>> After the script finishes ZK for the master cluster has the
>>>>>> replication znode and children of peers, master, and state. The slave
>>>>>> ZK didn't have a replication znode. I fixed that problem by rerunning
>>>>>> the script on the bk1 machine and commenting out the code to write to
>>>>>> the master ZK. Now the slave ZK has the /hbase/replication/master
>>>>>> znode with data (ds1:2181:/hbase). Everthing looked to be configured
>>>>>> correctly. I restarted the clusters. The logs of the master
>>>>>> regionservers stated:
>>>>>>
>>>>>> This cluster (ds1:2181:/hbase) is a master for replication, compared
>>>>>> with (ds1:2181:/hbase)
>>>>>>
>>>>>> The logs on the slave cluster stated:
>>>>>>
>>>>>> This cluster (bk1:2181:/hbase) is a slave for replication, compared
>>>>>> with (ds1:2181:/hbase)
>>>>>>
>>>>>> Using the hbase shell I put a row into the test table.
>>>>>>
>>>>>> The regionserver for that table had a log statement like:
>>>>>>
>>>>>> Going to report log #192.168.1.166%3A60020.1291757445179 for position
>>>>>> 15828 in 
>>>>>> hdfs://ds1:9000/hbase/.logs/ds1.internal,60020,1291757445059/192.168.1.166
>>>>>> <http://192.168.1.166/>%3A60020.1291757445179
>>>>>>
>>>>>> (192.168.1.166 is ds1)
>>>>>>
>>>>>> I wait and even after several minutes the row still does not appear in
>>>>>> the slave cluster table.
>>>>>>
>>>>>> Any help with what the problem might be is greatly appreciated.
>>>>>>
>>>>>> Both clusters are using a CDH3b3. The HBase version is exactly
>>>>>> 0.89.20100924+28.
>>>>>>
>>>>>> -Nathaniel Cook
>>>>>>
>>>>>
>>>>
>>>>
>>>>
>>>> --
>>>> -Nathaniel Cook
>>>>
>>>
>>>
>>>
>>> --
>>> -Nathaniel Cook
>>>
>>
>
>
>
> --
> -Nathaniel Cook
>

Reply via email to