You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hbase.apache.org by Jean-Daniel Cryans <jd...@apache.org> on 2010/05/27 03:09:30 UTC

Re: Review Request: HBASE-2223

-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/
-----------------------------------------------------------

(Updated 2010-05-26 18:09:30.362592)


Review request for hbase.


Changes
-------

This new patch takes care of almost all comments except:
ReplicationZookeeperHelper.java
- use a <pre> block to make this more readable in the HTML version of the javadoc.

ReplicationSink.java
- I think it would be good to document the fact that this method will typically be called from another thread than the thread that executes `run' so that other people reading the code will quickly get a good grasp of what are the concurrency / locking requirements.

- So Delete operations are "unbuffered" unlike Put operations, which you "buffer" in the `puts' list.  Does that mean that a Delete can be executed before the Put that was creating the data in the first place, and that the Delete will fail first and the Put will survive second?

// Should we log rejected edits in a file for replay?
- I vote yes


The major change I did was removing ReplicationConnectionManager and using HCM directly since it was the same code (so the comments left by Benoit still apply, but to HCM). Other than that it's mostly refactoring and fixing nits.


Summary
-------

This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.


This addresses bug HBASE-2223.
    http://issues.apache.org/jira/browse/HBASE-2223


Diffs (updated)
-----

  src/main/java/org/apache/hadoop/hbase/HConstants.java 13aff26 
  src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 4cbe52a 
  src/main/java/org/apache/hadoop/hbase/master/ServerManager.java a197b8f 
  src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b5ff43a 
  src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 12a3cd8 
  src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 7c1184c 
  src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java ed8709f 
  src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 

Diff: http://review.hbase.org/r/76/diff


Testing
-------


Thanks,

Jean-Daniel


Re: Review Request: HBASE-2223

Posted by Jean-Daniel Cryans <jd...@apache.org>.

> On 2010-05-27 12:55:57, Benoit Sigoure wrote:
> > src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java, line 378
> > <http://review.hbase.org/r/76/diff/2/?file=669#file669line378>
> >
> >     This seems brittle to me.

Kinda, but it's currently the best I found.


> On 2010-05-27 12:55:57, Benoit Sigoure wrote:
> > src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java, line 385
> > <http://review.hbase.org/r/76/diff/2/?file=669#file669line385>
> >
> >     ?

There are some race conditions currently preventing that to work. Same reason I need to make sure in this test that the master cluster has all the edits when I kill a region server GC-style. See this code:

    // Test we actually have all the rows, we may miss some because we
    // don't have IO fencing.
    if (res.length != initialCount) {
      LOG.warn("We lost some rows on the master cluster!");
      // We don't really expect the other cluster to have more rows
      initialCount = res.length;
    }

So if I can lose rows on the master cluster, I can also lose rows on the slave cluster... but rows that are lost because of a lack of IO-fencing or because of a bug in the replication will be the same from the client perspective.


> On 2010-05-27 12:55:57, Benoit Sigoure wrote:
> > src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java, line 135
> > <http://review.hbase.org/r/76/diff/2/?file=671#file671line135>
> >
> >     I don't understand why you're doing +=2 and /2 instead of just setting the upper bound of the loop to BATCH_SIZE/2.

/me feels dumb


> On 2010-05-27 12:55:57, Benoit Sigoure wrote:
> > src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java, line 198
> > <http://review.hbase.org/r/76/diff/2/?file=672#file672line198>
> >
> >     Uh?  Manual loop unrolling?

Mostly cruft 


- Jean-Daniel


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/#review86
-----------------------------------------------------------


On 2010-05-26 18:09:30, Jean-Daniel Cryans wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> http://review.hbase.org/r/76/
> -----------------------------------------------------------
> 
> (Updated 2010-05-26 18:09:30)
> 
> 
> Review request for hbase.
> 
> 
> Summary
> -------
> 
> This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.
> 
> 
> This addresses bug HBASE-2223.
>     http://issues.apache.org/jira/browse/HBASE-2223
> 
> 
> Diffs
> -----
> 
>   src/main/java/org/apache/hadoop/hbase/HConstants.java 13aff26 
>   src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 4cbe52a 
>   src/main/java/org/apache/hadoop/hbase/master/ServerManager.java a197b8f 
>   src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b5ff43a 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 12a3cd8 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 7c1184c 
>   src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java ed8709f 
>   src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 
> 
> Diff: http://review.hbase.org/r/76/diff
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Jean-Daniel
> 
>


Re: Review Request: HBASE-2223

Posted by Benoit Sigoure <ts...@gmail.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/#review86
-----------------------------------------------------------



src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
<http://review.hbase.org/r/76/#comment472>

    This test is pretty long and tests quite a few different things.  Consider to break it down into multiple smaller unit tests.



src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
<http://review.hbase.org/r/76/#comment473>

    I'm not sure to understand why this test is named like that.  Could you comment this test a bit more to explain what the intent is?



src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
<http://review.hbase.org/r/76/#comment474>

    This seems brittle to me.



src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
<http://review.hbase.org/r/76/#comment475>

    ?



src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
<http://review.hbase.org/r/76/#comment476>

    I believe this method can be made `static'.



src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
<http://review.hbase.org/r/76/#comment477>

    Copyright header.



src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
<http://review.hbase.org/r/76/#comment486>

    I'm not a big fan of this.



src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
<http://review.hbase.org/r/76/#comment478>

    private.



src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
<http://review.hbase.org/r/76/#comment479>

    Do you really need to define the 3 following empty methods?



src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
<http://review.hbase.org/r/76/#comment484>

    Consider writing a comment explaining what this test does.



src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
<http://review.hbase.org/r/76/#comment480>

    Missing space before `conf'.



src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
<http://review.hbase.org/r/76/#comment481>

    +"" is an idiom I don't like.



src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
<http://review.hbase.org/r/76/#comment482>

    Missing spaces.



src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
<http://review.hbase.org/r/76/#comment483>

    Missing spaces.



src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
<http://review.hbase.org/r/76/#comment485>

    Don't do this.



src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
<http://review.hbase.org/r/76/#comment487>

    Don't do this.



src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
<http://review.hbase.org/r/76/#comment488>

    Why protected?



src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
<http://review.hbase.org/r/76/#comment489>

    Is this really needed?



src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
<http://review.hbase.org/r/76/#comment490>

    I don't understand why you're doing +=2 and /2 instead of just setting the upper bound of the loop to BATCH_SIZE/2.



src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
<http://review.hbase.org/r/76/#comment491>

    Missing space before `scanRes'.



src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment492>

    Why protected?



src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment493>

    ?



src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment494>

    Uh?  Manual loop unrolling?


- Benoit


On 2010-05-26 18:09:30, Jean-Daniel Cryans wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> http://review.hbase.org/r/76/
> -----------------------------------------------------------
> 
> (Updated 2010-05-26 18:09:30)
> 
> 
> Review request for hbase.
> 
> 
> Summary
> -------
> 
> This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.
> 
> 
> This addresses bug HBASE-2223.
>     http://issues.apache.org/jira/browse/HBASE-2223
> 
> 
> Diffs
> -----
> 
>   src/main/java/org/apache/hadoop/hbase/HConstants.java 13aff26 
>   src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 4cbe52a 
>   src/main/java/org/apache/hadoop/hbase/master/ServerManager.java a197b8f 
>   src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b5ff43a 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 12a3cd8 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 7c1184c 
>   src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java ed8709f 
>   src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 
> 
> Diff: http://review.hbase.org/r/76/diff
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Jean-Daniel
> 
>


Re: Review Request: HBASE-2223

Posted by Jean-Daniel Cryans <jd...@apache.org>.

> On 2010-05-26 23:48:35, Benoit Sigoure wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java, line 120
> > <http://review.hbase.org/r/76/diff/2/?file=664#file664line120>
> >
> >     I don't find `replication.source.size.capacity' particularly explicit.  What is it supposed to control?

I'll put more details into their attributes. Also if we document those configs I will add details in hbase-default.xml


> On 2010-05-26 23:48:35, Benoit Sigoure wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java, line 167
> > <http://review.hbase.org/r/76/diff/2/?file=664#file664line167>
> >
> >     Nit pick: mapOfAddr since in English "Address" starts with "Addr" :)

Avoue que ça te mélanges autant que moi ;)


> On 2010-05-26 23:48:35, Benoit Sigoure wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java, line 179
> > <http://review.hbase.org/r/76/diff/2/?file=664#file664line179>
> >
> >     If HServerAddress respects the equal contract, consider using a set instead of a map.

And then, even better, might as well just use a HashSet :P


> On 2010-05-26 23:48:35, Benoit Sigoure wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java, line 208
> > <http://review.hbase.org/r/76/diff/2/?file=664#file664line208>
> >
> >     Is there any reason why you're checking this here?

It's a symptom of a problem with my code, I'll fix that.


> On 2010-05-26 23:48:35, Benoit Sigoure wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java, line 229
> > <http://review.hbase.org/r/76/diff/2/?file=664#file664line229>
> >
> >     I find this comment confusing.  Can you make it a bit more explicit, especially towards the "normally has a position" part?

Yeah, in fact I could just remove it... It means that we recovered a queue from another region server, and normally you are tailing a file but a RS could fail between 2 logs and that means that the new one wasn't read from yet.


> On 2010-05-26 23:48:35, Benoit Sigoure wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java, line 268
> > <http://review.hbase.org/r/76/diff/2/?file=664#file664line268>
> >
> >     All the double-negations with noIOE are harder to read than if instead you were starting with boolean gotIOE = false and set it to true when you get an IOE.

y


> On 2010-05-26 23:48:35, Benoit Sigoure wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java, line 270
> > <http://review.hbase.org/r/76/diff/2/?file=664#file664line270>
> >
> >     This `try' block is massive, would it be possible to refactor it using a private method to make the code a bit more readable?

I'll do my best.


> On 2010-05-26 23:48:35, Benoit Sigoure wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java, line 329
> > <http://review.hbase.org/r/76/diff/2/?file=664#file664line329>
> >
> >     It seems that considerDumping will always be true except when you fail to stat() the file due to an unexpected error.  That seems suspicious to me.  Is this intended?  If yes, can you explain?

I agree that part is weird, I built this while HDFS wasn't stable yet for tail'ing files and ended up having to decide if I should ditch a file that was broken. Currently you consider dumping the file when you get an EOF and if:
 - The queue was recovered and is empty. It's very suspicious. Usually you get EOFs on file that are opened but totally empty. Might as well get rid of it?
 - The number of entries read is greater than 0. That means you were reading the file, and instead of just getting null at the end you get an EOF in the face. By experience, it's usually a race condition à la HDFS-1057 then just redo the reading. If it failed 10 times, it means that that file is totally broken. 

Comments?


> On 2010-05-26 23:48:35, Benoit Sigoure wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java, line 109
> > <http://review.hbase.org/r/76/diff/2/?file=666#file666line109>
> >
> >     Does this statement need to be in the synchronized block?  If yes, why?

Nah doesn't need to.


> On 2010-05-26 23:48:35, Benoit Sigoure wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java, line 191
> > <http://review.hbase.org/r/76/diff/2/?file=666#file666line191>
> >
> >     Document.

Already documented in the interface


> On 2010-05-26 23:48:35, Benoit Sigoure wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java, line 206
> > <http://review.hbase.org/r/76/diff/2/?file=666#file666line206>
> >
> >     Document.

>>From the interface


> On 2010-05-26 23:48:35, Benoit Sigoure wrote:
> > src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java, line 12
> > <http://review.hbase.org/r/76/diff/2/?file=668#file668line12>
> >
> >     No javadoc in this file?

All from the implemented interface


> On 2010-05-26 23:48:35, Benoit Sigoure wrote:
> > src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java, line 55
> > <http://review.hbase.org/r/76/diff/2/?file=669#file669line55>
> >
> >     Why is this `protected' instead of `private'?

Excellent question...


> On 2010-05-26 23:48:35, Benoit Sigoure wrote:
> > src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java, line 126
> > <http://review.hbase.org/r/76/diff/2/?file=669#file669line126>
> >
> >     Pardon my ignorance but what's the `2' supposed to mean here?

Start 2 region servers


> On 2010-05-26 23:48:35, Benoit Sigoure wrote:
> > src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java, line 151
> > <http://review.hbase.org/r/76/diff/2/?file=669#file669line151>
> >
> >     Maybe you can add a watcher for yourself too so you get notified when you can continue instead of sleeping SLEEP_TIME and hoping that the timing will work out OK?

Seems the class would be even more complicated.


> On 2010-05-26 23:48:35, Benoit Sigoure wrote:
> > src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java, line 182
> > <http://review.hbase.org/r/76/diff/2/?file=669#file669line182>
> >
> >     Do you need an empty method?

Was keeping it around in case I need it again.


- Jean-Daniel


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/#review84
-----------------------------------------------------------


On 2010-05-26 18:09:30, Jean-Daniel Cryans wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> http://review.hbase.org/r/76/
> -----------------------------------------------------------
> 
> (Updated 2010-05-26 18:09:30)
> 
> 
> Review request for hbase.
> 
> 
> Summary
> -------
> 
> This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.
> 
> 
> This addresses bug HBASE-2223.
>     http://issues.apache.org/jira/browse/HBASE-2223
> 
> 
> Diffs
> -----
> 
>   src/main/java/org/apache/hadoop/hbase/HConstants.java 13aff26 
>   src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 4cbe52a 
>   src/main/java/org/apache/hadoop/hbase/master/ServerManager.java a197b8f 
>   src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b5ff43a 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 12a3cd8 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 7c1184c 
>   src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java ed8709f 
>   src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 
> 
> Diff: http://review.hbase.org/r/76/diff
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Jean-Daniel
> 
>


Re: Review Request: HBASE-2223

Posted by Benoit Sigoure <ts...@gmail.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/#review84
-----------------------------------------------------------


I'll pick up where I stopped in TestReplication.java tomorrow.


src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment412>

    Terminate this sentence with a period.
    
    Generally speak you must terminate the first sentence of a javadoc with a period.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment413>

    I don't find `replication.source.size.capacity' particularly explicit.  What is it supposed to control?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment414>

    Ditto.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment415>

    This is unnecessary.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment416>

    Nit pick: mapOfAddr since in English "Address" starts with "Addr" :)



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment417>

    s/adr/addr/ :)



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment418>

    How about using a do-while instead of repeating this line of code?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment419>

    If HServerAddress respects the equal contract, consider using a set instead of a map.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment420>

    Move this outside of the try block.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment422>

    The explicit call to toString() here is useless.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment421>

    Is there any reason why you're checking this here?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment423>

    Log a message too.
    This will call Log#error(java.lang.Object message) and I presume you meant to call Log#error(java.lang.Object message, java.lang.Throwable t)



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment424>

    I find this comment confusing.  Can you make it a bit more explicit, especially towards the "normally has a position" part?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment425>

    Don't lock in the `try' block.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment426>

    Why are you checking this again here?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment432>

    All the double-negations with noIOE are harder to read than if instead you were starting with boolean gotIOE = false and set it to true when you get an IOE.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment427>

    This `try' block is massive, would it be possible to refactor it using a private method to make the code a bit more readable?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment428>

    Use a local variable for entry.getKey() instead of calling it so many times in a row.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment429>

    Unnecessary call to toString().



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment430>

    It seems that considerDumping will always be true except when you fail to stat() the file due to an unexpected error.  That seems suspicious to me.  Is this intended?  If yes, can you explain?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment431>

    Need a message in first argument.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment433>

    Please document.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment434>

    Please document everything properly.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment435>

    Document.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment436>

    How about storing the result of edit.getKeyValues() in a local instead of repeating that code and repeating the call twice at every loop?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment437>

    Please document everything.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment438>

    Put this in a try-finally block.  Just because you're nulling one reference doesn't protect you from this thread getting interrupted and then you're left with a locked lock.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment439>

    Why are you doing this?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment440>

    s/adr/addr/



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment441>

    :)



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
<http://review.hbase.org/r/76/#comment442>

    Please document.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment443>

    You need to enclose te <li> in <ul> or <ol> and close the tags properly.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment444>

    Make this private.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment445>

    Can you please document everything non obvious.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment446>

    s/ and that/.  This/



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment447>

    ??



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment448>

    Document.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment449>

    queueRecovered should be documented too.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment450>

    Does this statement need to be in the synchronized block?  If yes, why?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment451>

    Unnecessary call to toString().



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment452>

    "Add" vs "tries" – be consistent.  Use imperative everywhere or don't use it.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment453>

    Since log messages can be intertwined if the server is busy logging a lot, how about making this and the previous line a single message.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment457>

    Document.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment454>

    This doesn't need to be in the synchronized block, does it?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment456>

    Document.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment455>

    Document.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment458>

    Remove this line and the next to avoid code duplication with line 250/251.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment459>

    Don't do this.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment460>

    Document.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment461>

    Don't you only need to synchronize on those two lines instead of all the other ones too?



src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
<http://review.hbase.org/r/76/#comment464>

    No copyright?



src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
<http://review.hbase.org/r/76/#comment463>

    No javadoc in this file?



src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
<http://review.hbase.org/r/76/#comment462>

    Not properly indented.



src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
<http://review.hbase.org/r/76/#comment465>

    Why is this `protected' instead of `private'?



src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
<http://review.hbase.org/r/76/#comment466>

    Pardon my ignorance but what's the `2' supposed to mean here?



src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
<http://review.hbase.org/r/76/#comment467>

    I'm not sure whether this is useful since junit will already give us all the info in the event of an unexpected exception.  Furthermore, the first argument to LOG.error must be a message.



src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
<http://review.hbase.org/r/76/#comment468>

    This would be a better API if it received a boolean in argument and transformed it in a String itself.



src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
<http://review.hbase.org/r/76/#comment470>

    Maybe you can add a watcher for yourself too so you get notified when you can continue instead of sleeping SLEEP_TIME and hoping that the timing will work out OK?



src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
<http://review.hbase.org/r/76/#comment469>

    ??



src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
<http://review.hbase.org/r/76/#comment471>

    Do you need an empty method?


- Benoit


On 2010-05-26 18:09:30, Jean-Daniel Cryans wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> http://review.hbase.org/r/76/
> -----------------------------------------------------------
> 
> (Updated 2010-05-26 18:09:30)
> 
> 
> Review request for hbase.
> 
> 
> Summary
> -------
> 
> This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.
> 
> 
> This addresses bug HBASE-2223.
>     http://issues.apache.org/jira/browse/HBASE-2223
> 
> 
> Diffs
> -----
> 
>   src/main/java/org/apache/hadoop/hbase/HConstants.java 13aff26 
>   src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 4cbe52a 
>   src/main/java/org/apache/hadoop/hbase/master/ServerManager.java a197b8f 
>   src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b5ff43a 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 12a3cd8 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 7c1184c 
>   src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java ed8709f 
>   src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 
> 
> Diff: http://review.hbase.org/r/76/diff
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Jean-Daniel
> 
>


Re: Review Request: HBASE-2223

Posted by Jean-Daniel Cryans <jd...@apache.org>.

> On 2010-06-11 12:45:29, stack wrote:
> > bin/replication/add_peer.rb, line 21
> > <http://review.hbase.org/r/76/diff/5/?file=1104#file1104line21>
> >
> >     Should you point at some replication documentation here?  Is there such a thing?

package.html later, should I point to it?


> On 2010-06-11 12:45:29, stack wrote:
> > bin/replication/copy_tables_desc.rb, line 58
> > <http://review.hbase.org/r/76/diff/5/?file=1105#file1105line58>
> >
> >     This could get a bit annoying I'd say.

It helped me a lot, remove if people complain?


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/HConstants.java, line 342
> > <http://review.hbase.org/r/76/diff/5/?file=1107#file1107line342>
> >
> >     This has to go here?  Can it go into one of the replication classes?

Used by master and region server, to me it belongs there.


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/master/ServerManager.java, line 156
> > <http://review.hbase.org/r/76/diff/5/?file=1109#file1109line156>
> >
> >     Can't you just do c.get("key", defaultvalue)?

No, I also do a check on replication.


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java, line 929
> > <http://review.hbase.org/r/76/diff/5/?file=1110#file1110line929>
> >
> >     You writing startcode into zk?  Why not write servername -- the host+port+startcode combo?

To be coherent with the rest of the code that uses zookeeper.


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java, line 1075
> > <http://review.hbase.org/r/76/diff/5/?file=1110#file1110line1075>
> >
> >     Is this directory name?  Confusingly named given rootdir+regLogPathStr only adds up to repLogPath.

I don't understand you, but this code is going to be removed in my next patch as I'm simplifying RepSink.


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java, line 55
> > <http://review.hbase.org/r/76/diff/5/?file=1113#file1113line55>
> >
> >     Peers are named '1', '2'?  Can't we have more meaningful names here?

We agreed that peers are identified with a short internally as it is stored. We could use an external mapping of short->cute_name.


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java, line 59
> > <http://review.hbase.org/r/76/diff/5/?file=1113#file1113line59>
> >
> >     Use servername instead of startcode

Same comment as before, needs to be coherent.


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java, line 60
> > <http://review.hbase.org/r/76/diff/5/?file=1113#file1113line60>
> >
> >     All RS's in a master cluster replicate?

Yep... was that an implicit way of saying that I need to document that in RZH?


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java, line 107
> > <http://review.hbase.org/r/76/diff/5/?file=1113#file1113line107>
> >
> >     Should this class be called WRapper instaad of Helper?

Sure


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java, line 185
> > <http://review.hbase.org/r/76/diff/5/?file=1113#file1113line185>
> >
> >     You mean 'ensemble' here rather than 'quorum' (Patrick will kill you if he sees you calling it a 'quorum' when you mean the other)

Argh I'm trying to correct myself but I'm still missing some of them. Thx!


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java, line 263
> > <http://review.hbase.org/r/76/diff/5/?file=1113#file1113line263>
> >
> >     We keep up the replication position in zk?  How much do we replicate in one go?  Its not a single edit, is it?  We do this for every log file?

Yes. A defined amount specified in ReplicationSource. No. Every current log file, we only replicate one at a time per region server.


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java, line 328
> > <http://review.hbase.org/r/76/diff/5/?file=1113#file1113line328>
> >
> >     LOG.warn instead?
> >

I'll do like the rest and log.error


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java, line 354
> > <http://review.hbase.org/r/76/diff/5/?file=1113#file1113line354>
> >
> >     We return empty map if clusters size is == 1?  Should that be clusters.size == 0?

That part isn't clear enough, so the reason it's 1 and not 0 is that we put a lock in there so it's listed in the znodes we fetch. Actually this should be <= 1 rather than ==.


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java, line 356
> > <http://review.hbase.org/r/76/diff/5/?file=1113#file1113line356>
> >
> >     Whats this about?

See previous comment, we lock the dead region server's znode by putting a lock in there, but we don't want to process the hlogs under since... it's not a cluster. Could use more doc.


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java, line 402
> > <http://review.hbase.org/r/76/diff/5/?file=1113#file1113line402>
> >
> >     Just logging errors?  What if session expired (our discussion from last day)?

Yes I need to review how I handle it in RZH, but I'd also need to review ZKW since some methods will hid it in there.


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/package.html, line 41
> > <http://review.hbase.org/r/76/diff/5/?file=1115#file1115line41>
> >
> >     Call it alpha

yeah! (j/k)


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/package.html, line 64
> > <http://review.hbase.org/r/76/diff/5/?file=1115#file1115line64>
> >
> >     Whats this about?  You need to run zk yourself but no zoo.cfg?

I... don't remember why I wrote this.


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/package.html, line 73
> > <http://review.hbase.org/r/76/diff/5/?file=1115#file1115line73>
> >
> >     And if not?  What if replicating single-family only?

Forgot to update that after we added scoping, updating.


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/package.html, line 83
> > <http://review.hbase.org/r/76/diff/5/?file=1115#file1115line83>
> >
> >     Has to be offline?  Will this always be the case?

Currently everything is static, but I hope we can move on from that in the future.


> On 2010-06-11 12:45:29, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/package.html, line 108
> > <http://review.hbase.org/r/76/diff/5/?file=1115#file1115line108>
> >
> >     whats ratio?

This is a log snippet that's coming from a region server. Do you want to see more documentation about it in package.html or in the logging itself?


- Jean-Daniel


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/#review191
-----------------------------------------------------------


On 2010-06-08 17:54:19, Jean-Daniel Cryans wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> http://review.hbase.org/r/76/
> -----------------------------------------------------------
> 
> (Updated 2010-06-08 17:54:19)
> 
> 
> Review request for hbase.
> 
> 
> Summary
> -------
> 
> This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.
> 
> 
> This addresses bug HBASE-2223.
>     http://issues.apache.org/jira/browse/HBASE-2223
> 
> 
> Diffs
> -----
> 
>   bin/replication/add_peer.rb PRE-CREATION 
>   bin/replication/copy_tables_desc.rb PRE-CREATION 
>   pom.xml 03c6ec8 
>   src/main/java/org/apache/hadoop/hbase/HConstants.java 13aff26 
>   src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java b36f1df 
>   src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 82148a6 
>   src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java a1baff4 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 034690e 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 5d4cffe 
>   src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java 2f2f306 
>   src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 
> 
> Diff: http://review.hbase.org/r/76/diff
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Jean-Daniel
> 
>


Re: Review Request: HBASE-2223

Posted by st...@duboce.net.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/#review191
-----------------------------------------------------------


I got as far as ReplicationLogCleaner... will continue later.


bin/replication/add_peer.rb
<http://review.hbase.org/r/76/#comment868>

    Should you point at some replication documentation here?  Is there such a thing?



bin/replication/copy_tables_desc.rb
<http://review.hbase.org/r/76/#comment870>

    This could get a bit annoying I'd say.



src/main/java/org/apache/hadoop/hbase/HConstants.java
<http://review.hbase.org/r/76/#comment871>

    This has to go here?  Can it go into one of the replication classes?



src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
<http://review.hbase.org/r/76/#comment872>

    Can't you just do c.get("key", defaultvalue)?



src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<http://review.hbase.org/r/76/#comment873>

    You writing startcode into zk?  Why not write servername -- the host+port+startcode combo?



src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<http://review.hbase.org/r/76/#comment875>

    Is this directory name?  Confusingly named given rootdir+regLogPathStr only adds up to repLogPath.



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java
<http://review.hbase.org/r/76/#comment876>

    Replication needs package documentation or else an article (like metrics) -- oh, i see it later... nm



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java
<http://review.hbase.org/r/76/#comment877>

    Peers are named '1', '2'?  Can't we have more meaningful names here?



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java
<http://review.hbase.org/r/76/#comment878>

    s/replicates/replicate/
    



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java
<http://review.hbase.org/r/76/#comment879>

    Use servername instead of startcode



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java
<http://review.hbase.org/r/76/#comment880>

    All RS's in a master cluster replicate?



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java
<http://review.hbase.org/r/76/#comment881>

    Should this class be called WRapper instaad of Helper?



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java
<http://review.hbase.org/r/76/#comment882>

    Registers it where?



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java
<http://review.hbase.org/r/76/#comment883>

    You mean 'ensemble' here rather than 'quorum' (Patrick will kill you if he sees you calling it a 'quorum' when you mean the other)



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java
<http://review.hbase.org/r/76/#comment884>

    We keep up the replication position in zk?  How much do we replicate in one go?  Its not a single edit, is it?  We do this for every log file?



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java
<http://review.hbase.org/r/76/#comment885>

    LOG.warn instead?
    



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java
<http://review.hbase.org/r/76/#comment887>

    We return empty map if clusters size is == 1?  Should that be clusters.size == 0?



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java
<http://review.hbase.org/r/76/#comment886>

    Whats this about?



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java
<http://review.hbase.org/r/76/#comment888>

    Just logging errors?  What if session expired (our discussion from last day)?



src/main/java/org/apache/hadoop/hbase/replication/package.html
<http://review.hbase.org/r/76/#comment889>

    Call it alpha



src/main/java/org/apache/hadoop/hbase/replication/package.html
<http://review.hbase.org/r/76/#comment890>

    Whats this about?  You need to run zk yourself but no zoo.cfg?



src/main/java/org/apache/hadoop/hbase/replication/package.html
<http://review.hbase.org/r/76/#comment891>

    And if not?  What if replicating single-family only?



src/main/java/org/apache/hadoop/hbase/replication/package.html
<http://review.hbase.org/r/76/#comment892>

    Has to be offline?  Will this always be the case?



src/main/java/org/apache/hadoop/hbase/replication/package.html
<http://review.hbase.org/r/76/#comment893>

    whats ratio?


- stack


On 2010-06-08 17:54:19, Jean-Daniel Cryans wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> http://review.hbase.org/r/76/
> -----------------------------------------------------------
> 
> (Updated 2010-06-08 17:54:19)
> 
> 
> Review request for hbase.
> 
> 
> Summary
> -------
> 
> This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.
> 
> 
> This addresses bug HBASE-2223.
>     http://issues.apache.org/jira/browse/HBASE-2223
> 
> 
> Diffs
> -----
> 
>   bin/replication/add_peer.rb PRE-CREATION 
>   bin/replication/copy_tables_desc.rb PRE-CREATION 
>   pom.xml 03c6ec8 
>   src/main/java/org/apache/hadoop/hbase/HConstants.java 13aff26 
>   src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java b36f1df 
>   src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 82148a6 
>   src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java a1baff4 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 034690e 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 5d4cffe 
>   src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java 2f2f306 
>   src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 
> 
> Diff: http://review.hbase.org/r/76/diff
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Jean-Daniel
> 
>


Re: Review Request: HBASE-2223

Posted by Jean-Daniel Cryans <jd...@apache.org>.

> On 2010-06-11 15:31:37, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java, line 56
> > <http://review.hbase.org/r/76/diff/5/?file=1114#file1114line56>
> >
> >     For sure setConf will have been called before we get here?  So, stuff gets setup by setConf?  Can setConf be called more than once?  How do I know how to use this class?  Not doc'd.  Doesn't have a Constructor.

LogCleanerDelegate is the interface that defines the general behavior. Yes should have a constructor.


> On 2010-06-11 15:31:37, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java, line 111
> > <http://review.hbase.org/r/76/diff/5/?file=1114#file1114line111>
> >
> >     The way this is done, if I didn't want to wait on the ttl, then I'd have to write a new class.  Can't we have ttl and recplication be distinct and then if I want delete based off ttl and whether log up in zk, then chain them?

I don't follow, chaining is already how I do it.


> On 2010-06-11 15:31:37, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java, line 54
> > <http://review.hbase.org/r/76/diff/5/?file=1116#file1116line54>
> >
> >     I dont follow?

Yeah, RepSink is a mix of 2 solutions but only features the worst of both. The next patch will significantly make it better.


> On 2010-06-11 15:31:37, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java, line 126
> > <http://review.hbase.org/r/76/diff/5/?file=1117#file1117line126>
> >
> >     This ain't a constructor?

I ain't.. but it's used like one.


> On 2010-06-11 15:31:37, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java, line 483
> > <http://review.hbase.org/r/76/diff/5/?file=1117#file1117line483>
> >
> >     We have to copy?

This is the down side of the way of I'm caping the log entries by size or number. I'm reusing the same HLog.Entry[] entriesArray to read from HLogs (and the entries in it). For example, replicationQueueSizeCapacity=64MB and replicationQueueNbCapacity=25k. Let's say on a first run we reach 25k without reaching the size, so we'll replicate the whole array. Now on the second run let's say we reached 64MB after only 10k rows, then we only want to replicate that and not the 15k "leftovers".


> On 2010-06-11 15:31:37, stack wrote:
> > src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java, line 67
> > <http://review.hbase.org/r/76/diff/5/?file=1123#file1123line67>
> >
> >     No dfs in this test.  Thats intentional?

Nope, should fix.


> On 2010-06-11 15:31:37, stack wrote:
> > src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java, line 86
> > <http://review.hbase.org/r/76/diff/5/?file=1124#file1124line86>
> >
> >     Can't you squash some of these tests together?  They each start up own minidfscluster... just start it once?

They don't?

  @Before
  public void setUp() throws Exception {
    table1 = TEST_UTIL.truncateTable(TABLE_NAME1);
    table2 = TEST_UTIL.truncateTable(TABLE_NAME2);
    Thread.sleep(SLEEP_TIME);
  }


- Jean-Daniel


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/#review194
-----------------------------------------------------------


On 2010-06-08 17:54:19, Jean-Daniel Cryans wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> http://review.hbase.org/r/76/
> -----------------------------------------------------------
> 
> (Updated 2010-06-08 17:54:19)
> 
> 
> Review request for hbase.
> 
> 
> Summary
> -------
> 
> This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.
> 
> 
> This addresses bug HBASE-2223.
>     http://issues.apache.org/jira/browse/HBASE-2223
> 
> 
> Diffs
> -----
> 
>   bin/replication/add_peer.rb PRE-CREATION 
>   bin/replication/copy_tables_desc.rb PRE-CREATION 
>   pom.xml 03c6ec8 
>   src/main/java/org/apache/hadoop/hbase/HConstants.java 13aff26 
>   src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java b36f1df 
>   src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 82148a6 
>   src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java a1baff4 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 034690e 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 5d4cffe 
>   src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java 2f2f306 
>   src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 
> 
> Diff: http://review.hbase.org/r/76/diff
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Jean-Daniel
> 
>


Re: Review Request: HBASE-2223

Posted by st...@duboce.net.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/#review194
-----------------------------------------------------------


First pass.  Maybe by the 3rd pass I'll have an idea of whats going on.  General comment is that there is a lot of new code here but tests seem to test replication system.  There are few instances of unit tests ensuring newly added methods are working properly.


src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
<http://review.hbase.org/r/76/#comment894>

    For sure setConf will have been called before we get here?  So, stuff gets setup by setConf?  Can setConf be called more than once?  How do I know how to use this class?  Not doc'd.  Doesn't have a Constructor.



src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
<http://review.hbase.org/r/76/#comment895>

    The way this is done, if I didn't want to wait on the ttl, then I'd have to write a new class.  Can't we have ttl and recplication be distinct and then if I want delete based off ttl and whether log up in zk, then chain them?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
<http://review.hbase.org/r/76/#comment896>

    I dont follow?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
<http://review.hbase.org/r/76/#comment897>

    Should read this out of config. rather than hardcode 10.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
<http://review.hbase.org/r/76/#comment898>

    Same here.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
<http://review.hbase.org/r/76/#comment900>

    Long while loop; can break it up?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
<http://review.hbase.org/r/76/#comment899>

    Only operate on the first kv?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
<http://review.hbase.org/r/76/#comment901>

    Do you have to write position back to zk?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
<http://review.hbase.org/r/76/#comment903>

    Can code from HLog be used here?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment904>

    This ain't a constructor?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment907>

    We have to copy?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
<http://review.hbase.org/r/76/#comment908>

    Not a constructor.  If javadoc in an interface, you don't need to reproduce the javadoc in the implementation.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment909>

    This should be SortedSet, not TreeSet... or NavigableSet.



src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
<http://review.hbase.org/r/76/#comment910>

    Good



src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
<http://review.hbase.org/r/76/#comment911>

    What does this class do?



src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
<http://review.hbase.org/r/76/#comment912>

    No dfs in this test.  Thats intentional?



src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
<http://review.hbase.org/r/76/#comment913>

    Can't you squash some of these tests together?  They each start up own minidfscluster... just start it once?


- stack


On 2010-06-08 17:54:19, Jean-Daniel Cryans wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> http://review.hbase.org/r/76/
> -----------------------------------------------------------
> 
> (Updated 2010-06-08 17:54:19)
> 
> 
> Review request for hbase.
> 
> 
> Summary
> -------
> 
> This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.
> 
> 
> This addresses bug HBASE-2223.
>     http://issues.apache.org/jira/browse/HBASE-2223
> 
> 
> Diffs
> -----
> 
>   bin/replication/add_peer.rb PRE-CREATION 
>   bin/replication/copy_tables_desc.rb PRE-CREATION 
>   pom.xml 03c6ec8 
>   src/main/java/org/apache/hadoop/hbase/HConstants.java 13aff26 
>   src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java b36f1df 
>   src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 82148a6 
>   src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java a1baff4 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 034690e 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 5d4cffe 
>   src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java 2f2f306 
>   src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 
> 
> Diff: http://review.hbase.org/r/76/diff
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Jean-Daniel
> 
>


Re: Review Request: HBASE-2223

Posted by Jean-Daniel Cryans <jd...@apache.org>.

> On 2010-06-09 23:44:22, Benoit Sigoure wrote:
> > src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java, line 1171
> > <http://review.hbase.org/r/76/diff/3-5/?file=724#file724line1171>
> >
> >     Unnecessary call to `toString()'.

It's untouched by my patch, but ok ;)


- Jean-Daniel


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/#review174
-----------------------------------------------------------


On 2010-06-08 17:54:19, Jean-Daniel Cryans wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> http://review.hbase.org/r/76/
> -----------------------------------------------------------
> 
> (Updated 2010-06-08 17:54:19)
> 
> 
> Review request for hbase.
> 
> 
> Summary
> -------
> 
> This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.
> 
> 
> This addresses bug HBASE-2223.
>     http://issues.apache.org/jira/browse/HBASE-2223
> 
> 
> Diffs
> -----
> 
>   bin/replication/add_peer.rb PRE-CREATION 
>   bin/replication/copy_tables_desc.rb PRE-CREATION 
>   pom.xml 03c6ec8 
>   src/main/java/org/apache/hadoop/hbase/HConstants.java 13aff26 
>   src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java b36f1df 
>   src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 82148a6 
>   src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java a1baff4 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 034690e 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 5d4cffe 
>   src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java 2f2f306 
>   src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 
> 
> Diff: http://review.hbase.org/r/76/diff
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Jean-Daniel
> 
>


Re: Review Request: HBASE-2223

Posted by Benoit Sigoure <ts...@gmail.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/#review174
-----------------------------------------------------------



src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<http://review.hbase.org/r/76/#comment838>

    One space before `port'.



src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<http://review.hbase.org/r/76/#comment839>

    Unnecessary call to `toString()'.


- Benoit


On 2010-06-08 17:54:19, Jean-Daniel Cryans wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> http://review.hbase.org/r/76/
> -----------------------------------------------------------
> 
> (Updated 2010-06-08 17:54:19)
> 
> 
> Review request for hbase.
> 
> 
> Summary
> -------
> 
> This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.
> 
> 
> This addresses bug HBASE-2223.
>     http://issues.apache.org/jira/browse/HBASE-2223
> 
> 
> Diffs
> -----
> 
>   bin/replication/add_peer.rb PRE-CREATION 
>   bin/replication/copy_tables_desc.rb PRE-CREATION 
>   pom.xml 03c6ec8 
>   src/main/java/org/apache/hadoop/hbase/HConstants.java 13aff26 
>   src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java b36f1df 
>   src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 82148a6 
>   src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java a1baff4 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 034690e 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 5d4cffe 
>   src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java 2f2f306 
>   src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 
> 
> Diff: http://review.hbase.org/r/76/diff
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Jean-Daniel
> 
>


Re: Review Request: HBASE-2223

Posted by Jean-Daniel Cryans <jd...@apache.org>.

> On 2010-06-25 15:07:35, stack wrote:
> > src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java, line 149
> > <http://review.hbase.org/r/76/diff/7/?file=1590#file1590line149>
> >
> >     Duplicated code -- see up about ten lines.
> 
> Jean-Daniel Cryans wrote:
>     Not it's different, one inserts and the other deletes

Ok oh yeah there is a better way to do it.


- Jean-Daniel


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/#review281
-----------------------------------------------------------


On 2010-06-23 17:24:52, Jean-Daniel Cryans wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> http://review.hbase.org/r/76/
> -----------------------------------------------------------
> 
> (Updated 2010-06-23 17:24:52)
> 
> 
> Review request for hbase.
> 
> 
> Summary
> -------
> 
> This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.
> 
> 
> This addresses bug HBASE-2223.
>     http://issues.apache.org/jira/browse/HBASE-2223
> 
> 
> Diffs
> -----
> 
>   bin/replication/add_peer.rb PRE-CREATION 
>   bin/replication/copy_tables_desc.rb PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/HConstants.java 4357ee5 
>   src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 62617ac 
>   src/main/java/org/apache/hadoop/hbase/master/HMaster.java 5367638 
>   src/main/java/org/apache/hadoop/hbase/master/LogCleanerDelegate.java 4c5153e 
>   src/main/java/org/apache/hadoop/hbase/master/RegionServerOperationQueue.java 10f9dbd 
>   src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 9fb1cce 
>   src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 33b9a99 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 94f18c6 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 5d4cffe 
>   src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperWrapper.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java b26c071 
>   src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java 0e69709 
>   src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 
> 
> Diff: http://review.hbase.org/r/76/diff
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Jean-Daniel
> 
>


Re: Review Request: HBASE-2223

Posted by st...@duboce.net.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/#review281
-----------------------------------------------------------


Missing is more desciption of how this feature works.  There is package doc. on how to get it going but needs fuller description of how replication works (how zk is used, how it manages hlogs, how it sends edits to remote cluster, etc.).  There is insufficient description in javadoc.  Without this, there is a danger that only the author will be able to figure whats going on in here.

How does one get insight on to how well or badly replication is working?

Alot of errors are logged and then we just move on.  Thats fine for an alpha package but need a plan for making it all more robust?  What you thinking?


bin/replication/add_peer.rb
<http://review.hbase.org/r/76/#comment1187>

    These log messages add nothing?  Remove?  Just restating what was passed on cmdline.



bin/replication/add_peer.rb
<http://review.hbase.org/r/76/#comment1188>

    Why do this?  Its annoying?
    
    It'd be better spending time interrogating what the user passed to see if it makes sense throwing errors if incorrectly formatted or if it doesn't look like it makes sense?



bin/replication/add_peer.rb
<http://review.hbase.org/r/76/#comment1189>

    A message on end saying what it did can be comforting to users.



bin/replication/copy_tables_desc.rb
<http://review.hbase.org/r/76/#comment1190>

    See above comments.



src/main/java/org/apache/hadoop/hbase/HConstants.java
<http://review.hbase.org/r/76/#comment1191>

    Change name of this config to hbase.replication.



src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
<http://review.hbase.org/r/76/#comment1192>

    Does this method have prerequisites? For example, does replication have to be enabled?



src/main/java/org/apache/hadoop/hbase/master/HMaster.java
<http://review.hbase.org/r/76/#comment1193>

    Is this a leak from another patch?



src/main/java/org/apache/hadoop/hbase/master/RegionServerOperationQueue.java
<http://review.hbase.org/r/76/#comment1194>

    This is a leak from another patch?



src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
<http://review.hbase.org/r/76/#comment1195>

    Whats going on here?  We're setting into config the logcleaner to use but then in the lines after this we go ahead and create OldLogsCleaner anyways?



src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<http://review.hbase.org/r/76/#comment1196>

    Just call this 'replication' or 'replicationEnabled'



src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<http://review.hbase.org/r/76/#comment1198>

    We are a ReplicationSink or a ReplicationMaster?  One or the other?  Can we not move all of this Replication stuff into a Replication class including the setup of whether we are a Sink or Master rather than have it hang out here in HRS?  



src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<http://review.hbase.org/r/76/#comment1197>

    I don't grok the comment



src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<http://review.hbase.org/r/76/#comment1199>

    This could be null if we are a replication master?



src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java
<http://review.hbase.org/r/76/#comment1200>

    If the above mentioned Replication class were a singleton, it could be shared here with HRS



src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java
<http://review.hbase.org/r/76/#comment1201>

    It'd be good if we didn't have to create this byte [] per edit.  I should reinstitute the comparator I had that took KVs but only compared family portion of the KV... no need to create family byte [] then.



src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java
<http://review.hbase.org/r/76/#comment1202>

    YOu might say why its being moved.  Should be INFO level?  WALs are kinda important to track?



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperWrapper.java
<http://review.hbase.org/r/76/#comment1203>

    isReplicating is what you would name the method that accesses the boolean replicating... with this name the accessor should be named isIsReplicating.



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperWrapper.java
<http://review.hbase.org/r/76/#comment1204>

    Why do it this way?  Why an if/else?  Why not do if (...length != 3) throw .... no need of an else.



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperWrapper.java
<http://review.hbase.org/r/76/#comment1205>

    This is an error? Is it critical fail (We dropped recording a WAL)  Are we just logging and moving on?



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperWrapper.java
<http://review.hbase.org/r/76/#comment1206>

    Same here.... We just log and keep going?  Are any of these fatal?



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperWrapper.java
<http://review.hbase.org/r/76/#comment1207>

    If this fails, what are the repercusssions?



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperWrapper.java
<http://review.hbase.org/r/76/#comment1208>

    What happens if this server dies?  It gets cleaned up by master?
    
    What happens to replication if master dies?  Does master failover need to do anything special to pick up running replication?



src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperWrapper.java
<http://review.hbase.org/r/76/#comment1209>

    spelling
    



src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
<http://review.hbase.org/r/76/#comment1213>

    Why this implementation have to know about other implementations?  Can't we do a chain of decision classes? Any class can say no?  As soon as any decision class says no, we exit the chain.... So in this case, first on the chain would be the ttl decision... then would be this one... and third would be the snapshotting decision. You don't have to do the chain as part of this patch but please open an issue to implement.



src/main/java/org/apache/hadoop/hbase/replication/package.html
<http://review.hbase.org/r/76/#comment1210>

    Its not just MDC, right?



src/main/java/org/apache/hadoop/hbase/replication/package.html
<http://review.hbase.org/r/76/#comment1211>

    Is it user tables or CF in user tables?



src/main/java/org/apache/hadoop/hbase/replication/package.html
<http://review.hbase.org/r/76/#comment1212>

    Good doc.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
<http://review.hbase.org/r/76/#comment1214>

    What is the 'main thread'?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
<http://review.hbase.org/r/76/#comment1215>

    Can you make this clearer?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
<http://review.hbase.org/r/76/#comment1216>

    Whats this mean?  Lost edits?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
<http://review.hbase.org/r/76/#comment1217>

    Maybe say something about how it works... pool of HTables to write remote peer, etc.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
<http://review.hbase.org/r/76/#comment1218>

    Why again does replication keep its own set of logs rather than just keep positions on HRS logs (whether local HRS or the HRS of others?)  So it prevails across HRS failures?  Aren't logs archived now rather than thrown away so this should be fine?  Otherwise, we are writing edits to FS twice, right?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
<http://review.hbase.org/r/76/#comment1220>

    This should be inside a finally block?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
<http://review.hbase.org/r/76/#comment1221>

    Duplicated code -- see up about ten lines.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
<http://review.hbase.org/r/76/#comment1222>

    You are stopping the regionserver because replication failed?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment1223>

    One second seems short?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment1226>

    We carry edits in memory?  And its outside of our normal accounting -- e.g. cache and max for memstores?  How we get it into the mix?  Else we could OOME if we don't account for this memory payload.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment1224>

    Ratio?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment1227>

    Not a constructor



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment1228>

    Stop cluster or stop the host?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment1230>

    Whats this about?  IN particular, the 'handling story'...



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment1231>

    When would it make sense to ever replicate catalog table entries?
    
    Why we even in here reading a file if replicating == false



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment1232>

    Are you checking size of these edits?  You'll read 25k entries of any size?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
<http://review.hbase.org/r/76/#comment1233>

    Where do you explain replication hlog filenaming convention?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
<http://review.hbase.org/r/76/#comment1234>

    Can only do one sink, right?  Might want to say so.  How does the source know the sink to replicate to?  Should that be part of the Interface?
    
    The comment for the Interface could say how the Interface is used... init, then set sink, then per file archived, call ... etc.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment1235>

    Make this "This class manages all the replication sources"
    
    "There are two classes of them"... what you mean to say here?  Two classes of source?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment1236>

    I don't follow this commentary.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment1238>

    You should say more here about whats going on.. either here or up in the class comment.  It has watchers on all other RSs in current cluster and will try to take on the workloads of others on crash (?).



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment1239>

    Comment is cut off.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment1240>

    You reporting status or updating updated position into zk?  There is a running status being kept up in zk?  Shouldn't this log message have the log path in it too?
    
    This method does more than update position or report status... looks like it changes state in this class updating set of logs?



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment1241>

    Shouldn't this be nonmodifiable List that you give out here since it seems you have it synchronized internally here.



src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
<http://review.hbase.org/r/76/#comment1243>

    What other regionserver?  There is alot that goes unexplained in here?  Could point to the doc. on how replication works.  Same for the transer method above (I have an idea because you white boarded it for me).



src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
<http://review.hbase.org/r/76/#comment1244>

    Good.


- stack


On 2010-06-23 17:24:52, Jean-Daniel Cryans wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> http://review.hbase.org/r/76/
> -----------------------------------------------------------
> 
> (Updated 2010-06-23 17:24:52)
> 
> 
> Review request for hbase.
> 
> 
> Summary
> -------
> 
> This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.
> 
> 
> This addresses bug HBASE-2223.
>     http://issues.apache.org/jira/browse/HBASE-2223
> 
> 
> Diffs
> -----
> 
>   bin/replication/add_peer.rb PRE-CREATION 
>   bin/replication/copy_tables_desc.rb PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/HConstants.java 4357ee5 
>   src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 62617ac 
>   src/main/java/org/apache/hadoop/hbase/master/HMaster.java 5367638 
>   src/main/java/org/apache/hadoop/hbase/master/LogCleanerDelegate.java 4c5153e 
>   src/main/java/org/apache/hadoop/hbase/master/RegionServerOperationQueue.java 10f9dbd 
>   src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 9fb1cce 
>   src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 33b9a99 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 94f18c6 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 5d4cffe 
>   src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperWrapper.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java b26c071 
>   src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java 0e69709 
>   src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 
> 
> Diff: http://review.hbase.org/r/76/diff
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Jean-Daniel
> 
>


Re: Review Request: HBASE-2223

Posted by st...@duboce.net.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/#review297
-----------------------------------------------------------

Ship it!


+1 on commit if all tests pass.  I sat wé J-D and went over the changes.  All is good for first commit.

- stack


On 2010-06-30 15:36:59, Jean-Daniel Cryans wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> http://review.hbase.org/r/76/
> -----------------------------------------------------------
> 
> (Updated 2010-06-30 15:36:59)
> 
> 
> Review request for hbase.
> 
> 
> Summary
> -------
> 
> This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.
> 
> 
> This addresses bug HBASE-2223.
>     http://issues.apache.org/jira/browse/HBASE-2223
> 
> 
> Diffs
> -----
> 
>   bin/replication/add_peer.rb PRE-CREATION 
>   bin/replication/copy_tables_desc.rb PRE-CREATION 
>   pom.xml 400cc0c 
>   src/main/java/org/apache/hadoop/hbase/HConstants.java 4357ee5 
>   src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 62617ac 
>   src/main/java/org/apache/hadoop/hbase/master/HMaster.java 04bc95e 
>   src/main/java/org/apache/hadoop/hbase/master/LogCleanerDelegate.java 4c5153e 
>   src/main/java/org/apache/hadoop/hbase/master/OldLogsCleaner.java 07e69b4 
>   src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 33b9a99 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 94f18c6 
>   src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 5d4cffe 
>   src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperWrapper.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
>   src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java b26c071 
>   src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java d865e38 
>   src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java e96637f 
>   src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
>   src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 
> 
> Diff: http://review.hbase.org/r/76/diff
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Jean-Daniel
> 
>


Re: Review Request: HBASE-2223

Posted by Jean-Daniel Cryans <jd...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/
-----------------------------------------------------------

(Updated 2010-06-30 15:36:59.081075)


Review request for hbase.


Changes
-------

Final patch


Summary
-------

This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.


This addresses bug HBASE-2223.
    http://issues.apache.org/jira/browse/HBASE-2223


Diffs (updated)
-----

  bin/replication/add_peer.rb PRE-CREATION 
  bin/replication/copy_tables_desc.rb PRE-CREATION 
  pom.xml 400cc0c 
  src/main/java/org/apache/hadoop/hbase/HConstants.java 4357ee5 
  src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 62617ac 
  src/main/java/org/apache/hadoop/hbase/master/HMaster.java 04bc95e 
  src/main/java/org/apache/hadoop/hbase/master/LogCleanerDelegate.java 4c5153e 
  src/main/java/org/apache/hadoop/hbase/master/OldLogsCleaner.java 07e69b4 
  src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 33b9a99 
  src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 94f18c6 
  src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 5d4cffe 
  src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperWrapper.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java b26c071 
  src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java d865e38 
  src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java e96637f 
  src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 

Diff: http://review.hbase.org/r/76/diff


Testing
-------


Thanks,

Jean-Daniel


Re: Review Request: HBASE-2223

Posted by Jean-Daniel Cryans <jd...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/
-----------------------------------------------------------

(Updated 2010-06-23 17:24:52.371363)


Review request for hbase.


Changes
-------

This patch now passes its own tests (many core modifications made them unstable) and incorporates comments from Stack's review. It also has the patch from HBASE-2707 applied because without it it's too easy to fail TestReplication. 

It also relies on using this hadoop jar http://people.apache.org/~rawson/repo/org/apache/hadoop/hadoop-core/0.20.3-append-r956776+1240+tail/


Summary
-------

This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.


This addresses bug HBASE-2223.
    http://issues.apache.org/jira/browse/HBASE-2223


Diffs (updated)
-----

  bin/replication/add_peer.rb PRE-CREATION 
  bin/replication/copy_tables_desc.rb PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/HConstants.java 4357ee5 
  src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 62617ac 
  src/main/java/org/apache/hadoop/hbase/master/HMaster.java 5367638 
  src/main/java/org/apache/hadoop/hbase/master/LogCleanerDelegate.java 4c5153e 
  src/main/java/org/apache/hadoop/hbase/master/RegionServerOperationQueue.java 10f9dbd 
  src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 9fb1cce 
  src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 33b9a99 
  src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 94f18c6 
  src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 5d4cffe 
  src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperWrapper.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java b26c071 
  src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java 0e69709 
  src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 

Diff: http://review.hbase.org/r/76/diff


Testing
-------


Thanks,

Jean-Daniel


Re: Review Request: HBASE-2223

Posted by Jean-Daniel Cryans <jd...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/
-----------------------------------------------------------

(Updated 2010-06-17 12:57:07.815144)


Review request for hbase.


Changes
-------

Patch rebased on the new Zookeeper/Master and with a cleaned up ReplicationSink. Also some cleanup from Stack's comments.

Currently fails TestReplication, hitting https://issues.apache.org/jira/browse/HBASE-2741 that kills a master thread. The root cause could be in this code but the NPE doesn't help debugging.


Summary
-------

This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.


This addresses bug HBASE-2223.
    http://issues.apache.org/jira/browse/HBASE-2223


Diffs (updated)
-----

  bin/replication/add_peer.rb PRE-CREATION 
  bin/replication/copy_tables_desc.rb PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/HConstants.java f5d3e94 
  src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 62617ac 
  src/main/java/org/apache/hadoop/hbase/master/HMaster.java 66dc697 
  src/main/java/org/apache/hadoop/hbase/master/LogCleanerDelegate.java 4c5153e 
  src/main/java/org/apache/hadoop/hbase/master/RegionServerOperationQueue.java 10f9dbd 
  src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1d95258 
  src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 7ace16a 
  src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 05cf17f 
  src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 5d4cffe 
  src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java 04957ca 
  src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 

Diff: http://review.hbase.org/r/76/diff


Testing
-------


Thanks,

Jean-Daniel


Re: Review Request: HBASE-2223

Posted by Jean-Daniel Cryans <jd...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/
-----------------------------------------------------------

(Updated 2010-06-08 17:54:19.227006)


Review request for hbase.


Changes
-------

Refreshed patch for trunk.


Summary
-------

This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.


This addresses bug HBASE-2223.
    http://issues.apache.org/jira/browse/HBASE-2223


Diffs (updated)
-----

  bin/replication/add_peer.rb PRE-CREATION 
  bin/replication/copy_tables_desc.rb PRE-CREATION 
  pom.xml 03c6ec8 
  src/main/java/org/apache/hadoop/hbase/HConstants.java 13aff26 
  src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java b36f1df 
  src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 82148a6 
  src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java a1baff4 
  src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 034690e 
  src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 5d4cffe 
  src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java 2f2f306 
  src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 

Diff: http://review.hbase.org/r/76/diff


Testing
-------


Thanks,

Jean-Daniel


Re: Review Request: HBASE-2223

Posted by Jean-Daniel Cryans <jd...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/
-----------------------------------------------------------

(Updated 2010-05-28 16:02:44.219338)


Review request for hbase.


Changes
-------

The remaining issues and some resolutions:

ReplicationSink.java
- I think it would be good to document the fact that this method will typically be called from another thread than the thread that executes `run' so that other people reading the code will quickly get a good grasp of what are the concurrency / locking requirements.

I chatted with Benoit on this issue, the problem with ReplicationSink at the moment is it's a mix of two solutions: sync and async. It's sync'ed because the thread that writes the log file has to wait for the ReplicationSink to apply it… but instead it could work like the ReplicationSource and leave it up to ZK-based management of log files. It currently work as is, but it's not efficient. I can fix this in the scope of this jira, which is still named "Handle 10min+ network partitions between clusters", or in a later one. 

- So Delete operations are "unbuffered" unlike Put operations, which you "buffer" in the `puts' list.  Does that mean that a Delete can be executed before the Put that was creating the data in the first place, and that the Delete will fail first and the Put will survive second?

Created a test that inserts 2 rows, deletes the second one, and adds two more. I expect to see the row as deleted, and it works (yeah!). Included in this patch

// Should we log rejected edits in a file for replay?
- I vote yes

This will be in HBASE-2626.

ReplicationSource.java
- This `try' block is massive, would it be possible to refactor it using a private method to make the code a bit more readable?

Did some refactoring, but still not satisfied.


Summary
-------

This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.


This addresses bug HBASE-2223.
    http://issues.apache.org/jira/browse/HBASE-2223


Diffs (updated)
-----

  bin/replication/add_peer.rb PRE-CREATION 
  bin/replication/copy_tables_desc.rb PRE-CREATION 
  pom.xml 0a009cf 
  src/main/java/org/apache/hadoop/hbase/HConstants.java 13aff26 
  src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 4cbe52a 
  src/main/java/org/apache/hadoop/hbase/master/ServerManager.java a197b8f 
  src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b5ff43a 
  src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 12a3cd8 
  src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 7c1184c 
  src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java ed8709f 
  src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 

Diff: http://review.hbase.org/r/76/diff


Testing
-------


Thanks,

Jean-Daniel


Re: Review Request: HBASE-2223

Posted by Jean-Daniel Cryans <jd...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.hbase.org/r/76/
-----------------------------------------------------------

(Updated 2010-05-28 12:03:35.298094)


Review request for hbase.


Changes
-------

New patch that takes care of almost all Benoit's comments. My current TODO is:


ReplicationZookeeperHelper.java
- use a <pre> block to make this more readable in the HTML version of the javadoc.

ReplicationSink.java
- I think it would be good to document the fact that this method will typically be called from another thread than the thread that executes `run' so that other people reading the code will quickly get a good grasp of what are the concurrency / locking requirements.

- So Delete operations are "unbuffered" unlike Put operations, which you "buffer" in the `puts' list.  Does that mean that a Delete can be executed before the Put that was creating the data in the first place, and that the Delete will fail first and the Put will survive second?

// Should we log rejected edits in a file for replay?
- I vote yes

ReplicationSource.java
- This `try' block is massive, would it be possible to refactor it using a private method to make the code a bit more readable?


Summary
-------

This is HBASE-2223 AKA Replication 2.0, it is currently only a "preview patch" as it's pretty much feature complete, works on a cluster, has unit tests and whatnot, but it could use a lot more testing and cleaning and ideas from others.


This addresses bug HBASE-2223.
    http://issues.apache.org/jira/browse/HBASE-2223


Diffs (updated)
-----

  src/main/java/org/apache/hadoop/hbase/HConstants.java 13aff26 
  src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 4cbe52a 
  src/main/java/org/apache/hadoop/hbase/master/ServerManager.java a197b8f 
  src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b5ff43a 
  src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java 12a3cd8 
  src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java 7c1184c 
  src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeperHelper.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/package.html PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java PRE-CREATION 
  src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java ed8709f 
  src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java PRE-CREATION 
  src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java PRE-CREATION 

Diff: http://review.hbase.org/r/76/diff


Testing
-------


Thanks,

Jean-Daniel